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

[nifi] branch master updated: NIFI-7407 Replaced SSLContextFactory references to "TLS" with "TLSv1.2" (in shared constant). Changed JettyServer default SSL initialization and updated unit test. Removed SecurityStoreTypes (unused). Added StringUtils inverted blank and empty checks. Added TlsConfiguration container object. Enhanced KeystoreType enum. Added clean #createSSLContext() method to serve as base method for special cases/other method signatures. Added utility methods in KeyStoreUtils. Added generic TlsException [...]

This is an automated email from the ASF dual-hosted git repository.

alopresto pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/master by this push:
     new 441781c  NIFI-7407 Replaced SSLContextFactory references to "TLS" with "TLSv1.2" (in shared constant). Changed JettyServer default SSL initialization and updated unit test. Removed SecurityStoreTypes (unused). Added StringUtils inverted blank and empty checks. Added TlsConfiguration container object. Enhanced KeystoreType enum. Added clean #createSSLContext() method to serve as base method for special cases/other method signatures. Added utility methods in KeyStoreUtils. Added ge [...]
441781c is described below

commit 441781cec50f77d9f1e65093f55bbd614b8c5ec6
Author: Andy LoPresto <al...@apache.org>
AuthorDate: Tue Apr 28 17:03:13 2020 -0700

    NIFI-7407 Replaced SSLContextFactory references to "TLS" with "TLSv1.2" (in shared constant).
    Changed JettyServer default SSL initialization and updated unit test.
    Removed SecurityStoreTypes (unused).
    Added StringUtils inverted blank and empty checks.
    Added TlsConfiguration container object.
    Enhanced KeystoreType enum.
    Added clean #createSSLContext() method to serve as base method for special cases/other method signatures.
    Added utility methods in KeyStoreUtils.
    Added generic TlsException for callers that cannot resolve TLS-specific exceptions.
    Added utility methods for component object debugging.
    Enforced TLS protocol version on cluster comms socket creation.
    Added utility method for SSL server socket creation.
    Refactored (Server)SocketConfigurationFactoryBean to store relevant NiFiProperties in TlsConfiguration instead of stateful SSLContextFactory (Cluster comms now enforce modern TLS protocol version).
    Removed duplicate SSLContextFactory.
    Switched duplicate SslContextFactory to wrap shared SSLContextFactory.
    Refactored SslContextFactoryTest for clarity (will move any unique tests to nifi-security-utils class test).
    Added further validation & boundary checking in uses of TlsConfiguration.
    Provided SSLSocketFactory accessor in SslContextFactory.
    Refactored OkHttpReplicationClient tuple method.
    Refactored OcspCertificateValidator TLS logic.
    Added utility method to apply TLS configs to OkHttpClientBuilder.
    Removed references to duplicate SslContextFactory.
    Removed unnecessary SslContextFactory.
    Moved OkHttpClientUtils to nifi-web-util module.
    Updated module dependencies.
    Removed now empty nifi-security module.
    Enforced TLS protocol selection on LB server socket.
    Enforced TLS protocol selection on S2S server socket.
    Applied specified TLS protocol versions to S2S socket creation.
    Completed removal of legacy SSLContext creation methods from only remaining SslContextFactory.
    Replaced references to creation methods throughout codebase.
    Replaced references to unnecessary NiFiProperties file reads throughout tests.
    Removed duplicate ClientAuth enum from SSLContextService and changed all references to SslContextFactory.ClientAuth.
    Suppressed repeated TLS exceptions in cluster, S2S, and load balance socket listeners.
    Cleaned up legacy code.
    Added external timing check to timing test assertion.
    Made RestrictedSSLContextService TLS protocol versions allowable values explicit.
    Enabled TLSv1.3 on Java 11.
    Added explanations of TLS protocol versions in StandardSSLContextService and StandardRestrictedSSLContextService.
    Resolved additional Java 11 test failures for NiFi internal classes that don't support TLSv1.3. Filed NIFI-7468 as follow on task.
    
    This closes #4263.
    
    Signed-off-by: Nathan Gough <th...@gmail.com>
    Signed-off-by: Mark Payne <ma...@hotmail.com>
---
 .../notification/http/HttpNotificationService.java |  80 ++--
 .../http/TestHttpNotificationServiceSSL.java       | 211 +++++----
 .../java/org/apache/nifi/util/NiFiProperties.java  |  91 +++-
 .../java/org/apache/nifi/util/StringUtils.java     |   8 +
 .../nifi/security/util/CertificateUtils.java       | 120 ++++--
 .../apache/nifi/security/util/KeyStoreUtils.java   | 182 ++++++++
 .../apache/nifi/security/util/KeystoreType.java    |  35 +-
 .../nifi/security/util/SecurityStoreTypes.java     | 144 -------
 .../nifi/security/util/SslContextFactory.java      | 472 ++++++++------------
 .../nifi/security/util/TlsConfiguration.java       | 480 +++++++++++++++++++++
 .../apache/nifi/security/util/TlsException.java    |  27 +-
 .../nifi/security/util/CertificateUtilsTest.groovy | 137 ++++--
 .../security/util/SslContextFactoryTest.groovy     | 297 +++++++++++++
 .../nifi/security/util/TlsConfigurationTest.groovy | 212 +++++++++
 .../test/resources/TlsConfigurationKeystore.jks    | Bin 0 -> 2269 bytes
 .../test/resources/TlsConfigurationTruststore.jks  | Bin 0 -> 980 bytes
 .../src/test/resources/logback-test.xml            |   1 +
 .../src/test/resources/no-password-truststore.jks  | Bin
 .../src/test/resources/samepassword.jks            | Bin
 .../nifi/remote/client/SiteToSiteClient.java       |  34 +-
 .../nifi/remote/client/http/TestHttpClient.java    |   4 +
 .../apache/nifi/io/socket/SSLContextFactory.java   | 124 ------
 .../nifi/io/socket/ServerSocketConfiguration.java  |  23 +-
 .../apache/nifi/io/socket/SocketConfiguration.java |  23 +-
 .../org/apache/nifi/io/socket/SocketListener.java  |   9 +-
 .../org/apache/nifi/io/socket/SocketUtils.java     |  76 +++-
 .../nifi/io/socket/SSLContextFactoryTest.groovy    | 174 --------
 .../apache/nifi/io/socket/SocketUtilsTest.groovy   | 121 ++++++
 .../test/resources/TlsConfigurationKeystore.jks    | Bin 0 -> 2269 bytes
 .../test/resources/TlsConfigurationTruststore.jks  | Bin 0 -> 980 bytes
 .../src/test/resources/differentpassword.jks       | Bin 1988 -> 0 bytes
 nifi-commons/nifi-web-utils/pom.xml                |   5 +
 .../nifi/security/util/OkHttpClientUtils.java      |  60 +++
 .../LoginIdentityProviderConfigurationContext.java |  11 +-
 .../NonComponentConfigurationContext.java}         |  26 +-
 .../amqp/processors/AbstractAMQPProcessor.java     |  21 +-
 .../nifi/processors/aws/AbstractAWSProcessor.java  |   4 +-
 .../apache/nifi/processors/beats/ListenBeats.java  |  10 +-
 .../cassandra/AbstractCassandraProcessor.java      |  28 +-
 .../nifi/service/CassandraSessionProvider.java     |  20 +-
 .../schemaregistry/ConfluentSchemaRegistry.java    |  29 +-
 .../ElasticSearchClientServiceImpl.java            |  37 +-
 .../nifi-elasticsearch-processors/pom.xml          |   6 +
 .../AbstractElasticsearchHttpProcessor.java        |  71 +--
 .../apache/nifi/processors/email/ListenSMTP.java   |   9 +-
 .../nifi/processors/email/TestListenSMTP.java      |  12 +-
 .../nifi-framework-cluster-protocol/pom.xml        |   4 -
 .../protocol/impl/SocketProtocolListener.java      |  92 ++--
 .../ServerSocketConfigurationFactoryBean.java      |  14 +-
 .../spring/SocketConfigurationFactoryBean.java     |  14 +-
 .../okhttp/OkHttpReplicationClient.java            |  63 +--
 .../okhttp/OkHttpReplicationClientTest.groovy      |  10 +-
 .../TestThreadPoolRequestReplicator.java           |  10 +-
 .../src/test/resources/logback-test.xml            |   2 +
 .../nifi-framework/nifi-framework-core/pom.xml     |   4 -
 .../org/apache/nifi/controller/FlowController.java |  49 +--
 .../server/ConnectionLoadBalanceServer.java        |  85 +++-
 .../manager/StandardStateManagerProvider.java      |  36 +-
 .../registry/flow/StandardFlowRegistryClient.java  |  29 +-
 .../server/ConnectionLoadBalanceServerTest.groovy  | 197 +++++++++
 .../nifi/controller/StandardFlowServiceTest.java   |  19 +-
 .../nifi/controller/TestStandardProcessorNode.java |  55 ++-
 .../queue/clustered/LoadBalancedQueueIT.java       |  96 +++--
 .../reporting/TestStandardReportingContext.java    |  18 +-
 .../repository/TestFileSystemRepository.java       |  56 ++-
 .../TestWriteAheadFlowFileRepository.java          |  75 ++--
 .../claim/TestContentClaimWriteCache.java          |   4 +-
 .../StandardControllerServiceProviderIT.java       |  21 +-
 .../StandardControllerServiceProviderTest.java     |   7 +-
 .../TestStandardControllerServiceProvider.java     |  32 +-
 .../nifi/nar/NarThreadContextClassLoaderTest.java  |  17 +-
 .../StandardNiFiPropertiesGroovyTest.groovy        |  33 ++
 .../nifi-framework/nifi-security/pom.xml           |  40 --
 .../security/util/SslContextCreationException.java |  40 --
 .../framework/security/util/SslContextFactory.java | 104 -----
 .../nifi/framework/security/util/SslException.java |  40 --
 .../SslServerSocketFactoryCreationException.java   |  41 --
 .../security/util/SslContextFactoryTest.java       |  80 ----
 .../nifi-security/src/test/resources/keystore.jks  | Bin 3088 -> 0 bytes
 .../src/test/resources/log4j.properties            |  21 -
 .../src/test/resources/truststore.jks              | Bin 911 -> 0 bytes
 .../nifi-framework/nifi-site-to-site/pom.xml       |   4 -
 .../nifi/remote/SocketRemoteSiteListener.java      |  87 ++--
 .../remote/SocketRemoteSiteListenerTest.groovy     | 137 ++++++
 .../nifi/remote/TestHttpRemoteSiteListener.java    |  17 +-
 .../nifi/remote/TestStandardRemoteGroupPort.java   |  49 ++-
 .../http/TestHttpFlowFileServerProtocol.java       |  62 +--
 .../src/test/resources/localhost-ks.jks            | Bin 0 -> 3076 bytes
 .../src/test/resources/localhost-ts.jks            | Bin 0 -> 911 bytes
 .../apache/nifi/stateless/core/StatelessFlow.java  |   7 +-
 .../nifi-framework/nifi-web/nifi-jetty/pom.xml     |   5 -
 .../org/apache/nifi/web/server/JettyServer.java    |  71 +--
 .../nifi/web/server/JettyServerGroovyTest.groovy   | 203 ++++++++-
 .../nifi-jetty/src/test/resources/keystore.jks     | Bin 0 -> 3095 bytes
 .../nifi-jetty/src/test/resources/log4j.properties |   2 +-
 .../src/test/resources/logback-test.xml            |   7 +-
 .../nifi-jetty/src/test/resources/truststore.jks   | Bin 0 -> 911 bytes
 .../accesscontrol/AccessControlHelper.java         |  16 +-
 .../accesscontrol/ITAccessTokenEndpoint.java       |  15 +-
 .../OneWaySslAccessControlHelper.java              |  24 +-
 .../nifi/integration/util/NiFiTestServer.java      |  17 +-
 .../nifi/web/api/TestDataTransferResource.java     |  69 ++-
 .../x509/ocsp/OcspCertificateValidator.java        |  49 +--
 .../ocsp/OcspCertificateValidatorGroovyTest.groovy | 174 ++++----
 .../x509/X509AuthenticationProviderTest.java       |   4 +-
 .../nifi-framework-bundle/nifi-framework/pom.xml   |   1 -
 .../apache/nifi/processors/grpc/InvokeGRPC.java    |  49 +--
 .../apache/nifi/processors/grpc/ListenGRPC.java    |  59 ++-
 .../nifi/jms/cf/JMSConnectionFactoryHandler.java   |  25 +-
 .../java/org/apache/nifi/ldap/LdapProvider.java    |  69 +--
 .../nifi/ldap/tenants/LdapUserGroupProvider.java   |  88 ++--
 .../processors/lumberjack/ListenLumberjack.java    |   8 +-
 .../apache/nifi/mongodb/MongoDBClientService.java  |   3 +-
 .../processors/mongodb/AbstractMongoProcessor.java |  33 +-
 .../mongodb/AbstractMongoProcessorTest.java        |  15 +-
 .../nifi/mongodb/MongoDBControllerService.java     |  13 +-
 .../TestVolatileProvenanceRepository.java          |  23 +-
 .../reporting/AbstractSiteToSiteReportingTask.java |   6 +-
 .../apache/nifi/reporting/s2s/SiteToSiteUtils.java |  18 +-
 .../sink/SiteToSiteReportingRecordSink.java        |  55 +--
 .../org/apache/nifi/processors/solr/SolrUtils.java |  53 ++-
 .../processors/solr/MockSSLContextService.java     |  91 ++++
 .../apache/nifi/processors/solr/QuerySolrIT.java   | 105 +----
 .../processors/solr/TestPutSolrContentStream.java  | 111 ++---
 .../nifi/processors/solr/TestPutSolrRecord.java    | 110 +----
 .../controller/livy/LivySessionController.java     |  16 +-
 .../apache/nifi/processors/splunk/PutSplunk.java   |  28 +-
 .../nifi-standard-processors/pom.xml               |   6 +
 .../apache/nifi/processors/standard/GetHTTP.java   |   2 +-
 .../nifi/processors/standard/InvokeHTTP.java       | 124 +-----
 .../nifi/processors/standard/ListenRELP.java       |  29 +-
 .../nifi/processors/standard/ListenSyslog.java     |  51 ++-
 .../apache/nifi/processors/standard/ListenTCP.java |  31 +-
 .../nifi/processors/standard/ListenTCPRecord.java  |  49 ++-
 .../apache/nifi/processors/standard/PutSyslog.java |  34 +-
 .../apache/nifi/processors/standard/PutTCP.java    |  22 +-
 .../processors/standard/TestGetHTTPGroovy.groovy   |   5 +-
 .../processors/standard/TestPostHTTPGroovy.groovy  |   5 +-
 .../processors/standard/ITListenAndPutSyslog.java  |  11 +-
 .../standard/ITestHandleHttpRequest.java           | 349 +++++++--------
 .../nifi/processors/standard/TestInvokeHTTP.java   |  36 +-
 .../nifi/processors/standard/TestListenHTTP.java   | 361 ++++++++--------
 .../nifi/processors/standard/TestListenRELP.java   |  27 +-
 .../nifi/processors/standard/TestListenTCP.java    |  98 +++--
 .../processors/standard/TestListenTCPRecord.java   |  87 ++--
 .../nifi/processors/standard/TestPutTcpSSL.java    |  10 +-
 .../processors/standard/util/TCPTestServer.java    |  14 +-
 .../client/DistributedMapCacheClientService.java   |   5 +-
 .../client/DistributedSetCacheClientService.java   |   3 +-
 .../cache/server/DistributedSetCacheServer.java    |   4 +-
 .../server/map/DistributedMapCacheServer.java      |   4 +-
 .../nifi-lookup-services/pom.xml                   |   6 +
 .../org/apache/nifi/lookup/RestLookupService.java  |  77 +---
 .../ssl/StandardRestrictedSSLContextService.java   |   3 +-
 .../apache/nifi/ssl/StandardSSLContextService.java |  71 ++-
 .../nifi/ssl/StandardSSLContextServiceTest.groovy  |   5 +-
 .../nifi/ssl/RestrictedSSLContextServiceTest.java  |   4 +-
 .../org/apache/nifi/ssl/SSLContextServiceTest.java |   9 +-
 .../nifi-ssl-context-service-api/pom.xml           |   6 +
 .../nifi/ssl/RestrictedSSLContextService.java      |   5 +-
 .../org/apache/nifi/ssl/SSLContextService.java     |  34 +-
 .../cli/impl/client/nifi/NiFiClientConfig.java     |  18 +-
 .../TlsCertificateSigningRequestPerformer.java     |   2 +-
 .../server/TlsCertificateAuthorityService.java     |  18 +-
 .../tls/status/TlsToolkitGetStatusCommandLine.java |  47 +-
 165 files changed, 4722 insertions(+), 3790 deletions(-)

diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java
index 2ab7c72..fdb4c2d 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/notification/http/HttpNotificationService.java
@@ -16,6 +16,13 @@
  */
 package org.apache.nifi.bootstrap.notification.http;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.X509TrustManager;
 import okhttp3.Call;
 import okhttp3.MediaType;
 import okhttp3.OkHttpClient;
@@ -32,18 +39,7 @@ import org.apache.nifi.expression.AttributeExpression;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.security.util.SslContextFactory;
-import org.apache.nifi.util.Tuple;
-
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManager;
-import javax.net.ssl.X509TrustManager;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.stream.Collectors;
+import org.apache.nifi.security.util.TlsConfiguration;
 
 public class HttpNotificationService extends AbstractNotificationService {
 
@@ -139,6 +135,7 @@ public class HttpNotificationService extends AbstractNotificationService {
     private final AtomicReference<String> urlReference = new AtomicReference<>();
 
     private static final List<PropertyDescriptor> supportedProperties;
+
     static {
         supportedProperties = new ArrayList<>();
         supportedProperties.add(PROP_URL);
@@ -160,7 +157,7 @@ public class HttpNotificationService extends AbstractNotificationService {
     }
 
     @Override
-    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName){
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
         return new PropertyDescriptor.Builder()
                 .required(false)
                 .name(propertyDescriptorName)
@@ -193,25 +190,15 @@ public class HttpNotificationService extends AbstractNotificationService {
         // check if the keystore is set and add the factory if so
         if (url.toLowerCase().startsWith("https")) {
             try {
-                Tuple<SSLContext, TrustManager[]> sslContextTuple = SslContextFactory.createTrustSslContextWithTrustManagers(
-                        context.getProperty(HttpNotificationService.PROP_KEYSTORE).getValue(),
-                        context.getProperty(HttpNotificationService.PROP_KEYSTORE_PASSWORD).isSet()
-                                ? context.getProperty(HttpNotificationService.PROP_KEYSTORE_PASSWORD).getValue().toCharArray() : null,
-                        context.getProperty(HttpNotificationService.PROP_KEY_PASSWORD).isSet()
-                                ? context.getProperty(HttpNotificationService.PROP_KEY_PASSWORD).getValue().toCharArray() : null,
-                        context.getProperty(HttpNotificationService.PROP_KEYSTORE_TYPE).getValue(),
-                        context.getProperty(HttpNotificationService.PROP_TRUSTSTORE).getValue(),
-                        context.getProperty(HttpNotificationService.PROP_TRUSTSTORE_PASSWORD).isSet()
-                                ? context.getProperty(HttpNotificationService.PROP_TRUSTSTORE_PASSWORD).getValue().toCharArray() : null,
-                        context.getProperty(HttpNotificationService.PROP_TRUSTSTORE_TYPE).getValue(),
-                        SslContextFactory.ClientAuth.REQUIRED,
-                        context.getProperty(HttpNotificationService.SSL_ALGORITHM).getValue()
-                );
-                // Find the first X509TrustManager
-                List<X509TrustManager> x509TrustManagers = Arrays.stream(sslContextTuple.getValue())
-                        .filter(trustManager -> trustManager instanceof X509TrustManager)
-                        .map(trustManager -> (X509TrustManager) trustManager).collect(Collectors.toList());
-                okHttpClientBuilder.sslSocketFactory(sslContextTuple.getKey().getSocketFactory(), x509TrustManagers.get(0));
+                TlsConfiguration tlsConfiguration = createTlsConfigurationFromContext(context);
+                final SSLSocketFactory sslSocketFactory = SslContextFactory.createSSLSocketFactory(tlsConfiguration);
+                final X509TrustManager x509TrustManager = SslContextFactory.getX509TrustManager(tlsConfiguration);
+                if (sslSocketFactory != null && x509TrustManager != null) {
+                    okHttpClientBuilder.sslSocketFactory(sslSocketFactory, x509TrustManager);
+                } else {
+                    // If the TLS config couldn't be parsed, throw an exception
+                    throw new IllegalStateException("The HTTP notification service URL indicates HTTPS but the TLS properties are not valid");
+                }
             } catch (Exception e) {
                 throw new IllegalStateException(e);
             }
@@ -220,6 +207,17 @@ public class HttpNotificationService extends AbstractNotificationService {
         httpClientReference.set(okHttpClientBuilder.build());
     }
 
+    private static TlsConfiguration createTlsConfigurationFromContext(NotificationInitializationContext context) {
+        String keystorePath = context.getProperty(HttpNotificationService.PROP_KEYSTORE).getValue();
+        String keystorePassword = context.getProperty(HttpNotificationService.PROP_KEYSTORE_PASSWORD).getValue();
+        String keyPassword = context.getProperty(HttpNotificationService.PROP_KEY_PASSWORD).getValue();
+        String keystoreType = context.getProperty(HttpNotificationService.PROP_KEYSTORE_TYPE).getValue();
+        String truststorePath = context.getProperty(HttpNotificationService.PROP_TRUSTSTORE).getValue();
+        String truststorePassword = context.getProperty(HttpNotificationService.PROP_TRUSTSTORE_PASSWORD).getValue();
+        String truststoreType = context.getProperty(HttpNotificationService.PROP_TRUSTSTORE_TYPE).getValue();
+        return new TlsConfiguration(keystorePath, keystorePassword, keyPassword, keystoreType, truststorePath, truststorePassword, truststoreType);
+    }
+
     @Override
     public void notify(NotificationContext context, NotificationType notificationType, String subject, String message) throws NotificationFailedException {
         try {
@@ -231,7 +229,7 @@ public class HttpNotificationService extends AbstractNotificationService {
 
             Map<PropertyDescriptor, String> configuredProperties = context.getProperties();
 
-            for(PropertyDescriptor propertyDescriptor: configuredProperties.keySet()) {
+            for (PropertyDescriptor propertyDescriptor : configuredProperties.keySet()) {
                 if (propertyDescriptor.isDynamic()) {
                     String propertyValue = context.getProperty(propertyDescriptor).evaluateAttributeExpressions().getValue();
                     requestBuilder = requestBuilder.addHeader(propertyDescriptor.getDisplayName(), propertyValue);
@@ -246,14 +244,14 @@ public class HttpNotificationService extends AbstractNotificationService {
             final OkHttpClient httpClient = httpClientReference.get();
 
             final Call call = httpClient.newCall(request);
-             try (final Response response = call.execute()) {
-
-                 if (!response.isSuccessful()) {
-                     throw new NotificationFailedException("Failed to send Http Notification. Received an unsuccessful status code response '" + response.code() + "'. The message was '" +
-                             response.message() + "'");
-                 }
-             }
-        } catch (NotificationFailedException e){
+            try (final Response response = call.execute()) {
+
+                if (!response.isSuccessful()) {
+                    throw new NotificationFailedException("Failed to send Http Notification. Received an unsuccessful status code response '" + response.code() + "'. The message was '" +
+                            response.message() + "'");
+                }
+            }
+        } catch (NotificationFailedException e) {
             throw e;
         } catch (Exception e) {
             throw new NotificationFailedException("Failed to send Http Notification", e);
diff --git a/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java b/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java
index 2f54f1c..ac280cf 100644
--- a/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java
+++ b/nifi-bootstrap/src/test/java/org/apache/nifi/bootstrap/http/TestHttpNotificationServiceSSL.java
@@ -16,139 +16,128 @@
  */
 package org.apache.nifi.bootstrap.http;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import ch.qos.logback.classic.Logger;
 import ch.qos.logback.classic.spi.ILoggingEvent;
 import ch.qos.logback.core.read.ListAppender;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import javax.net.ssl.SSLContext;
+import javax.xml.parsers.ParserConfigurationException;
 import okhttp3.mockwebserver.MockWebServer;
 import org.apache.nifi.bootstrap.NotificationServiceManager;
+import org.apache.nifi.security.util.CertificateUtils;
 import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.internal.util.io.IOUtil;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
-import ch.qos.logback.classic.Logger;
-
-import javax.net.ssl.SSLContext;
-import javax.xml.parsers.ParserConfigurationException;
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-import java.util.List;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 public class TestHttpNotificationServiceSSL extends TestHttpNotificationServiceCommon {
 
-    static final String CONFIGURATION_FILE_TEXT = "\n"+
-            "<services>\n"+
-            "         <service>\n"+
-            "            <id>http-notification</id>\n"+
-            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
-            "            <property name=\"URL\">${test.server}</property>\n"+
-            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n"+
-            "            <property name=\"Truststore Type\">JKS</property>\n"+
-            "            <property name=\"Truststore Password\">passwordpassword</property>\n"+
-            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n"+
-            "            <property name=\"Keystore Type\">JKS</property>\n"+
-            "            <property name=\"Key Password\">passwordpassword</property>\n"+
-            "            <property name=\"Keystore Password\">passwordpassword</property>\n"+
-            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
-            "         </service>\n"+
+    static final String CONFIGURATION_FILE_TEXT = "\n" +
+            "<services>\n" +
+            "         <service>\n" +
+            "            <id>http-notification</id>\n" +
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n" +
+            "            <property name=\"URL\">${test.server}</property>\n" +
+            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n" +
+            "            <property name=\"Truststore Type\">JKS</property>\n" +
+            "            <property name=\"Truststore Password\">passwordpassword</property>\n" +
+            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n" +
+            "            <property name=\"Keystore Type\">JKS</property>\n" +
+            "            <property name=\"Key Password\">passwordpassword</property>\n" +
+            "            <property name=\"Keystore Password\">passwordpassword</property>\n" +
+            "            <property name=\"testProp\">${literal('testing')}</property>\n" +
+            "         </service>\n" +
             "</services>";
 
-    static final String CONFIGURATION_FILE_TEXT_NO_KEYSTORE_PASSWORD = "\n"+
-            "<services>\n"+
-            "         <service>\n"+
-            "            <id>http-notification</id>\n"+
-            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
-            "            <property name=\"URL\">${test.server}</property>\n"+
-            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n"+
-            "            <property name=\"Truststore Type\">JKS</property>\n"+
-            "            <property name=\"Truststore Password\">passwordpassword</property>\n"+
-            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n"+
-            "            <property name=\"Keystore Type\">JKS</property>\n"+
-            "            <property name=\"Key Password\">passwordpassword</property>\n"+
-            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
-            "         </service>\n"+
+    static final String CONFIGURATION_FILE_TEXT_NO_KEYSTORE_PASSWORD = "\n" +
+            "<services>\n" +
+            "         <service>\n" +
+            "            <id>http-notification</id>\n" +
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n" +
+            "            <property name=\"URL\">${test.server}</property>\n" +
+            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n" +
+            "            <property name=\"Truststore Type\">JKS</property>\n" +
+            "            <property name=\"Truststore Password\">passwordpassword</property>\n" +
+            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n" +
+            "            <property name=\"Keystore Type\">JKS</property>\n" +
+            "            <property name=\"Key Password\">passwordpassword</property>\n" +
+            "            <property name=\"testProp\">${literal('testing')}</property>\n" +
+            "         </service>\n" +
             "</services>";
 
-    static final String CONFIGURATION_FILE_TEXT_NO_KEY_PASSWORD = "\n"+
-            "<services>\n"+
-            "         <service>\n"+
-            "            <id>http-notification</id>\n"+
-            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
-            "            <property name=\"URL\">${test.server}</property>\n"+
-            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n"+
-            "            <property name=\"Truststore Type\">JKS</property>\n"+
-            "            <property name=\"Truststore Password\">passwordpassword</property>\n"+
-            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n"+
-            "            <property name=\"Keystore Type\">JKS</property>\n"+
-            "            <property name=\"Keystore Password\">passwordpassword</property>\n"+
-            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
-            "         </service>\n"+
+    static final String CONFIGURATION_FILE_TEXT_NO_KEY_PASSWORD = "\n" +
+            "<services>\n" +
+            "         <service>\n" +
+            "            <id>http-notification</id>\n" +
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n" +
+            "            <property name=\"URL\">${test.server}</property>\n" +
+            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n" +
+            "            <property name=\"Truststore Type\">JKS</property>\n" +
+            "            <property name=\"Truststore Password\">passwordpassword</property>\n" +
+            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n" +
+            "            <property name=\"Keystore Type\">JKS</property>\n" +
+            "            <property name=\"Keystore Password\">passwordpassword</property>\n" +
+            "            <property name=\"testProp\">${literal('testing')}</property>\n" +
+            "         </service>\n" +
             "</services>";
 
-    static final String CONFIGURATION_FILE_TEXT_BLANK_KEY_PASSWORD = "\n"+
-            "<services>\n"+
-            "         <service>\n"+
-            "            <id>http-notification</id>\n"+
-            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
-            "            <property name=\"URL\">${test.server}</property>\n"+
-            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n"+
-            "            <property name=\"Truststore Type\">JKS</property>\n"+
-            "            <property name=\"Truststore Password\">passwordpassword</property>\n"+
-            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n"+
-            "            <property name=\"Keystore Type\">JKS</property>\n"+
-            "            <property name=\"Keystore Password\">passwordpassword</property>\n"+
-            "            <property name=\"Key Password\"></property>\n"+
-            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
-            "         </service>\n"+
+    static final String CONFIGURATION_FILE_TEXT_BLANK_KEY_PASSWORD = "\n" +
+            "<services>\n" +
+            "         <service>\n" +
+            "            <id>http-notification</id>\n" +
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n" +
+            "            <property name=\"URL\">${test.server}</property>\n" +
+            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n" +
+            "            <property name=\"Truststore Type\">JKS</property>\n" +
+            "            <property name=\"Truststore Password\">passwordpassword</property>\n" +
+            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n" +
+            "            <property name=\"Keystore Type\">JKS</property>\n" +
+            "            <property name=\"Keystore Password\">passwordpassword</property>\n" +
+            "            <property name=\"Key Password\"></property>\n" +
+            "            <property name=\"testProp\">${literal('testing')}</property>\n" +
+            "         </service>\n" +
             "</services>";
 
-    static final String CONFIGURATION_FILE_TEXT_BLANK_KEYSTORE_PASSWORD = "\n"+
-            "<services>\n"+
-            "         <service>\n"+
-            "            <id>http-notification</id>\n"+
-            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n"+
-            "            <property name=\"URL\">${test.server}</property>\n"+
-            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n"+
-            "            <property name=\"Truststore Type\">JKS</property>\n"+
-            "            <property name=\"Truststore Password\">passwordpassword</property>\n"+
-            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n"+
-            "            <property name=\"Keystore Type\">JKS</property>\n"+
-            "            <property name=\"Keystore Password\"></property>\n"+
-            "            <property name=\"Key Password\">passwordpassword</property>\n"+
-            "            <property name=\"testProp\">${literal('testing')}</property>\n"+
-            "         </service>\n"+
+    static final String CONFIGURATION_FILE_TEXT_BLANK_KEYSTORE_PASSWORD = "\n" +
+            "<services>\n" +
+            "         <service>\n" +
+            "            <id>http-notification</id>\n" +
+            "            <class>org.apache.nifi.bootstrap.notification.http.HttpNotificationService</class>\n" +
+            "            <property name=\"URL\">${test.server}</property>\n" +
+            "            <property name=\"Truststore Filename\">./src/test/resources/truststore.jks</property>\n" +
+            "            <property name=\"Truststore Type\">JKS</property>\n" +
+            "            <property name=\"Truststore Password\">passwordpassword</property>\n" +
+            "            <property name=\"Keystore Filename\">./src/test/resources/keystore.jks</property>\n" +
+            "            <property name=\"Keystore Type\">JKS</property>\n" +
+            "            <property name=\"Keystore Password\"></property>\n" +
+            "            <property name=\"Key Password\">passwordpassword</property>\n" +
+            "            <property name=\"testProp\">${literal('testing')}</property>\n" +
+            "         </service>\n" +
             "</services>";
 
     @Before
-    public void startServer() throws IOException, UnrecoverableKeyException, CertificateException, NoSuchAlgorithmException, KeyStoreException, KeyManagementException {
+    public void startServer() throws IOException, TlsException {
         tempConfigFilePath = "./target/TestHttpNotificationService-config.xml";
 
         Files.deleteIfExists(Paths.get(tempConfigFilePath));
 
         mockWebServer = new MockWebServer();
 
-        final SSLContext sslContext = SslContextFactory.createSslContext(
-                "./src/test/resources/keystore.jks",
-                "passwordpassword".toCharArray(),
-                null,
-                "JKS",
-                "./src/test/resources/truststore.jks",
-                "passwordpassword".toCharArray(),
-                "JKS",
-                SslContextFactory.ClientAuth.REQUIRED,
-                "TLS");
-
+        TlsConfiguration tlsConfiguration = new TlsConfiguration("./src/test/resources/keystore.jks", "passwordpassword", null, "JKS",
+                "./src/test/resources/truststore.jks", "passwordpassword", "JKS", CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion());
+        final SSLContext sslContext = SslContextFactory.createSslContext(tlsConfiguration, SslContextFactory.ClientAuth.REQUIRED);
         mockWebServer.useHttps(sslContext.getSocketFactory(), false);
 
         String configFileOutput = CONFIGURATION_FILE_TEXT.replace("${test.server}", String.valueOf(mockWebServer.url("/")));
@@ -177,9 +166,9 @@ public class TestHttpNotificationServiceSSL extends TestHttpNotificationServiceC
 
         List<ILoggingEvent> logsList = listAppender.list;
         boolean notificationServiceFailed = false;
-        for(ILoggingEvent logMessage : logsList) {
-            if(logMessage.getFormattedMessage().contains("is not valid for the following reasons")) {
-                    notificationServiceFailed = true;
+        for (ILoggingEvent logMessage : logsList) {
+            if (logMessage.getFormattedMessage().contains("is not valid for the following reasons")) {
+                notificationServiceFailed = true;
             }
         }
 
@@ -202,8 +191,8 @@ public class TestHttpNotificationServiceSSL extends TestHttpNotificationServiceC
 
         List<ILoggingEvent> logsList = listAppender.list;
         boolean notificationServiceFailed = false;
-        for(ILoggingEvent logMessage : logsList) {
-            if(logMessage.getFormattedMessage().contains("is not valid for the following reasons")) {
+        for (ILoggingEvent logMessage : logsList) {
+            if (logMessage.getFormattedMessage().contains("is not valid for the following reasons")) {
                 notificationServiceFailed = true;
             }
         }
@@ -227,8 +216,8 @@ public class TestHttpNotificationServiceSSL extends TestHttpNotificationServiceC
 
         List<ILoggingEvent> logsList = listAppender.list;
         boolean notificationServiceFailed = false;
-        for(ILoggingEvent logMessage : logsList) {
-            if(logMessage.getFormattedMessage().contains("'Keystore Password' validated against '' is invalid because Keystore Password cannot be empty")) {
+        for (ILoggingEvent logMessage : logsList) {
+            if (logMessage.getFormattedMessage().contains("'Keystore Password' validated against '' is invalid because Keystore Password cannot be empty")) {
                 notificationServiceFailed = true;
             }
         }
@@ -252,8 +241,8 @@ public class TestHttpNotificationServiceSSL extends TestHttpNotificationServiceC
 
         List<ILoggingEvent> logsList = listAppender.list;
         boolean notificationServiceFailed = false;
-        for(ILoggingEvent logMessage : logsList) {
-            if(logMessage.getFormattedMessage().contains("'Key Password' validated against '' is invalid because Key Password cannot be empty")) {
+        for (ILoggingEvent logMessage : logsList) {
+            if (logMessage.getFormattedMessage().contains("'Key Password' validated against '' is invalid because Key Password cannot be empty")) {
                 notificationServiceFailed = true;
             }
         }
diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
index 5643de0..0943c87 100644
--- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
+++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java
@@ -1672,10 +1672,28 @@ public abstract class NiFiProperties {
      * framework for default property loading behavior or helpful in tests
      * needing to create specific instances of NiFiProperties. If properties
      * file specified cannot be found/read a runtime exception will be thrown.
-     * If one is not specified no properties will be loaded by default.
+     * If one is not specified an empty object will be returned.
      *
      * @param propertiesFilePath   if provided properties will be loaded from
-     *                             given file; else will be loaded from System property. Can be null.
+     *                             given file; else will be loaded from System property.
+     *                             Can be null. Passing {@code ""} skips any attempt to load from the file system.
+     * @return NiFiProperties
+     */
+    public static NiFiProperties createBasicNiFiProperties(final String propertiesFilePath) {
+        return createBasicNiFiProperties(propertiesFilePath, new Properties());
+    }
+
+    /**
+     * Creates an instance of NiFiProperties. This should likely not be called
+     * by any classes outside of the NiFi framework but can be useful by the
+     * framework for default property loading behavior or helpful in tests
+     * needing to create specific instances of NiFiProperties. If properties
+     * file specified cannot be found/read a runtime exception will be thrown.
+     * If one is not specified, only the provided properties will be returned.
+     *
+     * @param propertiesFilePath   if provided properties will be loaded from
+     *                             given file; else will be loaded from System property.
+     *                             Can be null. Passing {@code ""} skips any attempt to load from the file system.
      * @param additionalProperties allows overriding of properties with the
      *                             supplied values. these will be applied after loading from any properties
      *                             file. Can be null or empty.
@@ -1684,18 +1702,65 @@ public abstract class NiFiProperties {
     public static NiFiProperties createBasicNiFiProperties(final String propertiesFilePath, final Map<String, String> additionalProperties) {
         final Map<String, String> addProps = (additionalProperties == null) ? Collections.EMPTY_MAP : additionalProperties;
         final Properties properties = new Properties();
+        addProps.forEach(properties::put);
+
+        return createBasicNiFiProperties(propertiesFilePath, properties);
+    }
+
+    /**
+     * Creates an instance of NiFiProperties. This should likely not be called
+     * by any classes outside of the NiFi framework but can be useful by the
+     * framework for default property loading behavior or helpful in tests
+     * needing to create specific instances of NiFiProperties. If properties
+     * file specified cannot be found/read a runtime exception will be thrown.
+     * If one is not specified, only the provided properties will be returned.
+     *
+     * @param propertiesFilePath   if provided properties will be loaded from
+     *                             given file; else will be loaded from System property.
+     *                             Can be null. Passing {@code ""} skips any attempt to load from the file system.
+     * @param additionalProperties allows overriding of properties with the
+     *                             supplied values. these will be applied after loading from any properties
+     *                             file. Can be null or empty.
+     * @return NiFiProperties
+     */
+    public static NiFiProperties createBasicNiFiProperties(final String propertiesFilePath, final Properties additionalProperties) {
+        final Properties properties = new Properties();
+
+        // If the provided file path is null or provided, load from file. If it is "", skip this
+        if (propertiesFilePath == null || StringUtils.isNotBlank(propertiesFilePath)) {
+            readFromPropertiesFile(propertiesFilePath, properties);
+        }
+
+        // The Properties(Properties) constructor does NOT inherit the provided values, just uses them as default values
+        if (additionalProperties != null) {
+            additionalProperties.forEach(properties::put);
+        }
+        return new NiFiProperties() {
+            @Override
+            public String getProperty(String key) {
+                return properties.getProperty(key);
+            }
+
+            @Override
+            public Set<String> getPropertyKeys() {
+                return properties.stringPropertyNames();
+            }
+        };
+    }
+
+    private static void readFromPropertiesFile(String propertiesFilePath, Properties properties) {
         final String nfPropertiesFilePath = (propertiesFilePath == null)
                 ? System.getProperty(NiFiProperties.PROPERTIES_FILE_PATH)
                 : propertiesFilePath;
         if (nfPropertiesFilePath != null) {
             final File propertiesFile = new File(nfPropertiesFilePath.trim());
             if (!propertiesFile.exists()) {
-                throw new RuntimeException("Properties file doesn't exist \'"
-                        + propertiesFile.getAbsolutePath() + "\'");
+                throw new RuntimeException("Properties file doesn't exist '"
+                        + propertiesFile.getAbsolutePath() + "'");
             }
             if (!propertiesFile.canRead()) {
-                throw new RuntimeException("Properties file exists but cannot be read \'"
-                        + propertiesFile.getAbsolutePath() + "\'");
+                throw new RuntimeException("Properties file exists but cannot be read '"
+                        + propertiesFile.getAbsolutePath() + "'");
             }
             InputStream inStream = null;
             try {
@@ -1716,20 +1781,6 @@ public abstract class NiFiProperties {
                 }
             }
         }
-        addProps.entrySet().stream().forEach((entry) -> {
-            properties.setProperty(entry.getKey(), entry.getValue());
-        });
-        return new NiFiProperties() {
-            @Override
-            public String getProperty(String key) {
-                return properties.getProperty(key);
-            }
-
-            @Override
-            public Set<String> getPropertyKeys() {
-                return properties.stringPropertyNames();
-            }
-        };
     }
 
     /**
diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java
index 5418812..8948eaf 100644
--- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java
+++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java
@@ -41,10 +41,18 @@ public class StringUtils {
         return true;
     }
 
+    public static boolean isNotBlank(final String str) {
+        return !isBlank(str);
+    }
+
     public static boolean isEmpty(final String str) {
         return str == null || str.isEmpty();
     }
 
+    public static boolean isNotEmpty(final String str) {
+        return !isEmpty(str);
+    }
+
     public static boolean startsWith(final String str, final String prefix) {
         if (str == null || prefix == null) {
             return (str == null && prefix == null);
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java
index d51e951..ee0c33e 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java
@@ -38,9 +38,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 import javax.naming.InvalidNameException;
 import javax.naming.ldap.LdapName;
 import javax.naming.ldap.Rdn;
+import javax.net.ssl.SSLException;
 import javax.net.ssl.SSLPeerUnverifiedException;
 import javax.net.ssl.SSLSocket;
 import org.apache.commons.lang3.StringUtils;
@@ -79,6 +82,11 @@ public final class CertificateUtils {
     private static final String PEER_NOT_AUTHENTICATED_MSG = "peer not authenticated";
     private static final Map<ASN1ObjectIdentifier, Integer> dnOrderMap = createDnOrderMap();
 
+    public static final String JAVA_8_MAX_SUPPORTED_TLS_PROTOCOL_VERSION = "TLSv1.2";
+    public static final String JAVA_11_MAX_SUPPORTED_TLS_PROTOCOL_VERSION = "TLSv1.3";
+    public static final String[] JAVA_8_SUPPORTED_TLS_PROTOCOL_VERSIONS = new String[]{JAVA_8_MAX_SUPPORTED_TLS_PROTOCOL_VERSION};
+    public static final String[] JAVA_11_SUPPORTED_TLS_PROTOCOL_VERSIONS = new String[]{JAVA_11_MAX_SUPPORTED_TLS_PROTOCOL_VERSION, JAVA_8_MAX_SUPPORTED_TLS_PROTOCOL_VERSION};
+
     static {
         Security.addProvider(new BouncyCastleProvider());
     }
@@ -113,25 +121,6 @@ public final class CertificateUtils {
         return Collections.unmodifiableMap(orderMap);
     }
 
-    public enum ClientAuth {
-        NONE(0, "none"),
-        WANT(1, "want"),
-        NEED(2, "need");
-
-        private int value;
-        private String description;
-
-        ClientAuth(int value, String description) {
-            this.value = value;
-            this.description = description;
-        }
-
-        @Override
-        public String toString() {
-            return "Client Auth: " + this.description + " (" + this.value + ")";
-        }
-    }
-
     /**
      * Extracts the username from the specified DN. If the username cannot be extracted because the CN is in an unrecognized format, the entire CN is returned. If the CN cannot be extracted because
      * the DN is in an unrecognized format, the entire DN is returned.
@@ -211,7 +200,7 @@ public final class CertificateUtils {
 
             boolean clientMode = sslSocket.getUseClientMode();
             logger.debug("SSL Socket in {} mode", clientMode ? "client" : "server");
-            ClientAuth clientAuth = getClientAuthStatus(sslSocket);
+            SslContextFactory.ClientAuth clientAuth = getClientAuthStatus(sslSocket);
             logger.debug("SSL Socket client auth status: {}", clientAuth);
 
             if (clientMode) {
@@ -244,10 +233,10 @@ public final class CertificateUtils {
          * This method should throw an exception if none are provided for need, return null if none are provided for want, and return null (without checking) for none.
          */
 
-        ClientAuth clientAuth = getClientAuthStatus(sslSocket);
+        SslContextFactory.ClientAuth clientAuth = getClientAuthStatus(sslSocket);
         logger.debug("SSL Socket client auth status: {}", clientAuth);
 
-        if (clientAuth != ClientAuth.NONE) {
+        if (clientAuth != SslContextFactory.ClientAuth.NONE) {
             try {
                 final Certificate[] certChains = sslSocket.getSession().getPeerCertificates();
                 if (certChains != null && certChains.length > 0) {
@@ -260,7 +249,7 @@ public final class CertificateUtils {
                     logger.error("The incoming request did not contain client certificates and thus the DN cannot" +
                             " be extracted. Check that the other endpoint is providing a complete client certificate chain");
                 }
-                if (clientAuth == ClientAuth.WANT) {
+                if (clientAuth == SslContextFactory.ClientAuth.WANT) {
                     logger.warn("Suppressing missing client certificate exception because client auth is set to 'want'");
                     return dn;
                 }
@@ -299,8 +288,8 @@ public final class CertificateUtils {
         return dn;
     }
 
-    private static ClientAuth getClientAuthStatus(SSLSocket sslSocket) {
-        return sslSocket.getNeedClientAuth() ? ClientAuth.NEED : sslSocket.getWantClientAuth() ? ClientAuth.WANT : ClientAuth.NONE;
+    private static SslContextFactory.ClientAuth getClientAuthStatus(SSLSocket sslSocket) {
+        return sslSocket.getNeedClientAuth() ? SslContextFactory.ClientAuth.REQUIRED : sslSocket.getWantClientAuth() ? SslContextFactory.ClientAuth.WANT : SslContextFactory.ClientAuth.NONE;
     }
 
     /**
@@ -311,6 +300,7 @@ public final class CertificateUtils {
      * @return a new {@code java.security.cert.X509Certificate}
      * @throws CertificateException if there is an error generating the new certificate
      */
+    @SuppressWarnings("deprecation")
     public static X509Certificate convertLegacyX509Certificate(javax.security.cert.X509Certificate legacyCertificate) throws CertificateException {
         if (legacyCertificate == null) {
             throw new IllegalArgumentException("The X.509 certificate cannot be null");
@@ -616,6 +606,86 @@ public final class CertificateUtils {
         return null;
     }
 
+    /**
+     * Returns {@code true} if this exception is due to a TLS problem (either directly or because of its cause, if present). Traverses the cause chain recursively.
+     *
+     * @param e the exception to evaluate
+     * @return true if the direct or indirect cause of this exception was TLS-related
+     */
+    public static boolean isTlsError(Throwable e) {
+        if (e == null) {
+            return false;
+        } else {
+            if (e instanceof CertificateException || e instanceof TlsException || e instanceof SSLException) {
+                return true;
+            } else if (e.getCause() != null) {
+                return isTlsError(e.getCause());
+            } else {
+                return false;
+            }
+        }
+    }
+
+    /**
+     * Returns the JVM Java major version based on the System properties (e.g. {@code JVM 1.8.0.231} -> {code 8}).
+     *
+     * @return the Java major version
+     */
+    public static int getJavaVersion() {
+        String version = System.getProperty("java.version");
+        return parseJavaVersion(version);
+    }
+
+    /**
+     * Returns the major version parsed from the provided Java version string (e.g. {@code "1.8.0.231"} -> {@code 8}).
+     *
+     * @param version the Java version string
+     * @return the major version as an int
+     */
+    public static int parseJavaVersion(String version) {
+        String majorVersion;
+        if (version.startsWith("1.")) {
+            majorVersion = version.substring(2, 3);
+        } else {
+            Pattern majorVersion9PlusPattern = Pattern.compile("(\\d+).*");
+            Matcher m = majorVersion9PlusPattern.matcher(version);
+            if (m.find()) {
+                majorVersion = m.group(1);
+            } else {
+                throw new IllegalArgumentException("Could not detect major version of " + version);
+            }
+        }
+        return Integer.parseInt(majorVersion);
+    }
+
+    /**
+     * Returns a {@code String[]} of supported TLS protocol versions based on the current Java platform version.
+     *
+     * @return the supported TLS protocol version(s)
+     */
+    public static String[] getCurrentSupportedTlsProtocolVersions() {
+        int javaMajorVersion = getJavaVersion();
+        if (javaMajorVersion < 11) {
+            return JAVA_8_SUPPORTED_TLS_PROTOCOL_VERSIONS;
+        } else {
+            return JAVA_11_SUPPORTED_TLS_PROTOCOL_VERSIONS;
+        }
+    }
+
+    /**
+     * Returns the highest supported TLS protocol version based on the current Java platform version.
+     *
+     * @return the TLS protocol (e.g. {@code "TLSv1.2"})
+     */
+    public static String getHighestCurrentSupportedTlsProtocolVersion() {
+        int javaMajorVersion = getJavaVersion();
+        if (javaMajorVersion < 11) {
+            return JAVA_8_MAX_SUPPORTED_TLS_PROTOCOL_VERSION;
+        } else {
+            return JAVA_11_MAX_SUPPORTED_TLS_PROTOCOL_VERSION;
+        }
+    }
+
     private CertificateUtils() {
     }
 }
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeyStoreUtils.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeyStoreUtils.java
index d763b09..18c1ac8 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeyStoreUtils.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeyStoreUtils.java
@@ -18,14 +18,24 @@
 package org.apache.nifi.security.util;
 
 import java.io.BufferedInputStream;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.URL;
 import java.security.Key;
 import java.security.KeyStore;
 import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
 import java.security.Security;
 import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLParameters;
+import javax.net.ssl.SSLServerSocket;
+import javax.net.ssl.TrustManagerFactory;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.bouncycastle.jce.provider.BouncyCastleProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -89,6 +99,154 @@ public class KeyStoreUtils {
     }
 
     /**
+     * Returns a loaded {@link KeyStore} given the provided configuration values.
+     *
+     * @param keystorePath     the file path to the keystore
+     * @param keystorePassword the keystore password
+     * @param keystoreType     the keystore type ({@code JKS} or {@code PKCS12})
+     * @return the loaded keystore
+     * @throws TlsException if there is a problem loading the keystore
+     */
+    public static KeyStore loadKeyStore(String keystorePath, char[] keystorePassword, String keystoreType) throws TlsException {
+        final KeyStore keyStore;
+        try {
+            keyStore = KeyStoreUtils.getKeyStore(keystoreType);
+            try (final InputStream keyStoreStream = new FileInputStream(keystorePath)) {
+                keyStore.load(keyStoreStream, keystorePassword);
+            }
+            return keyStore;
+        } catch (KeyStoreException | IOException | NoSuchAlgorithmException | CertificateException e) {
+            logger.error("Encountered an error loading keystore: {}", e.getLocalizedMessage());
+            throw new TlsException("Error loading keystore", e);
+        }
+    }
+
+    /**
+     * Returns the {@link KeyManagerFactory} from the provided {@link KeyStore} object, initialized with the key or keystore password.
+     *
+     * @param keyStore         the loaded keystore
+     * @param keystorePassword the keystore password
+     * @param keyPassword      the key password
+     * @return the key manager factory
+     * @throws TlsException if there is a problem initializing or reading from the keystore
+     */
+    public static KeyManagerFactory getKeyManagerFactoryFromKeyStore(KeyStore keyStore, char[] keystorePassword, char[] keyPassword) throws TlsException {
+        try {
+            final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+            if (keyPassword == null) {
+                keyManagerFactory.init(keyStore, keystorePassword);
+            } else {
+                keyManagerFactory.init(keyStore, keyPassword);
+            }
+            return keyManagerFactory;
+        } catch (NoSuchAlgorithmException | KeyStoreException | UnrecoverableKeyException e) {
+            logger.error("Encountered an error loading keystore: {}", e.getLocalizedMessage());
+            throw new TlsException("Error loading keystore", e);
+        }
+    }
+
+    /**
+     * Returns the intialized {@link KeyManagerFactory}.
+     *
+     * @param tlsConfiguration the TLS configuration
+     * @return the initialized key manager factory
+     * @throws TlsException if there is a problem initializing or reading from the keystore
+     */
+    public static KeyManagerFactory loadKeyManagerFactory(TlsConfiguration tlsConfiguration) throws TlsException {
+        return loadKeyManagerFactory(tlsConfiguration.getKeystorePath(), tlsConfiguration.getKeystorePassword(),
+                tlsConfiguration.getFunctionalKeyPassword(), tlsConfiguration.getKeystoreType().getType());
+    }
+
+    /**
+     * Returns the initialized {@link KeyManagerFactory}.
+     *
+     * @param keystorePath     the file path to the keystore
+     * @param keystorePassword the keystore password
+     * @param keyPassword      the key password
+     * @param keystoreType     the keystore type ({@code JKS} or {@code PKCS12})
+     * @return the initialized key manager factory
+     * @throws TlsException if there is a problem initializing or reading from the keystore
+     */
+    public static KeyManagerFactory loadKeyManagerFactory(String keystorePath, String keystorePassword, String keyPassword, String keystoreType) throws TlsException {
+        if (StringUtils.isEmpty(keystorePassword)) {
+            throw new IllegalArgumentException("The keystore password cannot be null or empty");
+        }
+        final char[] keystorePasswordChars = keystorePassword.toCharArray();
+        final char[] keyPasswordChars = (StringUtils.isNotEmpty(keyPassword)) ? keyPassword.toCharArray() : keystorePasswordChars;
+        KeyStore keyStore = loadKeyStore(keystorePath, keystorePasswordChars, keystoreType);
+        return getKeyManagerFactoryFromKeyStore(keyStore, keystorePasswordChars, keyPasswordChars);
+    }
+
+
+    /**
+     * Returns a loaded {@link KeyStore} (acting as a truststore) given the provided configuration values.
+     *
+     * @param truststorePath     the file path to the truststore
+     * @param truststorePassword the truststore password
+     * @param truststoreType     the truststore type ({@code JKS} or {@code PKCS12})
+     * @return the loaded truststore
+     * @throws TlsException if there is a problem loading the truststore
+     */
+    public static KeyStore loadTrustStore(String truststorePath, char[] truststorePassword, String truststoreType) throws TlsException {
+        final KeyStore trustStore;
+        try {
+            trustStore = KeyStoreUtils.getTrustStore(truststoreType);
+            try (final InputStream trustStoreStream = new FileInputStream(truststorePath)) {
+                trustStore.load(trustStoreStream, truststorePassword);
+            }
+            return trustStore;
+        } catch (KeyStoreException | IOException | NoSuchAlgorithmException | CertificateException e) {
+            logger.error("Encountered an error loading truststore: {}", e.getLocalizedMessage());
+            throw new TlsException("Error loading truststore", e);
+        }
+    }
+
+    /**
+     * Returns the {@link TrustManagerFactory} from the provided {@link KeyStore} object, initialized.
+     *
+     * @param trustStore the loaded truststore
+     * @return the trust manager factory
+     * @throws TlsException if there is a problem initializing or reading from the truststore
+     */
+    public static TrustManagerFactory getTrustManagerFactoryFromTrustStore(KeyStore trustStore) throws TlsException {
+        try {
+            final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+            trustManagerFactory.init(trustStore);
+            return trustManagerFactory;
+        } catch (NoSuchAlgorithmException | KeyStoreException e) {
+            logger.error("Encountered an error loading truststore: {}", e.getLocalizedMessage());
+            throw new TlsException("Error loading truststore", e);
+        }
+    }
+
+    /**
+     * Returns the intialized {@link TrustManagerFactory}.
+     *
+     * @param tlsConfiguration the TLS configuration
+     * @return the initialized trust manager factory
+     * @throws TlsException if there is a problem initializing or reading from the truststore
+     */
+    public static TrustManagerFactory loadTrustManagerFactory(TlsConfiguration tlsConfiguration) throws TlsException {
+        return loadTrustManagerFactory(tlsConfiguration.getTruststorePath(), tlsConfiguration.getTruststorePassword(), tlsConfiguration.getTruststoreType().getType());
+    }
+
+    /**
+     * Returns the initialized {@link TrustManagerFactory}.
+     *
+     * @param truststorePath     the file path to the truststore
+     * @param truststorePassword the truststore password
+     * @param truststoreType     the truststore type ({@code JKS} or {@code PKCS12})
+     * @return the initialized trust manager factory
+     * @throws TlsException if there is a problem initializing or reading from the truststore
+     */
+    public static TrustManagerFactory loadTrustManagerFactory(String truststorePath, String truststorePassword, String truststoreType) throws TlsException {
+        // Legacy truststore passwords can be empty
+        final char[] truststorePasswordChars = StringUtils.isNotBlank(truststorePassword) ? truststorePassword.toCharArray() : null;
+        KeyStore trustStore = loadTrustStore(truststorePath, truststorePasswordChars, truststoreType);
+        return getTrustManagerFactoryFromTrustStore(trustStore);
+    }
+
+    /**
      * Returns true if the given keystore can be loaded using the given keystore type and password. Returns false otherwise.
      *
      * @param keystore     the keystore to validate
@@ -180,4 +338,28 @@ public class KeyStoreUtils {
             }
         }
     }
+
+    public static String sslContextToString(SSLContext sslContext) {
+        return new ToStringBuilder(sslContext)
+                .append("protocol", sslContext.getProtocol())
+                .append("provider", sslContext.getProvider().toString())
+                .toString();
+    }
+
+    public static String sslParametersToString(SSLParameters sslParameters) {
+        return new ToStringBuilder(sslParameters)
+                .append("protocols", sslParameters.getProtocols())
+                .append("wantClientAuth", sslParameters.getWantClientAuth())
+                .append("needClientAuth", sslParameters.getNeedClientAuth())
+                .toString();
+    }
+
+    public static String sslServerSocketToString(SSLServerSocket sslServerSocket) {
+        return new ToStringBuilder(sslServerSocket)
+                .append("enabledProtocols", sslServerSocket.getEnabledProtocols())
+                .append("needClientAuth", sslServerSocket.getNeedClientAuth())
+                .append("wantClientAuth", sslServerSocket.getWantClientAuth())
+                .append("useClientMode", sslServerSocket.getUseClientMode())
+                .toString();
+    }
 }
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java
index 18574bb..ea47463 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java
@@ -16,11 +16,42 @@
  */
 package org.apache.nifi.security.util;
 
+import java.util.Arrays;
+import java.util.stream.Collectors;
+import org.apache.nifi.util.StringUtils;
+
 /**
  * Keystore types.
  */
 public enum KeystoreType {
+    PKCS12("PKCS12", "A PKCS12 Keystore"),
+    JKS("JKS", "A Java Keystore");
+
+    private final String type;
+    private final String description;
+
+    KeystoreType(String type, String description) {
+        this.type = type;
+        this.description = description;
+    }
+
+    public String getType() {
+        return this.type;
+    }
+
+    public String getDescription() {
+        return this.description;
+    }
+
+    @Override
+    public String toString() {
+        return getType();
+    }
 
-    PKCS12,
-    JKS;
+    public static boolean isValidKeystoreType(String type) {
+        if (StringUtils.isBlank(type)) {
+            return false;
+        }
+        return (Arrays.stream(values()).map(kt -> kt.getType().toLowerCase()).collect(Collectors.toList()).contains(type.toLowerCase()));
+    }
 }
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java
deleted file mode 100644
index f448312..0000000
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * 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.security.util;
-
-import java.io.PrintWriter;
-import java.io.Writer;
-
-/**
- * Types of security stores and their related Java system properties.
- */
-public enum SecurityStoreTypes {
-
-    TRUSTSTORE(
-            "javax.net.ssl.trustStore",
-            "javax.net.ssl.trustStorePassword",
-            "javax.net.ssl.trustStoreType"),
-    KEYSTORE(
-            "javax.net.ssl.keyStore",
-            "javax.net.ssl.keyStorePassword",
-            "javax.net.ssl.keyStoreType");
-
-    /**
-     * Logs the keystore and truststore Java system property values to the given
-     * writer. It logPasswords is true, then the keystore and truststore
-     * password property values are logged.
-     *
-     * @param writer a writer to log to
-     *
-     * @param logPasswords true if passwords should be logged; false otherwise
-     */
-    public static void logProperties(final Writer writer,
-            final boolean logPasswords) {
-        if (writer == null) {
-            return;
-        }
-
-        PrintWriter pw = new PrintWriter(writer);
-
-        // keystore properties
-        pw.println(
-                KEYSTORE.getStoreProperty() + " = " + System.getProperty(KEYSTORE.getStoreProperty()));
-
-        if (logPasswords) {
-            pw.println(
-                    KEYSTORE.getStorePasswordProperty() + " = "
-                    + System.getProperty(KEYSTORE.getStoreProperty()));
-        }
-
-        pw.println(
-                KEYSTORE.getStoreTypeProperty() + " = "
-                + System.getProperty(KEYSTORE.getStoreTypeProperty()));
-
-        // truststore properties
-        pw.println(
-                TRUSTSTORE.getStoreProperty() + " = "
-                + System.getProperty(TRUSTSTORE.getStoreProperty()));
-
-        if (logPasswords) {
-            pw.println(
-                    TRUSTSTORE.getStorePasswordProperty() + " = "
-                    + System.getProperty(TRUSTSTORE.getStoreProperty()));
-        }
-
-        pw.println(
-                TRUSTSTORE.getStoreTypeProperty() + " = "
-                + System.getProperty(TRUSTSTORE.getStoreTypeProperty()));
-        pw.flush();
-    }
-
-    /**
-     * the Java system property for setting the keystore (or truststore) path
-     */
-    private String storeProperty = "";
-
-    /**
-     * the Java system property for setting the keystore (or truststore)
-     * password
-     */
-    private String storePasswordProperty = "";
-
-    /**
-     * the Java system property for setting the keystore (or truststore) type
-     */
-    private String storeTypeProperty = "";
-
-    /**
-     * Creates an instance.
-     *
-     * @param storeProperty the Java system property for setting the keystore or
-     * truststore path
-     * @param storePasswordProperty the Java system property for setting the
-     * keystore or truststore path
-     * @param storeTypeProperty the Java system property for setting the
-     * keystore or truststore type
-     */
-    SecurityStoreTypes(final String storeProperty,
-            final String storePasswordProperty,
-            final String storeTypeProperty) {
-        this.storeProperty = storeProperty;
-        this.storePasswordProperty = storePasswordProperty;
-        this.storeTypeProperty = storeTypeProperty;
-    }
-
-    /**
-     * Returns the keystore (or truststore) property.
-     *
-     * @return the keystore (or truststore) property
-     */
-    public String getStoreProperty() {
-        return storeProperty;
-    }
-
-    /**
-     * Returns the keystore (or truststore) password property.
-     *
-     * @return the keystore (or truststore) password property
-     */
-    public String getStorePasswordProperty() {
-        return storePasswordProperty;
-    }
-
-    /**
-     * Returns the keystore (or truststore) type property.
-     *
-     * @return the keystore (or truststore) type property
-     */
-    public String getStoreTypeProperty() {
-        return storeTypeProperty;
-    }
-}
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java
index 3e8058f..39dcafa 100644
--- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java
@@ -16,351 +16,251 @@
  */
 package org.apache.nifi.security.util;
 
-import org.apache.nifi.util.Tuple;
-
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
 import java.security.KeyManagementException;
-import java.security.KeyStore;
-import java.security.KeyStoreException;
 import java.security.NoSuchAlgorithmException;
 import java.security.SecureRandom;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.stream.Collectors;
 import javax.net.ssl.KeyManager;
 import javax.net.ssl.KeyManagerFactory;
 import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocketFactory;
 import javax.net.ssl.TrustManager;
 import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * A factory for creating SSL contexts using the application's security
- * properties.
- *
+ * A factory for creating SSL contexts using the application's security properties. By requiring callers to bundle
+ * the properties in a {@link TlsConfiguration} container object, much better validation and property matching can
+ * occur. The {@code public} methods are designed for easy use, while the {@code protected} methods provide more
+ * granular (but less common) access to intermediate objects if required.
  */
 public final class SslContextFactory {
-
-    public static enum ClientAuth {
-
-        WANT,
-        REQUIRED,
-        NONE
-    }
+    private static final Logger logger = LoggerFactory.getLogger(SslContextFactory.class);
 
     /**
-     * Creates an SSLContext instance using the given information. The password for the key is assumed to be the same
-     * as the password for the keystore. If this is not the case, the {@link #createSslContext(String, char[], char[], String, String, char[], String, ClientAuth, String)}
-     * method should be used instead
-     *
-     * @param keystore the full path to the keystore
-     * @param keystorePasswd the keystore password
-     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
-     * @param truststore the full path to the truststore
-     * @param truststorePasswd the truststore password
-     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
-     * @param clientAuth the type of client authentication
-     * @param protocol         the protocol to use for the SSL connection
-     *
-     * @return an SSLContext instance
-     * @throws java.security.KeyStoreException if any issues accessing the keystore
-     * @throws java.io.IOException for any problems loading the keystores
-     * @throws java.security.NoSuchAlgorithmException if an algorithm is found to be used but is unknown
-     * @throws java.security.cert.CertificateException if there is an issue with the certificate
-     * @throws java.security.UnrecoverableKeyException if the key is insufficient
-     * @throws java.security.KeyManagementException if unable to manage the key
+     * This enum is used to indicate the three possible options for a server requesting a client certificate during TLS handshake negotiation.
      */
-    public static SSLContext createSslContext(
-            final String keystore, final char[] keystorePasswd, final String keystoreType,
-            final String truststore, final char[] truststorePasswd, final String truststoreType,
-            final ClientAuth clientAuth, final String protocol)
-            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
+    public enum ClientAuth {
+        WANT("Want", "Requests the client certificate on handshake and validates if present but does not require it"),
+        REQUIRED("Required", "Requests the client certificate on handshake and rejects the connection if it is not present and valid"),
+        NONE("None", "Does not request the client certificate on handshake");
 
-        // Pass the keystore password as both the keystore password and the key password.
-        return createSslContext(keystore, keystorePasswd, keystorePasswd, keystoreType, truststore, truststorePasswd, truststoreType, clientAuth, protocol);
+        private final String type;
+        private final String description;
+
+        ClientAuth(String type, String description) {
+            this.type = type;
+            this.description = description;
+        }
+
+        public String getType() {
+            return this.type;
+        }
+
+        public String getDescription() {
+            return this.description;
+        }
+
+        @Override
+        public String toString() {
+            final ToStringBuilder builder = new ToStringBuilder(this);
+            ToStringBuilder.setDefaultStyle(ToStringStyle.SHORT_PREFIX_STYLE);
+            builder.append("Type", type);
+            builder.append("Description", description);
+            return builder.toString();
+        }
+
+        public static boolean isValidClientAuthType(String type) {
+            if (StringUtils.isBlank(type)) {
+                return false;
+            }
+            return (Arrays.stream(values()).map(ca -> ca.getType().toLowerCase()).collect(Collectors.toList()).contains(type.toLowerCase()));
+        }
     }
 
     /**
-     * Creates an SSLContext instance using the given information.
-     *
-     * @param keystore the full path to the keystore
-     * @param keystorePasswd the keystore password
-     * @param keyPasswd the password for the key within the keystore
-     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
-     * @param truststore the full path to the truststore
-     * @param truststorePasswd the truststore password
-     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
-     * @param clientAuth the type of client authentication
-     * @param protocol         the protocol to use for the SSL connection
+     * Returns a configured {@link SSLContext} from the provided TLS configuration. Hardcodes the
+     * client auth setting to {@link ClientAuth#REQUIRED} because this method is usually used when
+     * creating a context for a client, which ignores the setting anyway.
      *
-     * @return an SSLContext instance
-     * @throws java.security.KeyStoreException if any issues accessing the keystore
-     * @throws java.io.IOException for any problems loading the keystores
-     * @throws java.security.NoSuchAlgorithmException if an algorithm is found to be used but is unknown
-     * @throws java.security.cert.CertificateException if there is an issue with the certificate
-     * @throws java.security.UnrecoverableKeyException if the key is insufficient
-     * @throws java.security.KeyManagementException if unable to manage the key
+     * @param tlsConfiguration the TLS configuration container object
+     * @return the configured SSLContext
+     * @throws TlsException if there is a problem configuring the SSLContext
      */
-    public static SSLContext createSslContext(
-            final String keystore, final char[] keystorePasswd, final char[] keyPasswd, final String keystoreType,
-            final String truststore, final char[] truststorePasswd, final String truststoreType,
-            final ClientAuth clientAuth, final String protocol)
-            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-        return createSslContextWithTrustManagers(keystore, keystorePasswd, keyPasswd, keystoreType, truststore,
-                truststorePasswd, truststoreType, clientAuth, protocol).getKey();
+    public static SSLContext createSslContext(TlsConfiguration tlsConfiguration) throws TlsException {
+        return createSslContext(tlsConfiguration, ClientAuth.REQUIRED);
     }
 
     /**
-     * Creates an SSLContext instance paired with its TrustManager instances using the given information.
+     * Returns a configured {@link SSLContext} from the provided TLS configuration.
      *
-     * @param keystore the full path to the keystore
-     * @param keystorePasswd the keystore password
-     * @param keyPasswd the password for the key within the keystore
-     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
-     * @param truststore the full path to the truststore
-     * @param truststorePasswd the truststore password
-     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
-     * @param clientAuth the type of client authentication
-     * @param protocol         the protocol to use for the SSL connection
-     *
-     * @return a {@link Tuple} pairing an SSLContext instance with its TrustManagers
-     * @throws java.security.KeyStoreException if any issues accessing the keystore
-     * @throws java.io.IOException for any problems loading the keystores
-     * @throws java.security.NoSuchAlgorithmException if an algorithm is found to be used but is unknown
-     * @throws java.security.cert.CertificateException if there is an issue with the certificate
-     * @throws java.security.UnrecoverableKeyException if the key is insufficient
-     * @throws java.security.KeyManagementException if unable to manage the key
+     * @param tlsConfiguration the TLS configuration container object
+     * @param clientAuth       the {@link ClientAuth} setting
+     * @return the configured SSLContext
+     * @throws TlsException if there is a problem configuring the SSLContext
      */
-    public static Tuple<SSLContext, TrustManager[]> createSslContextWithTrustManagers(
-            final String keystore, final char[] keystorePasswd, final char[] keyPasswd, final String keystoreType,
-            final String truststore, final char[] truststorePasswd, final String truststoreType,
-            final ClientAuth clientAuth, final String protocol)
-            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-        // prepare the keystore
-        final KeyStore keyStore = KeyStoreUtils.getKeyStore(keystoreType);
-        try (final InputStream keyStoreStream = new FileInputStream(keystore)) {
-            keyStore.load(keyStoreStream, keystorePasswd);
-        }
-        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-        if (keyPasswd == null) {
-            keyManagerFactory.init(keyStore, keystorePasswd);
-        } else {
-            keyManagerFactory.init(keyStore, keyPasswd);
+    public static SSLContext createSslContext(TlsConfiguration tlsConfiguration, ClientAuth clientAuth) throws TlsException {
+        // If the object is null or neither keystore nor truststore properties are present, return null
+        if (TlsConfiguration.isEmpty(tlsConfiguration)) {
+            logger.debug("Cannot create SSLContext from empty TLS configuration; returning null");
+            return null;
         }
 
-        // prepare the truststore
-        final KeyStore trustStore = KeyStoreUtils.getTrustStore(truststoreType);
-        try (final InputStream trustStoreStream = new FileInputStream(truststore)) {
-            trustStore.load(trustStoreStream, truststorePasswd);
+        // If the keystore properties are present, truststore properties are required to be present as well
+        if (tlsConfiguration.isKeystorePopulated() && !tlsConfiguration.isTruststorePopulated()) {
+            logger.error("The TLS config keystore properties were populated but the truststore properties were not");
+            if (logger.isDebugEnabled()) {
+                logger.debug("Provided TLS config: {}", tlsConfiguration);
+            }
+            throw new TlsException("Truststore properties are required if keystore properties are present");
         }
-        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-        trustManagerFactory.init(trustStore);
 
-        // initialize the ssl context
-        final SSLContext sslContext = SSLContext.getInstance(protocol);
-        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom());
-        if (ClientAuth.REQUIRED == clientAuth) {
-            sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
-        } else if (ClientAuth.WANT == clientAuth) {
-            sslContext.getDefaultSSLParameters().setWantClientAuth(true);
-        } else {
-            sslContext.getDefaultSSLParameters().setWantClientAuth(false);
+        if (clientAuth == null) {
+            clientAuth = ClientAuth.REQUIRED;
+            logger.debug("ClientAuth was null so defaulting to {}", clientAuth);
         }
 
-        return new Tuple<>(sslContext, trustManagerFactory.getTrustManagers());
-
-    }
+        // Create the keystore components
+        KeyManager[] keyManagers = getKeyManagers(tlsConfiguration);
 
-    /**
-     * Creates an SSLContext instance using the given information. This method assumes that the key password is
-     * the same as the keystore password. If this is not the case, use the {@link #createSslContext(String, char[], char[], String, String)}
-     * method instead.
-     *
-     * @param keystore the full path to the keystore
-     * @param keystorePasswd the keystore password
-     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
-     * @param protocol the protocol to use for the SSL connection
-     *
-     * @return an SSLContext instance
-     * @throws java.security.KeyStoreException if any issues accessing the keystore
-     * @throws java.io.IOException for any problems loading the keystores
-     * @throws java.security.NoSuchAlgorithmException if an algorithm is found to be used but is unknown
-     * @throws java.security.cert.CertificateException if there is an issue with the certificate
-     * @throws java.security.UnrecoverableKeyException if the key is insufficient
-     * @throws java.security.KeyManagementException if unable to manage the key
-     */
-    public static SSLContext createSslContext(
-        final String keystore, final char[] keystorePasswd, final String keystoreType, final String protocol)
-        throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-        UnrecoverableKeyException, KeyManagementException {
+        // Create the truststore components
+        TrustManager[] trustManagers = getTrustManagers(tlsConfiguration);
 
-        // create SSL Context passing keystore password as the key password
-        return createSslContext(keystore, keystorePasswd, keystorePasswd, keystoreType, protocol);
+        // Initialize the ssl context
+        return initializeSSLContext(tlsConfiguration, clientAuth, keyManagers, trustManagers);
     }
 
     /**
-     * Creates an SSLContext instance using the given information.
+     * Returns a configured {@link X509TrustManager} for the provided configuration. Useful for
+     * constructing HTTP clients which require their own trust management rather than an
+     * {@link SSLContext}. Filters and removes any trust managers that are not
+     * {@link javax.net.ssl.X509TrustManager} implementations, and returns the <em>first</em>
+     * X.509 trust manager.
      *
-     * @param keystore the full path to the keystore
-     * @param keystorePasswd the keystore password
-     * @param keyPasswd the password for the key within the keystore
-     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
-     * @param protocol the protocol to use for the SSL connection
-     *
-     * @return an SSLContext instance
-     * @throws java.security.KeyStoreException if any issues accessing the keystore
-     * @throws java.io.IOException for any problems loading the keystores
-     * @throws java.security.NoSuchAlgorithmException if an algorithm is found to be used but is unknown
-     * @throws java.security.cert.CertificateException if there is an issue with the certificate
-     * @throws java.security.UnrecoverableKeyException if the key is insufficient
-     * @throws java.security.KeyManagementException if unable to manage the key
+     * @param tlsConfiguration the TLS configuration container object
+     * @return an X.509 TrustManager (can be {@code null})
+     * @throws TlsException if there is a problem reading the truststore to create the trust managers
      */
-    public static SSLContext createSslContext(
-        final String keystore, final char[] keystorePasswd, final char[] keyPasswd, final String keystoreType, final String protocol)
-            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-        return createSslContextWithTrustManagers(keystore, keystorePasswd, keyPasswd, keystoreType, protocol).getKey();
+    public static X509TrustManager getX509TrustManager(TlsConfiguration tlsConfiguration) throws TlsException {
+        TrustManager[] trustManagers = getTrustManagers(tlsConfiguration);
+        if (trustManagers == null) {
+            return null;
+        }
+        Optional<X509TrustManager> x509TrustManager = Arrays.stream(trustManagers)
+                .filter(tm -> tm instanceof X509TrustManager)
+                .map(tm -> (X509TrustManager) tm)
+                .findFirst();
+        return x509TrustManager.orElse(null);
     }
 
     /**
-     * Creates an SSLContext instance paired with its TrustManager instances using the given information.
+     * Convenience method to return the {@link SSLSocketFactory} from the created {@link SSLContext}
+     * because that is what most callers of {@link #createSslContext(TlsConfiguration, ClientAuth)}
+     * actually need and don't know what to provide for the {@link ClientAuth} parameter.
      *
-     * @param keystore the full path to the keystore
-     * @param keystorePasswd the keystore password
-     * @param keyPasswd the password for the key within the keystore
-     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
-     * @param protocol the protocol to use for the SSL connection
-     *
-     * @return a {@link Tuple} pairing an SSLContext instance paired with its TrustManager instances
-     * @throws java.security.KeyStoreException if any issues accessing the keystore
-     * @throws java.io.IOException for any problems loading the keystores
-     * @throws java.security.NoSuchAlgorithmException if an algorithm is found to be used but is unknown
-     * @throws java.security.cert.CertificateException if there is an issue with the certificate
-     * @throws java.security.UnrecoverableKeyException if the key is insufficient
-     * @throws java.security.KeyManagementException if unable to manage the key
+     * @param tlsConfiguration the TLS configuration container object
+     * @return the configured SSLSocketFactory (can be {@code null})
+     * @throws TlsException if there is a problem creating the SSLContext or SSLSocketFactory
      */
-    public static Tuple<SSLContext, TrustManager[]> createSslContextWithTrustManagers(
-            final String keystore, final char[] keystorePasswd, final char[] keyPasswd, final String keystoreType, final String protocol)
-            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-
-        // prepare the keystore
-        final KeyStore keyStore = KeyStoreUtils.getKeyStore(keystoreType);
-        try (final InputStream keyStoreStream = new FileInputStream(keystore)) {
-            keyStore.load(keyStoreStream, keystorePasswd);
-        }
-        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-        if (keyPasswd == null) {
-            keyManagerFactory.init(keyStore, keystorePasswd);
-        } else {
-            keyManagerFactory.init(keyStore, keyPasswd);
+    public static SSLSocketFactory createSSLSocketFactory(TlsConfiguration tlsConfiguration) throws TlsException {
+        SSLContext sslContext = createSslContext(tlsConfiguration, ClientAuth.REQUIRED);
+        if (sslContext == null) {
+            // Only display an error in the log if the provided config wasn't empty
+            if (!TlsConfiguration.isEmpty(tlsConfiguration)) {
+                logger.error("The SSLContext could not be formed from the provided TLS configuration. Check the provided keystore and truststore properties");
+            }
+            return null;
         }
-
-        // initialize the ssl context
-        final SSLContext ctx = SSLContext.getInstance(protocol);
-        TrustManager[] trustManagers = new TrustManager[0];
-        ctx.init(keyManagerFactory.getKeyManagers(), trustManagers, new SecureRandom());
-
-        return new Tuple<>(ctx, trustManagers);
+        return sslContext.getSocketFactory();
     }
 
     /**
-     * Creates an SSLContext instance using the given information.
-     *
-     * @param truststore the full path to the truststore
-     * @param truststorePasswd the truststore password
-     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
-     * @param protocol the protocol to use for the SSL connection
+     * Returns an array of {@link KeyManager}s for the provided configuration. Useful for constructing
+     * HTTP clients which require their own key management rather than an {@link SSLContext}. The result can be
+     * {@code null} or empty. If an empty configuration is provided, {@code null} is returned. However, if a partially-populated
+     * but invalid configuration is provided, a {@link TlsException} is thrown.
      *
-     * @return an SSLContext instance
-     * @throws java.security.KeyStoreException if any issues accessing the keystore
-     * @throws java.io.IOException for any problems loading the keystores
-     * @throws java.security.NoSuchAlgorithmException if an algorithm is found to be used but is unknown
-     * @throws java.security.cert.CertificateException if there is an issue with the certificate
-     * @throws java.security.UnrecoverableKeyException if the key is insufficient
-     * @throws java.security.KeyManagementException if unable to manage the key
+     * @param tlsConfiguration the TLS configuration container object with keystore properties
+     * @return an array of KeyManagers (can be {@code null})
+     * @throws TlsException if there is a problem reading the keystore to create the key managers
      */
-    public static SSLContext createTrustSslContext(
-            final String truststore, final char[] truststorePasswd, final String truststoreType, final String protocol)
-            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-
-        return createTrustSslContextWithTrustManagers(truststore, truststorePasswd, truststoreType, protocol).getKey();
-
+    @SuppressWarnings("RedundantCast")
+    protected static KeyManager[] getKeyManagers(TlsConfiguration tlsConfiguration) throws TlsException {
+        KeyManager[] keyManagers = null;
+        if (tlsConfiguration.isKeystoreValid()) {
+            KeyManagerFactory keyManagerFactory = KeyStoreUtils.loadKeyManagerFactory(tlsConfiguration);
+            keyManagers = keyManagerFactory.getKeyManagers();
+        } else {
+            // If some keystore properties were populated but the key managers are empty, throw an exception to inform the caller
+            if (tlsConfiguration.isAnyKeystorePopulated()) {
+                logger.warn("Some keystore properties are populated ({}, {}, {}, {}) but not valid", (Object[]) tlsConfiguration.getKeystorePropertiesForLogging());
+                throw new TlsException("The keystore properties are not valid");
+            } else {
+                // If they are empty, the caller was not expecting a valid response
+                logger.debug("The keystore properties are not populated");
+            }
+        }
+        return keyManagers;
     }
 
     /**
-     * Creates an SSLContext instance paired with its TrustManager instances using the given information.
-     *
-     * @param truststore the full path to the truststore
-     * @param truststorePasswd the truststore password
-     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
-     * @param protocol the protocol to use for the SSL connection
+     * Returns an array of {@link TrustManager} implementations based on the provided truststore configurations. The result can be
+     * {@code null} or empty. If an empty configuration is provided, {@code null} is returned. However, if a partially-populated
+     * but invalid configuration is provided, a {@link TlsException} is thrown.
+     * <p>
+     * Most callers do not need the full array and can use {@link #getX509TrustManager(TlsConfiguration)} directly.
      *
-     * @return a {@link Tuple} pairing an SSLContext instance paired with its TrustManager instances
-     * @throws KeyStoreException if any issues accessing the keystore
-     * @throws IOException for any problems loading the keystores
-     * @throws NoSuchAlgorithmException if an algorithm is found to be used but is unknown
-     * @throws CertificateException if there is an issue with the certificate
-     * @throws UnrecoverableKeyException if the key is insufficient
-     * @throws KeyManagementException if unable to manage the key
+     * @param tlsConfiguration the TLS configuration container object with truststore properties
+     * @return the loaded trust managers
+     * @throws TlsException if there is a problem reading from the truststore
      */
-    public static Tuple<SSLContext, TrustManager[]> createTrustSslContextWithTrustManagers(
-            final String truststore, final char[] truststorePasswd, final String truststoreType, final String protocol)
-            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-        // prepare the truststore
-        final KeyStore trustStore = KeyStoreUtils.getTrustStore(truststoreType);
-        try (final InputStream trustStoreStream = new FileInputStream(truststore)) {
-            trustStore.load(trustStoreStream, truststorePasswd);
+    @SuppressWarnings("RedundantCast")
+    protected static TrustManager[] getTrustManagers(TlsConfiguration tlsConfiguration) throws TlsException {
+        TrustManager[] trustManagers = null;
+        if (tlsConfiguration.isTruststoreValid()) {
+            TrustManagerFactory trustManagerFactory = KeyStoreUtils.loadTrustManagerFactory(tlsConfiguration);
+            trustManagers = trustManagerFactory.getTrustManagers();
+        } else {
+            // If some truststore properties were populated but the trust managers are empty, throw an exception to inform the caller
+            if (tlsConfiguration.isAnyTruststorePopulated()) {
+                logger.warn("Some truststore properties are populated ({}, {}, {}) but not valid", (Object[]) tlsConfiguration.getTruststorePropertiesForLogging());
+                throw new TlsException("The truststore properties are not valid");
+            } else {
+                // If they are empty, the caller was not expecting a valid response
+                logger.debug("The truststore properties are not populated");
+            }
         }
-        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-        trustManagerFactory.init(trustStore);
-
-        // initialize the ssl context
-        final SSLContext ctx = SSLContext.getInstance(protocol);
-        TrustManager[] trustManagers = trustManagerFactory.getTrustManagers();
-        ctx.init(new KeyManager[0], trustManagers, new SecureRandom());
-
-        return new Tuple<>(ctx, trustManagers);
+        return trustManagers;
     }
 
-    /**
-     * Creates an SSLContext instance paired with its TrustManager instances using the given information.
-     *
-     * @param keystore the full path to the keystore
-     * @param keystorePasswd the keystore password
-     * @param keyPasswd the password for the key within the keystore
-     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
-     * @param truststore the full path to the truststore
-     * @param truststorePasswd the truststore password
-     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
-     * @param clientAuth the type of client authentication
-     * @param protocol         the protocol to use for the SSL connection
-     *
-     * @return a {@link Tuple} pairing an SSLSocketFactory instance with its TrustManagers
-     *
-     */
-    public static Tuple<SSLContext, TrustManager[]> createTrustSslContextWithTrustManagers(
-            final String keystore, final char[] keystorePasswd, final char[] keyPasswd, final String keystoreType,
-            final String truststore, final char[] truststorePasswd, final String truststoreType,
-            final ClientAuth clientAuth, final String protocol) throws CertificateException, UnrecoverableKeyException,
-            NoSuchAlgorithmException, KeyStoreException, KeyManagementException, IOException {
-
-            final Tuple<SSLContext, TrustManager[]> sslContextWithTrustManagers;
-            if (keystore == null) {
-                sslContextWithTrustManagers = createTrustSslContextWithTrustManagers(truststore, truststorePasswd, truststoreType, protocol);
-            } else if (truststore == null) {
-                sslContextWithTrustManagers = createSslContextWithTrustManagers(keystore, keystorePasswd, keyPasswd, keystoreType, protocol);
-            } else {
-                sslContextWithTrustManagers = createSslContextWithTrustManagers(keystore, keystorePasswd, keyPasswd, keystoreType, truststore,
-                        truststorePasswd, truststoreType, clientAuth, protocol);
+    private static SSLContext initializeSSLContext(TlsConfiguration tlsConfiguration, ClientAuth clientAuth, KeyManager[] keyManagers, TrustManager[] trustManagers) throws TlsException {
+        final SSLContext sslContext;
+        try {
+            sslContext = SSLContext.getInstance(tlsConfiguration.getProtocol());
+            sslContext.init(keyManagers, trustManagers, new SecureRandom());
+            switch (clientAuth) {
+                case REQUIRED:
+                    sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
+                    break;
+                case WANT:
+                    sslContext.getDefaultSSLParameters().setWantClientAuth(true);
+                    break;
+                case NONE:
+                default:
+                    sslContext.getDefaultSSLParameters().setWantClientAuth(false);
             }
-            return new Tuple<>(sslContextWithTrustManagers.getKey(), sslContextWithTrustManagers.getValue());
 
+            return sslContext;
+        } catch (NoSuchAlgorithmException | KeyManagementException e) {
+            logger.error("Encountered an error creating SSLContext from TLS configuration ({}): {}", tlsConfiguration.toString(), e.getLocalizedMessage());
+            throw new TlsException("Error creating SSL context", e);
+        }
     }
 }
diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/TlsConfiguration.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/TlsConfiguration.java
new file mode 100644
index 0000000..021986b
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/TlsConfiguration.java
@@ -0,0 +1,480 @@
+/*
+ * 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.security.util;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.util.Objects;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.nifi.util.NiFiProperties;
+import org.apache.nifi.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class serves as an immutable domain object (acting as an internal DTO) for the various keystore and truststore configuration settings necessary for building {@link javax.net.ssl.SSLContext}s.
+ */
+public class TlsConfiguration {
+    private static final Logger logger = LoggerFactory.getLogger(TlsConfiguration.class);
+
+    private static final String TLS_PROTOCOL_VERSION = CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion();
+    private static final String MASKED_PASSWORD_LOG = "********";
+    private static final String NULL_LOG = "null";
+
+    private final String keystorePath;
+    private final String keystorePassword;
+    private final String keyPassword;
+    private final KeystoreType keystoreType;
+
+    private final String truststorePath;
+    private final String truststorePassword;
+    private final KeystoreType truststoreType;
+
+    private final String protocol;
+
+    /**
+     * Default constructor present for testing and completeness.
+     */
+    public TlsConfiguration() {
+        this(null, null, null, "", null, null, "", null);
+    }
+
+    /**
+     * Instantiates a container object with the given configuration values.
+     *
+     * @param keystorePath       the keystore path
+     * @param keystorePassword   the keystore password
+     * @param keystoreType       the keystore type
+     * @param truststorePath     the truststore path
+     * @param truststorePassword the truststore password
+     * @param truststoreType     the truststore type
+     */
+    public TlsConfiguration(String keystorePath, String keystorePassword, KeystoreType keystoreType, String truststorePath, String truststorePassword, KeystoreType truststoreType) {
+        this(keystorePath, keystorePassword, keystorePassword, keystoreType, truststorePath, truststorePassword, truststoreType, TLS_PROTOCOL_VERSION);
+    }
+
+    /**
+     * Instantiates a container object with the given configuration values.
+     *
+     * @param keystorePath       the keystore path
+     * @param keystorePassword   the keystore password
+     * @param keyPassword        the key password
+     * @param keystoreType       the keystore type
+     * @param truststorePath     the truststore path
+     * @param truststorePassword the truststore password
+     * @param truststoreType     the truststore type
+     */
+    public TlsConfiguration(String keystorePath, String keystorePassword, String keyPassword,
+                            KeystoreType keystoreType, String truststorePath, String truststorePassword, KeystoreType truststoreType) {
+        this(keystorePath, keystorePassword, keyPassword, keystoreType, truststorePath, truststorePassword, truststoreType, TLS_PROTOCOL_VERSION);
+    }
+
+    /**
+     * Instantiates a container object with the given configuration values.
+     *
+     * @param keystorePath       the keystore path
+     * @param keystorePassword   the keystore password
+     * @param keyPassword        the key password
+     * @param keystoreType       the keystore type as a String
+     * @param truststorePath     the truststore path
+     * @param truststorePassword the truststore password
+     * @param truststoreType     the truststore type as a String
+     */
+    public TlsConfiguration(String keystorePath, String keystorePassword, String keyPassword,
+                            String keystoreType, String truststorePath, String truststorePassword, String truststoreType) {
+        this(keystorePath, keystorePassword, keyPassword,
+                (KeystoreType.isValidKeystoreType(keystoreType) ? KeystoreType.valueOf(keystoreType.toUpperCase()) : null),
+                truststorePath, truststorePassword,
+                (KeystoreType.isValidKeystoreType(truststoreType) ? KeystoreType.valueOf(truststoreType.toUpperCase()) : null),
+                TLS_PROTOCOL_VERSION);
+    }
+
+    /**
+     * Instantiates a container object with the given configuration values.
+     *
+     * @param keystorePath       the keystore path
+     * @param keystorePassword   the keystore password
+     * @param keyPassword        the (optional) key password -- if {@code null}, the keystore password is assumed the same for the individual key
+     * @param keystoreType       the keystore type as a String
+     * @param truststorePath     the truststore path
+     * @param truststorePassword the truststore password
+     * @param truststoreType     the truststore type as a String
+     * @param protocol           the TLS protocol version string
+     */
+    public TlsConfiguration(String keystorePath, String keystorePassword, String keyPassword,
+                            String keystoreType, String truststorePath, String truststorePassword, String truststoreType, String protocol) {
+        this(keystorePath, keystorePassword, keyPassword,
+                (KeystoreType.isValidKeystoreType(keystoreType) ? KeystoreType.valueOf(keystoreType.toUpperCase()) : null),
+                truststorePath, truststorePassword,
+                (KeystoreType.isValidKeystoreType(truststoreType) ? KeystoreType.valueOf(truststoreType.toUpperCase()) : null),
+                protocol);
+    }
+
+    /**
+     * Instantiates a container object with the given configuration values.
+     *
+     * @param keystorePath       the keystore path
+     * @param keystorePassword   the keystore password
+     * @param keyPassword        the (optional) key password -- if {@code null}, the keystore password is assumed the same for the individual key
+     * @param keystoreType       the keystore type
+     * @param truststorePath     the truststore path
+     * @param truststorePassword the truststore password
+     * @param truststoreType     the truststore type
+     * @param protocol           the TLS protocol version string
+     */
+    public TlsConfiguration(String keystorePath, String keystorePassword, String keyPassword,
+                            KeystoreType keystoreType, String truststorePath, String truststorePassword, KeystoreType truststoreType, String protocol) {
+        this.keystorePath = keystorePath;
+        this.keystorePassword = keystorePassword;
+        this.keyPassword = keyPassword;
+        this.keystoreType = keystoreType;
+        this.truststorePath = truststorePath;
+        this.truststorePassword = truststorePassword;
+        this.truststoreType = truststoreType;
+        this.protocol = protocol;
+    }
+
+    /**
+     * Instantiates a container object with a deep copy of the given configuration values.
+     *
+     * @param other the configuration to copy
+     */
+    public TlsConfiguration(TlsConfiguration other) {
+        this.keystorePath = other.keystorePath;
+        this.keystorePassword = other.keystorePassword;
+        this.keyPassword = other.keyPassword;
+        this.keystoreType = other.keystoreType;
+        this.truststorePath = other.truststorePath;
+        this.truststorePassword = other.truststorePassword;
+        this.truststoreType = other.truststoreType;
+        this.protocol = other.protocol;
+    }
+
+    // Static factory method from NiFiProperties
+
+    /**
+     * Returns a {@link TlsConfiguration} instantiated from the relevant {@link NiFiProperties} properties.
+     *
+     * @param niFiProperties the NiFi properties
+     * @return a populated TlsConfiguration container object
+     */
+    public static TlsConfiguration fromNiFiProperties(NiFiProperties niFiProperties) {
+        if (niFiProperties == null) {
+            throw new IllegalArgumentException("The NiFi properties cannot be null");
+        }
+
+        String keystorePath = niFiProperties.getProperty(NiFiProperties.SECURITY_KEYSTORE);
+        String keystorePassword = niFiProperties.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD);
+        String keyPassword = niFiProperties.getProperty(NiFiProperties.SECURITY_KEY_PASSWD);
+        String keystoreType = niFiProperties.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE);
+        String truststorePath = niFiProperties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE);
+        String truststorePassword = niFiProperties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD);
+        String truststoreType = niFiProperties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE);
+        String protocol = TLS_PROTOCOL_VERSION;
+
+        final TlsConfiguration tlsConfiguration = new TlsConfiguration(keystorePath, keystorePassword, keyPassword,
+                keystoreType, truststorePath, truststorePassword,
+                truststoreType, protocol);
+        if (logger.isDebugEnabled()) {
+            logger.debug("Instantiating TlsConfiguration from NiFi properties: {}, {}, {}, {}, {}, {}, {}, {}",
+                    keystorePath, tlsConfiguration.getKeystorePasswordForLogging(), tlsConfiguration.getKeyPasswordForLogging(), keystoreType,
+                    truststorePath, tlsConfiguration.getTruststorePasswordForLogging(), truststoreType, protocol);
+        }
+
+        return tlsConfiguration;
+    }
+
+    /**
+     * Returns a {@link TlsConfiguration} instantiated from the relevant {@link NiFiProperties} properties for the truststore <em>only</em>. No keystore properties are read or used.
+     *
+     * @param niFiProperties the NiFi properties
+     * @return a populated TlsConfiguration container object
+     */
+    public static TlsConfiguration fromNiFiPropertiesTruststoreOnly(NiFiProperties niFiProperties) {
+        if (niFiProperties == null) {
+            throw new IllegalArgumentException("The NiFi properties cannot be null");
+        }
+
+        String truststorePath = niFiProperties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE);
+        String truststorePassword = niFiProperties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD);
+        String truststoreType = niFiProperties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE);
+        String protocol = TLS_PROTOCOL_VERSION;
+
+        final TlsConfiguration tlsConfiguration = new TlsConfiguration(null, null, null, null, truststorePath, truststorePassword,
+                truststoreType, protocol);
+        if (logger.isDebugEnabled()) {
+            logger.debug("Instantiating TlsConfiguration from NiFi properties: null x4, {}, {}, {}, {}",
+                    truststorePath, tlsConfiguration.getTruststorePasswordForLogging(), truststoreType, protocol);
+        }
+
+        return tlsConfiguration;
+    }
+
+    /**
+     * Returns {@code true} if the provided TlsConfiguration is {@code null} or <em>empty</em>
+     * (i.e. neither any of the keystore nor truststore properties are populated).
+     *
+     * @param tlsConfiguration the container object to check
+     * @return true if this container is empty or null
+     */
+    public static boolean isEmpty(TlsConfiguration tlsConfiguration) {
+        return tlsConfiguration == null || !(tlsConfiguration.isAnyKeystorePopulated() || tlsConfiguration.isAnyTruststorePopulated());
+    }
+
+    // Getters & setters
+
+    public String getKeystorePath() {
+        return keystorePath;
+    }
+
+    public String getKeystorePassword() {
+        return keystorePassword;
+    }
+
+    /**
+     * Returns {@code "********"} if the keystore password is populated, {@code "null"} if not.
+     *
+     * @return a loggable String representation of the keystore password
+     */
+    public String getKeystorePasswordForLogging() {
+        return maskPasswordForLog(keystorePassword);
+    }
+
+    public String getKeyPassword() {
+        return keyPassword;
+    }
+
+    /**
+     * Returns {@code "********"} if the key password is populated, {@code "null"} if not.
+     *
+     * @return a loggable String representation of the key password
+     */
+    public String getKeyPasswordForLogging() {
+        return maskPasswordForLog(keyPassword);
+    }
+
+    /**
+     * Returns the "working" key password -- if the key password is populated, it is returned; otherwise the {@link #getKeystorePassword()} is returned.
+     *
+     * @return the key or keystore password actually populated
+     */
+    public String getFunctionalKeyPassword() {
+        return StringUtils.isNotBlank(keyPassword) ? keyPassword : keystorePassword;
+    }
+
+    /**
+     * Returns {@code "********"} if the functional key password is populated, {@code "null"} if not.
+     *
+     * @return a loggable String representation of the functional key password
+     */
+    public String getFunctionalKeyPasswordForLogging() {
+        return maskPasswordForLog(getFunctionalKeyPassword());
+    }
+
+    public KeystoreType getKeystoreType() {
+        return keystoreType;
+    }
+
+    public String getTruststorePath() {
+        return truststorePath;
+    }
+
+    public String getTruststorePassword() {
+        return truststorePassword;
+    }
+
+    /**
+     * Returns {@code "********"} if the truststore password is populated, {@code "null"} if not.
+     *
+     * @return a loggable String representation of the truststore password
+     */
+    public String getTruststorePasswordForLogging() {
+        return maskPasswordForLog(truststorePassword);
+    }
+
+    public KeystoreType getTruststoreType() {
+        return truststoreType;
+    }
+
+    public String getProtocol() {
+        return protocol;
+    }
+
+    // Boolean validators for keystore & truststore
+
+    /**
+     * Returns {@code true} if the necessary properties are populated to instantiate a <strong>keystore</strong>. This does <em>not</em> validate the values (see {@link #isKeystoreValid()}).
+     *
+     * @return true if the path, password, and type are present
+     */
+    public boolean isKeystorePopulated() {
+        return isStorePopulated(keystorePath, keystorePassword, keystoreType, "keystore");
+    }
+
+    /**
+     * Returns {@code true} if <em>any</em> of the keystore properties is populated, indicating that the caller expects a valid keystore to be generated.
+     *
+     * @return true if any keystore properties are present
+     */
+    public boolean isAnyKeystorePopulated() {
+        return isAnyPopulated(keystorePath, keystorePassword, keystoreType);
+    }
+
+    /**
+     * Returns {@code true} if the necessary properties are populated and the keystore can be successfully instantiated (i.e. the path is valid and the password(s) are correct).
+     *
+     * @return true if the keystore properties are valid
+     */
+    public boolean isKeystoreValid() {
+        boolean simpleCheck = isStoreValid(keystorePath, keystorePassword, keystoreType, "keystore");
+        if (simpleCheck) {
+            return true;
+        } else if (StringUtils.isNotBlank(keyPassword) && !keystorePassword.equals(keyPassword)) {
+            logger.debug("Simple keystore validity check failed; trying with separate key password");
+            try {
+                return isKeystorePopulated()
+                        && KeyStoreUtils.isKeyPasswordCorrect(new File(keystorePath).toURI().toURL(), keystoreType, keystorePassword.toCharArray(),
+                        getFunctionalKeyPassword().toCharArray());
+            } catch (MalformedURLException e) {
+                logger.error("Encountered an error validating the keystore: " + e.getLocalizedMessage());
+                return false;
+            }
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Returns {@code true} if the necessary properties are populated to instantiate a <strong>truststore</strong>. This does <em>not</em> validate the values (see {@link #isTruststoreValid()}).
+     *
+     * @return true if the path, password, and type are present
+     */
+    public boolean isTruststorePopulated() {
+        return isStorePopulated(truststorePath, truststorePassword, truststoreType, "truststore");
+    }
+
+    /**
+     * Returns {@code true} if <em>any</em> of the truststore properties is populated, indicating that the caller expects a valid truststore to be generated.
+     *
+     * @return true if any truststore properties are present
+     */
+    public boolean isAnyTruststorePopulated() {
+        return isAnyPopulated(truststorePath, truststorePassword, truststoreType);
+    }
+
+    /**
+     * Returns {@code true} if the necessary properties are populated and the truststore can be successfully instantiated (i.e. the path is valid and the password is correct).
+     *
+     * @return true if the truststore properties are valid
+     */
+    public boolean isTruststoreValid() {
+        return isStoreValid(truststorePath, truststorePassword, truststoreType, "truststore");
+    }
+
+    /**
+     * Returns a {@code String[]} containing the keystore properties for logging. The order is
+     * {@link #getKeystorePath()}, {@link #getKeystorePasswordForLogging()},
+     * {@link #getFunctionalKeyPasswordForLogging()}, {@link #getKeystoreType()} (using the type or "null").
+     *
+     * @return a loggable String[]
+     */
+    public String[] getKeystorePropertiesForLogging() {
+        return new String[]{getKeystorePath(), getKeystorePasswordForLogging(), getFunctionalKeyPasswordForLogging(), getKeystoreType() != null ? getKeystoreType().getType() : NULL_LOG};
+    }
+
+    /**
+     * Returns a {@code String[]} containing the truststore properties for logging. The order is
+     * {@link #getTruststorePath()}, {@link #getTruststorePasswordForLogging()},
+     * {@link #getTruststoreType()} (using the type or "null").
+     *
+     * @return a loggable String[]
+     */
+    public String[] getTruststorePropertiesForLogging() {
+        return new String[]{getTruststorePath(), getTruststorePasswordForLogging(), getKeystoreType() != null ? getTruststoreType().getType() : NULL_LOG};
+    }
+
+    @Override
+    public String toString() {
+        return new ToStringBuilder(this)
+                .append("keystorePath", keystorePath)
+                .append("keystorePassword", getKeystorePasswordForLogging())
+                .append("keyPassword", getKeyPasswordForLogging())
+                .append("keystoreType", keystoreType)
+                .append("truststorePath", truststorePath)
+                .append("truststorePassword", getTruststorePasswordForLogging())
+                .append("truststoreType", truststoreType)
+                .append("protocol", protocol)
+                .toString();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        TlsConfiguration that = (TlsConfiguration) o;
+        return Objects.equals(keystorePath, that.keystorePath)
+                && Objects.equals(keystorePassword, that.keystorePassword)
+                && Objects.equals(keyPassword, that.keyPassword)
+                && keystoreType == that.keystoreType
+                && Objects.equals(truststorePath, that.truststorePath)
+                && Objects.equals(truststorePassword, that.truststorePassword)
+                && truststoreType == that.truststoreType
+                && Objects.equals(protocol, that.protocol);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(keystorePath, keystorePassword, keyPassword, keystoreType, truststorePath, truststorePassword, truststoreType, protocol);
+    }
+
+    private static String maskPasswordForLog(String password) {
+        return StringUtils.isNotBlank(password) ? MASKED_PASSWORD_LOG : NULL_LOG;
+    }
+
+    private boolean isAnyPopulated(String path, String password, KeystoreType type) {
+        return StringUtils.isNotBlank(path) || StringUtils.isNotBlank(password) || type != null;
+    }
+
+    private boolean isStorePopulated(String path, String password, KeystoreType type, String label) {
+        boolean isPopulated;
+        String passwordForLogging;
+
+        // Legacy truststores can be populated without a password; only check the path and type
+        isPopulated = StringUtils.isNotBlank(path) && type != null;
+        if ("truststore".equalsIgnoreCase(label)) {
+            passwordForLogging = getTruststorePasswordForLogging();
+        } else {
+            // Keystores require a password
+            isPopulated = isPopulated && StringUtils.isNotBlank(password);
+            passwordForLogging = getKeystorePasswordForLogging();
+        }
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("TLS config {} is {}: {}, {}, {}", label, isPopulated ? "populated" : "not populated", path, passwordForLogging, type);
+        }
+        return isPopulated;
+    }
+
+    private boolean isStoreValid(String path, String password, KeystoreType type, String label) {
+        try {
+            return isStorePopulated(path, password, type, label) && KeyStoreUtils.isStoreValid(new File(path).toURI().toURL(), type, password.toCharArray());
+        } catch (MalformedURLException e) {
+            logger.error("Encountered an error validating the " + label + ": " + e.getLocalizedMessage());
+            return false;
+        }
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslSocketFactoryCreationException.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/TlsException.java
similarity index 60%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslSocketFactoryCreationException.java
rename to nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/TlsException.java
index 492c3db..7d2b4a4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslSocketFactoryCreationException.java
+++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/TlsException.java
@@ -14,27 +14,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.framework.security.util;
+package org.apache.nifi.security.util;
+
+import java.security.GeneralSecurityException;
 
 /**
- * Represents the exceptional case when a SslSocketFactory failed creation.
- *
+ * A common exception for any TLS-configuration or -operation related exceptions in NiFi framework
+ * and extensions. The nested cause can be examined if the caller can resolve various root causes,
+ * or the top-level exception can be handled generically if no recovery is available.
  */
-public class SslSocketFactoryCreationException extends SslException {
-
-    public SslSocketFactoryCreationException(Throwable cause) {
-        super(cause);
-    }
-
-    public SslSocketFactoryCreationException(String message, Throwable cause) {
-        super(message, cause);
+public class TlsException extends GeneralSecurityException {
+    public TlsException() {
     }
 
-    public SslSocketFactoryCreationException(String message) {
+    public TlsException(String message) {
         super(message);
     }
 
-    public SslSocketFactoryCreationException() {
+    public TlsException(String message, Throwable cause) {
+        super(message, cause);
     }
 
+    public TlsException(Throwable cause) {
+        super(cause);
+    }
 }
diff --git a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/CertificateUtilsTest.groovy b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/CertificateUtilsTest.groovy
index 5deb278..d20cfeb 100644
--- a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/CertificateUtilsTest.groovy
+++ b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/CertificateUtilsTest.groovy
@@ -31,6 +31,7 @@ import org.junit.runners.JUnit4
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
 
+import javax.net.ssl.SSLException
 import javax.net.ssl.SSLPeerUnverifiedException
 import javax.net.ssl.SSLSession
 import javax.net.ssl.SSLSocket
@@ -51,7 +52,6 @@ import java.util.concurrent.Future
 import java.util.concurrent.TimeUnit
 import java.util.concurrent.atomic.AtomicBoolean
 
-import static org.junit.Assert.assertEquals
 import static org.junit.Assert.assertTrue
 
 @RunWith(JUnit4.class)
@@ -104,13 +104,7 @@ class CertificateUtilsTest extends GroovyTestCase {
      *
      * @param dn the DN
      * @return the certificate
-     * @throws IOException
-     * @throws NoSuchAlgorithmException
-     * @throws java.security.cert.CertificateException
-     * @throws java.security.NoSuchProviderException
-     * @throws java.security.SignatureException
-     * @throws java.security.InvalidKeyException
-     * @throws OperatorCreationException
+     * @throws IOException* @throws NoSuchAlgorithmException* @throws java.security.cert.CertificateException* @throws java.security.NoSuchProviderException* @throws java.security.SignatureException* @throws java.security.InvalidKeyException* @throws OperatorCreationException
      */
     private
     static X509Certificate generateCertificate(String dn) throws IOException, NoSuchAlgorithmException, CertificateException, NoSuchProviderException, SignatureException, InvalidKeyException, OperatorCreationException {
@@ -125,13 +119,7 @@ class CertificateUtilsTest extends GroovyTestCase {
      * @param issuerDn the issuer DN
      * @param issuerKey the issuer private key
      * @return the certificate
-     * @throws IOException
-     * @throws NoSuchAlgorithmException
-     * @throws CertificateException
-     * @throws NoSuchProviderException
-     * @throws SignatureException
-     * @throws InvalidKeyException
-     * @throws OperatorCreationException
+     * @throws IOException* @throws NoSuchAlgorithmException* @throws CertificateException* @throws NoSuchProviderException* @throws SignatureException* @throws InvalidKeyException* @throws OperatorCreationException
      */
     private
     static X509Certificate generateIssuedCertificate(String dn, X509Certificate issuer, KeyPair issuerKey) throws IOException, NoSuchAlgorithmException, CertificateException, NoSuchProviderException, SignatureException, InvalidKeyException, OperatorCreationException {
@@ -147,6 +135,7 @@ class CertificateUtilsTest extends GroovyTestCase {
         [certificate, issuerCertificate] as X509Certificate[]
     }
 
+    @SuppressWarnings("deprecation")
     private static javax.security.cert.X509Certificate generateLegacyCertificate(X509Certificate x509Certificate) {
         return javax.security.cert.X509Certificate.getInstance(x509Certificate.getEncoded())
     }
@@ -205,20 +194,19 @@ class CertificateUtilsTest extends GroovyTestCase {
         SSLSocket noneSocket = [getNeedClientAuth: { -> false }, getWantClientAuth: { -> false }] as SSLSocket
 
         // Act
-        CertificateUtils.ClientAuth needClientAuthStatus = CertificateUtils.getClientAuthStatus(needSocket)
+        SslContextFactory.ClientAuth needClientAuthStatus = CertificateUtils.getClientAuthStatus(needSocket)
         logger.info("Client auth (needSocket): ${needClientAuthStatus}")
-        CertificateUtils.ClientAuth wantClientAuthStatus = CertificateUtils.getClientAuthStatus(wantSocket)
+        SslContextFactory.ClientAuth wantClientAuthStatus = CertificateUtils.getClientAuthStatus(wantSocket)
         logger.info("Client auth (wantSocket): ${wantClientAuthStatus}")
-        CertificateUtils.ClientAuth noneClientAuthStatus = CertificateUtils.getClientAuthStatus(noneSocket)
+        SslContextFactory.ClientAuth noneClientAuthStatus = CertificateUtils.getClientAuthStatus(noneSocket)
         logger.info("Client auth (noneSocket): ${noneClientAuthStatus}")
 
         // Assert
-        assert needClientAuthStatus == CertificateUtils.ClientAuth.NEED
-        assert wantClientAuthStatus == CertificateUtils.ClientAuth.WANT
-        assert noneClientAuthStatus == CertificateUtils.ClientAuth.NONE
+        assert needClientAuthStatus == SslContextFactory.ClientAuth.REQUIRED
+        assert wantClientAuthStatus == SslContextFactory.ClientAuth.WANT
+        assert noneClientAuthStatus == SslContextFactory.ClientAuth.NONE
     }
 
-
     @Test
     void testShouldExtractClientCertificatesFromSSLServerSocketWithAnyClientAuth() {
         final String EXPECTED_DN = "CN=ncm.nifi.apache.org,OU=Security,O=Apache,ST=CA,C=US"
@@ -436,7 +424,7 @@ class CertificateUtilsTest extends GroovyTestCase {
     }
 
     @Test
-    public void testShouldGenerateSelfSignedCert() throws Exception {
+    void testShouldGenerateSelfSignedCert() throws Exception {
         String dn = "CN=testDN,O=testOrg"
 
         int days = 365
@@ -458,8 +446,8 @@ class CertificateUtilsTest extends GroovyTestCase {
     }
 
     @Test
-    public void testIssueCert() throws Exception {
-        int days = 365;
+    void testIssueCert() throws Exception {
+        int days = 365
         KeyPair issuerKeyPair = generateKeyPair()
         X509Certificate issuer = CertificateUtils.generateSelfSignedX509Certificate(issuerKeyPair, "CN=testCa,O=testOrg", SIGNATURE_ALGORITHM, days)
 
@@ -486,7 +474,7 @@ class CertificateUtilsTest extends GroovyTestCase {
     }
 
     @Test
-    public void reorderShouldPutElementsInCorrectOrder() {
+    void reorderShouldPutElementsInCorrectOrder() {
         String cn = "CN=testcn"
         String l = "L=testl"
         String st = "ST=testst"
@@ -504,8 +492,8 @@ class CertificateUtilsTest extends GroovyTestCase {
     }
 
     @Test
-    public void testUniqueSerialNumbers() {
-        def running = new AtomicBoolean(true);
+    void testUniqueSerialNumbers() {
+        def running = new AtomicBoolean(true)
         def executorService = Executors.newCachedThreadPool()
         def serialNumbers = Collections.newSetFromMap(new ConcurrentHashMap())
         try {
@@ -514,7 +502,7 @@ class CertificateUtilsTest extends GroovyTestCase {
                 futures.add(executorService.submit(new Callable<Integer>() {
                     @Override
                     Integer call() throws Exception {
-                        int count = 0;
+                        int count = 0
                         while (running.get()) {
                             def before = System.currentTimeMillis()
                             def serialNumber = CertificateUtils.getUniqueSerialNumber()
@@ -523,23 +511,23 @@ class CertificateUtilsTest extends GroovyTestCase {
                             assertTrue(serialNumberMillis >= before)
                             assertTrue(serialNumberMillis <= after)
                             assertTrue(serialNumbers.add(serialNumber))
-                            count++;
+                            count++
                         }
-                        return count;
+                        return count
                     }
-                }));
+                }))
             }
 
             Thread.sleep(1000)
 
             running.set(false)
 
-            def totalRuns = 0;
+            def totalRuns = 0
             for (int i = 0; i < futures.size(); i++) {
                 try {
                     def numTimes = futures.get(i).get()
                     logger.info("future $i executed $numTimes times")
-                    totalRuns += numTimes;
+                    totalRuns += numTimes
                 } catch (ExecutionException e) {
                     throw e.getCause()
                 }
@@ -586,4 +574,85 @@ class CertificateUtilsTest extends GroovyTestCase {
         assert certificate.getSubjectAlternativeNames().size() == SANS.size()
         assert certificate.getSubjectAlternativeNames()*.last().containsAll(SANS)
     }
+
+    @Test
+    void testShouldDetectTlsErrors() {
+        // Arrange
+        final String msg = "Test exception"
+
+        // SSLPeerUnverifiedException isn't specifically defined in the method, but is a subclass of SSLException so it should be caught
+        List<Throwable> directErrors = [new TlsException(msg), new SSLPeerUnverifiedException(msg), new CertificateException(msg), new SSLException(msg)]
+        List<Throwable> causedErrors = directErrors.collect { Throwable cause -> new Exception(msg, cause) } + [
+                new Exception(msg,
+                        new Exception("Nested $msg",
+                                new Exception("Double nested $msg",
+                                        new TlsException("Triple nested $msg"))))]
+        List<Throwable> unrelatedErrors = [new Exception(msg), new IllegalArgumentException(msg), new NullPointerException(msg)]
+
+        // Act
+        def directResults = directErrors.collect { Throwable e -> CertificateUtils.isTlsError(e) }
+        def causedResults = causedErrors.collect { Throwable e -> CertificateUtils.isTlsError(e) }
+        def unrelatedResults = unrelatedErrors.collect { Throwable e -> CertificateUtils.isTlsError(e) }
+
+        logger.info("Direct results: ${directResults}")
+        logger.info("Caused results: ${causedResults}")
+        logger.info("Unrelated results: ${unrelatedResults}")
+
+        // Assert
+        assert directResults.every()
+        assert causedResults.every()
+        assert !unrelatedResults.any()
+    }
+
+    @Test
+    void testShouldParseJavaVersion() {
+        // Arrange
+        def possibleVersions = ["1.5.0", "1.6.0", "1.7.0.123", "1.8.0.231", "9.0.1", "10.1.2", "11.2.3", "12.3.456"]
+
+        // Act
+        def majorVersions = possibleVersions.collect { String version ->
+            logger.debug("Attempting to determine major version of ${version}")
+            CertificateUtils.parseJavaVersion(version)
+        }
+        logger.info("Major versions: ${majorVersions}")
+
+        // Assert
+        assert majorVersions == (5..12)
+    }
+
+    @Test
+    void testShouldGetCurrentSupportedTlsProtocolVersions() {
+        // Arrange
+        int javaMajorVersion = CertificateUtils.getJavaVersion()
+        logger.debug("Running on Java version: ${javaMajorVersion}")
+
+        // Act
+        def tlsVersions = CertificateUtils.getCurrentSupportedTlsProtocolVersions()
+        logger.info("Supported protocol versions for ${javaMajorVersion}: ${tlsVersions}")
+
+        // Assert
+        if (javaMajorVersion < 11) {
+            assert tlsVersions == ["TLSv1.2"] as String[]
+        } else {
+            assert tlsVersions == ["TLSv1.3", "TLSv1.2"] as String[]
+        }
+    }
+
+    @Test
+    void testShouldGetMaxCurrentSupportedTlsProtocolVersion() {
+        // Arrange
+        int javaMajorVersion = CertificateUtils.getJavaVersion()
+        logger.debug("Running on Java version: ${javaMajorVersion}")
+
+        // Act
+        def tlsVersion = CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion()
+        logger.info("Highest supported protocol version for ${javaMajorVersion}: ${tlsVersion}")
+
+        // Assert
+        if (javaMajorVersion < 11) {
+            assert tlsVersion == "TLSv1.2"
+        } else {
+            assert tlsVersion == "TLSv1.3"
+        }
+    }
 }
diff --git a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/SslContextFactoryTest.groovy b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/SslContextFactoryTest.groovy
new file mode 100644
index 0000000..cff92ff
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/SslContextFactoryTest.groovy
@@ -0,0 +1,297 @@
+/*
+ * 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.security.util
+
+import org.apache.nifi.util.NiFiProperties
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.junit.After
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import javax.net.ssl.SSLContext
+import javax.net.ssl.SSLServerSocket
+import java.security.Security
+
+@RunWith(JUnit4.class)
+class SslContextFactoryTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(SslContextFactoryTest.class)
+
+    private static final String KEYSTORE_PATH = "src/test/resources/TlsConfigurationKeystore.jks"
+    private static final String KEYSTORE_PASSWORD = "keystorepassword"
+    private static final String KEY_PASSWORD = "keypassword"
+    private static final KeystoreType KEYSTORE_TYPE = KeystoreType.JKS
+
+    private static final String TRUSTSTORE_PATH = "src/test/resources/TlsConfigurationTruststore.jks"
+    private static final String TRUSTSTORE_PASSWORD = "truststorepassword"
+    private static final KeystoreType TRUSTSTORE_TYPE = KeystoreType.JKS
+
+    private static final String PROTOCOL = CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion()
+
+    // The default TLS protocol versions for different Java versions
+    private static final List<String> JAVA_8_TLS_PROTOCOL_VERSIONS = ["TLSv1.2", "TLSv1.1", "TLSv1"]
+    private static final List<String> JAVA_11_TLS_PROTOCOL_VERSIONS = ["TLSv1.3", "TLSv1.2", "TLSv1.1", "TLSv1"]
+
+    private static final Map<String, String> DEFAULT_PROPS = [
+            (NiFiProperties.SECURITY_KEYSTORE)         : KEYSTORE_PATH,
+            (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : KEYSTORE_PASSWORD,
+            (NiFiProperties.SECURITY_KEY_PASSWD)       : KEY_PASSWORD,
+            (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : KEYSTORE_TYPE.getType(),
+            (NiFiProperties.SECURITY_TRUSTSTORE)       : TRUSTSTORE_PATH,
+            (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): TRUSTSTORE_PASSWORD,
+            (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : TRUSTSTORE_TYPE.getType(),
+    ]
+
+    private NiFiProperties mockNiFiProperties = NiFiProperties.createBasicNiFiProperties("", DEFAULT_PROPS)
+
+    private TlsConfiguration tlsConfiguration
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() {
+        tlsConfiguration = new TlsConfiguration(KEYSTORE_PATH, KEYSTORE_PASSWORD, KEY_PASSWORD, KEYSTORE_TYPE, TRUSTSTORE_PATH, TRUSTSTORE_PASSWORD, TRUSTSTORE_TYPE)
+    }
+
+    @After
+    void tearDown() {
+
+    }
+
+    static List<String> getCurrentTlsProtocolVersions() {
+        if (CertificateUtils.getJavaVersion() < 11) {
+            return JAVA_8_TLS_PROTOCOL_VERSIONS
+        } else {
+            return JAVA_11_TLS_PROTOCOL_VERSIONS
+        }
+    }
+
+    /**
+     * Asserts that the protocol versions are correct. In recent versions of Java, this enforces order as well, but in older versions, it just enforces presence.
+     *
+     * @param enabledProtocols the actual protocols, either in {@code String[]} or {@code Collection<String>} form
+     * @param expectedProtocols the specific protocol versions to be present (ordered as desired)
+     */
+    void assertProtocolVersions(def enabledProtocols, def expectedProtocols) {
+        if (CertificateUtils.getJavaVersion() > 8) {
+            assert enabledProtocols == expectedProtocols as String[]
+        } else {
+            assert enabledProtocols as Set == expectedProtocols as Set
+        }
+    }
+
+    @Test
+    void testShouldCreateSslContextFromTlsConfiguration() {
+        // Arrange
+        logger.info("Creating SSL Context from TLS Configuration: ${tlsConfiguration}")
+
+        // Act
+        SSLContext sslContext = SslContextFactory.createSslContext(tlsConfiguration, SslContextFactory.ClientAuth.NONE)
+        logger.info("Created SSL Context: ${KeyStoreUtils.sslContextToString(sslContext)}")
+
+        // Assert
+        assert sslContext.protocol == tlsConfiguration.protocol
+
+        def defaultSSLParameters = sslContext.defaultSSLParameters
+        logger.info("Default SSL Parameters: ${KeyStoreUtils.sslParametersToString(defaultSSLParameters)}" as String)
+        assertProtocolVersions(defaultSSLParameters.protocols, getCurrentTlsProtocolVersions())
+        assert !defaultSSLParameters.needClientAuth
+        assert !defaultSSLParameters.wantClientAuth
+
+        // Check a socket created from this context
+        assertSocketProtocols(sslContext)
+    }
+
+    @Test
+    void testCreateSslContextFromTlsConfigurationShouldHandleEmptyKeyPassword() {
+        // Arrange
+
+        // Change the keystore to one with the same keystore and key password, but don't provide the key password
+        Map missingKeyPasswordProps = DEFAULT_PROPS + [
+                (NiFiProperties.SECURITY_KEYSTORE)  : "src/test/resources/samepassword.jks",
+                (NiFiProperties.SECURITY_KEY_PASSWD): "",
+        ]
+        NiFiProperties propertiesWithoutKeyPassword = NiFiProperties.createBasicNiFiProperties("", missingKeyPasswordProps)
+        TlsConfiguration configWithoutKeyPassword = TlsConfiguration.fromNiFiProperties(propertiesWithoutKeyPassword)
+        logger.info("Creating SSL Context from TLS Configuration: ${configWithoutKeyPassword}")
+
+        // Act
+        SSLContext sslContext = SslContextFactory.createSslContext(configWithoutKeyPassword, SslContextFactory.ClientAuth.NONE)
+        logger.info("Created SSL Context: ${KeyStoreUtils.sslContextToString(sslContext)}")
+
+        // Assert
+        assert sslContext.protocol == configWithoutKeyPassword.protocol
+
+        def defaultSSLParameters = sslContext.defaultSSLParameters
+        logger.info("Default SSL Parameters: ${KeyStoreUtils.sslParametersToString(defaultSSLParameters)}" as String)
+        assertProtocolVersions(defaultSSLParameters.protocols, getCurrentTlsProtocolVersions())
+        assert !defaultSSLParameters.needClientAuth
+        assert !defaultSSLParameters.wantClientAuth
+
+        // Check a socket created from this context
+        assertSocketProtocols(sslContext)
+    }
+
+    /**
+     * This test ensures that silent failures don't occur -- if some keystore/truststore properties
+     * are populated but not enough to be valid, throw an exception on failure.
+     */
+    @Test
+    void testCreateSslContextFromTlsConfigurationShouldFailOnInvalidProperties() {
+        // Arrange
+
+        // Set up configurations missing the keystore path and truststore path
+        Map missingKeystorePathProps = DEFAULT_PROPS + [
+                (NiFiProperties.SECURITY_KEYSTORE): "",
+        ]
+        NiFiProperties propsNoKeystorePath = NiFiProperties.createBasicNiFiProperties("", missingKeystorePathProps)
+        TlsConfiguration configNoKeystorePath = TlsConfiguration.fromNiFiProperties(propsNoKeystorePath)
+        logger.info("Creating SSL Context from TLS Configuration: ${configNoKeystorePath}")
+
+        Map missingTruststorePathProps = DEFAULT_PROPS + [
+                (NiFiProperties.SECURITY_TRUSTSTORE)     : "",
+                // Remove the keystore properties to ensure the right conditional is tested
+                (NiFiProperties.SECURITY_KEYSTORE)       : "",
+                (NiFiProperties.SECURITY_KEYSTORE_PASSWD): "",
+                (NiFiProperties.SECURITY_KEY_PASSWD)     : "",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE)  : "",
+        ]
+        NiFiProperties propsNoTruststorePath = NiFiProperties.createBasicNiFiProperties("", missingTruststorePathProps)
+        TlsConfiguration configNoTruststorePath = TlsConfiguration.fromNiFiProperties(propsNoTruststorePath)
+        logger.info("Creating SSL Context from TLS Configuration: ${configNoTruststorePath}")
+
+        // Act
+        def noKeystorePathMsg = shouldFail(TlsException) {
+            SSLContext sslContext = SslContextFactory.createSslContext(configNoKeystorePath, SslContextFactory.ClientAuth.NONE)
+            logger.info("Created SSL Context missing keystore path: ${KeyStoreUtils.sslContextToString(sslContext)}")
+        }
+
+        def noTruststorePathMsg = shouldFail(TlsException) {
+            SSLContext sslContext = SslContextFactory.createSslContext(configNoTruststorePath, SslContextFactory.ClientAuth.NONE)
+            logger.info("Created SSL Context missing truststore path: ${KeyStoreUtils.sslContextToString(sslContext)}")
+        }
+
+        // Assert
+        assert noKeystorePathMsg =~ "The keystore properties are not valid"
+        assert noTruststorePathMsg =~ "The truststore properties are not valid"
+    }
+
+    /**
+     * This is a regression test to ensure that a truststore without a password is allowed (legacy truststores did not require a password).
+     */
+    @Test
+    void testCreateSslContextFromTlsConfigurationShouldHandleEmptyTruststorePassword() {
+        // Arrange
+
+        // Change the truststore to one with no password
+        Map truststoreNoPasswordProps = DEFAULT_PROPS + [
+                (NiFiProperties.SECURITY_TRUSTSTORE)       : "src/test/resources/no-password-truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "",
+        ]
+        NiFiProperties propertiesNoTruststorePassword = NiFiProperties.createBasicNiFiProperties("", truststoreNoPasswordProps)
+        TlsConfiguration configNoTruststorePassword = TlsConfiguration.fromNiFiProperties(propertiesNoTruststorePassword)
+        logger.info("Creating SSL Context from TLS Configuration: ${configNoTruststorePassword}")
+
+        // Act
+        SSLContext sslContext = SslContextFactory.createSslContext(configNoTruststorePassword, SslContextFactory.ClientAuth.NONE)
+        logger.info("Created SSL Context: ${KeyStoreUtils.sslContextToString(sslContext)}")
+
+        // Assert
+        assert sslContext.protocol == configNoTruststorePassword.protocol
+
+        def defaultSSLParameters = sslContext.defaultSSLParameters
+        logger.info("Default SSL Parameters: ${KeyStoreUtils.sslParametersToString(defaultSSLParameters)}" as String)
+        assertProtocolVersions(defaultSSLParameters.protocols, getCurrentTlsProtocolVersions())
+        assert !defaultSSLParameters.needClientAuth
+        assert !defaultSSLParameters.wantClientAuth
+
+        // Check a socket created from this context
+        assertSocketProtocols(sslContext)
+    }
+
+    /**
+     * This test is for legacy expectations in nifi-framework-core. That {@code SslContextFactory}
+     * implementation threw an exception if the keystore properties were present and the truststore
+     * properties were not.
+     */
+    @Test
+    void testCreateSslContextFromTlsConfigurationShouldHandleKeystorePropsWithoutTruststoreProps() {
+        // Arrange
+
+        // Change the keystore to one with the same keystore and key password, but don't provide the key password
+        Map keystoreOnlyProps = DEFAULT_PROPS.findAll { k, v -> k.contains("keystore") }
+        NiFiProperties keystoreNiFiProperties = NiFiProperties.createBasicNiFiProperties("", keystoreOnlyProps)
+        TlsConfiguration keystoreOnlyConfig = TlsConfiguration.fromNiFiProperties(keystoreNiFiProperties)
+        logger.info("Creating SSL Context from TLS Configuration: ${keystoreOnlyConfig}")
+
+        // Act
+        def msg = shouldFail(TlsException) {
+            SSLContext sslContext = SslContextFactory.createSslContext(keystoreOnlyConfig, SslContextFactory.ClientAuth.NONE)
+            logger.info("Created SSL Context: ${KeyStoreUtils.sslContextToString(sslContext)}")
+        }
+        logger.expected(msg)
+
+        // Assert
+        assert msg =~ "Truststore properties are required if keystore properties are present"
+    }
+
+    /**
+     * This test is for legacy expectations in nifi-framework-core. That {@code SslContextFactory}
+     * implementation returned {@code null} if none of the properties were populated.
+     */
+    @Test
+    void testCreateSslContextFromTlsConfigurationShouldHandleEmptyConfiguration() {
+        // Arrange
+        TlsConfiguration emptyConfig = new TlsConfiguration()
+        logger.info("Creating SSL Context from TLS Configuration: ${emptyConfig}")
+
+        // Act
+        SSLContext sslContext = SslContextFactory.createSslContext(emptyConfig, SslContextFactory.ClientAuth.NONE)
+
+        // Assert
+        assert !sslContext
+    }
+
+    /**
+     * Asserts an {@link SSLServerSocket} from the provided {@link SSLContext} has the proper TLS protocols set.
+     *
+     * @param sslContext the context under test
+     */
+    void assertSocketProtocols(SSLContext sslContext) {
+        SSLServerSocket sslSocket = sslContext.serverSocketFactory.createServerSocket() as SSLServerSocket
+        logger.info("Created SSL (server) socket: ${sslSocket}")
+        assert sslSocket.enabledProtocols.contains("TLSv1.2")
+
+        // Override the SSL parameters protocol version
+        SSLServerSocket customSslSocket = sslContext.serverSocketFactory.createServerSocket() as SSLServerSocket
+        def customParameters = customSslSocket.getSSLParameters()
+        customParameters.setProtocols(["TLSv1.2"] as String[])
+        customSslSocket.setSSLParameters(customParameters)
+        assertProtocolVersions(customSslSocket.enabledProtocols, ["TLSv1.2"])
+    }
+}
\ No newline at end of file
diff --git a/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/TlsConfigurationTest.groovy b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/TlsConfigurationTest.groovy
new file mode 100644
index 0000000..29ba36d
--- /dev/null
+++ b/nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/TlsConfigurationTest.groovy
@@ -0,0 +1,212 @@
+/*
+ * 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.security.util
+
+
+import org.apache.nifi.util.NiFiProperties
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.junit.After
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import java.security.Security
+
+@RunWith(JUnit4.class)
+class TlsConfigurationTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(TlsConfigurationTest.class)
+
+    private static final String KEYSTORE_PATH = "src/test/resources/TlsConfigurationKeystore.jks"
+    private static final String KEYSTORE_PASSWORD = "keystorepassword"
+    private static final String KEY_PASSWORD = "keypassword"
+    private static final KeystoreType KEYSTORE_TYPE = KeystoreType.JKS
+
+    private static final String TRUSTSTORE_PATH = "src/test/resources/TlsConfigurationTruststore.jks"
+    private static final String TRUSTSTORE_PASSWORD = "truststorepassword"
+    private static final KeystoreType TRUSTSTORE_TYPE = KeystoreType.JKS
+
+    private static final Map<String, String> DEFAULT_PROPS = [
+            (NiFiProperties.SECURITY_KEYSTORE)         : KEYSTORE_PATH,
+            (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : KEYSTORE_PASSWORD,
+            (NiFiProperties.SECURITY_KEY_PASSWD)       : KEY_PASSWORD,
+            (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : KEYSTORE_TYPE.getType(),
+            (NiFiProperties.SECURITY_TRUSTSTORE)       : TRUSTSTORE_PATH,
+            (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): TRUSTSTORE_PASSWORD,
+            (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : TRUSTSTORE_TYPE.getType(),
+    ]
+
+    private NiFiProperties mockNiFiProperties = NiFiProperties.createBasicNiFiProperties("", DEFAULT_PROPS)
+
+    private TlsConfiguration tlsConfiguration
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() throws Exception {
+        tlsConfiguration = new TlsConfiguration(KEYSTORE_PATH, KEYSTORE_PASSWORD, KEY_PASSWORD, KEYSTORE_TYPE, TRUSTSTORE_PATH, TRUSTSTORE_PASSWORD, TRUSTSTORE_TYPE)
+    }
+
+    @After
+    void tearDown() throws Exception {
+    }
+
+    @Test
+    void testShouldCreateFromNiFiProperties() {
+        // Arrange
+
+        // Act
+        TlsConfiguration fromProperties = TlsConfiguration.fromNiFiProperties(mockNiFiProperties)
+        logger.info("Created TlsConfiguration: ${fromProperties}")
+
+        // Assert
+        assert fromProperties == tlsConfiguration
+    }
+
+    @Test
+    void testCreateFromNiFiPropertiesShouldHandleNullKeystoreTypes() {
+        // Arrange
+        def noKeystoreTypesProps = NiFiProperties.createBasicNiFiProperties("", DEFAULT_PROPS +
+                [(NiFiProperties.SECURITY_KEYSTORE_TYPE)  : "",
+                 (NiFiProperties.SECURITY_TRUSTSTORE_TYPE): ""
+                ])
+
+        // Act
+        TlsConfiguration fromProperties = TlsConfiguration.fromNiFiProperties(noKeystoreTypesProps)
+        logger.info("Created TlsConfiguration: ${fromProperties}")
+
+        // Assert
+        assert fromProperties.keystoreType == null
+        assert fromProperties.truststoreType == null
+    }
+
+    @Test
+    void testShouldGetFunctionalKeyPassword() {
+        // Arrange
+        TlsConfiguration withKeyPassword = tlsConfiguration
+
+        // A container where the keystore password is explicitly set as the key password as well
+        TlsConfiguration withoutKeyPassword = new TlsConfiguration(KEYSTORE_PATH, KEYSTORE_PASSWORD, KEYSTORE_TYPE, TRUSTSTORE_PATH, TRUSTSTORE_PASSWORD, TRUSTSTORE_TYPE)
+
+        // A container where null is explicitly set as the key password
+        TlsConfiguration withNullPassword = new TlsConfiguration(KEYSTORE_PATH, KEYSTORE_PASSWORD, null, KEYSTORE_TYPE, TRUSTSTORE_PATH, TRUSTSTORE_PASSWORD, TRUSTSTORE_TYPE)
+
+        // Act
+        String actualKeyPassword = withKeyPassword.getKeyPassword()
+        String functionalKeyPassword = withKeyPassword.getFunctionalKeyPassword()
+
+        String duplicateKeystorePassword = withoutKeyPassword.getKeyPassword()
+        String duplicateFunctionalKeyPassword = withoutKeyPassword.getFunctionalKeyPassword()
+
+        String missingKeyPassword = withNullPassword.getKeyPassword()
+        String keystorePassword = withNullPassword.getFunctionalKeyPassword()
+
+        // Assert
+        assert actualKeyPassword == KEY_PASSWORD
+        assert functionalKeyPassword == KEY_PASSWORD
+
+        assert duplicateKeystorePassword == KEYSTORE_PASSWORD
+        assert duplicateFunctionalKeyPassword == KEYSTORE_PASSWORD
+
+        assert missingKeyPassword == null
+        assert keystorePassword == KEYSTORE_PASSWORD
+    }
+
+    @Test
+    void testShouldCheckKeystorePopulation() {
+        // Arrange
+        TlsConfiguration empty = new TlsConfiguration()
+        TlsConfiguration noKeystorePassword = new TlsConfiguration(KEYSTORE_PATH, "", KEY_PASSWORD, KEYSTORE_TYPE, TRUSTSTORE_PATH, TRUSTSTORE_PASSWORD, TRUSTSTORE_TYPE)
+
+        // Act
+        boolean normalIsPopulated = tlsConfiguration.isKeystorePopulated()
+        boolean emptyIsPopulated = empty.isKeystorePopulated()
+        boolean noPasswordIsPopulated = noKeystorePassword.isKeystorePopulated()
+
+        // Assert
+        assert normalIsPopulated
+        assert !emptyIsPopulated
+        assert !noPasswordIsPopulated
+    }
+
+    @Test
+    void testShouldCheckTruststorePopulation() {
+        // Arrange
+        TlsConfiguration empty = new TlsConfiguration()
+        TlsConfiguration noTruststorePassword = new TlsConfiguration(KEYSTORE_PATH, KEYSTORE_PASSWORD, KEY_PASSWORD, KEYSTORE_TYPE, TRUSTSTORE_PATH, "", TRUSTSTORE_TYPE)
+
+        // Act
+        boolean normalIsPopulated = tlsConfiguration.isTruststorePopulated()
+        boolean emptyIsPopulated = empty.isTruststorePopulated()
+        boolean noPasswordIsPopulated = noTruststorePassword.isTruststorePopulated()
+
+        // Assert
+        assert normalIsPopulated
+        assert !emptyIsPopulated
+        assert noPasswordIsPopulated
+    }
+
+    @Test
+    void testShouldValidateKeystoreConfiguration() {
+        // Arrange
+        TlsConfiguration empty = new TlsConfiguration()
+        TlsConfiguration wrongPassword = new TlsConfiguration(KEYSTORE_PATH, KEYSTORE_PASSWORD.reverse(), KEY_PASSWORD.reverse(), KEYSTORE_TYPE, TRUSTSTORE_PATH, TRUSTSTORE_PASSWORD.reverse(), TRUSTSTORE_TYPE)
+        TlsConfiguration invalid = new TlsConfiguration(KEYSTORE_PATH.reverse(), KEYSTORE_PASSWORD.reverse(), KEY_PASSWORD.reverse(), KEYSTORE_TYPE, TRUSTSTORE_PATH.reverse(), TRUSTSTORE_PASSWORD.reverse(), TRUSTSTORE_TYPE)
+
+        // Act
+        boolean normalIsValid = tlsConfiguration.isKeystoreValid()
+        boolean emptyIsValid = empty.isKeystoreValid()
+        boolean wrongPasswordIsValid = wrongPassword.isKeystoreValid()
+        boolean invalidIsValid = invalid.isKeystoreValid()
+
+        // Assert
+        assert normalIsValid
+        assert !emptyIsValid
+        assert !wrongPasswordIsValid
+        assert !invalidIsValid
+    }
+
+    @Test
+    void testShouldValidateTruststoreConfiguration() {
+        // Arrange
+        TlsConfiguration empty = new TlsConfiguration()
+        TlsConfiguration wrongPassword = new TlsConfiguration(KEYSTORE_PATH, KEYSTORE_PASSWORD.reverse(), KEY_PASSWORD.reverse(), KEYSTORE_TYPE, TRUSTSTORE_PATH, TRUSTSTORE_PASSWORD.reverse(), TRUSTSTORE_TYPE)
+        TlsConfiguration invalid = new TlsConfiguration(KEYSTORE_PATH.reverse(), KEYSTORE_PASSWORD.reverse(), KEY_PASSWORD.reverse(), KEYSTORE_TYPE, TRUSTSTORE_PATH.reverse(), TRUSTSTORE_PASSWORD.reverse(), TRUSTSTORE_TYPE)
+
+        // Act
+        boolean normalIsValid = tlsConfiguration.isTruststoreValid()
+        boolean emptyIsValid = empty.isTruststoreValid()
+        boolean wrongPasswordIsValid = wrongPassword.isTruststoreValid()
+        boolean invalidIsValid = invalid.isTruststoreValid()
+
+        // Assert
+        assert normalIsValid
+        assert !emptyIsValid
+        assert !wrongPasswordIsValid
+        assert !invalidIsValid
+    }
+}
diff --git a/nifi-commons/nifi-security-utils/src/test/resources/TlsConfigurationKeystore.jks b/nifi-commons/nifi-security-utils/src/test/resources/TlsConfigurationKeystore.jks
new file mode 100644
index 0000000..5dd2609
Binary files /dev/null and b/nifi-commons/nifi-security-utils/src/test/resources/TlsConfigurationKeystore.jks differ
diff --git a/nifi-commons/nifi-security-utils/src/test/resources/TlsConfigurationTruststore.jks b/nifi-commons/nifi-security-utils/src/test/resources/TlsConfigurationTruststore.jks
new file mode 100644
index 0000000..4946966
Binary files /dev/null and b/nifi-commons/nifi-security-utils/src/test/resources/TlsConfigurationTruststore.jks differ
diff --git a/nifi-commons/nifi-security-utils/src/test/resources/logback-test.xml b/nifi-commons/nifi-security-utils/src/test/resources/logback-test.xml
index b5036d4..b771380 100644
--- a/nifi-commons/nifi-security-utils/src/test/resources/logback-test.xml
+++ b/nifi-commons/nifi-security-utils/src/test/resources/logback-test.xml
@@ -30,6 +30,7 @@
 
 
     <logger name="org.apache.nifi" level="INFO"/>
+    <logger name="org.apache.nifi.security.util" level="DEBUG"/>
     <logger name="org.apache.nifi.security.util.crypto" level="DEBUG"/>
 
     <root level="INFO">
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/no-password-truststore.jks b/nifi-commons/nifi-security-utils/src/test/resources/no-password-truststore.jks
similarity index 100%
rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/no-password-truststore.jks
rename to nifi-commons/nifi-security-utils/src/test/resources/no-password-truststore.jks
diff --git a/nifi-commons/nifi-socket-utils/src/test/resources/samepassword.jks b/nifi-commons/nifi-security-utils/src/test/resources/samepassword.jks
similarity index 100%
rename from nifi-commons/nifi-socket-utils/src/test/resources/samepassword.jks
rename to nifi-commons/nifi-security-utils/src/test/resources/samepassword.jks
diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
index b805d03..7af6cce 100644
--- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
+++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
@@ -16,6 +16,21 @@
  */
 package org.apache.nifi.remote.client;
 
+import java.io.Closeable;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.security.KeyStore;
+import java.security.SecureRandom;
+import java.util.LinkedHashSet;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManagerFactory;
 import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.remote.Transaction;
@@ -29,24 +44,9 @@ import org.apache.nifi.remote.exception.UnknownPortException;
 import org.apache.nifi.remote.protocol.DataPacket;
 import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
 import org.apache.nifi.remote.protocol.http.HttpProxy;
+import org.apache.nifi.security.util.CertificateUtils;
 import org.apache.nifi.security.util.KeyStoreUtils;
 
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManagerFactory;
-import java.io.Closeable;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.Serializable;
-import java.net.InetAddress;
-import java.security.KeyStore;
-import java.security.SecureRandom;
-import java.util.LinkedHashSet;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
 /**
  * <p>
  * The SiteToSiteClient provides a mechanism for sending data to a remote
@@ -919,7 +919,7 @@ public interface SiteToSiteClient extends Closeable {
             if (keyManagerFactory != null && trustManagerFactory != null) {
                 try {
                     // initialize the ssl context
-                    final SSLContext sslContext = SSLContext.getInstance("TLS");
+                    final SSLContext sslContext = SSLContext.getInstance(CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion());
                     sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom());
                     sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
 
diff --git a/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java b/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java
index ded1db1..2b90745 100644
--- a/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java
+++ b/nifi-commons/nifi-site-to-site-client/src/test/java/org/apache/nifi/remote/client/http/TestHttpClient.java
@@ -65,6 +65,7 @@ import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
 import org.apache.nifi.remote.protocol.http.HttpHeaders;
 import org.apache.nifi.remote.protocol.http.HttpProxy;
 import org.apache.nifi.remote.util.StandardDataPacket;
+import org.apache.nifi.security.util.CertificateUtils;
 import org.apache.nifi.stream.io.StreamUtils;
 import org.apache.nifi.web.api.dto.ControllerDTO;
 import org.apache.nifi.web.api.dto.PortDTO;
@@ -456,6 +457,8 @@ public class TestHttpClient {
         sslContextFactory.setKeyStorePath("src/test/resources/certs/keystore.jks");
         sslContextFactory.setKeyStorePassword("passwordpassword");
         sslContextFactory.setKeyStoreType("JKS");
+        sslContextFactory.setProtocol(CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion());
+        sslContextFactory.setExcludeProtocols("TLS", "TLSv1", "TLSv1.1");
 
         httpConnector = new ServerConnector(server);
 
@@ -464,6 +467,7 @@ public class TestHttpClient {
         sslConnector = new ServerConnector(server,
                 new SslConnectionFactory(sslContextFactory, "http/1.1"),
                 new HttpConnectionFactory(https));
+        logger.info("SSL Connector: " + sslConnector.dump());
 
         server.setConnectors(new Connector[] { httpConnector, sslConnector });
 
diff --git a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java
deleted file mode 100644
index fd09a80..0000000
--- a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.io.socket;
-
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.security.KeyManagementException;
-import java.security.KeyStore;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.SecureRandom;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-import java.util.Arrays;
-
-import javax.net.ssl.KeyManager;
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManager;
-import javax.net.ssl.TrustManagerFactory;
-
-import org.apache.nifi.security.util.KeyStoreUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.file.FileUtils;
-
-public class SSLContextFactory {
-
-    private final String keystore;
-    private final char[] keystorePass;
-    private final String keystoreType;
-    private final char[] keyPassword;
-    private final String truststore;
-    private final char[] truststorePass;
-    private final String truststoreType;
-
-    private final KeyManager[] keyManagers;
-    private final TrustManager[] trustManagers;
-
-    public SSLContextFactory(final NiFiProperties properties) throws NoSuchAlgorithmException, CertificateException, FileNotFoundException, IOException, KeyStoreException, UnrecoverableKeyException {
-        keystore = properties.getProperty(NiFiProperties.SECURITY_KEYSTORE);
-        keystorePass = getPass(properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD));
-        keystoreType = properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE);
-        keyPassword = getPass(properties.getProperty(NiFiProperties.SECURITY_KEY_PASSWD));
-
-        truststore = properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE);
-        truststorePass = getPass(properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD));
-        truststoreType = properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE);
-
-        // prepare the keystore
-        final KeyStore keyStore = KeyStoreUtils.getKeyStore(keystoreType);
-        final FileInputStream keyStoreStream = new FileInputStream(keystore);
-        try {
-            keyStore.load(keyStoreStream, keystorePass);
-        } finally {
-            FileUtils.closeQuietly(keyStoreStream);
-        }
-        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-        if (isKeyAndKeystorePasswordDifferent()) {
-            keyManagerFactory.init(keyStore, keyPassword);
-        } else {
-            keyManagerFactory.init(keyStore, keystorePass);
-        }
-
-        // prepare the truststore
-        final KeyStore trustStore = KeyStoreUtils.getTrustStore(truststoreType);
-        final FileInputStream trustStoreStream = new FileInputStream(truststore);
-        try {
-            trustStore.load(trustStoreStream, truststorePass);
-        } finally {
-            FileUtils.closeQuietly(trustStoreStream);
-        }
-        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-        trustManagerFactory.init(trustStore);
-
-        keyManagers = keyManagerFactory.getKeyManagers();
-        trustManagers = trustManagerFactory.getTrustManagers();
-    }
-
-    private boolean isKeyAndKeystorePasswordDifferent() {
-        return keyPassword != null && !(new String(keyPassword).trim().isEmpty()) && !Arrays.equals(keyPassword, keystorePass);
-    }
-
-    private static char[] getPass(final String password) {
-        return password == null ? null : password.toCharArray();
-    }
-
-    /**
-     * Creates a SSLContext instance using the given information.
-     *
-     * @return a SSLContext instance
-     * @throws java.security.KeyStoreException         if problem with keystore
-     * @throws java.io.IOException                     if unable to create context
-     * @throws java.security.NoSuchAlgorithmException  if algorithm isn't known
-     * @throws java.security.cert.CertificateException if certificate is invalid
-     * @throws java.security.UnrecoverableKeyException if the key cannot be recovered
-     * @throws java.security.KeyManagementException    if the key is improper
-     */
-    public SSLContext createSslContext() throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
-            UnrecoverableKeyException, KeyManagementException {
-
-        // initialize the ssl context
-        final SSLContext sslContext = SSLContext.getInstance("TLS");
-        sslContext.init(keyManagers, trustManagers, new SecureRandom());
-        sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
-
-        return sslContext;
-
-    }
-}
diff --git a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java
index d6aca92..d33a48a 100644
--- a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java
+++ b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java
@@ -16,15 +16,10 @@
  */
 package org.apache.nifi.io.socket;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-
 import javax.net.ssl.SSLContext;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
 
 public final class ServerSocketConfiguration {
 
@@ -32,18 +27,18 @@ public final class ServerSocketConfiguration {
     private Integer socketTimeout;
     private Boolean reuseAddress;
     private Integer receiveBufferSize;
-    private SSLContextFactory sslContextFactory;
+    private TlsConfiguration tlsConfiguration;
 
     public ServerSocketConfiguration() {
     }
 
-    public SSLContext createSSLContext()
-            throws KeyManagementException, NoSuchAlgorithmException, UnrecoverableKeyException, KeyStoreException, CertificateException, FileNotFoundException, IOException {
-        return sslContextFactory == null ? null : sslContextFactory.createSslContext();
+    public SSLContext createSSLContext() throws TlsException {
+        // ClientAuth was hardcoded to REQUIRED in removed SSLContextFactory and overridden in SocketUtils when the socket is created
+        return SslContextFactory.createSslContext(tlsConfiguration, SslContextFactory.ClientAuth.REQUIRED);
     }
 
-    public void setSSLContextFactory(final SSLContextFactory sslContextFactory) {
-        this.sslContextFactory = sslContextFactory;
+    public void setTlsConfiguration(final TlsConfiguration tlsConfiguration) {
+        this.tlsConfiguration = tlsConfiguration;
     }
 
     public Integer getSocketTimeout() {
diff --git a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java
index 8b803dc..88709f5 100644
--- a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java
+++ b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java
@@ -16,15 +16,10 @@
  */
 package org.apache.nifi.io.socket;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-
 import javax.net.ssl.SSLContext;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
 
 public final class SocketConfiguration {
 
@@ -36,15 +31,15 @@ public final class SocketConfiguration {
     private Boolean oobInline;
     private Boolean tcpNoDelay;
     private Integer trafficClass;
-    private SSLContextFactory sslContextFactory;
+    private TlsConfiguration tlsConfiguration;
 
-    public SSLContext createSSLContext()
-            throws KeyManagementException, NoSuchAlgorithmException, UnrecoverableKeyException, KeyStoreException, CertificateException, FileNotFoundException, IOException {
-        return sslContextFactory == null ? null : sslContextFactory.createSslContext();
+    public SSLContext createSSLContext() throws TlsException {
+        // This is only used for client sockets, so the client auth setting is ignored
+        return SslContextFactory.createSslContext(tlsConfiguration, SslContextFactory.ClientAuth.NONE);
     }
 
-    public void setSSLContextFactory(final SSLContextFactory sslContextFactory) {
-        this.sslContextFactory = sslContextFactory;
+    public void setTlsConfiguration(final TlsConfiguration tlsConfiguration) {
+        this.tlsConfiguration = tlsConfiguration;
     }
 
     public Integer getSocketTimeout() {
diff --git a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java
index 718a386..1183735 100644
--- a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java
+++ b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java
@@ -21,18 +21,13 @@ import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.SocketException;
 import java.net.SocketTimeoutException;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-
+import org.apache.nifi.security.util.TlsException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -83,7 +78,7 @@ public abstract class SocketListener {
 
         try {
             serverSocket = SocketUtils.createServerSocket(port, configuration);
-        } catch (KeyManagementException | UnrecoverableKeyException | NoSuchAlgorithmException | KeyStoreException | CertificateException e) {
+        } catch (TlsException e) {
             throw new IOException(e);
         }
 
diff --git a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java
index 27d676a..453cbb2 100644
--- a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java
+++ b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java
@@ -20,18 +20,12 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLServerSocket;
 import javax.net.ssl.SSLSocket;
-
 import org.apache.nifi.logging.NiFiLog;
-
+import org.apache.nifi.security.util.CertificateUtils;
+import org.apache.nifi.security.util.TlsException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,6 +33,14 @@ public final class SocketUtils {
 
     private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketUtils.class));
 
+    /**
+     * Returns a {@link Socket} (effectively used as a client socket) for the given address and configuration.
+     *
+     * @param address   the {@link InetSocketAddress} for the socket (used for hostname and port)
+     * @param config the {@link SocketConfiguration}
+     * @return the socket (can be configured for SSL)
+     * @throws IOException  if there is a problem creating the socket
+     */
     public static Socket createSocket(final InetSocketAddress address, final SocketConfiguration config) throws IOException {
         if (address == null) {
             throw new IllegalArgumentException("Socket address may not be null.");
@@ -58,7 +60,14 @@ public final class SocketUtils {
         if (sslContext == null) {
             socket = new Socket(address.getHostName(), address.getPort());
         } else {
-            socket = sslContext.getSocketFactory().createSocket(address.getHostName(), address.getPort());
+            /* This would ideally be refactored to a shared create method but Socket and ServerSocket
+             * do not share a common interface; Socket is effectively "client socket" in this context
+             */
+            Socket tempSocket = sslContext.getSocketFactory().createSocket(address.getHostName(), address.getPort());
+            final SSLSocket sslSocket = (SSLSocket) tempSocket;
+            // Enforce custom protocols on socket
+            sslSocket.setEnabledProtocols(CertificateUtils.getCurrentSupportedTlsProtocolVersions());
+            socket = sslSocket;
         }
 
         if (config.getSocketTimeout() != null) {
@@ -96,8 +105,17 @@ public final class SocketUtils {
         return socket;
     }
 
+    /**
+     * Returns a {@link ServerSocket} for the given port and configuration.
+     *
+     * @param port   the port for the socket
+     * @param config the {@link ServerSocketConfiguration}
+     * @return the server socket (can be configured for SSL)
+     * @throws IOException  if there is a problem creating the socket
+     * @throws TlsException if there is a problem creating the socket
+     */
     public static ServerSocket createServerSocket(final int port, final ServerSocketConfiguration config)
-            throws IOException, KeyManagementException, UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, CertificateException {
+            throws IOException, TlsException {
         if (config == null) {
             throw new NullPointerException("Configuration may not be null.");
         }
@@ -108,7 +126,10 @@ public final class SocketUtils {
             serverSocket = new ServerSocket(port);
         } else {
             serverSocket = sslContext.getServerSocketFactory().createServerSocket(port);
-            ((SSLServerSocket) serverSocket).setNeedClientAuth(config.getNeedClientAuth());
+            final SSLServerSocket sslServerSocket = (SSLServerSocket) serverSocket;
+            sslServerSocket.setNeedClientAuth(config.getNeedClientAuth());
+            // Enforce custom protocols on socket
+            sslServerSocket.setEnabledProtocols(CertificateUtils.getCurrentSupportedTlsProtocolVersions());
         }
 
         if (config.getSocketTimeout() != null) {
@@ -126,6 +147,29 @@ public final class SocketUtils {
         return serverSocket;
     }
 
+    /**
+     * Returns a {@link SSLServerSocket} for the given port and configuration.
+     *
+     * @param port                      the port for the socket
+     * @param serverSocketConfiguration the {@link ServerSocketConfiguration}
+     * @return the SSL server socket
+     * @throws TlsException if there was a problem creating the socket
+     */
+    public static SSLServerSocket createSSLServerSocket(final int port, final ServerSocketConfiguration serverSocketConfiguration) throws TlsException {
+        try {
+            ServerSocket serverSocket = createServerSocket(port, serverSocketConfiguration);
+            if (serverSocket instanceof SSLServerSocket) {
+                return ((SSLServerSocket) serverSocket);
+            } else {
+                throw new TlsException("Created server socket does not support SSL/TLS");
+            }
+        } catch (IOException e) {
+            logger.error("Encountered an error creating SSLServerSocket: {}", e.getLocalizedMessage());
+            throw new TlsException("Error creating SSLServerSocket", e);
+        }
+
+    }
+
     public static void closeQuietly(final Socket socket) {
         if (socket == null) {
             return;
@@ -133,17 +177,17 @@ public final class SocketUtils {
 
         try {
             try {
-                // can't shudown input/output individually with secure sockets
-                if ((socket instanceof SSLSocket) == false) {
-                    if (socket.isInputShutdown() == false) {
+                // Can't shutdown input/output individually with secure sockets
+                if (!(socket instanceof SSLSocket)) {
+                    if (!socket.isInputShutdown()) {
                         socket.shutdownInput();
                     }
-                    if (socket.isOutputShutdown() == false) {
+                    if (!socket.isOutputShutdown()) {
                         socket.shutdownOutput();
                     }
                 }
             } finally {
-                if (socket.isClosed() == false) {
+                if (!socket.isClosed()) {
                     socket.close();
                 }
             }
diff --git a/nifi-commons/nifi-socket-utils/src/test/groovy/org/apache/nifi/io/socket/SSLContextFactoryTest.groovy b/nifi-commons/nifi-socket-utils/src/test/groovy/org/apache/nifi/io/socket/SSLContextFactoryTest.groovy
deleted file mode 100644
index e67df6f..0000000
--- a/nifi-commons/nifi-socket-utils/src/test/groovy/org/apache/nifi/io/socket/SSLContextFactoryTest.groovy
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * 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.io.socket
-
-import org.apache.nifi.util.NiFiProperties
-import org.bouncycastle.jce.provider.BouncyCastleProvider
-import org.junit.After
-import org.junit.AfterClass
-import org.junit.Before
-import org.junit.BeforeClass
-import org.junit.Test
-import org.junit.runner.RunWith
-import org.junit.runners.JUnit4
-import org.slf4j.Logger
-import org.slf4j.LoggerFactory
-
-import javax.net.ssl.SSLContext
-import java.security.Security
-
-@RunWith(JUnit4.class)
-class SSLContextFactoryTest extends GroovyTestCase {
-    private static final Logger logger = LoggerFactory.getLogger(SSLContextFactoryTest.class)
-
-    private static String NF_PROPS_FILE = null
-
-    @BeforeClass
-    static void setUpOnce() throws Exception {
-        Security.addProvider(new BouncyCastleProvider())
-
-        if (System.getProperty(NiFiProperties.PROPERTIES_FILE_PATH)) {
-            NF_PROPS_FILE = System.getProperty(NiFiProperties.PROPERTIES_FILE_PATH)
-            System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, null)
-        }
-
-        logger.metaClass.methodMissing = { String name, args ->
-            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
-        }
-    }
-
-    @Before
-    void setUp() throws Exception {
-
-    }
-
-    @After
-    void tearDown() throws Exception {
-    }
-
-    @AfterClass
-    static void tearDownOnce() throws Exception {
-        if (NF_PROPS_FILE) {
-            System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, NF_PROPS_FILE)
-        }
-    }
-
-    /**
-     * Returns a {@link NiFiProperties} object configured with default values for accessing
-     * keystores and truststores. The values can be overridden by providing a map parameter.
-     *
-     * @param overrides an optional Map of overriding configuration values
-     * @return the configured NiFiProperties object
-     */
-    private static NiFiProperties buildNiFiProperties(Map<String, String> overrides = [:]) {
-        final Map DEFAULTS = [
-                (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : "keystorepassword",
-                (NiFiProperties.SECURITY_KEYSTORE)         : "src/test/resources/samepassword.jks",
-                (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : "JKS",
-                (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "changeit",
-                (NiFiProperties.SECURITY_TRUSTSTORE)       : buildCacertsPath(),
-                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : "JKS",
-        ]
-        DEFAULTS.putAll(overrides)
-        NiFiProperties.createBasicNiFiProperties(null, DEFAULTS)
-    }
-
-    /**
-     * Returns the file path to the {@code cacerts} default JRE truststore. Handles Java 8
-     * and earlier as well as Java 9 and later directory structures.
-     *
-     * @return the path to cacerts
-     */
-    private static String buildCacertsPath() {
-        String javaHome = System.getenv("JAVA_HOME")
-        if (System.getProperty("java.version").startsWith("1.")) {
-            javaHome + "/jre/lib/security/cacerts"
-        } else {
-            javaHome + "/lib/security/cacerts"
-        }
-    }
-
-    @Test
-    void testShouldVerifyKeystoreWithSameKeyPassword() throws Exception {
-        // Arrange
-
-        // Set up the keystore configuration as NiFiProperties object
-        NiFiProperties np = buildNiFiProperties()
-
-        // Create the SSLContextFactory with the config
-        SSLContextFactory sslcf = new SSLContextFactory(np)
-
-        // Act
-
-        // Access the SSLContextFactory to create an SSLContext
-        SSLContext sslContext = sslcf.createSslContext()
-
-        // Assert
-
-        // The SSLContext was accessible and correct
-        assert sslContext
-    }
-
-    @Test
-    void testShouldVerifyKeystoreWithDifferentKeyPassword() throws Exception {
-        // Arrange
-
-        // Set up the keystore configuration as NiFiProperties object
-        // (prior to NIFI-6830, an UnrecoverableKeyException was thrown due to the wrong password being provided)
-        NiFiProperties np = buildNiFiProperties([
-                (NiFiProperties.SECURITY_KEYSTORE)  : "src/test/resources/differentpassword.jks",
-                (NiFiProperties.SECURITY_KEY_PASSWD): "keypassword",
-        ])
-
-        // Create the SSLContextFactory with the config
-        SSLContextFactory sslcf = new SSLContextFactory(np)
-
-        // Act
-
-        // Access the SSLContextFactory to create an SSLContext
-        SSLContext sslContext = sslcf.createSslContext()
-
-        // Assert
-
-        // The SSLContext was accessible and correct
-        assert sslContext
-    }
-
-    @Test
-    void testShouldVerifyKeystoreWithEmptyKeyPassword() throws Exception {
-        // Arrange
-
-        // Set up the keystore configuration as NiFiProperties object
-        // (re-opened NIFI-6830, an UnrecoverableKeyException was thrown due to an empty password being provided)
-        NiFiProperties np = buildNiFiProperties([
-                (NiFiProperties.SECURITY_KEY_PASSWD): ""
-        ])
-
-        // Create the SSLContextFactory with the config
-        SSLContextFactory sslcf = new SSLContextFactory(np)
-
-        // Act
-
-        // Access the SSLContextFactory to create an SSLContext
-        SSLContext sslContext = sslcf.createSslContext()
-
-        // Assert
-
-        // The SSLContext was accessible and correct
-        assert sslContext
-    }
-}
diff --git a/nifi-commons/nifi-socket-utils/src/test/groovy/org/apache/nifi/io/socket/SocketUtilsTest.groovy b/nifi-commons/nifi-socket-utils/src/test/groovy/org/apache/nifi/io/socket/SocketUtilsTest.groovy
new file mode 100644
index 0000000..b0a62c8
--- /dev/null
+++ b/nifi-commons/nifi-socket-utils/src/test/groovy/org/apache/nifi/io/socket/SocketUtilsTest.groovy
@@ -0,0 +1,121 @@
+/*
+ * 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.io.socket
+
+import org.apache.nifi.security.util.CertificateUtils
+import org.apache.nifi.security.util.KeystoreType
+import org.apache.nifi.security.util.TlsConfiguration
+import org.apache.nifi.util.NiFiProperties
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.junit.After
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import javax.net.ssl.SSLServerSocket
+import java.security.Security
+
+@RunWith(JUnit4.class)
+class SocketUtilsTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(SocketUtilsTest.class)
+
+    private static final String KEYSTORE_PATH = "src/test/resources/TlsConfigurationKeystore.jks"
+    private static final String KEYSTORE_PASSWORD = "keystorepassword"
+    private static final String KEY_PASSWORD = "keypassword"
+    private static final KeystoreType KEYSTORE_TYPE = KeystoreType.JKS
+
+    private static final String TRUSTSTORE_PATH = "src/test/resources/TlsConfigurationTruststore.jks"
+    private static final String TRUSTSTORE_PASSWORD = "truststorepassword"
+    private static final KeystoreType TRUSTSTORE_TYPE = KeystoreType.JKS
+
+    private static final String PROTOCOL = CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion()
+
+    private static final Map<String, String> DEFAULT_PROPS = [
+            (NiFiProperties.SECURITY_KEYSTORE)         : KEYSTORE_PATH,
+            (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : KEYSTORE_PASSWORD,
+            (NiFiProperties.SECURITY_KEY_PASSWD)       : KEY_PASSWORD,
+            (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : KEYSTORE_TYPE.getType(),
+            (NiFiProperties.SECURITY_TRUSTSTORE)       : TRUSTSTORE_PATH,
+            (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): TRUSTSTORE_PASSWORD,
+            (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : TRUSTSTORE_TYPE.getType(),
+    ]
+
+    private NiFiProperties mockNiFiProperties = NiFiProperties.createBasicNiFiProperties(null, DEFAULT_PROPS)
+
+    // A static TlsConfiguration referencing the test resource keystore and truststore
+//    private static final TlsConfiguration TLS_CONFIGURATION = new TlsConfiguration(KEYSTORE_PATH, KEYSTORE_PASSWORD, KEY_PASSWORD, KEYSTORE_TYPE, TRUSTSTORE_PATH, TRUSTSTORE_PASSWORD, TRUSTSTORE_TYPE, PROTOCOL)
+//    private static final SSLContext sslContext = SslContextFactory.createSslContext(TLS_CONFIGURATION, SslContextFactory.ClientAuth.NONE)
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+    }
+
+    @Before
+    void setUp() {
+
+    }
+
+    @After
+    void tearDown() {
+
+    }
+
+    @Test
+    void testCreateSSLServerSocketShouldRestrictTlsProtocols() {
+        // Arrange
+        ServerSocketConfiguration mockServerSocketConfiguration = new ServerSocketConfiguration()
+        mockServerSocketConfiguration.setTlsConfiguration(TlsConfiguration.fromNiFiProperties(mockNiFiProperties))
+
+        // Act
+        SSLServerSocket sslServerSocket = SocketUtils.createSSLServerSocket(0, mockServerSocketConfiguration)
+        logger.info("Created SSL server socket: ${sslServerSocket}")
+
+        // Assert
+        String[] enabledProtocols = sslServerSocket.getEnabledProtocols()
+        logger.info("Enabled protocols: ${enabledProtocols}")
+        assert enabledProtocols == CertificateUtils.getCurrentSupportedTlsProtocolVersions()
+        assert !enabledProtocols.contains("TLSv1")
+        assert !enabledProtocols.contains("TLSv1.1")
+    }
+
+    @Test
+    void testCreateServerSocketShouldRestrictTlsProtocols() {
+        // Arrange
+        ServerSocketConfiguration mockServerSocketConfiguration = new ServerSocketConfiguration()
+        mockServerSocketConfiguration.setTlsConfiguration(TlsConfiguration.fromNiFiProperties(mockNiFiProperties))
+
+        // Act
+        SSLServerSocket sslServerSocket = SocketUtils.createServerSocket(0, mockServerSocketConfiguration) as SSLServerSocket
+        logger.info("Created SSL server socket: ${sslServerSocket}")
+
+        // Assert
+        String[] enabledProtocols = sslServerSocket.getEnabledProtocols()
+        logger.info("Enabled protocols: ${enabledProtocols}")
+        assert enabledProtocols == CertificateUtils.getCurrentSupportedTlsProtocolVersions()
+        assert !enabledProtocols.contains("TLSv1")
+        assert !enabledProtocols.contains("TLSv1.1")
+    }
+}
\ No newline at end of file
diff --git a/nifi-commons/nifi-socket-utils/src/test/resources/TlsConfigurationKeystore.jks b/nifi-commons/nifi-socket-utils/src/test/resources/TlsConfigurationKeystore.jks
new file mode 100644
index 0000000..5dd2609
Binary files /dev/null and b/nifi-commons/nifi-socket-utils/src/test/resources/TlsConfigurationKeystore.jks differ
diff --git a/nifi-commons/nifi-socket-utils/src/test/resources/TlsConfigurationTruststore.jks b/nifi-commons/nifi-socket-utils/src/test/resources/TlsConfigurationTruststore.jks
new file mode 100644
index 0000000..4946966
Binary files /dev/null and b/nifi-commons/nifi-socket-utils/src/test/resources/TlsConfigurationTruststore.jks differ
diff --git a/nifi-commons/nifi-socket-utils/src/test/resources/differentpassword.jks b/nifi-commons/nifi-socket-utils/src/test/resources/differentpassword.jks
deleted file mode 100644
index e791f4b..0000000
Binary files a/nifi-commons/nifi-socket-utils/src/test/resources/differentpassword.jks and /dev/null differ
diff --git a/nifi-commons/nifi-web-utils/pom.xml b/nifi-commons/nifi-web-utils/pom.xml
index df4d0e0..40d6455 100644
--- a/nifi-commons/nifi-web-utils/pom.xml
+++ b/nifi-commons/nifi-web-utils/pom.xml
@@ -91,5 +91,10 @@
             <version>4.5.6</version>
             <scope>compile</scope>
         </dependency>
+        <dependency>
+            <groupId>com.squareup.okhttp3</groupId>
+            <artifactId>okhttp</artifactId>
+            <version>3.14.4</version>
+        </dependency>
     </dependencies>
 </project>
diff --git a/nifi-commons/nifi-web-utils/src/main/java/org/apache/nifi/security/util/OkHttpClientUtils.java b/nifi-commons/nifi-web-utils/src/main/java/org/apache/nifi/security/util/OkHttpClientUtils.java
new file mode 100644
index 0000000..f35eba5
--- /dev/null
+++ b/nifi-commons/nifi-web-utils/src/main/java/org/apache/nifi/security/util/OkHttpClientUtils.java
@@ -0,0 +1,60 @@
+/*
+ * 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.security.util;
+
+import java.security.UnrecoverableKeyException;
+import javax.net.ssl.SSLSocketFactory;
+import javax.net.ssl.X509TrustManager;
+import okhttp3.OkHttpClient;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class contains utility methods for working with the {@link OkHttpClient} that many components use for external HTTP communication.
+ */
+public class OkHttpClientUtils {
+    private static final Logger logger = LoggerFactory.getLogger(OkHttpClientUtils.class);
+
+    /**
+     * If the {@link TlsConfiguration} contains valid properties to configure an
+     * {@link SSLSocketFactory}, parses the necessary values and applies the config to the client
+     * builder. If the properties are not populated, no action is taken.
+     *
+     * @param tlsConfiguration the TLS configuration container object
+     * @param okHttpClient     the OkHttp client builder
+     * @return true if the TLS configuration was applied to the builder
+     */
+    public static boolean applyTlsToOkHttpClientBuilder(TlsConfiguration tlsConfiguration, OkHttpClient.Builder okHttpClient) {
+        try {
+            final SSLSocketFactory sslSocketFactory = SslContextFactory.createSSLSocketFactory(tlsConfiguration);
+            final X509TrustManager x509TrustManager = SslContextFactory.getX509TrustManager(tlsConfiguration);
+            if (sslSocketFactory != null && x509TrustManager != null) {
+                okHttpClient.sslSocketFactory(sslSocketFactory, x509TrustManager);
+                return true;
+            }
+        } catch (TlsException e) {
+            if (e.getCause() instanceof UnrecoverableKeyException) {
+                logger.error("Key password may be incorrect or not set. Check your keystore passwords." + e.getMessage());
+            } else {
+                logger.error("Encountered an error configuring TLS: {}", e.getLocalizedMessage());
+                throw new ProcessException("Error configuring TLS", e);
+            }
+        }
+        return false;
+    }
+}
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProviderConfigurationContext.java b/nifi-framework-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProviderConfigurationContext.java
index 6fe61fc..69c1f87 100644
--- a/nifi-framework-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProviderConfigurationContext.java
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProviderConfigurationContext.java
@@ -17,11 +17,12 @@
 package org.apache.nifi.authentication;
 
 import java.util.Map;
+import org.apache.nifi.configuration.NonComponentConfigurationContext;
 
 /**
  *
  */
-public interface LoginIdentityProviderConfigurationContext {
+public interface LoginIdentityProviderConfigurationContext extends NonComponentConfigurationContext {
 
     /**
      * @return identifier for the authority provider
@@ -39,10 +40,10 @@ public interface LoginIdentityProviderConfigurationContext {
     Map<String, String> getProperties();
 
     /**
-     * @param property to lookup the descriptor and value of
-     * @return the value the component currently understands for the given
-     * PropertyDescriptor. This method does not substitute default
-     * PropertyDescriptor values, so the value returned will be null if not set
+     * Returns the value of the provided property. This method does not substitute default values, so the value returned will be {@code null} if not set.
+     *
+     * @param property the property to retrieve
+     * @return the current property value (can be null)
      */
     String getProperty(String property);
 }
diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProviderConfigurationContext.java b/nifi-framework-api/src/main/java/org/apache/nifi/configuration/NonComponentConfigurationContext.java
similarity index 55%
copy from nifi-framework-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProviderConfigurationContext.java
copy to nifi-framework-api/src/main/java/org/apache/nifi/configuration/NonComponentConfigurationContext.java
index 6fe61fc..92dc416 100644
--- a/nifi-framework-api/src/main/java/org/apache/nifi/authentication/LoginIdentityProviderConfigurationContext.java
+++ b/nifi-framework-api/src/main/java/org/apache/nifi/configuration/NonComponentConfigurationContext.java
@@ -14,35 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.authentication;
+package org.apache.nifi.configuration;
 
 import java.util.Map;
+import org.apache.nifi.controller.ConfigurationContext;
 
 /**
- *
+ * Shared interface for various feature-specific configuration contexts which allows common code to
+ * handle property retrieval without awareness of the specific implementation.
+ * <p>
+ * Note: This interface is <em>not</em> used as the basis for component-specific configuration contexts (see {@link ConfigurationContext}).
  */
-public interface LoginIdentityProviderConfigurationContext {
-
+public interface NonComponentConfigurationContext {
     /**
-     * @return identifier for the authority provider
+     * @return identifier for the caller entity
      */
     String getIdentifier();
 
     /**
-     * Retrieves all properties the component currently understands regardless
+     * Returns all properties the configuration context contains regardless
      * of whether a value has been set for them or not. If no value is present
-     * then its value is null and thus any registered default for the property
-     * descriptor applies.
+     * then its value is {@code null}.
      *
      * @return Map of all properties
      */
     Map<String, String> getProperties();
 
     /**
-     * @param property to lookup the descriptor and value of
-     * @return the value the component currently understands for the given
-     * PropertyDescriptor. This method does not substitute default
-     * PropertyDescriptor values, so the value returned will be null if not set
+     * Returns the value of the provided property. This method does not substitute default values, so the value returned will be {@code null} if not set.
+     *
+     * @param property the property to retrieve
+     * @return the current property value (can be null)
      */
     String getProperty(String property);
 }
diff --git a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessor.java b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessor.java
index 238b1be..4a2e49a 100644
--- a/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessor.java
+++ b/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/AbstractAMQPProcessor.java
@@ -19,6 +19,12 @@ package org.apache.nifi.amqp.processors;
 import com.rabbitmq.client.Connection;
 import com.rabbitmq.client.ConnectionFactory;
 import com.rabbitmq.client.DefaultSaslConfig;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import javax.net.ssl.SSLContext;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
@@ -31,13 +37,6 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 
-import javax.net.ssl.SSLContext;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
 
 /**
  * Base processor that uses RabbitMQ client API
@@ -118,7 +117,7 @@ abstract class AbstractAMQPProcessor<T extends AMQPWorker> extends AbstractProce
                     + "Possible values are REQUIRED, WANT, NONE. This property is only used when an SSL Context "
                     + "has been defined and enabled.")
             .required(false)
-            .allowableValues(SSLContextService.ClientAuth.values())
+            .allowableValues(SslContextFactory.ClientAuth.values())
             .defaultValue("REQUIRED")
             .build();
 
@@ -227,12 +226,12 @@ abstract class AbstractAMQPProcessor<T extends AMQPWorker> extends AbstractProce
         final String rawClientAuth = context.getProperty(CLIENT_AUTH).getValue();
 
         if (sslService != null) {
-            final SSLContextService.ClientAuth clientAuth;
+            final SslContextFactory.ClientAuth clientAuth;
             if (StringUtils.isBlank(rawClientAuth)) {
-                clientAuth = SSLContextService.ClientAuth.REQUIRED;
+                clientAuth = SslContextFactory.ClientAuth.REQUIRED;
             } else {
                 try {
-                    clientAuth = SSLContextService.ClientAuth.valueOf(rawClientAuth);
+                    clientAuth = SslContextFactory.ClientAuth.valueOf(rawClientAuth);
                 } catch (final IllegalArgumentException iae) {
                     throw new IllegalStateException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
                             rawClientAuth, StringUtils.join(SslContextFactory.ClientAuth.values(), ", ")));
diff --git a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
index 10d19ff..d56d9ee 100644
--- a/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
+++ b/nifi-nar-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/AbstractAWSProcessor.java
@@ -20,6 +20,7 @@ import com.amazonaws.AmazonWebServiceClient;
 import com.amazonaws.ClientConfiguration;
 import com.amazonaws.Protocol;
 import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.AnonymousAWSCredentials;
 import com.amazonaws.auth.BasicAWSCredentials;
 import com.amazonaws.auth.PropertiesCredentials;
@@ -57,6 +58,7 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.aws.credentials.provider.factory.CredentialPropertyDescriptors;
 import org.apache.nifi.proxy.ProxyConfiguration;
 import org.apache.nifi.proxy.ProxySpec;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 
 /**
@@ -225,7 +227,7 @@ public abstract class AbstractAWSProcessor<ClientType extends AmazonWebServiceCl
         if(this.getSupportedPropertyDescriptors().contains(SSL_CONTEXT_SERVICE)) {
             final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
             if (sslContextService != null) {
-                final SSLContext sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.NONE);
+                final SSLContext sslContext = sslContextService.createSSLContext(SslContextFactory.ClientAuth.NONE);
                 // NIFI-3788: Changed hostnameVerifier from null to DHV (BrowserCompatibleHostnameVerifier is deprecated)
                 SdkTLSSocketFactory sdkTLSSocketFactory = new SdkTLSSocketFactory(sslContext, new DefaultHostnameVerifier());
                 config.getApacheHttpClientConfig().setSslSocketFactory(sdkTLSSocketFactory);
diff --git a/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java b/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java
index 6a4a568..5509318 100644
--- a/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java
+++ b/nifi-nar-bundles/nifi-beats-bundle/nifi-beats-processors/src/main/java/org/apache/nifi/processors/beats/ListenBeats.java
@@ -26,11 +26,8 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
-
 import javax.net.ssl.SSLContext;
-import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.commons.lang3.StringUtils;
-
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
@@ -60,6 +57,7 @@ import org.apache.nifi.processors.beats.frame.BeatsEncoder;
 import org.apache.nifi.processors.beats.handler.BeatsSocketChannelHandlerFactory;
 import org.apache.nifi.processors.beats.response.BeatsChannelResponse;
 import org.apache.nifi.processors.beats.response.BeatsResponse;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.RestrictedSSLContextService;
 import org.apache.nifi.ssl.SSLContextService;
 
@@ -92,8 +90,8 @@ public class ListenBeats extends AbstractListenEventBatchingProcessor<BeatsEvent
         .displayName("Client Auth")
         .description("The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.")
         .required(false)
-        .allowableValues(RestrictedSSLContextService.ClientAuth.values())
-        .defaultValue(RestrictedSSLContextService.ClientAuth.REQUIRED.name())
+        .allowableValues(SslContextFactory.ClientAuth.values())
+        .defaultValue(SslContextFactory.ClientAuth.REQUIRED.name())
         .build();
 
     @Override
@@ -157,7 +155,7 @@ public class ListenBeats extends AbstractListenEventBatchingProcessor<BeatsEvent
         final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
         if (sslContextService != null) {
             final String clientAuthValue = context.getProperty(CLIENT_AUTH).getValue();
-            sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.valueOf(clientAuthValue));
+            sslContext = sslContextService.createSSLContext(SslContextFactory.ClientAuth.valueOf(clientAuthValue));
             clientAuth = SslContextFactory.ClientAuth.valueOf(clientAuthValue);
 
         }
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java
index 5988f6b..897f1b8 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java
@@ -28,6 +28,14 @@ import com.datastax.driver.core.Session;
 import com.datastax.driver.core.TypeCodec;
 import com.datastax.driver.core.exceptions.AuthenticationException;
 import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.SSLContext;
 import org.apache.avro.Schema;
 import org.apache.avro.SchemaBuilder;
 import org.apache.commons.lang3.StringUtils;
@@ -47,16 +55,6 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 
-import javax.net.ssl.SSLContext;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-
 /**
  * AbstractCassandraProcessor is a base class for Cassandra processors and contains logic and variables common to most
  * processors integrating with Apache Cassandra.
@@ -109,7 +107,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
                     + "Possible values are REQUIRED, WANT, NONE. This property is only used when an SSL Context "
                     + "has been defined and enabled.")
             .required(false)
-            .allowableValues(SSLContextService.ClientAuth.values())
+            .allowableValues(SslContextFactory.ClientAuth.values())
             .defaultValue("REQUIRED")
             .build();
 
@@ -258,13 +256,13 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
             final SSLContext sslContext;
 
             if (sslService != null) {
-                final SSLContextService.ClientAuth clientAuth;
+                final SslContextFactory.ClientAuth clientAuth;
 
                 if (StringUtils.isBlank(rawClientAuth)) {
-                    clientAuth = SSLContextService.ClientAuth.REQUIRED;
+                    clientAuth = SslContextFactory.ClientAuth.REQUIRED;
                 } else {
                     try {
-                        clientAuth = SSLContextService.ClientAuth.valueOf(rawClientAuth);
+                        clientAuth = SslContextFactory.ClientAuth.valueOf(rawClientAuth);
                     } catch (final IllegalArgumentException iae) {
                         throw new IllegalStateException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
                                 rawClientAuth, StringUtils.join(SslContextFactory.ClientAuth.values(), ", ")));
@@ -534,7 +532,7 @@ public abstract class AbstractCassandraProcessor extends AbstractProcessor {
         if (contactPointList == null) {
             return null;
         }
-        final List<String> contactPointStringList = Arrays.asList(contactPointList.split(","));
+        final String[] contactPointStringList = contactPointList.split(",");
         List<InetSocketAddress> contactPoints = new ArrayList<>();
 
         for (String contactPointEntry : contactPointStringList) {
diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services/src/main/java/org/apache/nifi/service/CassandraSessionProvider.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services/src/main/java/org/apache/nifi/service/CassandraSessionProvider.java
index 079b570..e645edc 100644
--- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services/src/main/java/org/apache/nifi/service/CassandraSessionProvider.java
+++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-services/src/main/java/org/apache/nifi/service/CassandraSessionProvider.java
@@ -22,6 +22,10 @@ import com.datastax.driver.core.JdkSSLOptions;
 import com.datastax.driver.core.Metadata;
 import com.datastax.driver.core.ProtocolOptions;
 import com.datastax.driver.core.Session;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.SSLContext;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
@@ -42,12 +46,6 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 
-import javax.net.ssl.SSLContext;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
 @Tags({"cassandra", "dbcp", "database", "connection", "pooling"})
 @CapabilityDescription("Provides connection session for Cassandra processors to work with Apache Cassandra.")
 public class CassandraSessionProvider extends AbstractControllerService implements CassandraSessionProviderService {
@@ -90,7 +88,7 @@ public class CassandraSessionProvider extends AbstractControllerService implemen
                     + "Possible values are REQUIRED, WANT, NONE. This property is only used when an SSL Context "
                     + "has been defined and enabled.")
             .required(false)
-            .allowableValues(SSLContextService.ClientAuth.values())
+            .allowableValues(SslContextFactory.ClientAuth.values())
             .defaultValue("REQUIRED")
             .build();
 
@@ -212,12 +210,12 @@ public class CassandraSessionProvider extends AbstractControllerService implemen
             final SSLContext sslContext;
 
             if (sslService != null) {
-                final SSLContextService.ClientAuth clientAuth;
+                final SslContextFactory.ClientAuth clientAuth;
                 if (StringUtils.isBlank(rawClientAuth)) {
-                    clientAuth = SSLContextService.ClientAuth.REQUIRED;
+                    clientAuth = SslContextFactory.ClientAuth.REQUIRED;
                 } else {
                     try {
-                        clientAuth = SSLContextService.ClientAuth.valueOf(rawClientAuth);
+                        clientAuth = SslContextFactory.ClientAuth.valueOf(rawClientAuth);
                     } catch (final IllegalArgumentException iae) {
                         throw new ProviderCreationException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
                                 rawClientAuth, StringUtils.join(SslContextFactory.ClientAuth.values(), ", ")));
@@ -264,7 +262,7 @@ public class CassandraSessionProvider extends AbstractControllerService implemen
             return null;
         }
 
-        final List<String> contactPointStringList = Arrays.asList(contactPointList.split(","));
+        final String[] contactPointStringList = contactPointList.split(",");
         List<InetSocketAddress> contactPoints = new ArrayList<>();
 
         for (String contactPointEntry : contactPointStringList) {
diff --git a/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/ConfluentSchemaRegistry.java b/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/ConfluentSchemaRegistry.java
index 3567392..e0db13f 100644
--- a/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/ConfluentSchemaRegistry.java
+++ b/nifi-nar-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/ConfluentSchemaRegistry.java
@@ -17,6 +17,19 @@
 
 package org.apache.nifi.confluent.schemaregistry;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalLong;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import javax.net.ssl.SSLContext;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
@@ -34,24 +47,10 @@ import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.schema.access.SchemaField;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.schemaregistry.services.SchemaRegistry;
+import org.apache.nifi.security.util.SslContextFactory.ClientAuth;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.SchemaIdentifier;
 import org.apache.nifi.ssl.SSLContextService;
-import org.apache.nifi.ssl.SSLContextService.ClientAuth;
-
-import javax.net.ssl.SSLContext;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Optional;
-import java.util.OptionalLong;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
 
 
 
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java
index f856c76..aa0aa3f 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java
@@ -19,6 +19,20 @@ package org.apache.nifi.elasticsearch;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import javax.net.ssl.SSLContext;
 import org.apache.commons.io.IOUtils;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpHost;
@@ -36,6 +50,7 @@ import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.util.StopWatch;
 import org.apache.nifi.util.StringUtils;
@@ -43,20 +58,6 @@ import org.elasticsearch.client.Response;
 import org.elasticsearch.client.RestClient;
 import org.elasticsearch.client.RestClientBuilder;
 
-import javax.net.ssl.SSLContext;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 public class ElasticSearchClientServiceImpl extends AbstractControllerService implements ElasticSearchClientService {
     private ObjectMapper mapper = new ObjectMapper();
 
@@ -125,7 +126,7 @@ public class ElasticSearchClientServiceImpl extends AbstractControllerService im
         final SSLContext sslContext;
         try {
             sslContext = (sslService != null && (sslService.isKeyStoreConfigured() || sslService.isTrustStoreConfigured()))
-                ? sslService.createSSLContext(SSLContextService.ClientAuth.NONE) : null;
+                ? sslService.createSSLContext(SslContextFactory.ClientAuth.NONE) : null;
         } catch (Exception e) {
             getLogger().error("Error building up SSL Context from the supplied configuration.", e);
             throw new InitializationException(e);
@@ -260,7 +261,7 @@ public class ElasticSearchClientServiceImpl extends AbstractControllerService im
             Response response = client.performRequest("POST", "/_bulk", Collections.emptyMap(), entity);
             watch.stop();
 
-            String rawResponse = IOUtils.toString(response.getEntity().getContent(), "UTF-8");
+            String rawResponse = IOUtils.toString(response.getEntity().getContent(), StandardCharsets.UTF_8);
 
             if (getLogger().isDebugEnabled()) {
                 getLogger().debug(String.format("Response was: %s", rawResponse));
@@ -303,7 +304,7 @@ public class ElasticSearchClientServiceImpl extends AbstractControllerService im
 
             if (getLogger().isDebugEnabled()) {
                 getLogger().debug(String.format("Response for bulk delete: %s",
-                        IOUtils.toString(response.getEntity().getContent(), "UTF-8")));
+                        IOUtils.toString(response.getEntity().getContent(), StandardCharsets.UTF_8)));
             }
 
             DeleteOperationResponse dor = new DeleteOperationResponse(watch.getDuration(TimeUnit.MILLISECONDS));
@@ -335,7 +336,7 @@ public class ElasticSearchClientServiceImpl extends AbstractControllerService im
             endpoint.append("/").append(id);
             Response response = client.performRequest("GET", endpoint.toString(), new BasicHeader("Content-Type", "application/json"));
 
-            String body = IOUtils.toString(response.getEntity().getContent(), "UTF-8");
+            String body = IOUtils.toString(response.getEntity().getContent(), StandardCharsets.UTF_8);
 
             return (Map<String, Object>) mapper.readValue(body, Map.class).get("_source");
         } catch (Exception ex) {
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/pom.xml
index 9142a89..977badc 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/pom.xml
@@ -126,6 +126,12 @@ language governing permissions and limitations under the License. -->
             <artifactId>nifi-standard-record-utils</artifactId>
             <version>1.12.0-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-utils</artifactId>
+            <version>1.12.0-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
     </dependencies>
 
     <build>
diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
index 49a1865..67a53c4 100644
--- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
+++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchHttpProcessor.java
@@ -18,6 +18,16 @@ package org.apache.nifi.processors.elasticsearch;
 
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.Proxy;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 import okhttp3.Authenticator;
 import okhttp3.Credentials;
 import okhttp3.OkHttpClient;
@@ -36,31 +46,10 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.proxy.ProxyConfiguration;
 import org.apache.nifi.proxy.ProxySpec;
-import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.OkHttpClientUtils;
+import org.apache.nifi.security.util.TlsConfiguration;
 import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.util.StringUtils;
-import org.apache.nifi.util.Tuple;
-
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManager;
-import javax.net.ssl.X509TrustManager;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.Proxy;
-import java.net.URL;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.stream.Collectors;
 
 /**
  * A base class for Elasticsearch processors that use the HTTP API
@@ -200,11 +189,11 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
             final Proxy proxy = proxyConfig.createProxy();
             okHttpClient.proxy(proxy);
 
-            if (proxyConfig.hasCredential()){
+            if (proxyConfig.hasCredential()) {
                 okHttpClient.proxyAuthenticator(new Authenticator() {
                     @Override
                     public Request authenticate(Route route, Response response) throws IOException {
-                        final String credential=Credentials.basic(proxyConfig.getProxyUserName(), proxyConfig.getProxyUserPassword());
+                        final String credential = Credentials.basic(proxyConfig.getProxyUserName(), proxyConfig.getProxyUserPassword());
                         return response.request().newBuilder()
                                 .header("Proxy-Authorization", credential)
                                 .build();
@@ -213,35 +202,15 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
             }
         }
 
-
         // Set timeouts
         okHttpClient.connectTimeout((context.getProperty(CONNECT_TIMEOUT).evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS).intValue()), TimeUnit.MILLISECONDS);
         okHttpClient.readTimeout(context.getProperty(RESPONSE_TIMEOUT).evaluateAttributeExpressions().asTimePeriod(TimeUnit.MILLISECONDS).intValue(), TimeUnit.MILLISECONDS);
 
+        // Apply the TLS configuration if present
         final SSLContextService sslService = context.getProperty(PROP_SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.NONE);
-
-        // check if the ssl context is set and add the factory if so
-        if (sslContext != null) {
-            try {
-                Tuple<SSLContext, TrustManager[]> sslContextTuple = SslContextFactory.createTrustSslContextWithTrustManagers(
-                        sslService.getKeyStoreFile(),
-                        sslService.getKeyStorePassword() != null ? sslService.getKeyStorePassword().toCharArray() : null,
-                        sslService.getKeyPassword() != null ? sslService.getKeyPassword().toCharArray() : null,
-                        sslService.getKeyStoreType(),
-                        sslService.getTrustStoreFile(),
-                        sslService.getTrustStorePassword() != null ? sslService.getTrustStorePassword().toCharArray() : null,
-                        sslService.getTrustStoreType(),
-                        SslContextFactory.ClientAuth.WANT,
-                        sslService.getSslAlgorithm()
-                );
-                List<X509TrustManager> x509TrustManagers = Arrays.stream(sslContextTuple.getValue())
-                        .filter(trustManager -> trustManager instanceof X509TrustManager)
-                        .map(trustManager -> (X509TrustManager) trustManager).collect(Collectors.toList());
-                okHttpClient.sslSocketFactory(sslContextTuple.getKey().getSocketFactory(), x509TrustManagers.get(0));
-            } catch (CertificateException | UnrecoverableKeyException | NoSuchAlgorithmException | KeyStoreException | KeyManagementException | IOException e) {
-                throw new ProcessException(e);
-            }
+        if (sslService != null) {
+            final TlsConfiguration tlsConfiguration = sslService.createTlsConfiguration();
+            OkHttpClientUtils.applyTlsToOkHttpClientBuilder(tlsConfiguration, okHttpClient);
         }
 
         okHttpClientAtomicReference.set(okHttpClient.build());
@@ -250,7 +219,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
     @Override
     protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
         List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
-        if(validationContext.getProperty(PROXY_HOST).isSet() != validationContext.getProperty(PROXY_PORT).isSet()) {
+        if (validationContext.getProperty(PROXY_HOST).isSet() != validationContext.getProperty(PROXY_PORT).isSet()) {
             results.add(new ValidationResult.Builder()
                     .valid(false)
                     .explanation("Proxy Host and Proxy Port must be both set or empty")
@@ -286,7 +255,7 @@ public abstract class AbstractElasticsearchHttpProcessor extends AbstractElastic
             throw new IllegalArgumentException("Elasticsearch REST API verb not supported by this processor: " + verb);
         }
 
-        if(!StringUtils.isEmpty(username) && !StringUtils.isEmpty(password)) {
+        if (!StringUtils.isEmpty(username) && !StringUtils.isEmpty(password)) {
             String credential = Credentials.basic(username, password);
             requestBuilder = requestBuilder.header("Authorization", credential);
         }
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
index bed09b1..b443fd4 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
@@ -26,11 +26,9 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
-
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLSocket;
 import javax.net.ssl.SSLSocketFactory;
-
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.TriggerSerially;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
@@ -51,6 +49,7 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.processors.email.smtp.SmtpConsumer;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.RestrictedSSLContextService;
 import org.apache.nifi.ssl.SSLContextService;
 import org.springframework.util.StringUtils;
@@ -134,7 +133,7 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
             .displayName("Client Auth")
             .description("The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.")
             .required(false)
-            .allowableValues(SSLContextService.ClientAuth.NONE.toString(), SSLContextService.ClientAuth.REQUIRED.toString())
+            .allowableValues(SslContextFactory.ClientAuth.NONE.name(), SslContextFactory.ClientAuth.REQUIRED.name())
             .build();
 
     protected static final PropertyDescriptor SMTP_HOSTNAME = new PropertyDescriptor.Builder()
@@ -250,12 +249,12 @@ public class ListenSMTP extends AbstractSessionFactoryProcessor {
             public SSLSocket createSSLSocket(Socket socket) throws IOException {
                 InetSocketAddress remoteAddress = (InetSocketAddress) socket.getRemoteSocketAddress();
                 String clientAuth = context.getProperty(CLIENT_AUTH).getValue();
-                SSLContext sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.valueOf(clientAuth));
+                SSLContext sslContext = sslContextService.createSSLContext(SslContextFactory.ClientAuth.valueOf(clientAuth));
                 SSLSocketFactory socketFactory = sslContext.getSocketFactory();
                 SSLSocket sslSocket = (SSLSocket) (socketFactory.createSocket(socket, remoteAddress.getHostName(), socket.getPort(), true));
                 sslSocket.setUseClientMode(false);
 
-                if (SSLContextService.ClientAuth.REQUIRED.toString().equals(clientAuth)) {
+                if (SslContextFactory.ClientAuth.REQUIRED.toString().equals(clientAuth)) {
                     this.setRequireTLS(true);
                     sslSocket.setNeedClientAuth(true);
                 }
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestListenSMTP.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestListenSMTP.java
index f092ce3..325bfaf 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestListenSMTP.java
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestListenSMTP.java
@@ -22,11 +22,10 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.commons.mail.Email;
-import org.apache.commons.mail.EmailException;
 import org.apache.commons.mail.SimpleEmail;
 import org.apache.nifi.remote.io.socket.NetworkUtils;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.ssl.StandardRestrictedSSLContextService;
 import org.apache.nifi.ssl.StandardSSLContextService;
@@ -51,7 +50,7 @@ public class TestListenSMTP {
     }
 
     @Test
-    public void validateSuccessfulInteraction() throws Exception, EmailException {
+    public void validateSuccessfulInteraction() throws Exception {
         int port = NetworkUtils.availablePort();
 
         TestRunner runner = TestRunners.newTestRunner(ListenSMTP.class);
@@ -90,7 +89,8 @@ public class TestListenSMTP {
     }
 
     @Test
-    public void validateSuccessfulInteractionWithTls() throws Exception, EmailException {
+    public void validateSuccessfulInteractionWithTls() throws Exception {
+        // TODO: Setting system properties without cleaning/restoring at the end of a test is an anti-pattern and can have side effects
         System.setProperty("mail.smtp.ssl.trust", "*");
         System.setProperty("javax.net.ssl.keyStore", "src/test/resources/keystore.jks");
         System.setProperty("javax.net.ssl.keyStorePassword", "passwordpassword");
@@ -113,7 +113,7 @@ public class TestListenSMTP {
 
         // and add the SSL context to the runner
         runner.setProperty(ListenSMTP.SSL_CONTEXT_SERVICE, "ssl-context");
-        runner.setProperty(ListenSMTP.CLIENT_AUTH, SSLContextService.ClientAuth.NONE.name());
+        runner.setProperty(ListenSMTP.CLIENT_AUTH, SslContextFactory.ClientAuth.NONE.name());
         runner.assertValid();
 
         int messageCount = 5;
@@ -152,7 +152,7 @@ public class TestListenSMTP {
     }
 
     @Test
-    public void validateTooLargeMessage() throws Exception, EmailException {
+    public void validateTooLargeMessage() throws Exception {
         int port = NetworkUtils.availablePort();
 
         TestRunner runner = TestRunners.newTestRunner(ListenSMTP.class);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml
index b28f4d8..40657cb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/pom.xml
@@ -45,10 +45,6 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-framework-core-api</artifactId>
         </dependency>
         
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java
index c588a68..bc81cca 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java
@@ -16,6 +16,21 @@
  */
 package org.apache.nifi.cluster.protocol.impl;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.Socket;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.stream.Collectors;
+import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.net.ssl.SSLSession;
+import javax.net.ssl.SSLSocket;
 import org.apache.nifi.cluster.protocol.NodeIdentifier;
 import org.apache.nifi.cluster.protocol.ProtocolContext;
 import org.apache.nifi.cluster.protocol.ProtocolException;
@@ -24,10 +39,10 @@ import org.apache.nifi.cluster.protocol.ProtocolListener;
 import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller;
 import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller;
 import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage;
-import org.apache.nifi.cluster.protocol.message.OffloadMessage;
 import org.apache.nifi.cluster.protocol.message.DisconnectMessage;
 import org.apache.nifi.cluster.protocol.message.FlowRequestMessage;
 import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
+import org.apache.nifi.cluster.protocol.message.OffloadMessage;
 import org.apache.nifi.cluster.protocol.message.ProtocolMessage;
 import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage;
 import org.apache.nifi.events.BulletinFactory;
@@ -41,25 +56,8 @@ import org.apache.nifi.util.StopWatch;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.net.ssl.SSLPeerUnverifiedException;
-import javax.net.ssl.SSLSession;
-import javax.net.ssl.SSLSocket;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.Socket;
-import java.security.cert.Certificate;
-import java.security.cert.CertificateException;
-import java.security.cert.X509Certificate;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.stream.Collectors;
-
 /**
  * Implements a listener for protocol messages sent over unicast socket.
- *
  */
 public class SocketProtocolListener extends SocketListener implements ProtocolListener {
 
@@ -68,6 +66,9 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
     private final Collection<ProtocolHandler> handlers = new CopyOnWriteArrayList<>();
     private volatile BulletinRepository bulletinRepository;
 
+    private static int EXCEPTION_THRESHOLD_MILLIS = 10_000;
+    private volatile long tlsErrorLastSeen = -1;
+
     public SocketProtocolListener(
             final int numThreads,
             final int port,
@@ -190,17 +191,56 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
             final NodeIdentifier nodeId = getNodeIdentifier(request);
             final String from = nodeId == null ? hostname : nodeId.toString();
             logger.info("Finished processing request {} (type={}, length={} bytes) from {} in {}",
-                requestId, request.getType(), countingIn.getBytesRead(), from, stopWatch.getDuration());
+                    requestId, request.getType(), countingIn.getBytesRead(), from, stopWatch.getDuration());
         } catch (final IOException | ProtocolException e) {
-            logger.warn("Failed processing protocol message from " + hostname + " due to " + e, e);
-
-            if (bulletinRepository != null) {
-                final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", String.format("Failed to process protocol message from %s due to: %s", hostname, e.toString()));
-                bulletinRepository.addBulletin(bulletin);
+            String msg = "Failed processing protocol message from " + hostname + " due to ";
+            // Suppress repeated TLS errors
+            if (CertificateUtils.isTlsError(e)) {
+                boolean printedAsWarning = handleTlsError(msg, e);
+
+                // TODO: Move into handleTlsError and refactor shared behavior
+                // If the error was printed as a warning, reset the last seen timer
+                if (printedAsWarning) {
+                    tlsErrorLastSeen = System.currentTimeMillis();
+                }
+            } else {
+                logger.warn(msg + e, e);
+                publishBulletinWarning(msg + e);
             }
         }
     }
 
+    private boolean handleTlsError(String msg, Throwable e) {
+        final String populatedMessage = msg + e.getLocalizedMessage();
+        if (tlsErrorRecentlySeen()) {
+            logger.debug(populatedMessage);
+            return false;
+        } else {
+            logger.warn(populatedMessage);
+            publishBulletinWarning(populatedMessage);
+            return true;
+        }
+    }
+
+    private void publishBulletinWarning(String message) {
+        if (bulletinRepository != null) {
+            final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", message);
+            bulletinRepository.addBulletin(bulletin);
+        }
+    }
+
+    /**
+     * Returns {@code true} if any related exception (determined by {@link CertificateUtils#isTlsError(Throwable)}) has occurred within the last
+     * {@link #EXCEPTION_THRESHOLD_MILLIS} milliseconds. Does not evaluate the error locally,
+     * simply checks the last time the timestamp was updated.
+     *
+     * @return true if the time since the last similar exception occurred is below the threshold
+     */
+    private boolean tlsErrorRecentlySeen() {
+        long now = System.currentTimeMillis();
+        return now - tlsErrorLastSeen < EXCEPTION_THRESHOLD_MILLIS;
+    }
+
     private NodeIdentifier getNodeIdentifier(final ProtocolMessage message) {
         if (message == null) {
             return null;
@@ -247,8 +287,8 @@ public class SocketProtocolListener extends SocketListener implements ProtocolLi
         cert.checkValidity();
 
         final Set<String> identities = CertificateUtils.getSubjectAlternativeNames(cert).stream()
-            .map(CertificateUtils::extractUsername)
-            .collect(Collectors.toSet());
+                .map(CertificateUtils::extractUsername)
+                .collect(Collectors.toSet());
 
         return identities;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java
index 1f38d8e..b7de635 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java
@@ -16,14 +16,13 @@
  */
 package org.apache.nifi.cluster.protocol.spring;
 
-import org.apache.nifi.io.socket.SSLContextFactory;
+import java.util.concurrent.TimeUnit;
 import org.apache.nifi.io.socket.ServerSocketConfiguration;
+import org.apache.nifi.security.util.TlsConfiguration;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.springframework.beans.factory.FactoryBean;
 
-import java.util.concurrent.TimeUnit;
-
 /**
  * Factory bean for creating a singleton ServerSocketConfiguration instance.
  */
@@ -38,11 +37,14 @@ public class ServerSocketConfigurationFactoryBean implements FactoryBean<ServerS
             configuration = new ServerSocketConfiguration();
             configuration.setNeedClientAuth(true);
 
-            final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterNodeReadTimeout(), TimeUnit.MILLISECONDS);
+            final int timeout = (int) FormatUtils.getPreciseTimeDuration(properties.getClusterNodeReadTimeout(), TimeUnit.MILLISECONDS);
             configuration.setSocketTimeout(timeout);
             configuration.setReuseAddress(true);
-            if (Boolean.valueOf(properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_IS_SECURE))) {
-                configuration.setSSLContextFactory(new SSLContextFactory(properties));
+
+            // If the cluster protocol is marked as secure
+            if (Boolean.parseBoolean(properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_IS_SECURE))) {
+                // Parse the TLS configuration from the properties
+                configuration.setTlsConfiguration(TlsConfiguration.fromNiFiProperties(properties));
             }
         }
         return configuration;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java
index 6128329..5458f1e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java
@@ -16,14 +16,13 @@
  */
 package org.apache.nifi.cluster.protocol.spring;
 
-import org.apache.nifi.io.socket.SSLContextFactory;
+import java.util.concurrent.TimeUnit;
 import org.apache.nifi.io.socket.SocketConfiguration;
+import org.apache.nifi.security.util.TlsConfiguration;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.springframework.beans.factory.FactoryBean;
 
-import java.util.concurrent.TimeUnit;
-
 /**
  * Factory bean for creating a singleton SocketConfiguration instance.
  */
@@ -38,11 +37,14 @@ public class SocketConfigurationFactoryBean implements FactoryBean<SocketConfigu
         if (configuration == null) {
             configuration = new SocketConfiguration();
 
-            final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterNodeReadTimeout(), TimeUnit.MILLISECONDS);
+            final int timeout = (int) FormatUtils.getPreciseTimeDuration(properties.getClusterNodeReadTimeout(), TimeUnit.MILLISECONDS);
             configuration.setSocketTimeout(timeout);
             configuration.setReuseAddress(true);
-            if (Boolean.valueOf(properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_IS_SECURE))) {
-                configuration.setSSLContextFactory(new SSLContextFactory(properties));
+
+            // If the cluster protocol is marked as secure
+            if (Boolean.parseBoolean(properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_IS_SECURE))) {
+                // Parse the TLS configuration from the properties
+                configuration.setTlsConfiguration(TlsConfiguration.fromNiFiProperties(properties));
             }
         }
         return configuration;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java
index a123d81..6f88b37 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClient.java
@@ -17,9 +17,6 @@
 
 package org.apache.nifi.cluster.coordination.http.replication.okhttp;
 
-import static org.apache.nifi.security.util.SslContextFactory.ClientAuth.WANT;
-import static org.apache.nifi.security.util.SslContextFactory.createTrustSslContextWithTrustManagers;
-
 import com.fasterxml.jackson.annotation.JsonInclude.Include;
 import com.fasterxml.jackson.annotation.JsonInclude.Value;
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -30,12 +27,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.URI;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -46,10 +37,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import java.util.zip.GZIPInputStream;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLSocketFactory;
-import javax.net.ssl.TrustManager;
-import javax.net.ssl.X509TrustManager;
 import javax.ws.rs.HttpMethod;
 import javax.ws.rs.core.MultivaluedHashMap;
 import javax.ws.rs.core.MultivaluedMap;
@@ -65,12 +52,12 @@ import okhttp3.RequestBody;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.cluster.coordination.http.replication.HttpReplicationClient;
 import org.apache.nifi.cluster.coordination.http.replication.PreparedRequest;
-import org.apache.nifi.framework.security.util.SslContextFactory;
 import org.apache.nifi.remote.protocol.http.HttpHeaders;
+import org.apache.nifi.security.util.OkHttpClientUtils;
+import org.apache.nifi.security.util.TlsConfiguration;
 import org.apache.nifi.stream.io.GZIPOutputStream;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
-import org.apache.nifi.util.Tuple;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.util.StreamUtils;
@@ -318,9 +305,9 @@ public class OkHttpReplicationClient implements HttpReplicationClient {
 
     private OkHttpClient createOkHttpClient(final NiFiProperties properties) {
         final String connectionTimeout = properties.getClusterNodeConnectionTimeout();
-        final long connectionTimeoutMs = FormatUtils.getTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS);
+        final long connectionTimeoutMs = (long) FormatUtils.getPreciseTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS);
         final String readTimeout = properties.getClusterNodeReadTimeout();
-        final long readTimeoutMs = FormatUtils.getTimeDuration(readTimeout, TimeUnit.MILLISECONDS);
+        final long readTimeoutMs = (long) FormatUtils.getPreciseTimeDuration(readTimeout, TimeUnit.MILLISECONDS);
 
         OkHttpClient.Builder okHttpClientBuilder = new OkHttpClient().newBuilder();
         okHttpClientBuilder.connectTimeout(connectionTimeoutMs, TimeUnit.MILLISECONDS);
@@ -329,42 +316,16 @@ public class OkHttpReplicationClient implements HttpReplicationClient {
         final int connectionPoolSize = properties.getClusterNodeMaxConcurrentRequests();
         okHttpClientBuilder.connectionPool(new ConnectionPool(connectionPoolSize, 5, TimeUnit.MINUTES));
 
-        final Tuple<SSLSocketFactory, X509TrustManager> tuple = createSslSocketFactory(properties);
-        if (tuple != null) {
-            okHttpClientBuilder.sslSocketFactory(tuple.getKey(), tuple.getValue());
-            tlsConfigured = true;
+        // Apply the TLS configuration, if present
+        try {
+            TlsConfiguration tlsConfiguration = TlsConfiguration.fromNiFiProperties(properties);
+            tlsConfigured = OkHttpClientUtils.applyTlsToOkHttpClientBuilder(tlsConfiguration, okHttpClientBuilder);
+        } catch (Exception e) {
+            // Legacy expectations around this client are that it does not throw an exception on invalid TLS configuration
+            // TODO: The only current use of this class is ThreadPoolRequestReplicatorFactoryBean#getObject() which should be evaluated to see if that can change
+            tlsConfigured = false;
         }
 
         return okHttpClientBuilder.build();
     }
-
-    private Tuple<SSLSocketFactory, X509TrustManager> createSslSocketFactory(final NiFiProperties properties) {
-        final SSLContext sslContext = SslContextFactory.createSslContext(properties);
-
-        if (sslContext == null) {
-            return null;
-        }
-
-        try {
-            Tuple<SSLContext, TrustManager[]> sslContextTuple = createTrustSslContextWithTrustManagers(
-                    properties.getProperty(NiFiProperties.SECURITY_KEYSTORE),
-                    StringUtils.isNotBlank(properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD)) ? properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD).toCharArray() : null,
-                    StringUtils.isNotBlank(properties.getProperty(NiFiProperties.SECURITY_KEY_PASSWD)) ? properties.getProperty(NiFiProperties.SECURITY_KEY_PASSWD).toCharArray() : null,
-                    properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE),
-                    properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE),
-                    StringUtils.isNotBlank(properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD)) ? properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD).toCharArray() : null,
-                    properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE),
-                    WANT,
-                    sslContext.getProtocol());
-            List<X509TrustManager> x509TrustManagers = Arrays.stream(sslContextTuple.getValue())
-                    .filter(trustManager -> trustManager instanceof X509TrustManager)
-                    .map(trustManager -> (X509TrustManager) trustManager).collect(Collectors.toList());
-            return new Tuple<>(sslContextTuple.getKey().getSocketFactory(), x509TrustManagers.get(0));
-        } catch(UnrecoverableKeyException e) {
-            logger.error("Key password may be incorrect or not set. Check your keystore passwords." + e.getMessage());
-            return null;
-        } catch (CertificateException | NoSuchAlgorithmException | KeyStoreException | KeyManagementException | IOException e) {
-            return null;
-        }
-    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
index 6c1d23d..efd4813 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/groovy/org/apache/nifi/cluster/coordination/http/replication/okhttp/OkHttpReplicationClientTest.groovy
@@ -212,7 +212,7 @@ class OkHttpReplicationClientTest extends GroovyTestCase {
     }
 
     @Test
-    void testShouldFailIfKeyPasswordIsBlankAndKeystorePassword() {
+    void testShouldFailIfKeyPasswordAndKeystorePasswordAreBlank() {
         // Arrange
         Map propsMap = [
                 (NiFiProperties.SECURITY_TRUSTSTORE)       : "./src/test/resources/conf/truststore.jks",
@@ -242,13 +242,13 @@ class OkHttpReplicationClientTest extends GroovyTestCase {
                         (NiFiProperties.WEB_HTTPS_PORT): "51552",]
 
         Map tlsPropsMap = [
-                (NiFiProperties.SECURITY_TRUSTSTORE)       : "./src/test/resources/conf/truststore.jks",
-                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : "JKS",
-                (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
                 (NiFiProperties.SECURITY_KEYSTORE)         : "./src/test/resources/conf/keystore.jks",
-                (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : "JKS",
                 (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : "passwordpassword",
                 (NiFiProperties.SECURITY_KEY_PASSWD)       : "",
+                (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : "JKS",
+                (NiFiProperties.SECURITY_TRUSTSTORE)       : "./src/test/resources/conf/truststore.jks",
+                (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): "passwordpassword",
+                (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : "JKS",
         ] + propsMap
 
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java
index b97e495..6dc5079 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java
@@ -36,13 +36,11 @@ import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import javax.ws.rs.HttpMethod;
 import javax.ws.rs.ProcessingException;
 import javax.ws.rs.core.MultivaluedHashMap;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status;
-
 import org.apache.nifi.authorization.user.NiFiUser;
 import org.apache.nifi.authorization.user.NiFiUserDetails;
 import org.apache.nifi.authorization.user.NiFiUserUtils;
@@ -232,7 +230,7 @@ public class TestThreadPoolRequestReplicator {
         when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenReturn(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED));
 
         final AtomicInteger requestCount = new AtomicInteger(0);
-        final NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null, null);
+        final NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null);
 
         final MockReplicationClient client = new MockReplicationClient();
         final RequestCompletionCallback requestCompletionCallback = (uri, method, responses) -> {
@@ -309,7 +307,7 @@ public class TestThreadPoolRequestReplicator {
         nodeMap.put(NodeConnectionState.CONNECTING, otherState);
 
         when(coordinator.getConnectionStates()).thenReturn(nodeMap);
-        final NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null, null);
+        final NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null);
 
         final MockReplicationClient client = new MockReplicationClient();
         final RequestCompletionCallback requestCompletionCallback = (uri, method, responses) -> {
@@ -372,7 +370,7 @@ public class TestThreadPoolRequestReplicator {
 
         final ClusterCoordinator coordinator = createClusterCoordinator();
         final AtomicInteger requestCount = new AtomicInteger(0);
-        final NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null, null);
+        final NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null);
 
         final MockReplicationClient client = new MockReplicationClient();
         final RequestCompletionCallback requestCompletionCallback = (uri, method, responses) -> {
@@ -587,7 +585,7 @@ public class TestThreadPoolRequestReplicator {
 
     private void withReplicator(final WithReplicator function, final Status status, final long delayMillis, final RuntimeException failure, final String expectedRequestChain) {
         final ClusterCoordinator coordinator = createClusterCoordinator();
-        final NiFiProperties nifiProps = NiFiProperties.createBasicNiFiProperties(null, null);
+        final NiFiProperties nifiProps = NiFiProperties.createBasicNiFiProperties(null);
         final MockReplicationClient client = new MockReplicationClient();
         final RequestCompletionCallback requestCompletionCallback = (uri, method, responses) -> {
         };
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/logback-test.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/logback-test.xml
index 79db284..88c8105 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/logback-test.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/resources/logback-test.xml
@@ -23,6 +23,8 @@
     
     <!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
     <logger name="org.apache.nifi" level="INFO"/>
+    <logger name="org.apache.nifi.security.util" level="DEBUG"/>
+    <logger name="org.apache.nifi.cluster.coordination.http.replication.okhttp" level="DEBUG"/>
     <logger name="org.apache.nifi.engine.FlowEngine" level="OFF" />
     <logger name="org.apache.nifi.cluster.coordination.node" level="DEBUG" />
     <logger name="org.apache.curator.framework.recipes.leader.LeaderSelector" level="OFF" />
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
index 09fbe7b..f8fced1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
@@ -114,10 +114,6 @@
             <artifactId>jersey-client</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security</artifactId>
-        </dependency>
-        <dependency>
             <groupId>org.springframework</groupId>
             <artifactId>spring-core</artifactId>
         </dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index d7c992a..9a68658 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -169,7 +169,6 @@ import org.apache.nifi.events.BulletinFactory;
 import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.framework.security.util.SslContextFactory;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.groups.StandardProcessGroup;
@@ -208,6 +207,9 @@ import org.apache.nifi.reporting.Severity;
 import org.apache.nifi.reporting.StandardEventAccess;
 import org.apache.nifi.reporting.UserAwareEventAccess;
 import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
 import org.apache.nifi.services.FlowService;
 import org.apache.nifi.stream.io.LimitingInputStream;
 import org.apache.nifi.stream.io.StreamUtils;
@@ -223,39 +225,6 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.management.NotificationEmitter;
-import javax.net.ssl.SSLContext;
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.management.GarbageCollectorMXBean;
-import java.lang.management.ManagementFactory;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import java.util.stream.Collectors;
-
-import static java.util.Objects.requireNonNull;
-
 public class FlowController implements ReportingTaskProvider, Authorizable, NodeTypeProvider {
 
     // default repository implementations
@@ -490,7 +459,6 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         this.nifiProperties = nifiProperties;
         this.heartbeatMonitor = heartbeatMonitor;
         this.leaderElectionManager = leaderElectionManager;
-        this.sslContext = SslContextFactory.createSslContext(nifiProperties);
         this.extensionManager = extensionManager;
         this.clusterCoordinator = clusterCoordinator;
         this.authorizer = authorizer;
@@ -498,6 +466,15 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
         this.configuredForClustering = configuredForClustering;
         this.flowRegistryClient = flowRegistryClient;
 
+        try {
+            // Form the container object from the properties
+            TlsConfiguration tlsConfiguration = TlsConfiguration.fromNiFiProperties(nifiProperties);
+            this.sslContext = SslContextFactory.createSslContext(tlsConfiguration);
+        } catch (TlsException e) {
+            LOG.error("Unable to start the flow controller because the TLS configuration was invalid: {}", e.getLocalizedMessage());
+            throw new IllegalStateException("Flow controller TLS configuration is invalid", e);
+        }
+
         timerDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxTimerDrivenThreads.get(), "Timer-Driven Process"));
         eventDrivenEngineRef = new AtomicReference<>(new FlowEngine(maxEventDrivenThreads.get(), "Event-Driven Process"));
 
@@ -635,7 +612,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
                 zooKeeperStateServer = ZooKeeperStateServer.create(nifiProperties);
                 zooKeeperStateServer.start();
             } catch (final IOException | ConfigException e) {
-                throw new IllegalStateException("Unable to initailize Flow because NiFi was configured to start an Embedded Zookeeper server but failed to do so", e);
+                throw new IllegalStateException("Unable to initialize Flow because NiFi was configured to start an Embedded Zookeeper server but failed to do so", e);
             }
         } else {
             zooKeeperStateServer = null;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/server/ConnectionLoadBalanceServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/server/ConnectionLoadBalanceServer.java
index 920e08a..b2f91ce 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/server/ConnectionLoadBalanceServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/server/ConnectionLoadBalanceServer.java
@@ -17,13 +17,6 @@
 
 package org.apache.nifi.controller.queue.clustered.server;
 
-import org.apache.nifi.events.EventReporter;
-import org.apache.nifi.reporting.Severity;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLServerSocket;
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.IOException;
@@ -38,6 +31,14 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicLong;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLPeerUnverifiedException;
+import javax.net.ssl.SSLServerSocket;
+import org.apache.nifi.events.EventReporter;
+import org.apache.nifi.reporting.Severity;
+import org.apache.nifi.security.util.CertificateUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ConnectionLoadBalanceServer {
     private static final Logger logger = LoggerFactory.getLogger(ConnectionLoadBalanceServer.class);
@@ -113,24 +114,31 @@ public class ConnectionLoadBalanceServer {
         if (sslContext == null) {
             return new ServerSocket(port, 50, InetAddress.getByName(hostname));
         } else {
-            final ServerSocket serverSocket = sslContext.getServerSocketFactory().createServerSocket(port, 50, inetAddress);
-            ((SSLServerSocket) serverSocket).setNeedClientAuth(true);
+            final SSLServerSocket serverSocket = (SSLServerSocket) sslContext.getServerSocketFactory().createServerSocket(port, 50, inetAddress);
+            serverSocket.setNeedClientAuth(true);
+            // Enforce custom protocols on socket
+            serverSocket.setEnabledProtocols(CertificateUtils.getCurrentSupportedTlsProtocolVersions());
             return serverSocket;
         }
     }
 
-
-    private class CommunicateAction implements Runnable {
+    // Use a static nested class and pass the ER in the constructor to avoid instantiation issues in tests
+    protected static class CommunicateAction implements Runnable {
         private final LoadBalanceProtocol loadBalanceProtocol;
         private final Socket socket;
         private final InputStream in;
         private final OutputStream out;
+        private final EventReporter eventReporter;
 
         private volatile boolean stopped = false;
 
-        public CommunicateAction(final LoadBalanceProtocol loadBalanceProtocol, final Socket socket) throws IOException {
+        private static int EXCEPTION_THRESHOLD_MILLIS = 10_000;
+        private volatile long tlsErrorLastSeen = -1;
+
+        public CommunicateAction(final LoadBalanceProtocol loadBalanceProtocol, final Socket socket, final EventReporter eventReporter) throws IOException {
             this.loadBalanceProtocol = loadBalanceProtocol;
             this.socket = socket;
+            this.eventReporter = eventReporter;
 
             this.in = new BufferedInputStream(socket.getInputStream());
             this.out = new BufferedOutputStream(socket.getOutputStream());
@@ -164,12 +172,59 @@ public class ConnectionLoadBalanceServer {
                         }
                     }
 
-                    logger.error("Failed to communicate with Peer {}", peerDescription, e);
-                    eventReporter.reportEvent(Severity.ERROR, "Load Balanced Connection", "Failed to receive FlowFiles for Load Balancing due to " + e);
+                    /* The exceptions can fill the log very quickly and make it difficult to use. SSLPeerUnverifiedExceptions
+                    especially repeat and have a long stacktrace, and are not likely to be resolved instantaneously. Suppressing
+                    them for a period of time is helpful */
+                    if (CertificateUtils.isTlsError(e)) {
+                        handleTlsError(peerDescription, e);
+                    } else {
+                        logger.error("Failed to communicate with Peer {}", peerDescription, e);
+                        eventReporter.reportEvent(Severity.ERROR, "Load Balanced Connection", "Failed to receive FlowFiles for Load Balancing due to " + e);
+                    }
                     return;
                 }
             }
         }
+
+        /**
+         * Determines how to record the TLS-related error
+         * ({@link org.apache.nifi.security.util.TlsException}, {@link SSLPeerUnverifiedException},
+         * {@link java.security.cert.CertificateException}, etc.) to the log, based on how recently it was last seen.
+         *
+         * @param peerDescription the peer's String representation for the log message
+         * @param e               the exception
+         * @return true if the error was printed at ERROR severity and reported to the event reporter
+         */
+        private boolean handleTlsError(String peerDescription, Throwable e) {
+            final String populatedMessage = "Failed to communicate with Peer " + peerDescription + " due to " + e.getLocalizedMessage();
+            // If the exception has been seen recently, log as debug
+            if (tlsErrorRecentlySeen()) {
+                logger.debug(populatedMessage);
+                return false;
+            } else {
+                // If this is the first exception in X seconds, log as error
+                logger.error(populatedMessage);
+                logger.info("\tPrinted above error because it has been {} ms since the last printing", System.currentTimeMillis() - tlsErrorLastSeen);
+                eventReporter.reportEvent(Severity.ERROR, "Load Balanced Connection", populatedMessage);
+
+                // Reset the timer
+                tlsErrorLastSeen = System.currentTimeMillis();
+                return true;
+            }
+        }
+
+
+        /**
+         * Returns {@code true} if any related exception (determined by {@link CertificateUtils#isTlsError(Throwable)}) has occurred within the last
+         * {@link #EXCEPTION_THRESHOLD_MILLIS} milliseconds. Does not evaluate the error locally,
+         * simply checks the last time the timestamp was updated.
+         *
+         * @return true if the time since the last similar exception occurred is below the threshold
+         */
+        private boolean tlsErrorRecentlySeen() {
+            long now = System.currentTimeMillis();
+            return now - tlsErrorLastSeen < EXCEPTION_THRESHOLD_MILLIS;
+        }
     }
 
 
@@ -204,7 +259,7 @@ public class ConnectionLoadBalanceServer {
 
                     socket.setSoTimeout(connectionTimeoutMillis);
 
-                    final CommunicateAction communicateAction = new CommunicateAction(loadBalanceProtocol, socket);
+                    final CommunicateAction communicateAction = new CommunicateAction(loadBalanceProtocol, socket, eventReporter);
                     final Thread commsThread = new Thread(communicateAction);
                     commsThread.setName("Load-Balance Server Thread-" + threadCounter.getAndIncrement());
                     commsThread.start();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java
index 6b0ee95..a8d96a3 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/state/manager/StandardStateManagerProvider.java
@@ -17,8 +17,16 @@
 
 package org.apache.nifi.controller.state.manager;
 
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import javax.net.ssl.SSLContext;
 import org.apache.commons.lang3.ArrayUtils;
-import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.components.PropertyDescriptor;
@@ -37,30 +45,23 @@ import org.apache.nifi.controller.state.StandardStateManager;
 import org.apache.nifi.controller.state.StandardStateProviderInitializationContext;
 import org.apache.nifi.controller.state.config.StateManagerConfiguration;
 import org.apache.nifi.controller.state.config.StateProviderConfiguration;
-import org.apache.nifi.framework.security.util.SslContextFactory;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.nar.ExtensionManager;
 import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.parameter.ExpressionLanguageAwareParameterParser;
+import org.apache.nifi.parameter.ParameterLookup;
 import org.apache.nifi.parameter.ParameterParser;
 import org.apache.nifi.parameter.ParameterTokenList;
-import org.apache.nifi.parameter.ExpressionLanguageAwareParameterParser;
 import org.apache.nifi.processor.SimpleProcessLogger;
 import org.apache.nifi.processor.StandardValidationContext;
 import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.net.ssl.SSLContext;
-import java.io.File;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
 public class StandardStateManagerProvider implements StateManagerProvider{
     private static final Logger logger = LoggerFactory.getLogger(StandardStateManagerProvider.class);
 
@@ -215,7 +216,14 @@ public class StandardStateManagerProvider implements StateManagerProvider{
             propertyMap.put(descriptor, new StandardPropertyValue(entry.getValue(),null, parameterLookup, variableRegistry));
         }
 
-        final SSLContext sslContext = SslContextFactory.createSslContext(properties);
+        final SSLContext sslContext;
+        try {
+            sslContext = SslContextFactory.createSslContext(TlsConfiguration.fromNiFiProperties(properties));
+        } catch (TlsException e) {
+            logger.error("Encountered an error configuring TLS for state manager: ", e);
+            throw new IllegalStateException("Error configuring TLS for state manager", e);
+        }
+
         final ComponentLog logger = new SimpleProcessLogger(providerId, provider);
         final StateProviderInitializationContext initContext = new StandardStateProviderInitializationContext(providerId, propertyMap, sslContext, logger);
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClient.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClient.java
index 4a695fb..265eb2c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClient.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClient.java
@@ -17,15 +17,16 @@
 
 package org.apache.nifi.registry.flow;
 
-import org.apache.nifi.framework.security.util.SslContextFactory;
-import org.apache.nifi.util.NiFiProperties;
-
-import javax.net.ssl.SSLContext;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import javax.net.ssl.SSLContext;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
+import org.apache.nifi.util.NiFiProperties;
 
 public class StandardFlowRegistryClient implements FlowRegistryClient {
     private NiFiProperties nifiProperties;
@@ -75,15 +76,21 @@ public class StandardFlowRegistryClient implements FlowRegistryClient {
 
         final FlowRegistry registry;
         if (uriScheme.equalsIgnoreCase("http") || uriScheme.equalsIgnoreCase("https")) {
-            final SSLContext sslContext = SslContextFactory.createSslContext(nifiProperties);
-            if (sslContext == null && uriScheme.equalsIgnoreCase("https")) {
+            try {
+                final SSLContext sslContext = SslContextFactory.createSslContext(TlsConfiguration.fromNiFiProperties(nifiProperties));
+
+                if (sslContext == null && uriScheme.equalsIgnoreCase("https")) {
+                    throw new IllegalStateException("Failed to create Flow Registry for URI " + registryUrl
+                            + " because this NiFi is not configured with a Keystore/Truststore, so it is not capable of communicating with a secure Registry. "
+                            + "Please populate NiFi's Keystore/Truststore properties or connect to a NiFi Registry over http instead of https.");
+                }
+
+                registry = new RestBasedFlowRegistry(this, registryId, registryBaseUrl, sslContext, registryName);
+                registry.setDescription(description);
+            } catch (TlsException e) {
                 throw new IllegalStateException("Failed to create Flow Registry for URI " + registryUrl
-                    + " because this NiFi is not configured with a Keystore/Truststore, so it is not capable of communicating with a secure Registry. "
-                    + "Please populate NiFi's Keystore/Truststore properties or connect to a NiFi Registry over http instead of https.");
+                        + " because this NiFi instance has an invalid TLS configuration", e);
             }
-
-            registry = new RestBasedFlowRegistry(this, registryId, registryBaseUrl, sslContext, registryName);
-            registry.setDescription(description);
         } else {
             throw new IllegalArgumentException("Cannot create Flow Registry with URI of " + registryUrl
                 + " because there are no known implementations of Flow Registries that can handle URIs of scheme " + uriScheme);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/queue/clustered/server/ConnectionLoadBalanceServerTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/queue/clustered/server/ConnectionLoadBalanceServerTest.groovy
new file mode 100644
index 0000000..90fb5ec
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/groovy/org/apache/nifi/controller/queue/clustered/server/ConnectionLoadBalanceServerTest.groovy
@@ -0,0 +1,197 @@
+/*
+ * 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.queue.clustered.server
+
+import org.apache.nifi.events.EventReporter
+import org.apache.nifi.reporting.Severity
+import org.apache.nifi.security.util.CertificateUtils
+import org.apache.nifi.security.util.KeyStoreUtils
+import org.apache.nifi.security.util.KeystoreType
+import org.apache.nifi.security.util.SslContextFactory
+import org.apache.nifi.security.util.TlsConfiguration
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.junit.After
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import javax.net.ssl.SSLContext
+import javax.net.ssl.SSLPeerUnverifiedException
+import javax.net.ssl.SSLServerSocket
+import java.security.Security
+
+@RunWith(JUnit4.class)
+class ConnectionLoadBalanceServerTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(ConnectionLoadBalanceServerTest.class)
+
+    private static final String KEYSTORE_PATH = "src/test/resources/localhost-ks.jks"
+    private static final String KEYSTORE_PASSWORD = "OI7kMpWzzVNVx/JGhTL/0uO4+PWpGJ46uZ/pfepbkwI"
+    private static final KeystoreType KEYSTORE_TYPE = KeystoreType.JKS
+
+    private static final String TRUSTSTORE_PATH = "src/test/resources/localhost-ts.jks"
+    private static final String TRUSTSTORE_PASSWORD = "wAOR0nQJ2EXvOP0JZ2EaqA/n7W69ILS4sWAHghmIWCc"
+    private static final KeystoreType TRUSTSTORE_TYPE = KeystoreType.JKS
+
+    private static final String HOSTNAME = "localhost"
+    private static final int PORT = 54321
+    private static final int NUM_THREADS = 1
+    private static final int TIMEOUT_MS = 1000
+
+    private static TlsConfiguration tlsConfiguration
+    private static SSLContext sslContext
+
+    private ConnectionLoadBalanceServer lbServer
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+
+        tlsConfiguration = new TlsConfiguration(KEYSTORE_PATH, KEYSTORE_PASSWORD, KEYSTORE_TYPE, TRUSTSTORE_PATH, TRUSTSTORE_PASSWORD, TRUSTSTORE_TYPE)
+        sslContext = SslContextFactory.createSslContext(tlsConfiguration)
+    }
+
+    @Before
+    void setUp() {
+    }
+
+    @After
+    void tearDown() {
+        if (lbServer) {
+            lbServer.stop()
+        }
+    }
+
+    /**
+     * Asserts that the protocol versions in the parameters object are correct. In recent versions of Java, this enforces order as well, but in older versions, it just enforces presence.
+     *
+     * @param enabledProtocols the actual protocols, either in {@code String[]} or {@code Collection<String>} form
+     * @param expectedProtocols the specific protocol versions to be present (ordered as desired)
+     */
+    void assertProtocolVersions(def enabledProtocols, def expectedProtocols) {
+        if (CertificateUtils.getJavaVersion() > 8) {
+            assert enabledProtocols == expectedProtocols as String[]
+        } else {
+            assert enabledProtocols as Set == expectedProtocols as Set
+        }
+    }
+
+    @Test
+    void testRequestPeerListShouldUseTLS() {
+        // Arrange
+        logger.info("Creating SSL Context from TLS Configuration: ${tlsConfiguration}")
+        SSLContext sslContext = SslContextFactory.createSslContext(tlsConfiguration, SslContextFactory.ClientAuth.NONE)
+        logger.info("Created SSL Context: ${KeyStoreUtils.sslContextToString(sslContext)}")
+
+        def mockLBP = [
+                receiveFlowFiles: { Socket s, InputStream i, OutputStream o -> null }
+        ] as LoadBalanceProtocol
+        def mockER = [:] as EventReporter
+
+        lbServer = new ConnectionLoadBalanceServer(HOSTNAME, PORT, sslContext, NUM_THREADS, mockLBP, mockER, TIMEOUT_MS)
+
+        // Act
+        lbServer.start()
+
+        // Assert
+
+        // Assert that the default parameters (which can't be modified) still have legacy protocols and no client auth
+        def defaultSSLParameters = sslContext.defaultSSLParameters
+        logger.info("Default SSL Parameters: ${KeyStoreUtils.sslParametersToString(defaultSSLParameters)}" as String)
+        assertProtocolVersions(defaultSSLParameters.protocols, CertificateUtils.getCurrentSupportedTlsProtocolVersions() + ["TLSv1.1", "TLSv1"])
+        assert !defaultSSLParameters.needClientAuth
+
+        // Assert that the actual socket is set correctly due to the override in the LB server
+        SSLServerSocket socket = lbServer.serverSocket as SSLServerSocket
+        logger.info("Created SSL server socket: ${KeyStoreUtils.sslServerSocketToString(socket)}" as String)
+        assertProtocolVersions(socket.enabledProtocols, CertificateUtils.getCurrentSupportedTlsProtocolVersions())
+        assert socket.needClientAuth
+
+        // Clean up
+        lbServer.stop()
+    }
+
+    @Test
+    void testShouldHandleSSLPeerUnverifiedException() {
+        // Arrange
+        final long testStartMillis = System.currentTimeMillis()
+        final int CONNECTION_ATTEMPTS = 100
+        // If this test takes longer than 3 seconds, it's likely because of external delays, which would invalidate the assertions
+        final long MAX_TEST_DURATION_MILLIS = 3000
+        final String peerDescription = "Test peer"
+        final SSLPeerUnverifiedException e = new SSLPeerUnverifiedException("Test exception")
+
+        InputStream socketInputStream = new ByteArrayInputStream("This is the socket input stream".bytes)
+        OutputStream socketOutputStream = new ByteArrayOutputStream()
+
+        Socket mockSocket = [
+                getInputStream : { -> socketInputStream },
+                getOutputStream: { -> socketOutputStream },
+        ] as Socket
+        LoadBalanceProtocol mockLBProtocol = [
+                receiveFlowFiles: { Socket s, InputStream i, OutputStream o -> null }
+        ] as LoadBalanceProtocol
+        EventReporter mockER = [
+                reportEvent: { Severity s, String c, String m -> logger.mock("${s}: ${c} | ${m}") }
+        ] as EventReporter
+
+        def output = [debug: 0, error: 0]
+
+        ConnectionLoadBalanceServer.CommunicateAction communicateAction = new ConnectionLoadBalanceServer.CommunicateAction(mockLBProtocol, mockSocket, mockER)
+
+        // Override the threshold to 100 ms
+        communicateAction.EXCEPTION_THRESHOLD_MILLIS = 100
+
+        long listenerStart = System.currentTimeMillis()
+
+        // Act
+        CONNECTION_ATTEMPTS.times { int i ->
+            long now = System.currentTimeMillis()
+            logger.debug("Attempting connection ${i + 1} at ${now} [${now - listenerStart}]")
+            boolean printedError = communicateAction.handleTlsError(peerDescription, e)
+            if (printedError) {
+                output.error++
+            } else {
+                output.debug++
+            }
+            sleep(10)
+        }
+        logger.info("After ${CONNECTION_ATTEMPTS} attempts, debug: ${output.debug}, error: ${output.error}")
+
+        // Assert
+        logger.info("output.debug (${output.debug}) > output.error (${output.error}): ${output.debug > output.error}")
+
+        // Only enforce if the test completed in a reasonable amount of time (i.e. external delays did not influence the timing)
+        long testStopMillis = System.currentTimeMillis()
+        long testDurationMillis = testStopMillis - testStartMillis
+        if (testDurationMillis > MAX_TEST_DURATION_MILLIS) {
+            logger.warn("The test took ${testDurationMillis} ms, which is longer than the max duration ${MAX_TEST_DURATION_MILLIS} ms, so the timing may be suspect and the assertion will not be enforced")
+        } else {
+            assert output.debug > output.error
+        }
+
+        // Clean up
+        communicateAction.stop()
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
index df8a126..75b85ea 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
@@ -16,6 +16,14 @@
  */
 package org.apache.nifi.controller;
 
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
 import org.apache.commons.io.IOUtils;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.authorization.Authorizer;
@@ -48,15 +56,6 @@ import org.junit.Ignore;
 import org.junit.Test;
 import org.w3c.dom.Document;
 
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-
 /**
  */
 @Ignore
@@ -80,7 +79,7 @@ public class StandardFlowServiceTest {
 
     @Before
     public void setup() throws Exception {
-        properties = NiFiProperties.createBasicNiFiProperties(null, null);
+        properties = NiFiProperties.createBasicNiFiProperties(null);
 
 
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
index 62e1288..1293f48 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
@@ -17,6 +17,32 @@
 
 package org.apache.nifi.controller;
 
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
@@ -69,33 +95,6 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.io.File;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.hamcrest.CoreMatchers.containsString;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.junit.Assume.assumeTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 public class TestStandardProcessorNode {
 
     private MockVariableRegistry variableRegistry;
@@ -108,7 +107,7 @@ public class TestStandardProcessorNode {
     @Before
     public void setup() {
         variableRegistry = new MockVariableRegistry();
-        niFiProperties = NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties", null);
+        niFiProperties = NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties");
 
         systemBundle = SystemBundle.create(niFiProperties);
         extensionManager = new StandardExtensionDiscoveringManager() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/LoadBalancedQueueIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/LoadBalancedQueueIT.java
index cace897..354135f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/LoadBalancedQueueIT.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/LoadBalancedQueueIT.java
@@ -17,6 +17,42 @@
 
 package org.apache.nifi.controller.queue.clustered;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyCollection;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.security.KeyManagementException;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.SSLContext;
 import org.apache.nifi.cluster.coordination.ClusterCoordinator;
 import org.apache.nifi.cluster.coordination.ClusterTopologyEventListener;
 import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
@@ -56,50 +92,17 @@ import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
 import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
 import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.provenance.ProvenanceRepository;
+import org.apache.nifi.security.util.CertificateUtils;
+import org.apache.nifi.security.util.KeystoreType;
 import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import javax.net.ssl.SSLContext;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyCollection;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 public class LoadBalancedQueueIT {
     private final LoadBalanceAuthorizer ALWAYS_AUTHORIZED = (sslSocket) -> sslSocket == null ? null : "authorized.mydomain.com";
     private final LoadBalanceAuthorizer NEVER_AUTHORIZED = (sslSocket) -> {
@@ -137,7 +140,7 @@ public class LoadBalancedQueueIT {
     private final AtomicReference<LoadBalanceCompression> compressionReference = new AtomicReference<>();
 
     @Before
-    public void setup() throws IOException, UnrecoverableKeyException, CertificateException, NoSuchAlgorithmException, KeyStoreException, KeyManagementException {
+    public void setup() throws IOException, UnrecoverableKeyException, CertificateException, NoSuchAlgorithmException, KeyStoreException, KeyManagementException, TlsException {
         compressionReference.set(LoadBalanceCompression.DO_NOT_COMPRESS);
 
         nodeIdentifiers = new HashSet<>();
@@ -189,9 +192,9 @@ public class LoadBalancedQueueIT {
         final String keyPass = keystorePass;
         final String truststore = "src/test/resources/localhost-ts.jks";
         final String truststorePass = "wAOR0nQJ2EXvOP0JZ2EaqA/n7W69ILS4sWAHghmIWCc";
-        sslContext = SslContextFactory.createSslContext(keystore, keystorePass.toCharArray(), keyPass.toCharArray(), "JKS",
-                truststore, truststorePass.toCharArray(), "JKS",
-                SslContextFactory.ClientAuth.REQUIRED, "TLS");
+        TlsConfiguration tlsConfiguration = new TlsConfiguration(keystore, keystorePass, keyPass, KeystoreType.JKS,
+                truststore, truststorePass, KeystoreType.JKS, CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion());
+        sslContext = SslContextFactory.createSslContext(tlsConfiguration, SslContextFactory.ClientAuth.REQUIRED);
     }
 
 
@@ -262,7 +265,7 @@ public class LoadBalancedQueueIT {
 
                 clusterEventListeners.forEach(listener -> listener.onNodeAdded(nodeId));
 
-                for (int j=0; j < 2; j++) {
+                for (int j = 0; j < 2; j++) {
                     final Map<String, String> attributes = new HashMap<>();
                     attributes.put("greeting", "hello");
 
@@ -531,7 +534,7 @@ public class LoadBalancedQueueIT {
             clientThread.start();
 
             final SocketLoadBalancedFlowFileQueue flowFileQueue = new SocketLoadBalancedFlowFileQueue(queueId, new NopConnectionEventListener(), processScheduler, clientFlowFileRepo, clientProvRepo,
-                clientContentRepo, resourceClaimManager, clusterCoordinator, clientRegistry, flowFileSwapManager, swapThreshold, eventReporter);
+                    clientContentRepo, resourceClaimManager, clusterCoordinator, clientRegistry, flowFileSwapManager, swapThreshold, eventReporter);
             flowFileQueue.setFlowFilePartitioner(new RoundRobinPartitioner());
 
             try {
@@ -602,7 +605,7 @@ public class LoadBalancedQueueIT {
             clientThread.start();
 
             final SocketLoadBalancedFlowFileQueue flowFileQueue = new SocketLoadBalancedFlowFileQueue(queueId, new NopConnectionEventListener(), processScheduler, clientFlowFileRepo, clientProvRepo,
-                clientContentRepo, resourceClaimManager, clusterCoordinator, clientRegistry, flowFileSwapManager, swapThreshold, eventReporter);
+                    clientContentRepo, resourceClaimManager, clusterCoordinator, clientRegistry, flowFileSwapManager, swapThreshold, eventReporter);
             flowFileQueue.setFlowFilePartitioner(new RoundRobinPartitioner());
             flowFileQueue.setLoadBalanceCompression(LoadBalanceCompression.COMPRESS_ATTRIBUTES_ONLY);
 
@@ -692,7 +695,7 @@ public class LoadBalancedQueueIT {
             clientThread.start();
 
             final SocketLoadBalancedFlowFileQueue flowFileQueue = new SocketLoadBalancedFlowFileQueue(queueId, new NopConnectionEventListener(), processScheduler, clientFlowFileRepo, clientProvRepo,
-                clientContentRepo, resourceClaimManager, clusterCoordinator, clientRegistry, flowFileSwapManager, swapThreshold, eventReporter);
+                    clientContentRepo, resourceClaimManager, clusterCoordinator, clientRegistry, flowFileSwapManager, swapThreshold, eventReporter);
             flowFileQueue.setFlowFilePartitioner(new RoundRobinPartitioner());
             flowFileQueue.setLoadBalanceCompression(LoadBalanceCompression.COMPRESS_ATTRIBUTES_AND_CONTENT);
 
@@ -1087,6 +1090,7 @@ public class LoadBalancedQueueIT {
                 public boolean isRebalanceOnClusterResize() {
                     return true;
                 }
+
                 @Override
                 public boolean isRebalanceOnFailure() {
                     return true;
@@ -1242,7 +1246,7 @@ public class LoadBalancedQueueIT {
             clientThread.start();
 
             final SocketLoadBalancedFlowFileQueue flowFileQueue = new SocketLoadBalancedFlowFileQueue(queueId, new NopConnectionEventListener(), processScheduler, clientFlowFileRepo, clientProvRepo,
-                clientContentRepo, resourceClaimManager, clusterCoordinator, clientRegistry, flowFileSwapManager, swapThreshold, eventReporter);
+                    clientContentRepo, resourceClaimManager, clusterCoordinator, clientRegistry, flowFileSwapManager, swapThreshold, eventReporter);
             flowFileQueue.setFlowFilePartitioner(new RoundRobinPartitioner());
 
             try {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
index d16cd5a..f9fad7b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
@@ -16,6 +16,14 @@
  */
 package org.apache.nifi.controller.reporting;
 
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
 import org.apache.commons.io.FileUtils;
 import org.apache.nifi.admin.service.AuditService;
 import org.apache.nifi.authorization.AbstractPolicyBasedAuthorizer;
@@ -47,15 +55,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import java.io.File;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.assertEquals;
-
 public class TestStandardReportingContext {
 
     private static final String DEFAULT_SENSITIVE_PROPS_KEY = "nififtw!";
@@ -74,7 +73,6 @@ public class TestStandardReportingContext {
 
     @Before
     public void setup() {
-
         flowFileEventRepo = Mockito.mock(FlowFileEventRepository.class);
         auditService = Mockito.mock(AuditService.class);
         final Map<String, String> otherProps = new HashMap<>();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
index 5657a11..e47dd51 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
@@ -16,29 +16,17 @@
  */
 package org.apache.nifi.controller.repository;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+
 import ch.qos.logback.classic.Level;
 import ch.qos.logback.classic.Logger;
 import ch.qos.logback.classic.spi.ILoggingEvent;
 import ch.qos.logback.core.read.ListAppender;
-import org.apache.commons.lang3.SystemUtils;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.controller.repository.claim.ResourceClaim;
-import org.apache.nifi.controller.repository.claim.StandardContentClaim;
-import org.apache.nifi.controller.repository.claim.StandardResourceClaim;
-import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
-import org.apache.nifi.controller.repository.util.DiskUtils;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.stream.io.StreamUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Assume;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.slf4j.LoggerFactory;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -62,13 +50,24 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNotSame;
-import static org.junit.Assert.assertTrue;
+import org.apache.commons.lang3.SystemUtils;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaim;
+import org.apache.nifi.controller.repository.claim.StandardContentClaim;
+import org.apache.nifi.controller.repository.claim.StandardResourceClaim;
+import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
+import org.apache.nifi.controller.repository.util.DiskUtils;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.LoggerFactory;
 
 public class TestFileSystemRepository {
 
@@ -88,8 +87,7 @@ public class TestFileSystemRepository {
 
     @Before
     public void setup() throws IOException {
-        System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestFileSystemRepository.class.getResource("/conf/nifi.properties").getFile());
-        nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
+        nifiProperties = NiFiProperties.createBasicNiFiProperties(TestFileSystemRepository.class.getResource("/conf/nifi.properties").getFile());
         if (rootFile.exists()) {
             DiskUtils.deleteRecursively(rootFile);
         }
@@ -149,7 +147,7 @@ public class TestFileSystemRepository {
         root.addAppender(testAppender);
         final Map<String, String> addProps = new HashMap<>();
         addProps.put(NiFiProperties.CONTENT_ARCHIVE_CLEANUP_FREQUENCY, "1 millis");
-        final NiFiProperties localProps = NiFiProperties.createBasicNiFiProperties(null, addProps);
+        final NiFiProperties localProps = NiFiProperties.createBasicNiFiProperties(TestFileSystemRepository.class.getResource("/conf/nifi.properties").getFile(), addProps);
         repository = new FileSystemRepository(localProps);
         repository.initialize(new StandardResourceClaimManager());
         repository.purge();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
index c4acf9a..a6edbf5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
@@ -16,6 +16,28 @@
  */
 package org.apache.nifi.controller.repository;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.queue.DropFlowFileStatus;
@@ -45,7 +67,6 @@ import org.apache.nifi.util.file.FileUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -54,39 +75,22 @@ import org.mockito.stubbing.Answer;
 import org.wali.MinimalLockingWriteAheadLog;
 import org.wali.WriteAheadRepository;
 
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.when;
-
 @SuppressWarnings("deprecation")
 public class TestWriteAheadFlowFileRepository {
 
-    @BeforeClass
-    public static void setupProperties() {
-        System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestWriteAheadFlowFileRepository.class.getResource("/conf/nifi.properties").getFile());
-    }
+    private static NiFiProperties niFiProperties;
 
     @Before
+    public void setUp() throws Exception {
+        niFiProperties = NiFiProperties.createBasicNiFiProperties(TestWriteAheadFlowFileRepository.class.getResource("/conf/nifi.properties").getFile());
+        clearRepo();
+    }
+
     @After
+    public void tearDown() throws Exception {
+        clearRepo();
+    }
+
     public void clearRepo() throws IOException {
         final File target = new File("target");
         final File testRepo = new File(target, "test-repo");
@@ -95,7 +99,6 @@ public class TestWriteAheadFlowFileRepository {
         }
     }
 
-
     @Test
     @Ignore("Intended only for local performance testing before/after making changes")
     public void testUpdatePerformance() throws IOException, InterruptedException {
@@ -418,7 +421,7 @@ public class TestWriteAheadFlowFileRepository {
             FileUtils.deleteFile(path.toFile(), true);
         }
 
-        final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null));
+        final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(niFiProperties);
         repo.initialize(new StandardResourceClaimManager());
 
         final TestQueueProvider queueProvider = new TestQueueProvider();
@@ -447,7 +450,7 @@ public class TestWriteAheadFlowFileRepository {
         repo.close();
 
         // restore
-        final WriteAheadFlowFileRepository repo2 = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null));
+        final WriteAheadFlowFileRepository repo2 = new WriteAheadFlowFileRepository(niFiProperties);
         repo2.initialize(new StandardResourceClaimManager());
         repo2.loadFlowFiles(queueProvider);
         assertTrue(repo2.isValidSwapLocationSuffix("swap123"));
@@ -462,7 +465,7 @@ public class TestWriteAheadFlowFileRepository {
             FileUtils.deleteFile(path.toFile(), true);
         }
 
-        final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null));
+        final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(niFiProperties);
         repo.initialize(new StandardResourceClaimManager());
 
         final TestQueueProvider queueProvider = new TestQueueProvider();
@@ -521,7 +524,7 @@ public class TestWriteAheadFlowFileRepository {
         // Create a flowfile repo, update it once with a FlowFile that points to one resource claim. Then,
         // indicate that a FlowFile was swapped out. We should then be able to recover these FlowFiles and the
         // resource claims' counts should be updated for both the swapped out FlowFile and the non-swapped out FlowFile
-        try (final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null))) {
+        try (final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(niFiProperties)) {
             repo.initialize(claimManager);
             repo.loadFlowFiles(queueProvider);
 
@@ -556,7 +559,7 @@ public class TestWriteAheadFlowFileRepository {
         }
 
         final ResourceClaimManager recoveryClaimManager = new StandardResourceClaimManager();
-        try (final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null))) {
+        try (final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(niFiProperties)) {
             repo.initialize(recoveryClaimManager);
             final long largestId = repo.loadFlowFiles(queueProvider);
 
@@ -587,7 +590,7 @@ public class TestWriteAheadFlowFileRepository {
             FileUtils.deleteFile(path.toFile(), true);
         }
 
-        final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null));
+        final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(niFiProperties);
         repo.initialize(new StandardResourceClaimManager());
 
         final TestQueueProvider queueProvider = new TestQueueProvider();
@@ -641,7 +644,7 @@ public class TestWriteAheadFlowFileRepository {
         repo.close();
 
         // restore
-        final WriteAheadFlowFileRepository repo2 = new WriteAheadFlowFileRepository(NiFiProperties.createBasicNiFiProperties(null, null));
+        final WriteAheadFlowFileRepository repo2 = new WriteAheadFlowFileRepository(niFiProperties);
         repo2.initialize(new StandardResourceClaimManager());
         repo2.loadFlowFiles(queueProvider);
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/claim/TestContentClaimWriteCache.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/claim/TestContentClaimWriteCache.java
index fc08f55..8cc068b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/claim/TestContentClaimWriteCache.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/claim/TestContentClaimWriteCache.java
@@ -24,7 +24,6 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-
 import org.apache.nifi.controller.repository.FileSystemRepository;
 import org.apache.nifi.controller.repository.TestFileSystemRepository;
 import org.apache.nifi.controller.repository.util.DiskUtils;
@@ -44,8 +43,7 @@ public class TestContentClaimWriteCache {
 
     @Before
     public void setup() throws IOException {
-        System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestFileSystemRepository.class.getResource("/conf/nifi.properties").getFile());
-        nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
+        nifiProperties = NiFiProperties.createBasicNiFiProperties(TestFileSystemRepository.class.getResource("/conf/nifi.properties").getFile());
         if (rootFile.exists()) {
             DiskUtils.deleteRecursively(rootFile);
         }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderIT.java
index 2bcf3d9..e73ad23 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderIT.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderIT.java
@@ -17,6 +17,15 @@
 
 package org.apache.nifi.controller.service;
 
+import static org.junit.Assert.assertTrue;
+
+import java.beans.PropertyDescriptor;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.state.StateManager;
@@ -43,16 +52,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import java.beans.PropertyDescriptor;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-
-import static org.junit.Assert.assertTrue;
-
 public class StandardControllerServiceProviderIT {
     private static Bundle systemBundle;
     private static NiFiProperties niFiProperties;
@@ -85,7 +84,7 @@ public class StandardControllerServiceProviderIT {
     @BeforeClass
     public static void setNiFiProps() {
         System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestStandardControllerServiceProvider.class.getResource("/conf/nifi.properties").getFile());
-        niFiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
+        niFiProperties = NiFiProperties.createBasicNiFiProperties(null);
 
         // load the system bundle
         systemBundle = SystemBundle.create(niFiProperties);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
index f70ce6e..96df54c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.controller.service;
 
+import java.util.Collections;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.state.StateManagerProvider;
@@ -38,8 +39,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import java.util.Collections;
-
 
 
 public class StandardControllerServiceProviderTest {
@@ -54,8 +53,7 @@ public class StandardControllerServiceProviderTest {
 
     @BeforeClass
     public static void setupSuite() {
-        System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, StandardControllerServiceProviderTest.class.getResource("/conf/nifi.properties").getFile());
-        nifiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
+        nifiProperties = NiFiProperties.createBasicNiFiProperties(StandardControllerServiceProviderTest.class.getResource("/conf/nifi.properties").getFile());
 
         // load the system bundle
         systemBundle = SystemBundle.create(nifiProperties);
@@ -68,7 +66,6 @@ public class StandardControllerServiceProviderTest {
         Mockito.when(flowController.getExtensionManager()).thenReturn(extensionManager);
     }
 
-
     @Before
     public void setup() throws Exception {
         String id = "id";
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
index 0f348c7..893b101 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
@@ -17,6 +17,20 @@
  */
 package org.apache.nifi.controller.service;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.bundle.BundleCoordinate;
 import org.apache.nifi.components.PropertyDescriptor;
@@ -63,21 +77,6 @@ import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-
 public class TestStandardControllerServiceProvider {
 
     private static StateManagerProvider stateManagerProvider = new StateManagerProvider() {
@@ -111,8 +110,7 @@ public class TestStandardControllerServiceProvider {
 
     @BeforeClass
     public static void setNiFiProps() {
-        System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestStandardControllerServiceProvider.class.getResource("/conf/nifi.properties").getFile());
-        niFiProperties = NiFiProperties.createBasicNiFiProperties(null, null);
+        niFiProperties = NiFiProperties.createBasicNiFiProperties(TestStandardControllerServiceProvider.class.getResource("/conf/nifi.properties").getFile());
 
         // load the system bundle
         systemBundle = SystemBundle.create(niFiProperties);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
index 0039214..585d9af 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/NarThreadContextClassLoaderTest.java
@@ -16,6 +16,12 @@
  */
 package org.apache.nifi.nar;
 
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.processor.AbstractProcessor;
 import org.apache.nifi.processor.ProcessContext;
@@ -24,18 +30,11 @@ import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.util.NiFiProperties;
 import org.junit.Test;
 
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
 public class NarThreadContextClassLoaderTest {
 
     @Test
     public void validateWithPropertiesConstructor() throws Exception {
-        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
+        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties");
         Bundle systemBundle = SystemBundle.create(properties);
         ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
         extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
@@ -60,7 +59,7 @@ public class NarThreadContextClassLoaderTest {
 
     @Test
     public void validateWithDefaultConstructor() throws Exception {
-        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties", null);
+        NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties");
         Bundle systemBundle = SystemBundle.create(properties);
         ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager();
         extensionManager.discoverExtensions(systemBundle, Collections.emptySet());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/StandardNiFiPropertiesGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/StandardNiFiPropertiesGroovyTest.groovy
index a1ed963..aaae1e9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/StandardNiFiPropertiesGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/StandardNiFiPropertiesGroovyTest.groovy
@@ -958,4 +958,37 @@ class StandardNiFiPropertiesGroovyTest extends GroovyTestCase {
         // Assert
         assert hosts.size() == 0
     }
+
+    @Test
+    void testStaticFactoryMethodShouldAcceptRawProperties() throws Exception {
+        // Arrange
+        Properties rawProperties = new Properties()
+        rawProperties.setProperty("key", "value")
+        logger.info("rawProperties has ${rawProperties.size()} properties: ${rawProperties.stringPropertyNames()}")
+        assert rawProperties.size() == 1
+
+        // Act
+        NiFiProperties niFiProperties = NiFiProperties.createBasicNiFiProperties("", rawProperties)
+        logger.info("niFiProperties has ${niFiProperties.size()} properties: ${niFiProperties.getPropertyKeys()}")
+
+        // Assert
+        assert niFiProperties.size() == 1
+        assert niFiProperties.getPropertyKeys() == ["key"] as Set
+    }
+
+    @Test
+    void testStaticFactoryMethodShouldAcceptMap() throws Exception {
+        // Arrange
+        def mapProperties = ["key": "value"]
+        logger.info("rawProperties has ${mapProperties.size()} properties: ${mapProperties.keySet()}")
+        assert mapProperties.size() == 1
+
+        // Act
+        NiFiProperties niFiProperties = NiFiProperties.createBasicNiFiProperties("", mapProperties)
+        logger.info("niFiProperties has ${niFiProperties.size()} properties: ${niFiProperties.getPropertyKeys()}")
+
+        // Assert
+        assert niFiProperties.size() == 1
+        assert niFiProperties.getPropertyKeys() == ["key"] as Set
+    }
 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/pom.xml
deleted file mode 100644
index 470b50a..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/pom.xml
+++ /dev/null
@@ -1,40 +0,0 @@
-<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 https://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <!--
-      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.
-    -->
-    <modelVersion>4.0.0</modelVersion>
-    <parent>
-        <groupId>org.apache.nifi</groupId>
-        <artifactId>nifi-framework</artifactId>
-        <version>1.12.0-SNAPSHOT</version>
-    </parent>
-    <artifactId>nifi-security</artifactId>
-    <description>Contains security functionality common to NiFi.</description>
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-properties</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security-utils</artifactId>
-            <version>1.12.0-SNAPSHOT</version>
-        </dependency>
-        <dependency>
-            <groupId>commons-io</groupId>
-            <artifactId>commons-io</artifactId>
-        </dependency>
-    </dependencies>
-</project>
-
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslContextCreationException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslContextCreationException.java
deleted file mode 100644
index 3d7fd27..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslContextCreationException.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.framework.security.util;
-
-/**
- * Represents the exceptional case when a SSL context failed creation.
- *
- */
-public class SslContextCreationException extends SslException {
-
-    public SslContextCreationException(Throwable cause) {
-        super(cause);
-    }
-
-    public SslContextCreationException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public SslContextCreationException(String message) {
-        super(message);
-    }
-
-    public SslContextCreationException() {
-    }
-
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslContextFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslContextFactory.java
deleted file mode 100644
index 6159264..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslContextFactory.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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.framework.security.util;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.security.util.KeyStoreUtils;
-import org.apache.nifi.util.NiFiProperties;
-
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManagerFactory;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.security.KeyManagementException;
-import java.security.KeyStore;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-
-/**
- * A factory for creating SSL contexts using the application's security
- * properties.
- *
- */
-public final class SslContextFactory {
-
-    public static SSLContext createSslContext(final NiFiProperties props)
-            throws SslContextCreationException {
-
-        if (hasKeystoreProperties(props) == false) {
-            return null;
-        } else if (hasTruststoreProperties(props) == false) {
-            throw new SslContextCreationException("SSL context cannot be created because truststore properties have not been configured.");
-        }
-
-        try {
-            // prepare the trust store
-            final KeyStore trustStore;
-            if (hasTruststoreProperties(props)) {
-                trustStore = KeyStoreUtils.getTrustStore(props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE));
-                try (final InputStream trustStoreStream = new FileInputStream(props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE))) {
-                    trustStore.load(trustStoreStream, props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD).toCharArray());
-                }
-            } else {
-                trustStore = null;
-            }
-            final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-            trustManagerFactory.init(trustStore);
-
-            // prepare the key store
-            final KeyStore keyStore = KeyStoreUtils.getKeyStore(props.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE));
-            try (final InputStream keyStoreStream = new FileInputStream(props.getProperty(NiFiProperties.SECURITY_KEYSTORE))) {
-                keyStore.load(keyStoreStream, props.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD).toCharArray());
-            }
-            KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-
-            // if the key password is provided, try to use that - otherwise default to the keystore password
-            if (StringUtils.isNotBlank(props.getProperty(NiFiProperties.SECURITY_KEY_PASSWD))) {
-                keyManagerFactory.init(keyStore, props.getProperty(NiFiProperties.SECURITY_KEY_PASSWD).toCharArray());
-            } else {
-                keyManagerFactory.init(keyStore, props.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD).toCharArray());
-            }
-
-            // initialize the ssl context
-            final SSLContext sslContext = SSLContext.getInstance("TLS");
-            sslContext.init(keyManagerFactory.getKeyManagers(),
-                    trustManagerFactory.getTrustManagers(), null);
-            sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
-
-            return sslContext;
-
-        } catch (final KeyStoreException | IOException | NoSuchAlgorithmException | CertificateException | UnrecoverableKeyException | KeyManagementException e) {
-            throw new SslContextCreationException(e);
-        }
-    }
-
-    private static boolean hasKeystoreProperties(final NiFiProperties props) {
-        return (StringUtils.isNotBlank(props.getProperty(NiFiProperties.SECURITY_KEYSTORE))
-                && StringUtils.isNotBlank(props.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD))
-                && StringUtils.isNotBlank(props.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE)));
-    }
-
-    private static boolean hasTruststoreProperties(final NiFiProperties props) {
-        return (StringUtils.isNotBlank(props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE))
-                && StringUtils.isNotBlank(props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE)));
-    }
-
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslException.java
deleted file mode 100644
index cae2043..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslException.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.framework.security.util;
-
-/**
- * Base class for SSL related exceptions.
- *
- */
-public class SslException extends RuntimeException {
-
-    public SslException(Throwable cause) {
-        super(cause);
-    }
-
-    public SslException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public SslException(String message) {
-        super(message);
-    }
-
-    public SslException() {
-    }
-
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslServerSocketFactoryCreationException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslServerSocketFactoryCreationException.java
deleted file mode 100644
index 83ccb88..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/main/java/org/apache/nifi/framework/security/util/SslServerSocketFactoryCreationException.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.framework.security.util;
-
-/**
- * Represents the exceptional case when a SslServerSocketFactory failed
- * creation.
- *
- */
-public class SslServerSocketFactoryCreationException extends SslException {
-
-    public SslServerSocketFactoryCreationException(Throwable cause) {
-        super(cause);
-    }
-
-    public SslServerSocketFactoryCreationException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public SslServerSocketFactoryCreationException(String message) {
-        super(message);
-    }
-
-    public SslServerSocketFactoryCreationException() {
-    }
-
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/java/org/apache/nifi/framework/security/util/SslContextFactoryTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/java/org/apache/nifi/framework/security/util/SslContextFactoryTest.java
deleted file mode 100644
index 6bcaa9f..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/java/org/apache/nifi/framework/security/util/SslContextFactoryTest.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.framework.security.util;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.File;
-import org.apache.nifi.security.util.KeystoreType;
-import org.apache.nifi.util.NiFiProperties;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- */
-public class SslContextFactoryTest {
-
-    private NiFiProperties mutualAuthProps;
-    private NiFiProperties authProps;
-    private NiFiProperties noPasswordTruststore;
-
-    @Before
-    public void setUp() throws Exception {
-
-        final File ksFile = new File(SslContextFactoryTest.class.getResource("/keystore.jks").toURI());
-        final File trustFile = new File(SslContextFactoryTest.class.getResource("/truststore.jks").toURI());
-        final File noPasswordTrustFile = new File(SslContextFactoryTest.class.getResource("/no-password-truststore.jks").toURI());
-
-        authProps = mock(NiFiProperties.class);
-        when(authProps.getProperty(NiFiProperties.SECURITY_KEYSTORE)).thenReturn(ksFile.getAbsolutePath());
-        when(authProps.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE)).thenReturn(KeystoreType.JKS.toString());
-        when(authProps.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD)).thenReturn("passwordpassword");
-
-        mutualAuthProps = mock(NiFiProperties.class);
-        when(mutualAuthProps.getProperty(NiFiProperties.SECURITY_KEYSTORE)).thenReturn(ksFile.getAbsolutePath());
-        when(mutualAuthProps.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE)).thenReturn(KeystoreType.JKS.toString());
-        when(mutualAuthProps.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD)).thenReturn("passwordpassword");
-        when(mutualAuthProps.getProperty(NiFiProperties.SECURITY_TRUSTSTORE)).thenReturn(trustFile.getAbsolutePath());
-        when(mutualAuthProps.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE)).thenReturn(KeystoreType.JKS.toString());
-        when(mutualAuthProps.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD)).thenReturn("passwordpassword");
-
-        noPasswordTruststore = mock(NiFiProperties.class);
-        when(noPasswordTruststore.getProperty(NiFiProperties.SECURITY_KEYSTORE)).thenReturn(ksFile.getAbsolutePath());
-        when(noPasswordTruststore.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE)).thenReturn(KeystoreType.JKS.toString());
-        when(noPasswordTruststore.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD)).thenReturn("passwordpassword");
-        when(noPasswordTruststore.getProperty(NiFiProperties.SECURITY_TRUSTSTORE)).thenReturn(noPasswordTrustFile.getAbsolutePath());
-        when(noPasswordTruststore.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE)).thenReturn(KeystoreType.JKS.toString());
-        when(noPasswordTruststore.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD)).thenReturn("");
-    }
-
-    @Test
-    public void testCreateSslContextWithMutualAuth() {
-        Assert.assertNotNull(SslContextFactory.createSslContext(mutualAuthProps));
-    }
-
-    @Test(expected = SslContextCreationException.class)
-    public void testCreateSslContextWithNoMutualAuth() {
-        SslContextFactory.createSslContext(authProps);
-    }
-
-    @Test
-    public void testCreateSslContextWithNoPasswordTruststore() {
-        Assert.assertNotNull(SslContextFactory.createSslContext(noPasswordTruststore));
-    }
-}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/keystore.jks b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/keystore.jks
deleted file mode 100644
index 246fe88..0000000
Binary files a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/keystore.jks and /dev/null differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/log4j.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/log4j.properties
deleted file mode 100644
index a253bdd..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,21 +0,0 @@
-# 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.
-
-log4j.rootLogger=DEBUG,console
-
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target=System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/truststore.jks b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/truststore.jks
deleted file mode 100644
index 87f4be1..0000000
Binary files a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-security/src/test/resources/truststore.jks and /dev/null differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
index abcc5b9..c71b862 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/pom.xml
@@ -24,10 +24,6 @@
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-security-utils</artifactId>
             <version>1.12.0-SNAPSHOT</version>
         </dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
index 830d043..8a6d993 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java
@@ -16,25 +16,6 @@
  */
 package org.apache.nifi.remote;
 
-import org.apache.nifi.groups.ProcessGroup;
-import org.apache.nifi.remote.cluster.ClusterNodeInformation;
-import org.apache.nifi.remote.cluster.NodeInformant;
-import org.apache.nifi.remote.cluster.NodeInformation;
-import org.apache.nifi.remote.exception.BadRequestException;
-import org.apache.nifi.remote.exception.HandshakeException;
-import org.apache.nifi.remote.exception.NotAuthorizedException;
-import org.apache.nifi.remote.exception.RequestExpiredException;
-import org.apache.nifi.remote.io.socket.SocketCommunicationsSession;
-import org.apache.nifi.remote.protocol.CommunicationsSession;
-import org.apache.nifi.remote.protocol.RequestType;
-import org.apache.nifi.remote.protocol.ServerProtocol;
-import org.apache.nifi.security.util.CertificateUtils;
-import org.apache.nifi.util.NiFiProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLServerSocket;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.EOFException;
@@ -51,6 +32,24 @@ import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLServerSocket;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.remote.cluster.ClusterNodeInformation;
+import org.apache.nifi.remote.cluster.NodeInformant;
+import org.apache.nifi.remote.cluster.NodeInformation;
+import org.apache.nifi.remote.exception.BadRequestException;
+import org.apache.nifi.remote.exception.HandshakeException;
+import org.apache.nifi.remote.exception.NotAuthorizedException;
+import org.apache.nifi.remote.exception.RequestExpiredException;
+import org.apache.nifi.remote.io.socket.SocketCommunicationsSession;
+import org.apache.nifi.remote.protocol.CommunicationsSession;
+import org.apache.nifi.remote.protocol.RequestType;
+import org.apache.nifi.remote.protocol.ServerProtocol;
+import org.apache.nifi.security.util.CertificateUtils;
+import org.apache.nifi.util.NiFiProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class SocketRemoteSiteListener implements RemoteSiteListener {
 
@@ -61,6 +60,9 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
     private final NiFiProperties nifiProperties;
     private final PeerDescriptionModifier peerDescriptionModifier;
 
+    private static int EXCEPTION_THRESHOLD_MILLIS = 10_000;
+    private volatile long tlsErrorLastSeen = -1;
+
     private final AtomicBoolean stopped = new AtomicBoolean(false);
 
     private static final Logger LOG = LoggerFactory.getLogger(SocketRemoteSiteListener.class);
@@ -168,9 +170,22 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
                                 dn = null;
                             }
                         } catch (final Exception e) {
-                            LOG.error("RemoteSiteListener Unable to accept connection from {} due to {}", socket, e.toString());
-                            if (LOG.isDebugEnabled()) {
-                                LOG.error("", e);
+                            // TODO: Add SocketProtocolListener#handleTlsError logic here
+                            String msg = String.format("RemoteSiteListener Unable to accept connection from {} due to {}", socket, e.getLocalizedMessage());
+                            // Suppress repeated TLS errors
+                            if (CertificateUtils.isTlsError(e)) {
+                                boolean printedAsWarning = handleTlsError(msg);
+
+                                // TODO: Move into handleTlsError and refactor shared behavior
+                                // If the error was printed as a warning, reset the last seen timer
+                                if (printedAsWarning) {
+                                    tlsErrorLastSeen = System.currentTimeMillis();
+                                }
+                            } else {
+                                LOG.error(msg);
+                                if (LOG.isDebugEnabled()) {
+                                    LOG.error("", e);
+                                }
                             }
                             return;
                         }
@@ -304,10 +319,34 @@ public class SocketRemoteSiteListener implements RemoteSiteListener {
         listenerThread.start();
     }
 
+    private boolean handleTlsError(String msg) {
+        if (tlsErrorRecentlySeen()) {
+            LOG.debug(msg);
+            return false;
+        } else {
+            LOG.error(msg);
+            return true;
+        }
+    }
+
+    /**
+     * Returns {@code true} if any related exception (determined by {@link CertificateUtils#isTlsError(Throwable)}) has occurred within the last
+     * {@link #EXCEPTION_THRESHOLD_MILLIS} milliseconds. Does not evaluate the error locally,
+     * simply checks the last time the timestamp was updated.
+     *
+     * @return true if the time since the last similar exception occurred is below the threshold
+     */
+    private boolean tlsErrorRecentlySeen() {
+        long now = System.currentTimeMillis();
+        return now - tlsErrorLastSeen < EXCEPTION_THRESHOLD_MILLIS;
+    }
+
     private ServerSocket createServerSocket() throws IOException {
         if (sslContext != null) {
-            final ServerSocket serverSocket = sslContext.getServerSocketFactory().createServerSocket(socketPort);
-            ((SSLServerSocket) serverSocket).setNeedClientAuth(true);
+            final SSLServerSocket serverSocket = (SSLServerSocket) sslContext.getServerSocketFactory().createServerSocket(socketPort);
+            serverSocket.setNeedClientAuth(true);
+            // Enforce custom protocols on socket
+            serverSocket.setEnabledProtocols(CertificateUtils.getCurrentSupportedTlsProtocolVersions());
             return serverSocket;
         } else {
             return new ServerSocket(socketPort);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/groovy/org/apache/nifi/remote/SocketRemoteSiteListenerTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/groovy/org/apache/nifi/remote/SocketRemoteSiteListenerTest.groovy
new file mode 100644
index 0000000..3955f49
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/groovy/org/apache/nifi/remote/SocketRemoteSiteListenerTest.groovy
@@ -0,0 +1,137 @@
+/*
+ * 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.remote
+
+import org.apache.nifi.security.util.CertificateUtils
+import org.apache.nifi.security.util.KeyStoreUtils
+import org.apache.nifi.security.util.KeystoreType
+import org.apache.nifi.security.util.SslContextFactory
+import org.apache.nifi.security.util.TlsConfiguration
+import org.apache.nifi.util.NiFiProperties
+import org.bouncycastle.jce.provider.BouncyCastleProvider
+import org.junit.After
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.JUnit4
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
+
+import javax.net.ssl.SSLContext
+import javax.net.ssl.SSLServerSocket
+import java.security.Security
+
+@RunWith(JUnit4.class)
+class SocketRemoteSiteListenerTest extends GroovyTestCase {
+    private static final Logger logger = LoggerFactory.getLogger(SocketRemoteSiteListenerTest.class)
+
+    private static final String KEYSTORE_PATH = "src/test/resources/localhost-ks.jks"
+    private static final String KEYSTORE_PASSWORD = "OI7kMpWzzVNVx/JGhTL/0uO4+PWpGJ46uZ/pfepbkwI"
+    private static final KeystoreType KEYSTORE_TYPE = KeystoreType.JKS
+
+    private static final String TRUSTSTORE_PATH = "src/test/resources/localhost-ts.jks"
+    private static final String TRUSTSTORE_PASSWORD = "wAOR0nQJ2EXvOP0JZ2EaqA/n7W69ILS4sWAHghmIWCc"
+    private static final KeystoreType TRUSTSTORE_TYPE = KeystoreType.JKS
+
+    private static final String HOSTNAME = "localhost"
+    private static final int PORT = 0
+
+    // The nifi.properties in src/test/resources has 0.x properties and should be removed or updated
+    private static final Map<String, String> DEFAULT_PROPS = [
+            (NiFiProperties.SECURITY_KEYSTORE)         : KEYSTORE_PATH,
+            (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : KEYSTORE_PASSWORD,
+            (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : KEYSTORE_TYPE.getType(),
+            (NiFiProperties.SECURITY_TRUSTSTORE)       : TRUSTSTORE_PATH,
+            (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): TRUSTSTORE_PASSWORD,
+            (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : TRUSTSTORE_TYPE.getType(),
+            (NiFiProperties.REMOTE_INPUT_HOST): HOSTNAME,
+            (NiFiProperties.REMOTE_INPUT_PORT): PORT as String,
+            "nifi.remote.input.secure": "true"
+    ]
+
+    private NiFiProperties mockNiFiProperties = NiFiProperties.createBasicNiFiProperties("", DEFAULT_PROPS)
+
+    private static TlsConfiguration tlsConfiguration
+    private static SSLContext sslContext
+
+    private SocketRemoteSiteListener srsListener
+
+    @BeforeClass
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
+
+        logger.metaClass.methodMissing = { String name, args ->
+            logger.info("[${name?.toUpperCase()}] ${(args as List).join(" ")}")
+        }
+
+        tlsConfiguration = new TlsConfiguration(KEYSTORE_PATH, KEYSTORE_PASSWORD, KEYSTORE_TYPE, TRUSTSTORE_PATH, TRUSTSTORE_PASSWORD, TRUSTSTORE_TYPE)
+        sslContext = SslContextFactory.createSslContext(tlsConfiguration)
+    }
+
+    @Before
+    void setUp() {
+    }
+
+    @After
+    void tearDown() {
+        if (srsListener) {
+            srsListener.stop()
+        }
+    }
+
+    /**
+     * Asserts that the protocol versions in the parameters object are correct. In recent versions of Java, this enforces order as well, but in older versions, it just enforces presence.
+     *
+     * @param enabledProtocols the actual protocols, either in {@code String[]} or {@code Collection<String>} form
+     * @param expectedProtocols the specific protocol versions to be present (ordered as desired)
+     */
+    void assertProtocolVersions(def enabledProtocols, def expectedProtocols) {
+        if (CertificateUtils.getJavaVersion() > 8) {
+            assert enabledProtocols == expectedProtocols as String[]
+        } else {
+            assert enabledProtocols as Set == expectedProtocols as Set
+        }
+    }
+
+    @Test
+    void testShouldCreateSecureServer() {
+        // Arrange
+        logger.info("Creating SSL Context from TLS Configuration: ${tlsConfiguration}")
+        SSLContext sslContext = SslContextFactory.createSslContext(tlsConfiguration, SslContextFactory.ClientAuth.NONE)
+        logger.info("Created SSL Context: ${KeyStoreUtils.sslContextToString(sslContext)}")
+
+        srsListener = new SocketRemoteSiteListener(PORT, sslContext, mockNiFiProperties)
+
+        // Act
+        srsListener.start()
+
+        // Assert
+
+        // serverSocket isn't instance field like CLBS so have to use private method invocation to verify
+        SSLServerSocket sslServerSocket = srsListener.createServerSocket() as SSLServerSocket
+        logger.info("Created SSL server socket: ${KeyStoreUtils.sslServerSocketToString(sslServerSocket)}" as String)
+        assertProtocolVersions(sslServerSocket.enabledProtocols, CertificateUtils.getCurrentSupportedTlsProtocolVersions())
+        assert sslServerSocket.needClientAuth
+
+        // Assert that the default parameters (which can't be modified) still have legacy protocols and no client auth
+        def defaultSSLParameters = sslContext.defaultSSLParameters
+        logger.info("Default SSL Parameters: ${KeyStoreUtils.sslParametersToString(defaultSSLParameters)}" as String)
+        assertProtocolVersions(defaultSSLParameters.getProtocols(), CertificateUtils.getCurrentSupportedTlsProtocolVersions().sort().reverse() + ["TLSv1.1", "TLSv1"])
+        assert !defaultSSLParameters.needClientAuth
+    }
+}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestHttpRemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestHttpRemoteSiteListener.java
index 58d4d26..fbad70f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestHttpRemoteSiteListener.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestHttpRemoteSiteListener.java
@@ -16,7 +16,13 @@
  */
 package org.apache.nifi.remote;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.properties.StandardNiFiProperties;
 import org.apache.nifi.remote.protocol.FlowFileTransaction;
 import org.apache.nifi.remote.protocol.HandshakeProperties;
 import org.apache.nifi.util.NiFiProperties;
@@ -24,11 +30,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 public class TestHttpRemoteSiteListener {
 
     @BeforeClass
@@ -39,7 +40,7 @@ public class TestHttpRemoteSiteListener {
 
     @Test
     public void testNormalTransactionProgress() {
-        HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(new StandardNiFiProperties());
         String transactionId = transactionManager.createTransaction();
 
         assertTrue("Transaction should be active.", transactionManager.isTransactionActive(transactionId));
@@ -59,7 +60,7 @@ public class TestHttpRemoteSiteListener {
 
     @Test
     public void testDuplicatedTransactionId() {
-        HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(new StandardNiFiProperties());
         String transactionId = transactionManager.createTransaction();
 
         assertTrue("Transaction should be active.", transactionManager.isTransactionActive(transactionId));
@@ -78,7 +79,7 @@ public class TestHttpRemoteSiteListener {
 
     @Test
     public void testNoneExistingTransaction() {
-        HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(new StandardNiFiProperties());
 
         String transactionId = "does-not-exist-1";
         assertFalse("Transaction should not be active.", transactionManager.isTransactionActive(transactionId));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
index d7b423d..8c47cf8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/TestStandardRemoteGroupPort.java
@@ -16,6 +16,27 @@
  */
 package org.apache.nifi.remote;
 
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayInputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.IntStream;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.controller.ProcessScheduler;
 import org.apache.nifi.events.EventReporter;
@@ -26,6 +47,7 @@ import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.properties.StandardNiFiProperties;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventType;
 import org.apache.nifi.remote.client.SiteToSiteClient;
@@ -39,34 +61,11 @@ import org.apache.nifi.remote.util.StandardDataPacket;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.MockProcessContext;
 import org.apache.nifi.util.MockProcessSession;
+import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.SharedSessionState;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.ByteArrayInputStream;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.IntStream;
-
-import org.apache.nifi.util.NiFiProperties;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
 public class TestStandardRemoteGroupPort {
 
     private static final String ID = "remote-group-port-id";
@@ -121,7 +120,7 @@ public class TestStandardRemoteGroupPort {
         }
 
         port = spy(new StandardRemoteGroupPort(ID, ID, NAME,
-                remoteGroup, direction, connectableType, null, scheduler, NiFiProperties.createBasicNiFiProperties(null, null)));
+                remoteGroup, direction, connectableType, null, scheduler, new StandardNiFiProperties()));
 
         doReturn(true).when(remoteGroup).isTransmitting();
         doReturn(protocol).when(remoteGroup).getTransportProtocol();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/protocol/http/TestHttpFlowFileServerProtocol.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/protocol/http/TestHttpFlowFileServerProtocol.java
index 1f934ef..75f6b0c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/protocol/http/TestHttpFlowFileServerProtocol.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/java/org/apache/nifi/remote/protocol/http/TestHttpFlowFileServerProtocol.java
@@ -16,6 +16,32 @@
  */
 package org.apache.nifi.remote.protocol.http;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
@@ -25,6 +51,7 @@ import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.properties.StandardNiFiProperties;
 import org.apache.nifi.provenance.ProvenanceEventRecord;
 import org.apache.nifi.provenance.ProvenanceEventType;
 import org.apache.nifi.remote.HttpRemoteSiteListener;
@@ -39,8 +66,8 @@ import org.apache.nifi.remote.exception.HandshakeException;
 import org.apache.nifi.remote.io.http.HttpInput;
 import org.apache.nifi.remote.io.http.HttpServerCommunicationsSession;
 import org.apache.nifi.remote.protocol.DataPacket;
-import org.apache.nifi.remote.protocol.ResponseCode;
 import org.apache.nifi.remote.protocol.HandshakeProperty;
+import org.apache.nifi.remote.protocol.ResponseCode;
 import org.apache.nifi.remote.util.StandardDataPacket;
 import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.MockProcessContext;
@@ -50,33 +77,6 @@ import org.apache.nifi.util.SharedSessionState;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 public class TestHttpFlowFileServerProtocol {
 
     private SharedSessionState sessionState;
@@ -107,7 +107,7 @@ public class TestHttpFlowFileServerProtocol {
 
     private HttpFlowFileServerProtocol getDefaultHttpFlowFileServerProtocol() {
         final StandardVersionNegotiator versionNegotiator = new StandardVersionNegotiator(5, 4, 3, 2, 1);
-        return new StandardHttpFlowFileServerProtocol(versionNegotiator, NiFiProperties.createBasicNiFiProperties(null, null));
+        return new StandardHttpFlowFileServerProtocol(versionNegotiator, new StandardNiFiProperties());
     }
 
     @Test
@@ -365,7 +365,7 @@ public class TestHttpFlowFileServerProtocol {
             sessionState.getFlowFileQueue().offer(flowFile);
         }
 
-        final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance(new StandardNiFiProperties());
 
         serverProtocol.handshake(peer);
         assertTrue(serverProtocol.isHandshakeSuccessful());
@@ -520,7 +520,7 @@ public class TestHttpFlowFileServerProtocol {
     }
 
     private void receiveFlowFiles(final HttpFlowFileServerProtocol serverProtocol, final String transactionId, final Peer peer, final DataPacket ... dataPackets) throws IOException {
-        final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        final HttpRemoteSiteListener remoteSiteListener = HttpRemoteSiteListener.getInstance(new StandardNiFiProperties());
         final HttpServerCommunicationsSession commsSession = (HttpServerCommunicationsSession) peer.getCommunicationsSession();
 
         serverProtocol.handshake(peer);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/localhost-ks.jks b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/localhost-ks.jks
new file mode 100755
index 0000000..6db775d
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/localhost-ks.jks differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/localhost-ts.jks b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/localhost-ts.jks
new file mode 100755
index 0000000..5a4fd2d
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/test/resources/localhost-ts.jks differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlow.java
index 1329970..6ca5320 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlow.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-stateless/src/main/java/org/apache/nifi/stateless/core/StatelessFlow.java
@@ -54,7 +54,9 @@ import org.apache.nifi.registry.flow.VersionedProcessor;
 import org.apache.nifi.registry.flow.VersionedRemoteGroupPort;
 import org.apache.nifi.registry.flow.VersionedRemoteProcessGroup;
 import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.security.util.CertificateUtils;
 import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
 import org.apache.nifi.stateless.bootstrap.ExtensionDiscovery;
 import org.apache.nifi.stateless.bootstrap.InMemoryFlowFile;
 import org.apache.nifi.stateless.bootstrap.RunnableFlow;
@@ -370,8 +372,9 @@ public class StatelessFlow implements RunnableFlow {
             final String truststoreType = sslObject.get(TRUSTSTORE_TYPE).getAsString();
 
             try {
-                return SslContextFactory.createSslContext(keystore, keystorePass.toCharArray(), keyPass.toCharArray(), keystoreType,
-                    truststore, truststorePass.toCharArray(), truststoreType, SslContextFactory.ClientAuth.REQUIRED, "TLS");
+                TlsConfiguration tlsConfiguration = new TlsConfiguration(keystore, keystorePass, keyPass, keystoreType,
+                        truststore, truststorePass, truststoreType, CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion());
+                return SslContextFactory.createSslContext(tlsConfiguration, SslContextFactory.ClientAuth.REQUIRED);
             } catch (final Exception e) {
                 throw new RuntimeException("Failed to create Keystore", e);
             }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml
index 7b300f6..16c7cdb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/pom.xml
@@ -53,11 +53,6 @@
             <scope>compile</scope>
         </dependency>
         <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-security</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-server</artifactId>
         </dependency>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
index 8a151db..04df2bf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java
@@ -18,6 +18,37 @@ package org.apache.nifi.web.server;
 
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+import java.util.stream.Collectors;
+import javax.servlet.DispatcherType;
+import javax.servlet.Filter;
+import javax.servlet.ServletContext;
 import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.NiFiServer;
@@ -42,6 +73,7 @@ import org.apache.nifi.nar.NarLoader;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.StandardNarLoader;
 import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.security.util.CertificateUtils;
 import org.apache.nifi.security.util.KeyStoreUtils;
 import org.apache.nifi.services.FlowService;
 import org.apache.nifi.ui.extension.UiExtension;
@@ -51,11 +83,11 @@ import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.ContentAccess;
 import org.apache.nifi.web.NiFiWebConfigurationContext;
 import org.apache.nifi.web.UiExtensionType;
-import org.apache.nifi.web.security.requests.ContentLengthFilter;
 import org.apache.nifi.web.security.headers.ContentSecurityPolicyFilter;
 import org.apache.nifi.web.security.headers.StrictTransportSecurityFilter;
 import org.apache.nifi.web.security.headers.XFrameOptionsFilter;
 import org.apache.nifi.web.security.headers.XSSProtectionFilter;
+import org.apache.nifi.web.security.requests.ContentLengthFilter;
 import org.eclipse.jetty.annotations.AnnotationConfiguration;
 import org.eclipse.jetty.deploy.App;
 import org.eclipse.jetty.deploy.DeploymentManager;
@@ -88,38 +120,6 @@ import org.springframework.context.ApplicationContext;
 import org.springframework.web.context.WebApplicationContext;
 import org.springframework.web.context.support.WebApplicationContextUtils;
 
-import javax.servlet.DispatcherType;
-import javax.servlet.Filter;
-import javax.servlet.ServletContext;
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.jar.JarEntry;
-import java.util.jar.JarFile;
-import java.util.stream.Collectors;
-
 /**
  * Encapsulates the Jetty instance.
  */
@@ -930,6 +930,11 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader {
         // Previous to Jetty 9.4.15.v20190215, this defaulted to null, and now defaults to "HTTPS".
         contextFactory.setEndpointIdentificationAlgorithm(null);
 
+        // Explicitly exclude legacy TLS protocol versions
+        // contextFactory.setProtocol(CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion());
+        contextFactory.setIncludeProtocols(CertificateUtils.getCurrentSupportedTlsProtocolVersions());
+        contextFactory.setExcludeProtocols("TLS", "TLSv1", "TLSv1.1", "SSL", "SSLv2", "SSLv2Hello", "SSLv3");
+
         // require client auth when not supporting login, Kerberos service, or anonymous access
         if (props.isClientAuthRequiredForRestApi()) {
             contextFactory.setNeedClientAuth(true);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy
index 174efc0..59c38de 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/groovy/org/apache/nifi/web/server/JettyServerGroovyTest.groovy
@@ -20,14 +20,19 @@ import org.apache.log4j.AppenderSkeleton
 import org.apache.log4j.spi.LoggingEvent
 import org.apache.nifi.bundle.Bundle
 import org.apache.nifi.properties.StandardNiFiProperties
+import org.apache.nifi.security.util.CertificateUtils
+import org.apache.nifi.security.util.TlsConfiguration
 import org.apache.nifi.util.NiFiProperties
 import org.bouncycastle.jce.provider.BouncyCastleProvider
 import org.eclipse.jetty.server.Connector
 import org.eclipse.jetty.server.HttpConfiguration
 import org.eclipse.jetty.server.Server
 import org.eclipse.jetty.server.ServerConnector
+import org.eclipse.jetty.server.SslConnectionFactory
+import org.eclipse.jetty.util.ssl.SslContextFactory
 import org.junit.After
 import org.junit.AfterClass
+import org.junit.Assume
 import org.junit.Before
 import org.junit.BeforeClass
 import org.junit.Rule
@@ -41,6 +46,9 @@ import org.junit.runners.JUnit4
 import org.slf4j.Logger
 import org.slf4j.LoggerFactory
 
+import javax.net.ssl.SSLSocket
+import javax.net.ssl.SSLSocketFactory
+import java.nio.charset.StandardCharsets
 import java.security.Security
 
 @RunWith(JUnit4.class)
@@ -56,6 +64,36 @@ class JettyServerGroovyTest extends GroovyTestCase {
     @Rule
     public final SystemErrRule systemErrRule = new SystemErrRule().enableLog()
 
+    private static final int HTTPS_PORT = 8443
+    private static final String HTTPS_HOSTNAME = "localhost"
+
+    private static final String KEYSTORE_PATH = "src/test/resources/keystore.jks"
+    private static final String TRUSTSTORE_PATH = "src/test/resources/truststore.jks"
+    private static final String STORE_PASSWORD = "passwordpassword"
+    private static final String STORE_TYPE = "JKS"
+
+    private static final String TLS_1_2_PROTOCOL = "TLSv1.2"
+    private static final String TLS_1_3_PROTOCOL = "TLSv1.3"
+    private static final List<String> TLS_1_3_CIPHER_SUITES = ["TLS_AES_128_GCM_SHA256"]
+
+    // Depending if the test is run on Java 8 or Java 11, these values change (TLSv1.2 vs. TLSv1.3)
+    private static final CURRENT_TLS_PROTOCOL_VERSION = CertificateUtils.getHighestCurrentSupportedTlsProtocolVersion()
+    private static final List<String> CURRENT_TLS_PROTOCOL_VERSIONS = CertificateUtils.getCurrentSupportedTlsProtocolVersions()
+
+    // These protocol versions should not ever be supported
+    static private final List<String> LEGACY_TLS_PROTOCOLS = ["TLS", "TLSv1", "TLSv1.1", "SSL", "SSLv2", "SSLv2Hello", "SSLv3"]
+
+    NiFiProperties httpsProps = new StandardNiFiProperties(rawProperties: new Properties([
+            (NiFiProperties.WEB_HTTPS_PORT)            : HTTPS_PORT as String,
+            (NiFiProperties.WEB_HTTPS_HOST)            : HTTPS_HOSTNAME,
+            (NiFiProperties.SECURITY_KEYSTORE)         : KEYSTORE_PATH,
+            (NiFiProperties.SECURITY_KEYSTORE_PASSWD)  : STORE_PASSWORD,
+            (NiFiProperties.SECURITY_KEYSTORE_TYPE)    : STORE_TYPE,
+            (NiFiProperties.SECURITY_TRUSTSTORE)       : TRUSTSTORE_PATH,
+            (NiFiProperties.SECURITY_TRUSTSTORE_PASSWD): STORE_PASSWORD,
+            (NiFiProperties.SECURITY_TRUSTSTORE_TYPE)  : STORE_TYPE,
+    ]))
+
     @BeforeClass
     static void setUpOnce() throws Exception {
         Security.addProvider(new BouncyCastleProvider())
@@ -168,16 +206,16 @@ class JettyServerGroovyTest extends GroovyTestCase {
                 (NiFiProperties.WEB_HTTPS_HOST): "secure.host.com",
         ]
         NiFiProperties mockProps = [
-                getPort    : { -> 8080 },
-                getSslPort : { -> 8443 },
-                getProperty: { String prop ->
+                getPort            : { -> 8080 },
+                getSslPort         : { -> 8443 },
+                getProperty        : { String prop ->
                     String value = badProps[prop] ?: "no_value"
                     logger.mock("getProperty(${prop}) -> ${value}")
                     value
                 },
-                getWebThreads: { -> NiFiProperties.DEFAULT_WEB_THREADS },
+                getWebThreads      : { -> NiFiProperties.DEFAULT_WEB_THREADS },
                 getWebMaxHeaderSize: { -> NiFiProperties.DEFAULT_WEB_MAX_HEADER_SIZE },
-                isHTTPSConfigured: { -> true }
+                isHTTPSConfigured  : { -> true }
         ] as StandardNiFiProperties
 
         // The web server should fail to start and exit Java
@@ -187,11 +225,11 @@ class JettyServerGroovyTest extends GroovyTestCase {
                 final String standardErr = systemErrRule.getLog()
                 List<String> errLines = standardErr.split("\n")
 
-                assert errLines.any { it =~ "Failed to start web server: "}
-                assert errLines.any { it =~ "Shutting down..."}
+                assert errLines.any { it =~ "Failed to start web server: " }
+                assert errLines.any { it =~ "Shutting down..." }
             }
         })
-        
+
         // Act
         JettyServer jettyServer = new JettyServer(mockProps, [] as Set<Bundle>)
 
@@ -203,25 +241,152 @@ class JettyServerGroovyTest extends GroovyTestCase {
     @Test
     void testShouldConfigureHTTPSConnector() {
         // Arrange
-       NiFiProperties httpsProps = new StandardNiFiProperties(rawProperties: new Properties([
-//               (NiFiProperties.WEB_HTTP_PORT): null,
-//               (NiFiProperties.WEB_HTTP_HOST): null,
-               (NiFiProperties.WEB_HTTPS_PORT): "8443",
-               (NiFiProperties.WEB_HTTPS_HOST): "secure.host.com",
-       ]))
-        
+        final String externalHostname = "secure.host.com"
+
+        NiFiProperties httpsProps = new StandardNiFiProperties(rawProperties: new Properties([
+                (NiFiProperties.WEB_HTTPS_PORT): HTTPS_PORT as String,
+                (NiFiProperties.WEB_HTTPS_HOST): externalHostname,
+        ]))
+
+        Server internalServer = new Server()
+        JettyServer jetty = new JettyServer(internalServer, httpsProps)
+
+        // Act
+        jetty.configureHttpsConnector(internalServer, new HttpConfiguration())
+        List<Connector> connectors = Arrays.asList(internalServer.connectors)
+
+        // Assert
+
+        // Set the expected TLS protocols to null because no actual keystore/truststore is loaded here
+        assertServerConnector(connectors, "TLS", null, null, externalHostname, HTTPS_PORT)
+    }
+
+    @Test
+    void testShouldSupportTLSv1_3OnJava11() {
+        // Arrange
+        Assume.assumeTrue("This test should only run on Java 11+", CertificateUtils.getJavaVersion() >= 11)
+
+        Server internalServer = new Server()
+        JettyServer jetty = new JettyServer(internalServer, httpsProps)
+
+        jetty.configureConnectors(internalServer)
+        List<Connector> connectors = Arrays.asList(internalServer.connectors)
+        internalServer.start()
+
+        // Create a (client) socket which only supports TLSv1.3
+        TlsConfiguration tls13ClientConf = TlsConfiguration.fromNiFiProperties(httpsProps)
+        SSLSocketFactory socketFactory = org.apache.nifi.security.util.SslContextFactory.createSSLSocketFactory(tls13ClientConf)
+
+        SSLSocket socket = (SSLSocket) socketFactory.createSocket(HTTPS_HOSTNAME, HTTPS_PORT)
+        socket.setEnabledProtocols([TLS_1_3_PROTOCOL] as String[])
+        socket.setEnabledCipherSuites(TLS_1_3_CIPHER_SUITES as String[])
+
+        // Act
+        String response = makeTLSRequest(socket, "This is a TLS 1.3 request")
+
+        // Assert
+        assert response =~ "HTTP/1.1 400"
+
+        // Assert that the connector prefers TLSv1.3 but the JVM supports TLSv1.2 as well
+        assertServerConnector(connectors, "TLS", [CURRENT_TLS_PROTOCOL_VERSION], CURRENT_TLS_PROTOCOL_VERSIONS)
+
+        // Clean up
+        internalServer.stop()
+    }
+
+    @Test
+    void testShouldNotSupportTLSv1_3OnJava8() {
+        // Arrange
+        Assume.assumeTrue("This test should only run on Java 8", CertificateUtils.getJavaVersion() <= 8)
+
         Server internalServer = new Server()
         JettyServer jetty = new JettyServer(internalServer, httpsProps)
 
+        jetty.configureConnectors(internalServer)
+        List<Connector> connectors = Arrays.asList(internalServer.connectors)
+        internalServer.start()
+
+        TlsConfiguration tlsConfiguration = TlsConfiguration.fromNiFiProperties(httpsProps)
+
+        // Create a "default" (client) socket (which supports TLSv1.2)
+        SSLSocketFactory defaultSocketFactory = org.apache.nifi.security.util.SslContextFactory.createSSLSocketFactory(tlsConfiguration)
+        SSLSocket defaultSocket = (SSLSocket) defaultSocketFactory.createSocket(HTTPS_HOSTNAME, HTTPS_PORT)
+
         // Act
-       jetty.configureHttpsConnector(internalServer, new HttpConfiguration())
-       List<Connector> connectors = Arrays.asList(internalServer.connectors)
+        String tls12Response = makeTLSRequest(defaultSocket, "This is a default socket request")
+
+        def msg = shouldFail(IllegalArgumentException) {
+            // Create a (client) socket which only supports TLSv1.3
+            SSLSocketFactory tls13SocketFactory = org.apache.nifi.security.util.SslContextFactory.createSSLSocketFactory(tlsConfiguration)
+
+            SSLSocket tls13Socket = (SSLSocket) tls13SocketFactory.createSocket(HTTPS_HOSTNAME, HTTPS_PORT)
+            tls13Socket.setEnabledProtocols([TLS_1_3_PROTOCOL] as String[])
+            tls13Socket.setEnabledCipherSuites(TLS_1_3_CIPHER_SUITES as String[])
+
+            String tls13Response = makeTLSRequest(tls13Socket, "This is a TLSv1.3 socket request")
+        }
+        // The IAE message is just the invalid argument (i.e. "TLSv1.3")
+        logger.expected(msg)
 
         // Assert
+        assert tls12Response =~ "HTTP"
+        assert msg == "TLSv1.3"
+
+        // Assert that the connector only accepts TLSv1.2
+        assertServerConnector(connectors, "TLS", [CURRENT_TLS_PROTOCOL_VERSION], CURRENT_TLS_PROTOCOL_VERSIONS)
+
+        // Clean up
+        internalServer.stop()
+    }
+
+    /**
+     * Returns the server's response body as a String. Closes the socket connection.
+     *
+     * @param socket
+     * @param requestMessage
+     * @return
+     */
+    private static String makeTLSRequest(Socket socket, String requestMessage) {
+        InputStream socketInputStream = new BufferedInputStream(socket.getInputStream())
+        OutputStream socketOutputStream = new BufferedOutputStream(socket.getOutputStream())
+
+        socketOutputStream.write(requestMessage.getBytes())
+        socketOutputStream.flush()
+
+        byte[] data = new byte[2048]
+        int len = socketInputStream.read(data)
+        if (len <= 0) {
+            throw new IOException("no data received")
+        }
+        final String trimmedResponse = new String(data, 0, len, StandardCharsets.UTF_8)
+        logger.info("Client received ${len} bytes from server: \n${trimmedResponse}\n----End of response----")
+        socket.close()
+        trimmedResponse
+    }
+
+    private static void assertServerConnector(List<Connector> connectors,
+                                              String EXPECTED_TLS_PROTOCOL = "TLS",
+                                              List<String> EXPECTED_INCLUDED_PROTOCOLS = CertificateUtils.getCurrentSupportedTlsProtocolVersions(),
+                                              List<String> EXPECTED_SELECTED_PROTOCOLS = CertificateUtils.getCurrentSupportedTlsProtocolVersions(),
+                                              String EXPECTED_HOSTNAME = HTTPS_HOSTNAME,
+                                              int EXPECTED_PORT = HTTPS_PORT) {
+        // Assert the server connector is correct
         assert connectors.size() == 1
         ServerConnector connector = connectors.first() as ServerConnector
-        assert connector.host == "secure.host.com"
-        assert connector.port == 8443
+        assert connector.host == EXPECTED_HOSTNAME
+        assert connector.port == EXPECTED_PORT
+        assert connector.getProtocols() == ['ssl', 'http/1.1']
+
+        // This kind of testing is not ideal as it breaks encapsulation, but is necessary to enforce verification of the TLS protocol versions specified
+        SslConnectionFactory connectionFactory = connector.getConnectionFactory("ssl") as SslConnectionFactory
+        SslContextFactory sslContextFactory = connectionFactory._sslContextFactory as SslContextFactory
+        logger.debug("SSL Context Factory: ${sslContextFactory.dump()}")
+
+        // Using the getters is subject to NPE due to blind array copies
+        assert sslContextFactory._sslProtocol == EXPECTED_TLS_PROTOCOL
+        assert sslContextFactory._includeProtocols.containsAll(EXPECTED_INCLUDED_PROTOCOLS ?: Collections.emptySet())
+        assert (sslContextFactory._excludeProtocols as List<String>).containsAll(LEGACY_TLS_PROTOCOLS)
+        assert sslContextFactory._selectedProtocols == EXPECTED_SELECTED_PROTOCOLS as String[]
     }
 }
 
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/keystore.jks b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/keystore.jks
new file mode 100644
index 0000000..34a197f
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/keystore.jks differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/log4j.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/log4j.properties
index 4608f9e..162521f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/log4j.properties
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/log4j.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-log4j.rootLogger=DEBUG,console,test
+log4j.rootLogger=INFO,console,test
 
 log4j.appender.console=org.apache.log4j.ConsoleAppender
 log4j.appender.console.Target=System.err
diff --git a/nifi-commons/nifi-security-utils/src/test/resources/logback-test.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/logback-test.xml
similarity index 89%
copy from nifi-commons/nifi-security-utils/src/test/resources/logback-test.xml
copy to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/logback-test.xml
index b5036d4..4e4a789 100644
--- a/nifi-commons/nifi-security-utils/src/test/resources/logback-test.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/logback-test.xml
@@ -17,7 +17,7 @@
 <configuration>
     <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender">
         <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
-            <pattern>%-4r [%t] %-5p %c{3} - %m%n</pattern>
+            <pattern>%-4r [%t] %-5p %c - %m%n</pattern>
         </encoder>
     </appender>
 
@@ -30,10 +30,9 @@
 
 
     <logger name="org.apache.nifi" level="INFO"/>
-    <logger name="org.apache.nifi.security.util.crypto" level="DEBUG"/>
-
+    <logger name="org.apache.nifi.web.server" level="DEBUG"/>
+    <logger name="org.eclipse.jetty.server" level="DEBUG"/>
     <root level="INFO">
         <appender-ref ref="CONSOLE"/>
     </root>
-
 </configuration>
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/truststore.jks b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/truststore.jks
new file mode 100644
index 0000000..4bc1b20
Binary files /dev/null and b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/resources/truststore.jks differ
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
index b56f708..2637cc2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/AccessControlHelper.java
@@ -16,6 +16,12 @@
  */
 package org.apache.nifi.integration.accesscontrol;
 
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import javax.ws.rs.core.Response;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.integration.NiFiWebApiTest;
 import org.apache.nifi.integration.util.NiFiTestAuthorizer;
@@ -30,13 +36,6 @@ import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.StringUtils;
 
-import javax.ws.rs.core.Response;
-import java.io.File;
-import java.nio.file.Files;
-import java.nio.file.StandardCopyOption;
-
-import static org.junit.Assert.assertEquals;
-
 /**
  * Access control test for the dfm user.
  */
@@ -69,8 +68,7 @@ public class AccessControlHelper {
         // configure the location of the nifi properties
         File nifiPropertiesFile = new File(nifiPropertiesPath);
         System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, nifiPropertiesFile.getAbsolutePath());
-
-        NiFiProperties props = NiFiProperties.createBasicNiFiProperties(nifiPropertiesPath, null);
+        NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null);
         flowXmlPath = props.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE);
 
         final File libTargetDir = new File("target/test-classes/access-control/lib");
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java
index 5904929..9f1ae29 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/ITAccessTokenEndpoint.java
@@ -16,6 +16,12 @@
  */
 package org.apache.nifi.integration.accesscontrol;
 
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.StringJoiner;
+import javax.ws.rs.core.Response;
 import net.minidev.json.JSONObject;
 import org.apache.nifi.integration.util.SourceTestProcessor;
 import org.apache.nifi.web.api.dto.AccessConfigurationDTO;
@@ -31,13 +37,6 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import javax.ws.rs.core.Response;
-import java.util.Calendar;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.StringJoiner;
-
 /**
  * Access token endpoint test.
  */
@@ -220,6 +219,8 @@ public class ITAccessTokenEndpoint {
         Assert.assertEquals("ACTIVE", accessStatus.getStatus());
     }
 
+    // // TODO: Revisit the HTTP status codes in this test after logout functionality change
+    // @Ignore("This test is failing before refactoring")
     @Test
     public void testLogOutSuccess() throws Exception {
         String accessStatusUrl = helper.getBaseUrl() + "/access";
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/OneWaySslAccessControlHelper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/OneWaySslAccessControlHelper.java
index e5942e8..cf9721f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/OneWaySslAccessControlHelper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/accesscontrol/OneWaySslAccessControlHelper.java
@@ -16,6 +16,10 @@
  */
 package org.apache.nifi.integration.accesscontrol;
 
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import javax.ws.rs.client.Client;
 import org.apache.commons.io.FileUtils;
 import org.apache.nifi.bundle.Bundle;
 import org.apache.nifi.integration.util.NiFiTestServer;
@@ -27,15 +31,10 @@ import org.apache.nifi.nar.NarUnpacker;
 import org.apache.nifi.nar.StandardExtensionDiscoveringManager;
 import org.apache.nifi.nar.SystemBundle;
 import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.util.WebUtils;
 
-import javax.net.ssl.SSLContext;
-import javax.ws.rs.client.Client;
-import java.io.File;
-import java.nio.file.Files;
-import java.nio.file.StandardCopyOption;
-
 /**
  * Access control test for the dfm user.
  */
@@ -58,7 +57,7 @@ public class OneWaySslAccessControlHelper {
         File nifiPropertiesFile = new File(nifiPropertiesPath);
         System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, nifiPropertiesFile.getAbsolutePath());
 
-        NiFiProperties props = NiFiProperties.createBasicNiFiProperties(nifiPropertiesPath, null);
+        NiFiProperties props = NiFiProperties.createBasicNiFiProperties(nifiPropertiesPath);
         flowXmlPath = props.getProperty(NiFiProperties.FLOW_CONFIGURATION_FILE);
 
         // delete the database directory to avoid issues with re-registration in testRequestAccessUsingToken
@@ -90,8 +89,11 @@ public class OneWaySslAccessControlHelper {
         // get the base url
         baseUrl = server.getBaseUrl() + CONTEXT_PATH;
 
+        // Create a TlsConfiguration for the truststore properties only
+        TlsConfiguration trustOnlyTlsConfiguration = TlsConfiguration.fromNiFiPropertiesTruststoreOnly(props);
+
         // create the user
-        final Client client = WebUtils.createClient(null, createTrustContext(props));
+        final Client client = WebUtils.createClient(null, SslContextFactory.createSslContext(trustOnlyTlsConfiguration));
         user = new NiFiTestUser(client, null);
     }
 
@@ -103,12 +105,6 @@ public class OneWaySslAccessControlHelper {
         return baseUrl;
     }
 
-    private static SSLContext createTrustContext(final NiFiProperties props) throws Exception {
-        return SslContextFactory.createTrustSslContext(props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE),
-                props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD).toCharArray(),
-                props.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE), "TLS");
-    }
-
     public void cleanup() throws Exception {
         // shutdown the server
         server.shutdownServer();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
index 1a8a997..9379019 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/integration/util/NiFiTestServer.java
@@ -16,8 +16,13 @@
  */
 package org.apache.nifi.integration.util;
 
+import java.io.File;
+import java.util.Collections;
+import javax.servlet.ServletContext;
+import javax.ws.rs.client.Client;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.framework.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
 import org.apache.nifi.services.FlowService;
 import org.apache.nifi.ui.extension.UiExtensionMapping;
 import org.apache.nifi.util.NiFiProperties;
@@ -34,11 +39,6 @@ import org.slf4j.LoggerFactory;
 import org.springframework.web.context.WebApplicationContext;
 import org.springframework.web.context.support.WebApplicationContextUtils;
 
-import javax.servlet.ServletContext;
-import javax.ws.rs.client.Client;
-import java.io.File;
-import java.util.Collections;
-
 /**
  * Creates an embedded server for testing the NiFi REST API.
  */
@@ -75,6 +75,7 @@ public class NiFiTestServer {
         return jetty;
     }
 
+    // TODO: Refactor this method to use proper factory methods
     private void createSecureConnector() {
         org.eclipse.jetty.util.ssl.SslContextFactory contextFactory = new org.eclipse.jetty.util.ssl.SslContextFactory();
 
@@ -171,8 +172,8 @@ public class NiFiTestServer {
         return "https://localhost:" + getPort();
     }
 
-    public Client getClient() {
-        return WebUtils.createClient(null, SslContextFactory.createSslContext(properties));
+    public Client getClient() throws TlsException {
+        return WebUtils.createClient(null, org.apache.nifi.security.util.SslContextFactory.createSslContext(TlsConfiguration.fromNiFiProperties(properties)));
     }
 
     /**
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java
index fd72a60..cbe3030 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java
@@ -16,6 +16,31 @@
  */
 package org.apache.nifi.web.api;
 
+import static org.apache.nifi.web.api.ApplicationResource.PROXY_HOST_HTTP_HEADER;
+import static org.apache.nifi.web.api.ApplicationResource.PROXY_PORT_HTTP_HEADER;
+import static org.apache.nifi.web.api.ApplicationResource.PROXY_SCHEME_HTTP_HEADER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.InputStream;
+import java.lang.reflect.Field;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import javax.servlet.ServletContext;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.StreamingOutput;
+import javax.ws.rs.core.UriBuilder;
+import javax.ws.rs.core.UriInfo;
 import org.apache.nifi.authorization.AuthorizableLookup;
 import org.apache.nifi.authorization.resource.ResourceType;
 import org.apache.nifi.remote.HttpRemoteSiteListener;
@@ -33,32 +58,6 @@ import org.apache.nifi.web.api.entity.TransactionResultEntity;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import javax.servlet.ServletContext;
-import javax.servlet.http.HttpServletRequest;
-import javax.servlet.http.HttpServletResponse;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.StreamingOutput;
-import javax.ws.rs.core.UriBuilder;
-import javax.ws.rs.core.UriInfo;
-import java.io.InputStream;
-import java.lang.reflect.Field;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-
-import static org.apache.nifi.web.api.ApplicationResource.PROXY_HOST_HTTP_HEADER;
-import static org.apache.nifi.web.api.ApplicationResource.PROXY_PORT_HTTP_HEADER;
-import static org.apache.nifi.web.api.ApplicationResource.PROXY_SCHEME_HTTP_HEADER;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 public class TestDataTransferResource {
 
     @BeforeClass
@@ -231,7 +230,7 @@ public class TestDataTransferResource {
         final UriInfo uriInfo = mockUriInfo(locationUriStr);
         final InputStream inputStream = null;
 
-        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null));
         final String transactionId = transactionManager.createTransaction();
 
         final Response response = resource.extendPortTransactionTTL("input-ports", "port-id", transactionId, req, res, context, uriInfo, inputStream);
@@ -262,7 +261,7 @@ public class TestDataTransferResource {
         final ServletContext context = null;
         final InputStream inputStream = null;
 
-        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null));
         final String transactionId = transactionManager.createTransaction();
 
         final Response response = resource.receiveFlowFiles("port-id", transactionId, req, context, inputStream);
@@ -289,7 +288,7 @@ public class TestDataTransferResource {
         final ServletContext context = null;
         final InputStream inputStream = null;
 
-        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null));
         final String transactionId = transactionManager.createTransaction();
 
         final Response response = resource.receiveFlowFiles("port-id", transactionId, req, context, inputStream);
@@ -308,7 +307,7 @@ public class TestDataTransferResource {
         final ServletContext context = null;
         final InputStream inputStream = null;
 
-        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null));
         final String transactionId = transactionManager.createTransaction();
 
         final Response response = resource.commitInputPortTransaction(ResponseCode.CONFIRM_TRANSACTION.getCode(), "port-id", transactionId, req, context, inputStream);
@@ -331,7 +330,7 @@ public class TestDataTransferResource {
         final HttpServletResponse res = null;
         final InputStream inputStream = null;
 
-        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null));
         final String transactionId = transactionManager.createTransaction();
 
         final Response response = resource.transferFlowFiles("port-id", transactionId, req, res, context, inputStream);
@@ -353,7 +352,7 @@ public class TestDataTransferResource {
         final ServletContext context = null;
         final InputStream inputStream = null;
 
-        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null));
         final String transactionId = transactionManager.createTransaction();
 
         final Response response = resource.commitOutputPortTransaction(ResponseCode.CONFIRM_TRANSACTION.getCode(),
@@ -379,7 +378,7 @@ public class TestDataTransferResource {
         final ServletContext context = null;
         final InputStream inputStream = null;
 
-        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null, null));
+        final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null));
         final String transactionId = transactionManager.createTransaction();
 
         final Response response = resource.commitOutputPortTransaction(ResponseCode.CONFIRM_TRANSACTION.getCode(),
@@ -397,7 +396,7 @@ public class TestDataTransferResource {
         final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class);
 
         final HttpFlowFileServerProtocol serverProtocol = mock(HttpFlowFileServerProtocol.class);
-        final DataTransferResource resource = new DataTransferResource(NiFiProperties.createBasicNiFiProperties(null, null)) {
+        final DataTransferResource resource = new DataTransferResource(NiFiProperties.createBasicNiFiProperties(null)) {
             @Override
             protected void authorizeDataTransfer(AuthorizableLookup lookup, ResourceType resourceType, String identifier) {
             }
@@ -407,7 +406,7 @@ public class TestDataTransferResource {
                 return serverProtocol;
             }
         };
-        resource.setProperties(NiFiProperties.createBasicNiFiProperties(null, null));
+        resource.setProperties(NiFiProperties.createBasicNiFiProperties(null));
         resource.setServiceFacade(serviceFacade);
         return resource;
     }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidator.java
index 10aeedd..42e30e8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidator.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/main/java/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidator.java
@@ -20,9 +20,31 @@ import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.math.BigInteger;
+import java.net.URI;
+import java.security.KeyStore;
+import java.security.cert.CertificateException;
+import java.security.cert.CertificateFactory;
+import java.security.cert.X509Certificate;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import javax.net.ssl.X509TrustManager;
+import javax.security.auth.x500.X500Principal;
+import javax.ws.rs.ProcessingException;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.Response;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.framework.security.util.SslContextFactory;
 import org.apache.nifi.security.util.KeyStoreUtils;
+import org.apache.nifi.security.util.SslContextFactory;
+import org.apache.nifi.security.util.TlsConfiguration;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.security.x509.ocsp.OcspStatus.ValidationStatus;
@@ -53,28 +75,6 @@ import org.glassfish.jersey.client.ClientProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.net.ssl.TrustManager;
-import javax.net.ssl.TrustManagerFactory;
-import javax.net.ssl.X509TrustManager;
-import javax.security.auth.x500.X500Principal;
-import javax.ws.rs.ProcessingException;
-import javax.ws.rs.client.Client;
-import javax.ws.rs.client.Entity;
-import javax.ws.rs.client.WebTarget;
-import javax.ws.rs.core.Response;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.math.BigInteger;
-import java.net.URI;
-import java.security.KeyStore;
-import java.security.cert.CertificateException;
-import java.security.cert.CertificateFactory;
-import java.security.cert.X509Certificate;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 public class OcspCertificateValidator {
 
     private static final Logger logger = LoggerFactory.getLogger(OcspCertificateValidator.class);
@@ -107,7 +107,8 @@ public class OcspCertificateValidator {
 
                 // initialize the client
                 if (HTTPS.equalsIgnoreCase(validationAuthorityURI.getScheme())) {
-                    client = WebUtils.createClient(clientConfig, SslContextFactory.createSslContext(properties));
+                    TlsConfiguration tlsConfiguration = TlsConfiguration.fromNiFiProperties(properties);
+                    client = WebUtils.createClient(clientConfig, SslContextFactory.createSslContext(tlsConfiguration));
                 } else {
                     client = WebUtils.createClient(clientConfig);
                 }
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidatorGroovyTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidatorGroovyTest.groovy
index 52c4476..04206c0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidatorGroovyTest.groovy
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/groovy/org/apache/nifi/web/security/x509/ocsp/OcspCertificateValidatorGroovyTest.groovy
@@ -22,10 +22,10 @@ import com.google.common.cache.LoadingCache
 import org.apache.nifi.util.NiFiProperties
 import org.bouncycastle.asn1.x500.X500Name
 import org.bouncycastle.asn1.x509.ExtendedKeyUsage
+import org.bouncycastle.asn1.x509.Extension
 import org.bouncycastle.asn1.x509.KeyPurposeId
 import org.bouncycastle.asn1.x509.KeyUsage
 import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo
-import org.bouncycastle.asn1.x509.Extension
 import org.bouncycastle.cert.X509CertificateHolder
 import org.bouncycastle.cert.X509v3CertificateBuilder
 import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter
@@ -57,18 +57,18 @@ import java.security.cert.X509Certificate
 import static groovy.test.GroovyAssert.shouldFail
 import static org.junit.Assert.fail
 
-public class OcspCertificateValidatorGroovyTest {
-    private static final Logger logger = LoggerFactory.getLogger(OcspCertificateValidatorGroovyTest.class);
+class OcspCertificateValidatorGroovyTest {
+    private static final Logger logger = LoggerFactory.getLogger(OcspCertificateValidatorGroovyTest.class)
 
-    private static final int KEY_SIZE = 2048;
+    private static final int KEY_SIZE = 2048
 
-    private static final long YESTERDAY = System.currentTimeMillis() - 24 * 60 * 60 * 1000;
-    private static final long ONE_YEAR_FROM_NOW = System.currentTimeMillis() + 365 * 24 * 60 * 60 * 1000;
-    private static final String SIGNATURE_ALGORITHM = "SHA256withRSA";
-    private static final String PROVIDER = "BC";
+    private static final long YESTERDAY = System.currentTimeMillis() - 24 * 60 * 60 * 1000
+    private static final long ONE_YEAR_FROM_NOW = System.currentTimeMillis() + 365 * 24 * 60 * 60 * 1000
+    private static final String SIGNATURE_ALGORITHM = "SHA256withRSA"
+    private static final String PROVIDER = "BC"
 
-    private static final String SUBJECT_DN = "CN=NiFi Test Server,OU=Security,O=Apache,ST=CA,C=US";
-    private static final String ISSUER_DN = "CN=NiFi Test CA,OU=Security,O=Apache,ST=CA,C=US";
+    private static final String SUBJECT_DN = "CN=NiFi Test Server,OU=Security,O=Apache,ST=CA,C=US"
+    private static final String ISSUER_DN = "CN=NiFi Test CA,OU=Security,O=Apache,ST=CA,C=US"
 
     private NiFiProperties mockProperties
 
@@ -76,12 +76,12 @@ public class OcspCertificateValidatorGroovyTest {
     OcspCertificateValidator certificateValidator
 
     @BeforeClass
-    public static void setUpOnce() throws Exception {
-        Security.addProvider(new BouncyCastleProvider());
+    static void setUpOnce() throws Exception {
+        Security.addProvider(new BouncyCastleProvider())
     }
 
     @Before
-    public void setUp() throws Exception {
+    void setUp() throws Exception {
         mockProperties = new NiFiProperties() {
             @Override
             String getProperty(String key) {
@@ -96,7 +96,7 @@ public class OcspCertificateValidatorGroovyTest {
     }
 
     @After
-    public void tearDown() throws Exception {
+    void tearDown() throws Exception {
         certificateValidator?.metaClass = null
     }
 
@@ -107,9 +107,9 @@ public class OcspCertificateValidatorGroovyTest {
      * @throws NoSuchAlgorithmException if the RSA algorithm is not available
      */
     private static KeyPair generateKeyPair() throws NoSuchAlgorithmException {
-        KeyPairGenerator keyPairGenerator = KeyPairGenerator.getInstance("RSA");
-        keyPairGenerator.initialize(KEY_SIZE);
-        return keyPairGenerator.generateKeyPair();
+        KeyPairGenerator keyPairGenerator = KeyPairGenerator.getInstance("RSA")
+        keyPairGenerator.initialize(KEY_SIZE)
+        return keyPairGenerator.generateKeyPair()
     }
 
     /**
@@ -127,8 +127,8 @@ public class OcspCertificateValidatorGroovyTest {
      */
     private
     static X509Certificate generateCertificate(String dn) throws IOException, NoSuchAlgorithmException, CertificateException, NoSuchProviderException, SignatureException, InvalidKeyException, OperatorCreationException {
-        KeyPair keyPair = generateKeyPair();
-        return generateCertificate(dn, keyPair);
+        KeyPair keyPair = generateKeyPair()
+        return generateCertificate(dn, keyPair)
     }
 
     /**
@@ -147,34 +147,34 @@ public class OcspCertificateValidatorGroovyTest {
      */
     private
     static X509Certificate generateCertificate(String dn, KeyPair keyPair) throws IOException, NoSuchAlgorithmException, CertificateException, NoSuchProviderException, SignatureException, InvalidKeyException, OperatorCreationException {
-        PrivateKey privateKey = keyPair.getPrivate();
-        ContentSigner sigGen = new JcaContentSignerBuilder(SIGNATURE_ALGORITHM).setProvider(PROVIDER).build(privateKey);
-        SubjectPublicKeyInfo subPubKeyInfo = SubjectPublicKeyInfo.getInstance(keyPair.getPublic().getEncoded());
-        Date startDate = new Date(YESTERDAY);
-        Date endDate = new Date(ONE_YEAR_FROM_NOW);
+        PrivateKey privateKey = keyPair.getPrivate()
+        ContentSigner sigGen = new JcaContentSignerBuilder(SIGNATURE_ALGORITHM).setProvider(PROVIDER).build(privateKey)
+        SubjectPublicKeyInfo subPubKeyInfo = SubjectPublicKeyInfo.getInstance(keyPair.getPublic().getEncoded())
+        Date startDate = new Date(YESTERDAY)
+        Date endDate = new Date(ONE_YEAR_FROM_NOW)
 
         X509v3CertificateBuilder certBuilder = new X509v3CertificateBuilder(
                 new X500Name(dn),
                 BigInteger.valueOf(System.currentTimeMillis()),
                 startDate, endDate,
                 new X500Name(dn),
-                subPubKeyInfo);
+                subPubKeyInfo)
 
         // Set certificate extensions
         // (1) digitalSignature extension
         certBuilder.addExtension(Extension.keyUsage, true,
-                new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyEncipherment | KeyUsage.dataEncipherment | KeyUsage.keyAgreement));
+                new KeyUsage(KeyUsage.digitalSignature | KeyUsage.keyEncipherment | KeyUsage.dataEncipherment | KeyUsage.keyAgreement))
 
         // (2) extendedKeyUsage extension
-        Vector<KeyPurposeId> ekUsages = new Vector<>();
-        ekUsages.add(KeyPurposeId.id_kp_clientAuth);
-        ekUsages.add(KeyPurposeId.id_kp_serverAuth);
-        certBuilder.addExtension(Extension.extendedKeyUsage, false, new ExtendedKeyUsage(ekUsages));
+        Vector<KeyPurposeId> ekUsages = new Vector<>()
+        ekUsages.add(KeyPurposeId.id_kp_clientAuth)
+        ekUsages.add(KeyPurposeId.id_kp_serverAuth)
+        certBuilder.addExtension(Extension.extendedKeyUsage, false, new ExtendedKeyUsage(ekUsages))
 
         // Sign the certificate
-        X509CertificateHolder certificateHolder = certBuilder.build(sigGen);
+        X509CertificateHolder certificateHolder = certBuilder.build(sigGen)
         return new JcaX509CertificateConverter().setProvider(PROVIDER)
-                .getCertificate(certificateHolder);
+                .getCertificate(certificateHolder)
     }
 
     /**
@@ -194,8 +194,8 @@ public class OcspCertificateValidatorGroovyTest {
      */
     private
     static X509Certificate generateIssuedCertificate(String dn, String issuerDn, PrivateKey issuerKey) throws IOException, NoSuchAlgorithmException, CertificateException, NoSuchProviderException, SignatureException, InvalidKeyException, OperatorCreationException {
-        KeyPair keyPair = generateKeyPair();
-        return generateIssuedCertificate(dn, keyPair.getPublic(), issuerDn, issuerKey);
+        KeyPair keyPair = generateKeyPair()
+        return generateIssuedCertificate(dn, keyPair.getPublic(), issuerDn, issuerKey)
     }
 
     /**
@@ -216,98 +216,98 @@ public class OcspCertificateValidatorGroovyTest {
      */
     private
     static X509Certificate generateIssuedCertificate(String dn, PublicKey publicKey, String issuerDn, PrivateKey issuerKey) throws IOException, NoSuchAlgorithmException, CertificateException, NoSuchProviderException, SignatureException, InvalidKeyException, OperatorCreationException {
-        ContentSigner sigGen = new JcaContentSignerBuilder(SIGNATURE_ALGORITHM).setProvider(PROVIDER).build(issuerKey);
-        SubjectPublicKeyInfo subPubKeyInfo = SubjectPublicKeyInfo.getInstance(publicKey.getEncoded());
-        Date startDate = new Date(YESTERDAY);
-        Date endDate = new Date(ONE_YEAR_FROM_NOW);
+        ContentSigner sigGen = new JcaContentSignerBuilder(SIGNATURE_ALGORITHM).setProvider(PROVIDER).build(issuerKey)
+        SubjectPublicKeyInfo subPubKeyInfo = SubjectPublicKeyInfo.getInstance(publicKey.getEncoded())
+        Date startDate = new Date(YESTERDAY)
+        Date endDate = new Date(ONE_YEAR_FROM_NOW)
 
         X509v3CertificateBuilder v3CertGen = new X509v3CertificateBuilder(
                 new X500Name(issuerDn),
                 BigInteger.valueOf(System.currentTimeMillis()),
                 startDate, endDate,
                 new X500Name(dn),
-                subPubKeyInfo);
+                subPubKeyInfo)
 
-        X509CertificateHolder certificateHolder = v3CertGen.build(sigGen);
+        X509CertificateHolder certificateHolder = v3CertGen.build(sigGen)
         return new JcaX509CertificateConverter().setProvider(PROVIDER)
-                .getCertificate(certificateHolder);
+                .getCertificate(certificateHolder)
     }
 
     private static X509Certificate[] generateCertificateChain(String dn = SUBJECT_DN, String issuerDn = ISSUER_DN) {
-        final KeyPair issuerKeyPair = generateKeyPair();
-        final PrivateKey issuerPrivateKey = issuerKeyPair.getPrivate();
+        final KeyPair issuerKeyPair = generateKeyPair()
+        final PrivateKey issuerPrivateKey = issuerKeyPair.getPrivate()
 
-        final X509Certificate issuerCertificate = generateCertificate(issuerDn, issuerKeyPair);
-        final X509Certificate certificate = generateIssuedCertificate(dn, issuerDn, issuerPrivateKey);
+        final X509Certificate issuerCertificate = generateCertificate(issuerDn, issuerKeyPair)
+        final X509Certificate certificate = generateIssuedCertificate(dn, issuerDn, issuerPrivateKey)
         [certificate, issuerCertificate] as X509Certificate[]
     }
 
     @Test
-    public void testShouldGenerateCertificate() throws Exception {
+    void testShouldGenerateCertificate() throws Exception {
         // Arrange
-        final String testDn = "CN=This is a test";
+        final String testDn = "CN=This is a test"
 
         // Act
-        X509Certificate certificate = generateCertificate(testDn);
-        logger.info("Generated certificate: \n{}", certificate);
+        X509Certificate certificate = generateCertificate(testDn)
+        logger.info("Generated certificate: \n{}", certificate)
 
         // Assert
-        assert certificate.getSubjectDN().getName().equals(testDn);
-        assert certificate.getIssuerDN().getName().equals(testDn);
-        certificate.verify(certificate.getPublicKey());
+        assert certificate.getSubjectDN().getName() == testDn
+        assert certificate.getIssuerDN().getName() == testDn
+        certificate.verify(certificate.getPublicKey())
     }
 
     @Test
-    public void testShouldGenerateCertificateFromKeyPair() throws Exception {
+    void testShouldGenerateCertificateFromKeyPair() throws Exception {
         // Arrange
-        final String testDn = "CN=This is a test";
-        final KeyPair keyPair = generateKeyPair();
+        final String testDn = "CN=This is a test"
+        final KeyPair keyPair = generateKeyPair()
 
         // Act
-        X509Certificate certificate = generateCertificate(testDn, keyPair);
-        logger.info("Generated certificate: \n{}", certificate);
+        X509Certificate certificate = generateCertificate(testDn, keyPair)
+        logger.info("Generated certificate: \n{}", certificate)
 
         // Assert
-        assert certificate.getPublicKey().equals(keyPair.getPublic());
-        assert certificate.getSubjectDN().getName().equals(testDn);
-        assert certificate.getIssuerDN().getName().equals(testDn);
-        certificate.verify(certificate.getPublicKey());
+        assert certificate.getPublicKey() == keyPair.getPublic()
+        assert certificate.getSubjectDN().getName() == testDn
+        assert certificate.getIssuerDN().getName() == testDn
+        certificate.verify(certificate.getPublicKey())
     }
 
     @Test
-    public void testShouldGenerateIssuedCertificate() throws Exception {
+    void testShouldGenerateIssuedCertificate() throws Exception {
         // Arrange
-        final String testDn = "CN=This is a signed test";
-        final String issuerDn = "CN=Issuer CA";
-        final KeyPair issuerKeyPair = generateKeyPair();
-        final PrivateKey issuerPrivateKey = issuerKeyPair.getPrivate();
+        final String testDn = "CN=This is a signed test"
+        final String issuerDn = "CN=Issuer CA"
+        final KeyPair issuerKeyPair = generateKeyPair()
+        final PrivateKey issuerPrivateKey = issuerKeyPair.getPrivate()
 
-        final X509Certificate issuerCertificate = generateCertificate(issuerDn, issuerKeyPair);
-        logger.info("Generated issuer certificate: \n{}", issuerCertificate);
+        final X509Certificate issuerCertificate = generateCertificate(issuerDn, issuerKeyPair)
+        logger.info("Generated issuer certificate: \n{}", issuerCertificate)
 
         // Act
-        X509Certificate certificate = generateIssuedCertificate(testDn, issuerDn, issuerPrivateKey);
-        logger.info("Generated signed certificate: \n{}", certificate);
+        X509Certificate certificate = generateIssuedCertificate(testDn, issuerDn, issuerPrivateKey)
+        logger.info("Generated signed certificate: \n{}", certificate)
 
         // Assert
-        assert issuerCertificate.getPublicKey().equals(issuerKeyPair.getPublic());
-        assert certificate.getSubjectX500Principal().getName().equals(testDn);
-        assert certificate.getIssuerX500Principal().getName().equals(issuerDn);
-        certificate.verify(issuerCertificate.getPublicKey());
+        assert issuerCertificate.getPublicKey() == issuerKeyPair.getPublic()
+        assert certificate.getSubjectX500Principal().getName() == testDn
+        assert certificate.getIssuerX500Principal().getName() == issuerDn
+        certificate.verify(issuerCertificate.getPublicKey())
 
         try {
-            certificate.verify(certificate.getPublicKey());
-            fail("Should have thrown exception");
+            certificate.verify(certificate.getPublicKey())
+            fail("Should have thrown exception")
         } catch (Exception e) {
-            assert e instanceof SignatureException;
-            assert e.getMessage().contains("certificate does not verify with supplied key");
+            assert e instanceof SignatureException
+            assert e.getMessage().contains("certificate does not verify with supplied key")
         }
     }
 
     @Test
-    public void testShouldValidateCertificate() throws Exception {
+    void testShouldValidateCertificate() throws Exception {
         // Arrange
-        X509Certificate[] certificateChain = generateCertificateChain();
+        X509Certificate[] certificateChain = generateCertificateChain()
 
         certificateValidator = new OcspCertificateValidator(mockProperties)
 
@@ -333,14 +333,14 @@ public class OcspCertificateValidatorGroovyTest {
     // TODO - NIFI-1364
     @Ignore("To be implemented with Groovy test")
     @Test
-    public void testShouldNotValidateEmptyCertificate() throws Exception {
+    void testShouldNotValidateEmptyCertificate() throws Exception {
 
     }
 
     @Test
-    public void testShouldNotValidateRevokedCertificate() throws Exception {
+    void testShouldNotValidateRevokedCertificate() throws Exception {
         // Arrange
-        X509Certificate[] certificateChain = generateCertificateChain();
+        X509Certificate[] certificateChain = generateCertificateChain()
 
         certificateValidator = new OcspCertificateValidator(mockProperties)
 
@@ -368,24 +368,24 @@ public class OcspCertificateValidatorGroovyTest {
     LoadingCache<OcspRequest, OcspStatus> buildCacheWithContents(Map map) {
         CacheBuilder.newBuilder().build(new CacheLoader<OcspRequest, OcspStatus>() {
             @Override
-            public OcspStatus load(OcspRequest ocspRequest) throws Exception {
+            OcspStatus load(OcspRequest ocspRequest) throws Exception {
                 logger.info("Mock cache implementation load(${ocspRequest}) returns ${map.get(ocspRequest)}")
                 return map.get(ocspRequest) as OcspStatus
             }
-        });
+        })
     }
 
     // TODO - NIFI-1364
     @Ignore("To be implemented with Groovy test")
     @Test
-    public void testValidateShouldHandleUnsignedResponse() throws Exception {
+    void testValidateShouldHandleUnsignedResponse() throws Exception {
 
     }
 
     // TODO - NIFI-1364
     @Ignore("To be implemented with Groovy test")
     @Test
-    public void testValidateShouldHandleResponseWithIncorrectNonce() throws Exception {
+    void testValidateShouldHandleResponseWithIncorrectNonce() throws Exception {
 
     }
 }
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java
index fc4cf5f..e4a1c2a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java
@@ -95,7 +95,7 @@ public class X509AuthenticationProviderTest {
             return AuthorizationResult.approved();
         });
 
-        x509AuthenticationProvider = new X509AuthenticationProvider(certificateIdentityProvider, authorizer, NiFiProperties.createBasicNiFiProperties(null, null));
+        x509AuthenticationProvider = new X509AuthenticationProvider(certificateIdentityProvider, authorizer, NiFiProperties.createBasicNiFiProperties(null));
     }
 
     @Test(expected = InvalidAuthenticationException.class)
@@ -322,4 +322,4 @@ public class X509AuthenticationProviderTest {
         return certificate;
     }
 
-}
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
index 6d23a89..030fa06 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
@@ -27,7 +27,6 @@
         <module>nifi-client-dto</module>
         <module>nifi-nar-utils</module>
         <module>nifi-runtime</module>
-        <module>nifi-security</module>
         <module>nifi-site-to-site</module>
         <module>nifi-repository-models</module>
         <module>nifi-flowfile-repo-serialization</module>
diff --git a/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/src/main/java/org/apache/nifi/processors/grpc/InvokeGRPC.java b/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/src/main/java/org/apache/nifi/processors/grpc/InvokeGRPC.java
index 7d31f6c..582ff9e 100644
--- a/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/src/main/java/org/apache/nifi/processors/grpc/InvokeGRPC.java
+++ b/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/src/main/java/org/apache/nifi/processors/grpc/InvokeGRPC.java
@@ -17,7 +17,27 @@
 package org.apache.nifi.processors.grpc;
 
 import com.google.protobuf.ByteString;
-
+import io.grpc.CompressorRegistry;
+import io.grpc.DecompressorRegistry;
+import io.grpc.ManagedChannel;
+import io.grpc.netty.GrpcSslContexts;
+import io.grpc.netty.NettyChannelBuilder;
+import io.netty.handler.ssl.SslContextBuilder;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.security.KeyStore;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManagerFactory;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.behavior.EventDriven;
 import org.apache.nifi.annotation.behavior.InputRequirement;
@@ -38,32 +58,9 @@ import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 
-import java.io.FileInputStream;
-import java.io.InputStream;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.security.KeyStore;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManagerFactory;
-
-import io.grpc.CompressorRegistry;
-import io.grpc.DecompressorRegistry;
-import io.grpc.ManagedChannel;
-import io.grpc.netty.GrpcSslContexts;
-import io.grpc.netty.NettyChannelBuilder;
-import io.netty.handler.ssl.SslContextBuilder;
-
 @EventDriven
 @SupportsBatching
 @Tags({"grpc", "rpc", "client"})
@@ -243,7 +240,7 @@ public class InvokeGRPC extends AbstractProcessor {
         // configure whether or not we're using secure comms
         final boolean useSecure = context.getProperty(PROP_USE_SECURE).asBoolean();
         final SSLContextService sslContextService = context.getProperty(PROP_SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-        final SSLContext sslContext = sslContextService == null ? null : sslContextService.createSSLContext(SSLContextService.ClientAuth.NONE);
+        final SSLContext sslContext = sslContextService == null ? null : sslContextService.createSSLContext(SslContextFactory.ClientAuth.NONE);
 
         if (useSecure && sslContext != null) {
             SslContextBuilder sslContextBuilder = GrpcSslContexts.forClient();
diff --git a/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/src/main/java/org/apache/nifi/processors/grpc/ListenGRPC.java b/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/src/main/java/org/apache/nifi/processors/grpc/ListenGRPC.java
index 72d9aff..f34d1bc 100644
--- a/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/src/main/java/org/apache/nifi/processors/grpc/ListenGRPC.java
+++ b/nifi-nar-bundles/nifi-grpc-bundle/nifi-grpc-processors/src/main/java/org/apache/nifi/processors/grpc/ListenGRPC.java
@@ -17,7 +17,32 @@
 package org.apache.nifi.processors.grpc;
 
 import com.google.common.collect.Sets;
-
+import io.grpc.CompressorRegistry;
+import io.grpc.DecompressorRegistry;
+import io.grpc.Server;
+import io.grpc.ServerInterceptors;
+import io.grpc.netty.GrpcSslContexts;
+import io.grpc.netty.NettyServerBuilder;
+import io.netty.handler.ssl.ClientAuth;
+import io.netty.handler.ssl.SslContextBuilder;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManagerFactory;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
@@ -35,38 +60,10 @@ import org.apache.nifi.processor.ProcessSessionFactory;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.RestrictedSSLContextService;
 import org.apache.nifi.ssl.SSLContextService;
 
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.security.KeyStore;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Pattern;
-
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManagerFactory;
-
-import io.grpc.CompressorRegistry;
-import io.grpc.DecompressorRegistry;
-import io.grpc.Server;
-import io.grpc.ServerInterceptors;
-import io.grpc.netty.GrpcSslContexts;
-import io.grpc.netty.NettyServerBuilder;
-import io.netty.handler.ssl.ClientAuth;
-import io.netty.handler.ssl.SslContextBuilder;
-
 @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
 @CapabilityDescription("Starts a gRPC server and listens on the given port to transform the incoming messages into FlowFiles." +
         " The message format is defined by the standard gRPC protobuf IDL provided by NiFi. gRPC isn't intended to carry large payloads," +
@@ -174,7 +171,7 @@ public class ListenGRPC extends AbstractSessionFactoryProcessor {
         final Integer flowControlWindow = context.getProperty(PROP_FLOW_CONTROL_WINDOW).asDataSize(DataUnit.B).intValue();
         final Integer maxMessageSize = context.getProperty(PROP_MAX_MESSAGE_SIZE).asDataSize(DataUnit.B).intValue();
         final SSLContextService sslContextService = context.getProperty(PROP_SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-        final SSLContext sslContext = sslContextService == null ? null : sslContextService.createSSLContext(SSLContextService.ClientAuth.NONE);
+        final SSLContext sslContext = sslContextService == null ? null : sslContextService.createSSLContext(SslContextFactory.ClientAuth.NONE);
         final Pattern authorizedDnPattern = Pattern.compile(context.getProperty(PROP_AUTHORIZED_DN_PATTERN).getValue());
         final FlowFileIngestServiceInterceptor callInterceptor = new FlowFileIngestServiceInterceptor(getLogger());
         callInterceptor.enforceDNPattern(authorizedDnPattern);
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandler.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandler.java
index 4702e89..fe77500 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandler.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryHandler.java
@@ -16,24 +16,23 @@
  */
 package org.apache.nifi.jms.cf;
 
+import static org.apache.nifi.jms.cf.JMSConnectionFactoryProperties.JMS_BROKER_URI;
+import static org.apache.nifi.jms.cf.JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL;
+import static org.apache.nifi.jms.cf.JMSConnectionFactoryProperties.JMS_SSL_CONTEXT_SERVICE;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import javax.jms.ConnectionFactory;
+import javax.net.ssl.SSLContext;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.security.util.SslContextFactory.ClientAuth;
 import org.apache.nifi.ssl.SSLContextService;
-import org.apache.nifi.ssl.SSLContextService.ClientAuth;
-
-import javax.jms.ConnectionFactory;
-import javax.net.ssl.SSLContext;
-import java.lang.reflect.Method;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-
-import static org.apache.nifi.jms.cf.JMSConnectionFactoryProperties.JMS_BROKER_URI;
-import static org.apache.nifi.jms.cf.JMSConnectionFactoryProperties.JMS_CONNECTION_FACTORY_IMPL;
-import static org.apache.nifi.jms.cf.JMSConnectionFactoryProperties.JMS_SSL_CONTEXT_SERVICE;
 
 /**
  * Handler class to create a JMS Connection Factory by instantiating the vendor specific javax.jms.ConnectionFactory
@@ -241,7 +240,7 @@ public class JMSConnectionFactoryHandler implements IJMSConnectionFactoryProvide
      * 'queueManager' property will correspond to setQueueManager method name
      */
     private String toMethodName(String propertyName) {
-        char c[] = propertyName.toCharArray();
+        char[] c = propertyName.toCharArray();
         c[0] = Character.toUpperCase(c[0]);
         return "set" + new String(c);
     }
diff --git a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java
index 58751cd..8ae30a0 100644
--- a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java
+++ b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/LdapProvider.java
@@ -16,6 +16,11 @@
  */
 package org.apache.nifi.ldap;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import javax.naming.Context;
+import javax.net.ssl.SSLContext;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authentication.AuthenticationResponse;
 import org.apache.nifi.authentication.LoginCredentials;
@@ -26,8 +31,11 @@ import org.apache.nifi.authentication.exception.IdentityAccessException;
 import org.apache.nifi.authentication.exception.InvalidLoginCredentialsException;
 import org.apache.nifi.authentication.exception.ProviderCreationException;
 import org.apache.nifi.authentication.exception.ProviderDestructionException;
+import org.apache.nifi.configuration.NonComponentConfigurationContext;
 import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.security.util.SslContextFactory.ClientAuth;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
 import org.apache.nifi.util.FormatUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -47,18 +55,6 @@ import org.springframework.security.ldap.search.FilterBasedLdapUserSearch;
 import org.springframework.security.ldap.search.LdapUserSearch;
 import org.springframework.security.ldap.userdetails.LdapUserDetails;
 
-import javax.naming.Context;
-import javax.net.ssl.SSLContext;
-import java.io.IOException;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
 /**
  * Abstract LDAP based implementation of a login identity provider.
  */
@@ -240,17 +236,18 @@ public class LdapProvider implements LoginIdentityProvider {
         final String rawTimeout = configurationContext.getProperty(configurationProperty);
         if (StringUtils.isNotBlank(rawTimeout)) {
             try {
-                final Long timeout = FormatUtils.getTimeDuration(rawTimeout, TimeUnit.MILLISECONDS);
-                baseEnvironment.put(environmentKey, timeout.toString());
+                final long timeout = (long) FormatUtils.getPreciseTimeDuration(rawTimeout, TimeUnit.MILLISECONDS);
+                baseEnvironment.put(environmentKey, timeout);
             } catch (final IllegalArgumentException iae) {
                 throw new ProviderCreationException(String.format("The %s '%s' is not a valid time duration", configurationProperty, rawTimeout));
             }
         }
     }
 
-    private SSLContext getConfiguredSslContext(final LoginIdentityProviderConfigurationContext configurationContext) {
+    public static SSLContext getConfiguredSslContext(final NonComponentConfigurationContext configurationContext) {
         final String rawKeystore = configurationContext.getProperty("TLS - Keystore");
         final String rawKeystorePassword = configurationContext.getProperty("TLS - Keystore Password");
+        // TODO: Should support different key password
         final String rawKeystoreType = configurationContext.getProperty("TLS - Keystore Type");
         final String rawTruststore = configurationContext.getProperty("TLS - Truststore");
         final String rawTruststorePassword = configurationContext.getProperty("TLS - Truststore Password");
@@ -258,44 +255,14 @@ public class LdapProvider implements LoginIdentityProvider {
         final String rawClientAuth = configurationContext.getProperty("TLS - Client Auth");
         final String rawProtocol = configurationContext.getProperty("TLS - Protocol");
 
-        // create the ssl context
-        final SSLContext sslContext;
         try {
-            if (StringUtils.isBlank(rawKeystore) && StringUtils.isBlank(rawTruststore)) {
-                sslContext = null;
-            } else {
-                // ensure the protocol is specified
-                if (StringUtils.isBlank(rawProtocol)) {
-                    throw new ProviderCreationException("TLS - Protocol must be specified.");
-                }
-
-                if (StringUtils.isBlank(rawKeystore)) {
-                    sslContext = SslContextFactory.createTrustSslContext(rawTruststore, rawTruststorePassword.toCharArray(), rawTruststoreType, rawProtocol);
-                } else if (StringUtils.isBlank(rawTruststore)) {
-                    sslContext = SslContextFactory.createSslContext(rawKeystore, rawKeystorePassword.toCharArray(), rawKeystoreType, rawProtocol);
-                } else {
-                    // determine the client auth if specified
-                    final ClientAuth clientAuth;
-                    if (StringUtils.isBlank(rawClientAuth)) {
-                        clientAuth = ClientAuth.NONE;
-                    } else {
-                        try {
-                            clientAuth = ClientAuth.valueOf(rawClientAuth);
-                        } catch (final IllegalArgumentException iae) {
-                            throw new ProviderCreationException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
-                                    rawClientAuth, StringUtils.join(ClientAuth.values(), ", ")));
-                        }
-                    }
-
-                    sslContext = SslContextFactory.createSslContext(rawKeystore, rawKeystorePassword.toCharArray(), rawKeystoreType,
-                            rawTruststore, rawTruststorePassword.toCharArray(), rawTruststoreType, clientAuth, rawProtocol);
-                }
-            }
-        } catch (final KeyStoreException | NoSuchAlgorithmException | CertificateException | UnrecoverableKeyException | KeyManagementException | IOException e) {
-            throw new ProviderCreationException(e.getMessage(), e);
+            TlsConfiguration tlsConfiguration = new TlsConfiguration(rawKeystore, rawKeystorePassword, null, rawKeystoreType, rawTruststore, rawTruststorePassword, rawTruststoreType, rawProtocol);
+            ClientAuth clientAuth = ClientAuth.isValidClientAuthType(rawClientAuth) ? ClientAuth.valueOf(rawClientAuth) : ClientAuth.NONE;
+            return SslContextFactory.createSslContext(tlsConfiguration, clientAuth);
+        } catch (TlsException e) {
+            logger.error("Encountered an error configuring TLS for LDAP identity provider: {}", e.getLocalizedMessage());
+            throw new ProviderCreationException("Error configuring TLS for LDAP identity provider", e);
         }
-
-        return sslContext;
     }
 
     @Override
diff --git a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/tenants/LdapUserGroupProvider.java b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/tenants/LdapUserGroupProvider.java
index 9cb4af3..9d4bab0 100644
--- a/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/tenants/LdapUserGroupProvider.java
+++ b/nifi-nar-bundles/nifi-ldap-iaa-providers-bundle/nifi-ldap-iaa-providers/src/main/java/org/apache/nifi/ldap/tenants/LdapUserGroupProvider.java
@@ -16,7 +16,26 @@
  */
 package org.apache.nifi.ldap.tenants;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.naming.Context;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.SearchControls;
+import javax.net.ssl.SSLContext;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.authentication.exception.ProviderCreationException;
 import org.apache.nifi.authentication.exception.ProviderDestructionException;
 import org.apache.nifi.authorization.AuthorizerConfigurationContext;
 import org.apache.nifi.authorization.Group;
@@ -35,6 +54,8 @@ import org.apache.nifi.ldap.LdapsSocketFactory;
 import org.apache.nifi.ldap.ReferralStrategy;
 import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.security.util.SslContextFactory.ClientAuth;
+import org.apache.nifi.security.util.TlsConfiguration;
+import org.apache.nifi.security.util.TlsException;
 import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.slf4j.Logger;
@@ -56,31 +77,6 @@ import org.springframework.ldap.filter.AndFilter;
 import org.springframework.ldap.filter.EqualsFilter;
 import org.springframework.ldap.filter.HardcodedFilter;
 
-import javax.naming.Context;
-import javax.naming.NamingEnumeration;
-import javax.naming.NamingException;
-import javax.naming.directory.Attribute;
-import javax.naming.directory.SearchControls;
-import javax.net.ssl.SSLContext;
-import java.io.IOException;
-import java.security.KeyManagementException;
-import java.security.KeyStoreException;
-import java.security.NoSuchAlgorithmException;
-import java.security.UnrecoverableKeyException;
-import java.security.cert.CertificateException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
 /**
  * Abstract LDAP based implementation of a login identity provider.
  */
@@ -827,44 +823,14 @@ public class LdapUserGroupProvider implements UserGroupProvider {
         final String rawClientAuth = configurationContext.getProperty("TLS - Client Auth").getValue();
         final String rawProtocol = configurationContext.getProperty("TLS - Protocol").getValue();
 
-        // create the ssl context
-        final SSLContext sslContext;
         try {
-            if (StringUtils.isBlank(rawKeystore) && StringUtils.isBlank(rawTruststore)) {
-                sslContext = null;
-            } else {
-                // ensure the protocol is specified
-                if (StringUtils.isBlank(rawProtocol)) {
-                    throw new AuthorizerCreationException("TLS - Protocol must be specified.");
-                }
-
-                if (StringUtils.isBlank(rawKeystore)) {
-                    sslContext = SslContextFactory.createTrustSslContext(rawTruststore, rawTruststorePassword.toCharArray(), rawTruststoreType, rawProtocol);
-                } else if (StringUtils.isBlank(rawTruststore)) {
-                    sslContext = SslContextFactory.createSslContext(rawKeystore, rawKeystorePassword.toCharArray(), rawKeystoreType, rawProtocol);
-                } else {
-                    // determine the client auth if specified
-                    final ClientAuth clientAuth;
-                    if (StringUtils.isBlank(rawClientAuth)) {
-                        clientAuth = ClientAuth.NONE;
-                    } else {
-                        try {
-                            clientAuth = ClientAuth.valueOf(rawClientAuth);
-                        } catch (final IllegalArgumentException iae) {
-                            throw new AuthorizerCreationException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
-                                    rawClientAuth, StringUtils.join(ClientAuth.values(), ", ")));
-                        }
-                    }
-
-                    sslContext = SslContextFactory.createSslContext(rawKeystore, rawKeystorePassword.toCharArray(), rawKeystoreType,
-                            rawTruststore, rawTruststorePassword.toCharArray(), rawTruststoreType, clientAuth, rawProtocol);
-                }
-            }
-        } catch (final KeyStoreException | NoSuchAlgorithmException | CertificateException | UnrecoverableKeyException | KeyManagementException | IOException e) {
-            throw new AuthorizerCreationException(e.getMessage(), e);
+            TlsConfiguration tlsConfiguration = new TlsConfiguration(rawKeystore, rawKeystorePassword, null, rawKeystoreType, rawTruststore, rawTruststorePassword, rawTruststoreType, rawProtocol);
+            ClientAuth clientAuth = ClientAuth.isValidClientAuthType(rawClientAuth) ? ClientAuth.valueOf(rawClientAuth) : ClientAuth.NONE;
+            return SslContextFactory.createSslContext(tlsConfiguration, clientAuth);
+        } catch (TlsException e) {
+            logger.error("Encountered an error configuring TLS for LDAP user group provider: {}", e.getLocalizedMessage());
+            throw new ProviderCreationException("Error configuring TLS for LDAP user group provider", e);
         }
-
-        return sslContext;
     }
 
 }
diff --git a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/ListenLumberjack.java b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/ListenLumberjack.java
index d189fac..ec9ffde 100644
--- a/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/ListenLumberjack.java
+++ b/nifi-nar-bundles/nifi-lumberjack-bundle/nifi-lumberjack-processors/src/main/java/org/apache/nifi/processors/lumberjack/ListenLumberjack.java
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.processors.lumberjack;
 
+import com.google.gson.Gson;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
@@ -26,9 +27,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
-
 import javax.net.ssl.SSLContext;
-
 import org.apache.nifi.annotation.behavior.InputRequirement;
 import org.apache.nifi.annotation.behavior.WritesAttribute;
 import org.apache.nifi.annotation.behavior.WritesAttributes;
@@ -58,11 +57,10 @@ import org.apache.nifi.processors.lumberjack.frame.LumberjackEncoder;
 import org.apache.nifi.processors.lumberjack.handler.LumberjackSocketChannelHandlerFactory;
 import org.apache.nifi.processors.lumberjack.response.LumberjackChannelResponse;
 import org.apache.nifi.processors.lumberjack.response.LumberjackResponse;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.RestrictedSSLContextService;
 import org.apache.nifi.ssl.SSLContextService;
 
-import com.google.gson.Gson;
-
 @Deprecated
 @InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
 @Tags({"listen", "lumberjack", "tcp", "logs"})
@@ -143,7 +141,7 @@ public class ListenLumberjack extends AbstractListenEventBatchingProcessor<Lumbe
         SSLContext sslContext = null;
         final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
         if (sslContextService != null) {
-            sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.REQUIRED);
+            sslContext = sslContextService.createSSLContext(SslContextFactory.ClientAuth.REQUIRED);
         }
 
         // if we decide to support SSL then get the context and pass it in here
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-client-service-api/src/main/java/org/apache/nifi/mongodb/MongoDBClientService.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-client-service-api/src/main/java/org/apache/nifi/mongodb/MongoDBClientService.java
index 2ae9265..e00bed4 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-client-service-api/src/main/java/org/apache/nifi/mongodb/MongoDBClientService.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-client-service-api/src/main/java/org/apache/nifi/mongodb/MongoDBClientService.java
@@ -24,6 +24,7 @@ import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 import org.bson.Document;
 
@@ -58,7 +59,7 @@ public interface MongoDBClientService extends ControllerService {
                     + "Possible values are REQUIRED, WANT, NONE. This property is only used when an SSL Context "
                     + "has been defined and enabled.")
             .required(false)
-            .allowableValues(SSLContextService.ClientAuth.values())
+            .allowableValues(SslContextFactory.ClientAuth.values())
             .defaultValue("REQUIRED")
             .build();
 
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
index df918d9..524f6fb 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessor.java
@@ -26,6 +26,18 @@ import com.mongodb.MongoClientURI;
 import com.mongodb.WriteConcern;
 import com.mongodb.client.MongoCollection;
 import com.mongodb.client.MongoDatabase;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import javax.net.ssl.SSLContext;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.lifecycle.OnScheduled;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
@@ -46,19 +58,6 @@ import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 import org.bson.Document;
 
-import javax.net.ssl.SSLContext;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-
 public abstract class AbstractMongoProcessor extends AbstractProcessor {
     static final String WRITE_CONCERN_ACKNOWLEDGED = "ACKNOWLEDGED";
     static final String WRITE_CONCERN_UNACKNOWLEDGED = "UNACKNOWLEDGED";
@@ -136,7 +135,7 @@ public abstract class AbstractMongoProcessor extends AbstractProcessor {
                     + "Possible values are REQUIRED, WANT, NONE. This property is only used when an SSL Context "
                     + "has been defined and enabled.")
             .required(false)
-            .allowableValues(SSLContextService.ClientAuth.values())
+            .allowableValues(SslContextFactory.ClientAuth.values())
             .defaultValue("REQUIRED")
             .build();
 
@@ -246,12 +245,12 @@ public abstract class AbstractMongoProcessor extends AbstractProcessor {
         final SSLContext sslContext;
 
         if (sslService != null) {
-            final SSLContextService.ClientAuth clientAuth;
+            final SslContextFactory.ClientAuth clientAuth;
             if (StringUtils.isBlank(rawClientAuth)) {
-                clientAuth = SSLContextService.ClientAuth.REQUIRED;
+                clientAuth = SslContextFactory.ClientAuth.REQUIRED;
             } else {
                 try {
-                    clientAuth = SSLContextService.ClientAuth.valueOf(rawClientAuth);
+                    clientAuth = SslContextFactory.ClientAuth.valueOf(rawClientAuth);
                 } catch (final IllegalArgumentException iae) {
                     throw new IllegalStateException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
                             rawClientAuth, StringUtils.join(SslContextFactory.ClientAuth.values(), ", ")));
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessorTest.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessorTest.java
index 1ae2b82..8489af0 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessorTest.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/test/java/org/apache/nifi/processors/mongodb/AbstractMongoProcessorTest.java
@@ -16,25 +16,24 @@
  */
 package org.apache.nifi.processors.mongodb;
 
+import static org.junit.Assert.assertNotNull;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 import com.mongodb.MongoClientOptions;
 import com.mongodb.MongoClientOptions.Builder;
+import javax.net.ssl.SSLContext;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.ProcessSession;
 import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.security.util.SslContextFactory.ClientAuth;
 import org.apache.nifi.ssl.SSLContextService;
-import org.apache.nifi.ssl.SSLContextService.ClientAuth;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.Before;
 import org.junit.Test;
 
-import javax.net.ssl.SSLContext;
-
-import static org.junit.Assert.assertNotNull;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 public class AbstractMongoProcessorTest {
 
     MockAbstractMongoProcessor processor;
diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/MongoDBControllerService.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/MongoDBControllerService.java
index 6fdebf5..252e0d1 100644
--- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/MongoDBControllerService.java
+++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-services/src/main/java/org/apache/nifi/mongodb/MongoDBControllerService.java
@@ -22,6 +22,9 @@ import com.mongodb.MongoClientOptions;
 import com.mongodb.MongoClientURI;
 import com.mongodb.WriteConcern;
 import com.mongodb.client.MongoDatabase;
+import java.util.ArrayList;
+import java.util.List;
+import javax.net.ssl.SSLContext;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
@@ -34,10 +37,6 @@ import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.SSLContextService;
 
-import javax.net.ssl.SSLContext;
-import java.util.ArrayList;
-import java.util.List;
-
 @Tags({"mongo", "mongodb", "service"})
 @CapabilityDescription(
         "Provides a controller service that configures a connection to MongoDB and provides access to that connection to " +
@@ -75,12 +74,12 @@ public class MongoDBControllerService extends AbstractControllerService implemen
         final SSLContext sslContext;
 
         if (sslService != null) {
-            final SSLContextService.ClientAuth clientAuth;
+            final SslContextFactory.ClientAuth clientAuth;
             if (StringUtils.isBlank(rawClientAuth)) {
-                clientAuth = SSLContextService.ClientAuth.REQUIRED;
+                clientAuth = SslContextFactory.ClientAuth.REQUIRED;
             } else {
                 try {
-                    clientAuth = SSLContextService.ClientAuth.valueOf(rawClientAuth);
+                    clientAuth = SslContextFactory.ClientAuth.valueOf(rawClientAuth);
                 } catch (final IllegalArgumentException iae) {
                     throw new IllegalStateException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
                             rawClientAuth, StringUtils.join(SslContextFactory.ClientAuth.values(), ", ")));
diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
index 5ccf6ea..dfbe15b 100644
--- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
+++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java
@@ -16,14 +16,7 @@
  */
 package org.apache.nifi.provenance;
 
-import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.provenance.search.Query;
-import org.apache.nifi.provenance.search.QuerySubmission;
-import org.apache.nifi.provenance.search.SearchTerms;
-import org.apache.nifi.util.NiFiProperties;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -32,8 +25,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-
-import static org.junit.Assert.assertEquals;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.provenance.search.Query;
+import org.apache.nifi.provenance.search.QuerySubmission;
+import org.apache.nifi.provenance.search.SearchTerms;
+import org.apache.nifi.util.NiFiProperties;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 public class TestVolatileProvenanceRepository {
 
@@ -46,7 +45,7 @@ public class TestVolatileProvenanceRepository {
 
     @Test
     public void testAddAndGet() throws IOException, InterruptedException {
-        repo = new VolatileProvenanceRepository(NiFiProperties.createBasicNiFiProperties(null, null));
+        repo = new VolatileProvenanceRepository(NiFiProperties.createBasicNiFiProperties(null));
 
         final Map<String, String> attributes = new HashMap<>();
         attributes.put("abc", "xyz");
@@ -79,7 +78,7 @@ public class TestVolatileProvenanceRepository {
 
     @Test
     public void testIndexAndCompressOnRolloverAndSubsequentSearchAsync() throws InterruptedException {
-        repo = new VolatileProvenanceRepository(NiFiProperties.createBasicNiFiProperties(null, null));
+        repo = new VolatileProvenanceRepository(NiFiProperties.createBasicNiFiProperties(null));
 
         final String uuid = "00000000-0000-0000-0000-000000000000";
         final Map<String, String> attributes = new HashMap<>();
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java
index 17a4720..5f8328f 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java
@@ -29,11 +29,9 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.function.Supplier;
-
 import javax.json.JsonArray;
 import javax.json.JsonObjectBuilder;
 import javax.json.JsonValue;
-
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
@@ -118,7 +116,7 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
 
     public void setup(final ReportingContext reportContext) throws IOException {
         if (siteToSiteClient == null) {
-            siteToSiteClient = SiteToSiteUtils.getClient(reportContext, getLogger());
+            siteToSiteClient = SiteToSiteUtils.getClient(reportContext, getLogger(), null);
         }
     }
 
@@ -269,7 +267,7 @@ public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingT
             return recordSchema;
         }
 
-        private JsonNode getNextJsonNode() throws JsonParseException, IOException, MalformedRecordException {
+        private JsonNode getNextJsonNode() throws IOException, MalformedRecordException {
             if (!firstObjectConsumed) {
                 firstObjectConsumed = true;
                 return firstJsonNode;
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/s2s/SiteToSiteUtils.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/s2s/SiteToSiteUtils.java
index feea3dd..8d6f10c 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/s2s/SiteToSiteUtils.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/s2s/SiteToSiteUtils.java
@@ -16,10 +16,14 @@
  */
 package org.apache.nifi.reporting.s2s;
 
+import java.util.concurrent.TimeUnit;
+import javax.net.ssl.SSLContext;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
 import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.components.Validator;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.context.PropertyContext;
 import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
@@ -29,13 +33,11 @@ import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
 import org.apache.nifi.remote.protocol.http.HttpProxy;
 import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
 import org.apache.nifi.reporting.ReportingContext;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.ssl.RestrictedSSLContextService;
 import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.util.StringUtils;
 
-import javax.net.ssl.SSLContext;
-import java.util.concurrent.TimeUnit;
-
 public class SiteToSiteUtils {
 
     public static final PropertyDescriptor DESTINATION_URL = new PropertyDescriptor.Builder()
@@ -143,9 +145,9 @@ public class SiteToSiteUtils {
             .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
             .build();
 
-    public static SiteToSiteClient getClient(ReportingContext reportContext, ComponentLog logger) {
+    public static SiteToSiteClient getClient(PropertyContext reportContext, ComponentLog logger, StateManager stateManager) {
         final SSLContextService sslContextService = reportContext.getProperty(SiteToSiteUtils.SSL_CONTEXT).asControllerService(SSLContextService.class);
-        final SSLContext sslContext = sslContextService == null ? null : sslContextService.createSSLContext(SSLContextService.ClientAuth.REQUIRED);
+        final SSLContext sslContext = sslContextService == null ? null : sslContextService.createSSLContext(SslContextFactory.ClientAuth.REQUIRED);
         final EventReporter eventReporter = (EventReporter) (severity, category, message) -> {
             switch (severity) {
                 case WARNING:
@@ -165,6 +167,10 @@ public class SiteToSiteUtils {
                 : new HttpProxy(reportContext.getProperty(SiteToSiteUtils.HTTP_PROXY_HOSTNAME).getValue(), reportContext.getProperty(SiteToSiteUtils.HTTP_PROXY_PORT).asInteger(),
                 reportContext.getProperty(SiteToSiteUtils.HTTP_PROXY_USERNAME).getValue(), reportContext.getProperty(SiteToSiteUtils.HTTP_PROXY_PASSWORD).getValue());
 
+        // If no state manager was provided and this context supports retrieving it, do so
+        if (stateManager == null && reportContext instanceof ReportingContext) {
+            stateManager = ((ReportingContext) reportContext).getStateManager();
+        }
         return new SiteToSiteClient.Builder()
                 .urls(SiteToSiteRestApiClient.parseClusterUrls(destinationUrl))
                 .portName(reportContext.getProperty(SiteToSiteUtils.PORT_NAME).getValue())
@@ -174,7 +180,7 @@ public class SiteToSiteUtils {
                 .timeout(reportContext.getProperty(SiteToSiteUtils.TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)
                 .transportProtocol(mode)
                 .httpProxy(httpProxy)
-                .stateManager(reportContext.getStateManager())
+                .stateManager(stateManager)
                 .build();
     }
 
diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/sink/SiteToSiteReportingRecordSink.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/sink/SiteToSiteReportingRecordSink.java
index b2ed107..45acdb5 100644
--- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/sink/SiteToSiteReportingRecordSink.java
+++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/sink/SiteToSiteReportingRecordSink.java
@@ -16,6 +16,12 @@
  */
 package org.apache.nifi.reporting.sink;
 
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnDisabled;
@@ -25,16 +31,12 @@ import org.apache.nifi.components.state.StateManager;
 import org.apache.nifi.controller.AbstractControllerService;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerServiceInitializationContext;
-import org.apache.nifi.events.EventReporter;
 import org.apache.nifi.flowfile.attributes.CoreAttributes;
 import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.record.sink.RecordSinkService;
 import org.apache.nifi.remote.Transaction;
 import org.apache.nifi.remote.TransferDirection;
 import org.apache.nifi.remote.client.SiteToSiteClient;
-import org.apache.nifi.remote.protocol.SiteToSiteTransportProtocol;
-import org.apache.nifi.remote.protocol.http.HttpProxy;
-import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
 import org.apache.nifi.reporting.InitializationException;
 import org.apache.nifi.reporting.s2s.SiteToSiteUtils;
 import org.apache.nifi.serialization.RecordSetWriter;
@@ -43,17 +45,6 @@ import org.apache.nifi.serialization.WriteResult;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.RecordSet;
-import org.apache.nifi.ssl.SSLContextService;
-import org.apache.nifi.util.StringUtils;
-
-import javax.net.ssl.SSLContext;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
 
 @Tags({ "db", "s2s", "site", "record"})
 @CapabilityDescription("Provides a service to write records using a configured RecordSetWriter over a Site-to-Site connection.")
@@ -92,40 +83,8 @@ public class SiteToSiteReportingRecordSink extends AbstractControllerService imp
     @OnEnabled
     public void onEnabled(final ConfigurationContext context) throws InitializationException {
         try {
-            final SSLContextService sslContextService = context.getProperty(SiteToSiteUtils.SSL_CONTEXT).asControllerService(SSLContextService.class);
-            final SSLContext sslContext = sslContextService == null ? null : sslContextService.createSSLContext(SSLContextService.ClientAuth.REQUIRED);
             final ComponentLog logger = getLogger();
-            final EventReporter eventReporter = (EventReporter) (severity, category, message) -> {
-                switch (severity) {
-                    case WARNING:
-                        logger.warn(message);
-                        break;
-                    case ERROR:
-                        logger.error(message);
-                        break;
-                    default:
-                        break;
-                }
-            };
-
-            final String destinationUrl = context.getProperty(SiteToSiteUtils.DESTINATION_URL).evaluateAttributeExpressions().getValue();
-
-            final SiteToSiteTransportProtocol mode = SiteToSiteTransportProtocol.valueOf(context.getProperty(SiteToSiteUtils.TRANSPORT_PROTOCOL).getValue());
-            final HttpProxy httpProxy = mode.equals(SiteToSiteTransportProtocol.RAW) || StringUtils.isEmpty(context.getProperty(SiteToSiteUtils.HTTP_PROXY_HOSTNAME).getValue()) ? null
-                    : new HttpProxy(context.getProperty(SiteToSiteUtils.HTTP_PROXY_HOSTNAME).getValue(), context.getProperty(SiteToSiteUtils.HTTP_PROXY_PORT).asInteger(),
-                    context.getProperty(SiteToSiteUtils.HTTP_PROXY_USERNAME).getValue(), context.getProperty(SiteToSiteUtils.HTTP_PROXY_PASSWORD).getValue());
-
-            siteToSiteClient = new SiteToSiteClient.Builder()
-                    .urls(SiteToSiteRestApiClient.parseClusterUrls(destinationUrl))
-                    .portName(context.getProperty(SiteToSiteUtils.PORT_NAME).getValue())
-                    .useCompression(context.getProperty(SiteToSiteUtils.COMPRESS).asBoolean())
-                    .eventReporter(eventReporter)
-                    .sslContext(sslContext)
-                    .stateManager(stateManager)
-                    .timeout(context.getProperty(SiteToSiteUtils.TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)
-                    .transportProtocol(mode)
-                    .httpProxy(httpProxy)
-                    .build();
+            siteToSiteClient = SiteToSiteUtils.getClient(context, logger, stateManager);
 
             writerFactory = context.getProperty(RECORD_WRITER_FACTORY).asControllerService(RecordSetWriterFactory.class);
         } catch(Exception e) {
diff --git a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java
index 68a7ff2..60fa072 100644
--- a/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java
+++ b/nifi-nar-bundles/nifi-solr-bundle/nifi-solr-processors/src/main/java/org/apache/nifi/processors/solr/SolrUtils.java
@@ -18,14 +18,33 @@
  */
 package org.apache.nifi.processors.solr;
 
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import javax.net.ssl.SSLContext;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.http.client.HttpClient;
 import org.apache.http.config.Registry;
 import org.apache.http.config.RegistryBuilder;
-import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
 import org.apache.http.conn.socket.ConnectionSocketFactory;
 import org.apache.http.conn.socket.PlainConnectionSocketFactory;
+import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
 import org.apache.nifi.components.AllowableValue;
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.context.PropertyContext;
@@ -36,6 +55,7 @@ import org.apache.nifi.kerberos.KerberosCredentialsService;
 import org.apache.nifi.processor.ProcessContext;
 import org.apache.nifi.processor.io.OutputStreamCallback;
 import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.security.util.SslContextFactory;
 import org.apache.nifi.serialization.RecordSetWriterFactory;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.ListRecordSet;
@@ -61,27 +81,6 @@ import org.apache.solr.common.params.MultiMapSolrParams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.net.ssl.SSLContext;
-import java.io.IOException;
-import java.io.OutputStream;
... 3864 lines suppressed ...