You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/05/11 01:29:19 UTC

[GitHub] [nifi] alopresto opened a new pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

alopresto opened a new pull request #4263:
URL: https://github.com/apache/nifi/pull/4263


   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _In order to streamline the creation of `SSLContext` and `SSLSocket` objects throughout the application, I refactored the various near-duplicate but slightly-different factory objects. This also involved refactoring duplicate enums and removing legacy modules which were no longer necessary. Additional comments, unit tests, and regression tests were introduced._
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [x] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `master`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [x] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [x] Have you written or updated unit tests to verify your changes?
   - [x] Have you verified that the full build is successful on both JDK 8 and JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


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

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



[GitHub] [nifi] alopresto commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto commented on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-628984876


   Force pushed as I had to fix the referenced Jira number in the recent commit messages. 


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

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



[GitHub] [nifi] thenatog commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

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


   Looks like there might be a small issue with JDK11 tests, if we can fix that I'll +1. Thanks for the huge contribution, Andy! Definitely valuable changes here - the SSL Contexts have needed this improvement for a long time.


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

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



[GitHub] [nifi] alopresto commented on a change in pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto commented on a change in pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#discussion_r424763709



##########
File path: 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,185 @@
+/*
+ * 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, ["TLSv1.2", "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

Review comment:
       Added test case timing check. 




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

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



[GitHub] [nifi] alopresto commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto commented on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-630485289


   Running a full build this time because some of the tests were failing on ordering. 


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

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



[GitHub] [nifi] alopresto commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto commented on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-630562444


   Thanks for finding all the edge cases @thenatog & @markap14. I think this is ready for your +1. I'll then merge. 


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

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



[GitHub] [nifi] thenatog commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

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


   +1, looks good. Thanks Andy!


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

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



[GitHub] [nifi] alopresto edited a comment on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto edited a comment on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-628794810


   I made the dropdown for `RestrictedSSLContextService` more explicit where it now provides `TLS, TLSv1.2` on Java 8 and `TLS, TLSv1.2, TLSv1.3` on Java 11. Selecting `TLS` will allow connections over `TLSv1.2` _and_ `TLSv1.3` (on Java 11 _only_. Java 8 does not support `TLSv1.3`). 
   
   ### With `TLSv1.2` selected:
   
   ```
   
   # TLSv1.2 is successful
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_2
   CONNECTED(00000003)
   depth=1 OU = NIFI, CN = ca.nifi
   verify return:1
   depth=0 OU = NIFI, CN = node1.nifi
   verify return:1
   ---
   Certificate chain
    0 s:OU = NIFI, CN = node1.nifi
      i:OU = NIFI, CN = ca.nifi
    1 s:OU = NIFI, CN = ca.nifi
      i:OU = NIFI, CN = ca.nifi
   ---
   ...
   ---
   SSL handshake has read 2289 bytes and written 1464 bytes
   Verification: OK
   ---
   New, TLSv1.2, Cipher is ECDHE-RSA-AES256-GCM-SHA384
   Server public key is 2048 bit
   Secure Renegotiation IS supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   SSL-Session:
       Protocol  : TLSv1.2
       Cipher    : ECDHE-RSA-AES256-GCM-SHA384
       Session-ID: BA2FC4...0D2790
       Session-ID-ctx:
       Master-Key: C773AC...A85A19
       PSK identity: None
       PSK identity hint: None
       SRP username: None
       Start Time: 1589478477
       Timeout   : 7200 (sec)
       Verify return code: 0 (ok)
       Extended master secret: yes
   ---
   DONE
   
   # TLSv1.3 fails
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_3
   CONNECTED(00000003)
   4570201536:error:1409442E:SSL routines:ssl3_read_bytes:tlsv1 alert protocol version:ssl/record/rec_layer_s3.c:1544:SSL alert number 70
   ---
   no peer certificate available
   ---
   No client certificate CA names sent
   ---
   SSL handshake has read 7 bytes and written 234 bytes
   Verification: OK
   ---
   New, (NONE), Cipher is (NONE)
   Secure Renegotiation IS NOT supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   Early data was not sent
   Verify return code: 0 (ok)
   ---
    ✘  ..oolkit-1.11.4   master ● 
   ```
   
   # With `TLS` selected:
   
   ```
   
   ### TLSv1.3 is successful
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_3
   CONNECTED(00000003)
   depth=1 OU = NIFI, CN = ca.nifi
   verify return:1
   depth=0 OU = NIFI, CN = node1.nifi
   verify return:1
   ---
   Certificate chain
    0 s:OU = NIFI, CN = node1.nifi
      i:OU = NIFI, CN = ca.nifi
    1 s:OU = NIFI, CN = ca.nifi
      i:OU = NIFI, CN = ca.nifi
   ---
   ...
   ---
   SSL handshake has read 2510 bytes and written 1800 bytes
   Verification: OK
   ---
   New, TLSv1.3, Cipher is TLS_AES_128_GCM_SHA256
   Server public key is 2048 bit
   Secure Renegotiation IS NOT supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   Early data was not sent
   Verify return code: 0 (ok)
   ---
   DONE
   
   # TLSv1.2 is successful
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_2
   CONNECTED(00000003)
   depth=1 OU = NIFI, CN = ca.nifi
   verify return:1
   depth=0 OU = NIFI, CN = node1.nifi
   verify return:1
   ---
   Certificate chain
    0 s:OU = NIFI, CN = node1.nifi
      i:OU = NIFI, CN = ca.nifi
    1 s:OU = NIFI, CN = ca.nifi
      i:OU = NIFI, CN = ca.nifi
   ---
   ...
   ---
   SSL handshake has read 2293 bytes and written 1464 bytes
   Verification: OK
   ---
   New, TLSv1.2, Cipher is ECDHE-RSA-AES256-GCM-SHA384
   Server public key is 2048 bit
   Secure Renegotiation IS supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   SSL-Session:
       Protocol  : TLSv1.2
       Cipher    : ECDHE-RSA-AES256-GCM-SHA384
       Session-ID: 7E5D46...1F4E63
       Session-ID-ctx:
       Master-Key: AB80DE...4FCC9A
       PSK identity: None
       PSK identity hint: None
       SRP username: None
       Start Time: 1589478427
       Timeout   : 7200 (sec)
       Verify return code: 0 (ok)
       Extended master secret: yes
   ---
   DONE
   ```


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

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



[GitHub] [nifi] thenatog edited a comment on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
thenatog edited a comment on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-628899529


   Looks like there's currently a test error for JDK11.
   
   My testing:
   
   Java 8
   - Secure cluster
   - ListenHTTP
   - InvokeHTTP
   - Checked TLS negotiation for cluster comms data (cluster.node.protocol.port) with Wireshark which was TLSv1.2
   - Clustered Site to Site back to the same cluster (had errors)
   - openssl s_client protocol version tests:
   https://docs.google.com/spreadsheets/d/1Vm17iqMdaPkqKtIYjGBUxG_TtRcdzhFRBnr_kaVTBVg/edit?usp=sharing
   
   Java 11
   - Secure cluster
   - ListenHTTP
   - InvokeHTTP
   - Checked TLS negotiation for cluster comms data (cluster.node.protocol.port) with Wireshark which was TLSv1.2
   - Clustered Site to Site back to the same cluster (had errors)
   - openssl s_client protocol version tests: https://docs.google.com/spreadsheets/d/1Vm17iqMdaPkqKtIYjGBUxG_TtRcdzhFRBnr_kaVTBVg/edit?usp=sharing
   
   Saw errors with site to site when using the HTTP protocol. I'm not certain if it's related to these changes or not:
   `"2020-05-14 15:16:06,799 WARN [Timer-Driven Process Thread-9] o.apache.nifi.remote.client.PeerSelector Could not communicate with node0.com:9551 to determine which nodes exist in the remote NiFi cluster, due to javax.net.ssl.SSLPeerUnverifiedException: Certificate for <node0.com> doesn't match any of the subject alternative names: [node1.com]"`
   It's possible these errors only happen for a cluster hosted on the same machine/localhost.


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

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



[GitHub] [nifi] alopresto edited a comment on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto edited a comment on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-626427812


   High level description of changes:
   * Previously there were `SslContextFactory` implementations in multiple modules. I enhanced the one in `nifi-security-utils` which is now used throughout the project, and removed the implementations in `nifi-framework` and `nifi-socket-utils`. 
     * Part of this refactoring was removing `public static` methods which created an `SSLContext` object from various combinations of explicit keystore and truststore properties. These were being used in an inconsistent manner. I introduced a container object called `TlsConfiguration` which wraps the state of the configuration and provides internal validation checks. This encapsulates the need to check for different combinations of configuration presence/validity in each use case (components, framework, etc.) and relieves the calling developer of re-implementing this logic every time. 
     * I also provided static convenience methods like `getX509TrustManager()` and `createSSLSocketFactory()` because in most cases that is what the calling code needs, rather than an intermediate `SSLContext` object they need to further configure. This reduced the need for `Tuple<>` return values throughout the code. 
     * Duplicate code to transform the various return values and configure the `OkHttpClient` and its `Builder` were refactored to utility methods. 
   * Duplicate enums were refactored. 
   * Unnecessary code dealing with client authentication settings when creating a client connection/socket were removed (these settings would be ignored, as only an SSL/TLS server can decide to enforce/request client authentication). 
   * Some tests were refactored to make mocking easier. 
   * Removed extraneous file loading during `NiFiProperties` construction in many tests. 
   * Enforced modern TLS protocol versions in various internal socket creations. 
   
   The easiest way to test these changes is to configure and deploy a secured cluster (see [Apache NiFi Walkthroughs: Creating and Securing a NiFi Cluster with the TLS Toolkit](https://nifi.apache.org/docs/nifi-docs/html/walkthroughs.html#creating-and-securing-a-nifi-cluster-with-the-tls-toolkit)) and run a flow which handles incoming secured connections such as `ListenHTTP`, `HandleHttpRequest`, etc. 


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

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



[GitHub] [nifi] alopresto commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto commented on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-628794810


   I made the dropdown for `RestrictedSSLContextService` more explicit where it now provides `TLS, TLSv1.2` on Java 8 and `TLS, TLSv1.2, TLSv1.3` on Java 11. Selecting `TLS` will allow connections over `TLSv1.2` _and_ `TLSv1.3`. 
   
   # With `TLSv1.2` selected:
   
   ```
   
   # TLSv1.2 is successful
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_2
   CONNECTED(00000003)
   depth=1 OU = NIFI, CN = ca.nifi
   verify return:1
   depth=0 OU = NIFI, CN = node1.nifi
   verify return:1
   ---
   Certificate chain
    0 s:OU = NIFI, CN = node1.nifi
      i:OU = NIFI, CN = ca.nifi
    1 s:OU = NIFI, CN = ca.nifi
      i:OU = NIFI, CN = ca.nifi
   ---
   ...
   ---
   SSL handshake has read 2289 bytes and written 1464 bytes
   Verification: OK
   ---
   New, TLSv1.2, Cipher is ECDHE-RSA-AES256-GCM-SHA384
   Server public key is 2048 bit
   Secure Renegotiation IS supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   SSL-Session:
       Protocol  : TLSv1.2
       Cipher    : ECDHE-RSA-AES256-GCM-SHA384
       Session-ID: BA2FC4...0D2790
       Session-ID-ctx:
       Master-Key: C773AC...A85A19
       PSK identity: None
       PSK identity hint: None
       SRP username: None
       Start Time: 1589478477
       Timeout   : 7200 (sec)
       Verify return code: 0 (ok)
       Extended master secret: yes
   ---
   DONE
   
   # TLSv1.3 fails
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_3
   CONNECTED(00000003)
   4570201536:error:1409442E:SSL routines:ssl3_read_bytes:tlsv1 alert protocol version:ssl/record/rec_layer_s3.c:1544:SSL alert number 70
   ---
   no peer certificate available
   ---
   No client certificate CA names sent
   ---
   SSL handshake has read 7 bytes and written 234 bytes
   Verification: OK
   ---
   New, (NONE), Cipher is (NONE)
   Secure Renegotiation IS NOT supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   Early data was not sent
   Verify return code: 0 (ok)
   ---
    ✘  ..oolkit-1.11.4   master ● 
   ```
   
   # With `TLS` selected:
   
   ```
   
   # TLSv1.3 is successful
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_3
   CONNECTED(00000003)
   depth=1 OU = NIFI, CN = ca.nifi
   verify return:1
   depth=0 OU = NIFI, CN = node1.nifi
   verify return:1
   ---
   Certificate chain
    0 s:OU = NIFI, CN = node1.nifi
      i:OU = NIFI, CN = ca.nifi
    1 s:OU = NIFI, CN = ca.nifi
      i:OU = NIFI, CN = ca.nifi
   ---
   ...
   ---
   SSL handshake has read 2510 bytes and written 1800 bytes
   Verification: OK
   ---
   New, TLSv1.3, Cipher is TLS_AES_128_GCM_SHA256
   Server public key is 2048 bit
   Secure Renegotiation IS NOT supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   Early data was not sent
   Verify return code: 0 (ok)
   ---
   DONE
   
   # TLSv1.2 is successful
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_2
   CONNECTED(00000003)
   depth=1 OU = NIFI, CN = ca.nifi
   verify return:1
   depth=0 OU = NIFI, CN = node1.nifi
   verify return:1
   ---
   Certificate chain
    0 s:OU = NIFI, CN = node1.nifi
      i:OU = NIFI, CN = ca.nifi
    1 s:OU = NIFI, CN = ca.nifi
      i:OU = NIFI, CN = ca.nifi
   ---
   ...
   ---
   SSL handshake has read 2293 bytes and written 1464 bytes
   Verification: OK
   ---
   New, TLSv1.2, Cipher is ECDHE-RSA-AES256-GCM-SHA384
   Server public key is 2048 bit
   Secure Renegotiation IS supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   SSL-Session:
       Protocol  : TLSv1.2
       Cipher    : ECDHE-RSA-AES256-GCM-SHA384
       Session-ID: 7E5D46...1F4E63
       Session-ID-ctx:
       Master-Key: AB80DE...4FCC9A
       PSK identity: None
       PSK identity hint: None
       SRP username: None
       Start Time: 1589478427
       Timeout   : 7200 (sec)
       Verify return code: 0 (ok)
       Extended master secret: yes
   ---
   DONE
   ```


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

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



[GitHub] [nifi] asfgit closed pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #4263:
URL: https://github.com/apache/nifi/pull/4263


   


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

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



[GitHub] [nifi] alopresto edited a comment on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto edited a comment on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-628794810


   I made the dropdown for `RestrictedSSLContextService` more explicit where it now provides `TLS, TLSv1.2` on Java 8 and `TLS, TLSv1.2, TLSv1.3` on Java 11. Selecting `TLS` will allow connections over `TLSv1.2` _and_ `TLSv1.3`. 
   
   ### With `TLSv1.2` selected:
   
   ```
   
   # TLSv1.2 is successful
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_2
   CONNECTED(00000003)
   depth=1 OU = NIFI, CN = ca.nifi
   verify return:1
   depth=0 OU = NIFI, CN = node1.nifi
   verify return:1
   ---
   Certificate chain
    0 s:OU = NIFI, CN = node1.nifi
      i:OU = NIFI, CN = ca.nifi
    1 s:OU = NIFI, CN = ca.nifi
      i:OU = NIFI, CN = ca.nifi
   ---
   ...
   ---
   SSL handshake has read 2289 bytes and written 1464 bytes
   Verification: OK
   ---
   New, TLSv1.2, Cipher is ECDHE-RSA-AES256-GCM-SHA384
   Server public key is 2048 bit
   Secure Renegotiation IS supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   SSL-Session:
       Protocol  : TLSv1.2
       Cipher    : ECDHE-RSA-AES256-GCM-SHA384
       Session-ID: BA2FC4...0D2790
       Session-ID-ctx:
       Master-Key: C773AC...A85A19
       PSK identity: None
       PSK identity hint: None
       SRP username: None
       Start Time: 1589478477
       Timeout   : 7200 (sec)
       Verify return code: 0 (ok)
       Extended master secret: yes
   ---
   DONE
   
   # TLSv1.3 fails
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_3
   CONNECTED(00000003)
   4570201536:error:1409442E:SSL routines:ssl3_read_bytes:tlsv1 alert protocol version:ssl/record/rec_layer_s3.c:1544:SSL alert number 70
   ---
   no peer certificate available
   ---
   No client certificate CA names sent
   ---
   SSL handshake has read 7 bytes and written 234 bytes
   Verification: OK
   ---
   New, (NONE), Cipher is (NONE)
   Secure Renegotiation IS NOT supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   Early data was not sent
   Verify return code: 0 (ok)
   ---
    ✘  ..oolkit-1.11.4   master ● 
   ```
   
   # With `TLS` selected:
   
   ```
   
   ### TLSv1.3 is successful
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_3
   CONNECTED(00000003)
   depth=1 OU = NIFI, CN = ca.nifi
   verify return:1
   depth=0 OU = NIFI, CN = node1.nifi
   verify return:1
   ---
   Certificate chain
    0 s:OU = NIFI, CN = node1.nifi
      i:OU = NIFI, CN = ca.nifi
    1 s:OU = NIFI, CN = ca.nifi
      i:OU = NIFI, CN = ca.nifi
   ---
   ...
   ---
   SSL handshake has read 2510 bytes and written 1800 bytes
   Verification: OK
   ---
   New, TLSv1.3, Cipher is TLS_AES_128_GCM_SHA256
   Server public key is 2048 bit
   Secure Renegotiation IS NOT supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   Early data was not sent
   Verify return code: 0 (ok)
   ---
   DONE
   
   # TLSv1.2 is successful
   
    ..oolkit-1.11.4   master ●  echo Q | openssl s_client -connect node1.nifi:9999 -key nifi-key.key -cert nifi-cert.pem -CAfile nifi-cert.pem -tls1_2
   CONNECTED(00000003)
   depth=1 OU = NIFI, CN = ca.nifi
   verify return:1
   depth=0 OU = NIFI, CN = node1.nifi
   verify return:1
   ---
   Certificate chain
    0 s:OU = NIFI, CN = node1.nifi
      i:OU = NIFI, CN = ca.nifi
    1 s:OU = NIFI, CN = ca.nifi
      i:OU = NIFI, CN = ca.nifi
   ---
   ...
   ---
   SSL handshake has read 2293 bytes and written 1464 bytes
   Verification: OK
   ---
   New, TLSv1.2, Cipher is ECDHE-RSA-AES256-GCM-SHA384
   Server public key is 2048 bit
   Secure Renegotiation IS supported
   Compression: NONE
   Expansion: NONE
   No ALPN negotiated
   SSL-Session:
       Protocol  : TLSv1.2
       Cipher    : ECDHE-RSA-AES256-GCM-SHA384
       Session-ID: 7E5D46...1F4E63
       Session-ID-ctx:
       Master-Key: AB80DE...4FCC9A
       PSK identity: None
       PSK identity hint: None
       SRP username: None
       Start Time: 1589478427
       Timeout   : 7200 (sec)
       Verify return code: 0 (ok)
       Extended master secret: yes
   ---
   DONE
   ```


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

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



[GitHub] [nifi] alopresto commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto commented on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-630446175


   I decided to do the S2S refactor in a separate Jira as it grew larger than anticipated. @thenatog if you're satisfied with what's here, please give a +1 and I'll merge. Thanks. 


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

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



[GitHub] [nifi] pvillard31 commented on a change in pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on a change in pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#discussion_r423006603



##########
File path: nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/SslContextFactoryTest.groovy
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.CURRENT_TLS_PROTOCOL_VERSION
+
+    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() {
+
+    }
+
+    @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)
+        assert defaultSSLParameters.getProtocols() == ["TLSv1.2", "TLSv1.1", "TLSv1"] as String[]

Review comment:
       ```suggestion
           assert defaultSSLParameters.getProtocols() as Set == ["TLSv1.2", "TLSv1.1", "TLSv1"] as Set
   ```

##########
File path: nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/SslContextFactoryTest.groovy
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.CURRENT_TLS_PROTOCOL_VERSION
+
+    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() {
+
+    }
+
+    @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)
+        assert defaultSSLParameters.getProtocols() == ["TLSv1.2", "TLSv1.1", "TLSv1"] as String[]
+        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)
+        assert defaultSSLParameters.getProtocols() == ["TLSv1.2", "TLSv1.1", "TLSv1"] as String[]

Review comment:
       ```suggestion
           assert defaultSSLParameters.getProtocols() as Set == ["TLSv1.2", "TLSv1.1", "TLSv1"] as Set
   ```

##########
File path: nifi-commons/nifi-security-utils/src/test/groovy/org/apache/nifi/security/util/SslContextFactoryTest.groovy
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.CURRENT_TLS_PROTOCOL_VERSION
+
+    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() {
+
+    }
+
+    @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)
+        assert defaultSSLParameters.getProtocols() == ["TLSv1.2", "TLSv1.1", "TLSv1"] as String[]
+        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)
+        assert defaultSSLParameters.getProtocols() == ["TLSv1.2", "TLSv1.1", "TLSv1"] as String[]
+        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)
+        assert defaultSSLParameters.getProtocols() == ["TLSv1.2", "TLSv1.1", "TLSv1"] as String[]

Review comment:
       ```suggestion
           assert defaultSSLParameters.getProtocols() as Set == ["TLSv1.2", "TLSv1.1", "TLSv1"] as Set
   ```




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

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



[GitHub] [nifi] alopresto commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto commented on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-628981634


   Thanks @markap14 and @thenatog for the extensive testing. I pushed another commit which enables TLSv1.3 for the Java 11 UI/API port and should resolve the test error. 
   
   I can reproduce the S2S issue mentioned above on a secure 3 node cluster pointing back to itself when all nodes are hosted on the same machine. I don't think the PR changed how this worked, so I suspect this existed previously, but I'll try to address it here as well. I also encountered trouble retrieving S2S peers so I will add some unit tests there to see what I can isolate and fix. 


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

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



[GitHub] [nifi] thenatog commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

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


   Looks like there's currently a test error for JDK11.
   
   My testing:
   Java 8
   	- Secure cluster
   	- ListenHTTP
   	- InvokeHTTP
   	- Clustered Site to Site back to the same cluster (had errors)
   	- openssl s_client protocol version tests: https://docs.google.com/spreadsheets/d/1Vm17iqMdaPkqKtIYjGBUxG_TtRcdzhFRBnr_kaVTBVg/edit?usp=sharing
   
   Java 11
   	- Secure cluster
   	- ListenHTTP
   	- InvokeHTTP
   	- Clustered Site to Site back to the same cluster (had errors)
   	- openssl s_client protocol version tests: https://docs.google.com/spreadsheets/d/1Vm17iqMdaPkqKtIYjGBUxG_TtRcdzhFRBnr_kaVTBVg/edit?usp=sharing
   
   Saw errors with site to site when using the HTTP protocol. I'm not certain if it's related to these changes or not:
   `"2020-05-14 15:16:06,799 WARN [Timer-Driven Process Thread-9] o.apache.nifi.remote.client.PeerSelector Could not communicate with node0.com:9551 to determine which nodes exist in the remote NiFi cluster, due to javax.net.ssl.SSLPeerUnverifiedException: Certificate for <node0.com> doesn't match any of the subject alternative names: [node1.com]"`
   It's possible these errors only happen for a cluster hosted on the same machine/localhost.


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

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



[GitHub] [nifi] alopresto commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto commented on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-626427812


   High level description of changes:
   * Previously there were `SslContextFactory` implementations in multiple modules. I enhanced the one in `nifi-security-utils` which is now used throughout the project, and removed the implementations in `nifi-framework` and `nifi-socket-utils`. 
   ** Part of this refactoring was removing `public static` methods which created an `SSLContext` object from various combinations of explicit keystore and truststore properties. These were being used in an inconsistent manner. I introduced a container object called `TlsConfiguration` which wraps the state of the configuration and provides internal validation checks. This encapsulates the need to check for different combinations of configuration presence/validity in each use case (components, framework, etc.) and relieves the calling developer of re-implementing this logic every time. 
   ** I also provided static convenience methods like `getX509TrustManager()` and `createSSLSocketFactory()` because in most cases that is what the calling code needs, rather than an intermediate `SSLContext` object they need to further configure. This reduced the need for `Tuple<>` return values throughout the code. 
   ** Duplicate code to transform the various return values and configure the `OkHttpClient` and its `Builder` were refactored to utility methods. 
   * Duplicate enums were refactored. 
   * Unnecessary code dealing with client authentication settings when creating a client connection/socket were removed (these settings would be ignored, as only an SSL/TLS server can decide to enforce/request client authentication). 
   * Some tests were refactored to make mocking easier. 
   * Removed extraneous file loading during `NiFiProperties` construction in many tests. 
   * Enforced modern TLS protocol versions in various internal socket creations. 
   
   The easiest way to test these changes is to configure and deploy a secured cluster (see [Apache NiFi Walkthroughs: Creating and Securing a NiFi Cluster with the TLS Toolkit](https://nifi.apache.org/docs/nifi-docs/html/walkthroughs.html#creating-and-securing-a-nifi-cluster-with-the-tls-toolkit)) and run a flow which handles incoming secured connections such as `ListenHTTP`, `HandleHttpRequest`, etc. 


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

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



[GitHub] [nifi] alopresto commented on a change in pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto commented on a change in pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#discussion_r424712012



##########
File path: 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,185 @@
+/*
+ * 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, ["TLSv1.2", "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

Review comment:
       May want to change to IT as GHA can encounter delays which might invalidate the assertions in this test. 




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

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



[GitHub] [nifi] alopresto commented on pull request #4263: NIFI-7407 Refactored SSL context generation throughout framework and extensions.

Posted by GitBox <gi...@apache.org>.
alopresto commented on pull request #4263:
URL: https://github.com/apache/nifi/pull/4263#issuecomment-630447908


   There was another Java 11 unit test failure. Resolved that. 


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

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