You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/10/21 22:51:15 UTC

[GitHub] [cassandra] dineshjoshi commented on a change in pull request #770: CASSANDRA-13325 configure which TLS protocols should be accepted

dineshjoshi commented on a change in pull request #770:
URL: https://github.com/apache/cassandra/pull/770#discussion_r509774148



##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -204,6 +208,79 @@ public void setOptional(boolean optional) {
         this.optional = optional;
     }
 
+    /**
+     * Sets accepted TLS protocol for this channel. Note that this should only be called by
+     * the configuration parser or tests. It is public only for that purpose, mutating protocol state
+     * is probably a bad idea.
+     * @param protocol value to set
+     */
+    public void setProtocol(String protocol) {
+        this.protocol = protocol;
+    }
+
+    /**
+     * Sets accepted TLS protocols for this channel. Note that this should only be called by
+     * the configuration parser or tests. It is public only for that purpose, mutating protocl state
+     * is probably a bad idea.
+     * @param accepted_protocols value to set
+     */
+    public void setaccepted_protocols(List<String> accepted_protocols) {

Review comment:
       Why are we using underscores? We use CamelCase convention. Likely typo? :)

##########
File path: src/java/org/apache/cassandra/security/SSLFactory.java
##########
@@ -397,41 +390,121 @@ public static synchronized void initHotReloading(EncryptionOptions.ServerEncrypt
         isHotReloadingInitialized = true;
     }
 
-
-    /**
-     * Sanity checks all certificates to ensure we can actually load them
+    // Non-logging
+    /*
+     * This class will filter all requested ciphers out that are not supported by the current {@link SSLEngine},
+     * logging messages for all dropped ciphers, and throws an exception if no ciphers are supported
      */
-    public static void validateSslCerts(EncryptionOptions.ServerEncryptionOptions serverOpts, EncryptionOptions clientOpts) throws IOException
+    public static final class LoggingCipherSuiteFilter implements CipherSuiteFilter
     {
-        try
+        // Version without logging the ciphers, make sure same filtering logic is used
+        // all the time, regardless of user output.
+        public static final CipherSuiteFilter QUIET_FILTER = new LoggingCipherSuiteFilter();
+        final String settingDescription;
+
+        private LoggingCipherSuiteFilter()
         {
-            // Ensure we're able to create both server & client SslContexts if they might ever be needed
-            if (serverOpts != null && serverOpts.tlsEncryptionPolicy() != EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED)
-            {
-                createNettySslContext(serverOpts, true, SocketType.SERVER, openSslIsAvailable());
-                createNettySslContext(serverOpts, true, SocketType.CLIENT, openSslIsAvailable());
-            }
+            this.settingDescription = null;
         }
-        catch (Exception e)
+
+        public LoggingCipherSuiteFilter(String settingDescription)
         {
-            throw new IOException("Failed to create SSL context using server_encryption_options!", e);
+            this.settingDescription = settingDescription;
         }
 
-        try
+
+        @Override
+        public String[] filterCipherSuites(Iterable<String> ciphers, List<String> defaultCiphers,
+                                           Set<String> supportedCiphers)
         {
-            // Ensure we're able to create both server & client SslContexts if they might ever be needed
-            if (clientOpts != null && clientOpts.tlsEncryptionPolicy() != EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED)
+            Objects.requireNonNull(defaultCiphers, "defaultCiphers");
+            Objects.requireNonNull(supportedCiphers, "supportedCiphers");
+
+            final List<String> newCiphers;
+            if (ciphers == null)
+            {
+                newCiphers = new ArrayList<>(defaultCiphers.size());
+                ciphers = defaultCiphers;
+            }
+            else
+            {
+                newCiphers = new ArrayList<>(supportedCiphers.size());
+            }
+            for (String c : ciphers)
             {
-                createNettySslContext(clientOpts, clientOpts.require_client_auth, SocketType.SERVER, openSslIsAvailable());
-                createNettySslContext(clientOpts, clientOpts.require_client_auth, SocketType.CLIENT, openSslIsAvailable());
+                if (c == null)
+                {
+                    break;
+                }
+                if (supportedCiphers.contains(c))
+                {
+                    newCiphers.add(c);
+                }
+                else
+                {
+                    if (settingDescription != null)
+                    {
+                        logger.warn("Dropping unsupported cipher_suite {} from {} configuration",
+                                    c, settingDescription.toLowerCase());
+                    }
+                }
             }
+            if (newCiphers.isEmpty())
+            {
+                throw new IllegalStateException("No ciphers left after filtering supported cipher suite");
+            }
+
+            return newCiphers.toArray(new String[0]);
         }
-        catch (Exception e)
+    }
+
+    public static void validateSslContext(String contextDescription, EncryptionOptions options, boolean buildTrustStore, boolean logProtocolAndCiphers) throws IOException
+    {
+        if (options != null && options.tlsEncryptionPolicy() != EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED)
         {
-            throw new IOException("Failed to create SSL context using client_encryption_options!", e);
+            try
+            {
+                CipherSuiteFilter loggingCipherSuiteFilter =logProtocolAndCiphers ?  new LoggingCipherSuiteFilter(contextDescription)
+                                                                                  : LoggingCipherSuiteFilter.QUIET_FILTER;
+                SslContext sslContext = createNettySslContext(options, buildTrustStore, SocketType.SERVER, openSslIsAvailable(), loggingCipherSuiteFilter);
+                SSLEngine engine = sslContext.newEngine(ByteBufAllocator.DEFAULT);
+
+                if (logProtocolAndCiphers)
+                {
+                    String[] supportedProtocols = engine.getSupportedProtocols();
+                    String[] supportedCiphers = engine.getSupportedCipherSuites();
+                    String[] enabledProtocols = engine.getEnabledProtocols();
+                    String[] enabledCiphers = engine.getEnabledCipherSuites();
+
+                    logger.debug("{} supported TLS protocols: {}", contextDescription,
+                                 supportedProtocols == null ? "system default" : String.join(", ", supportedProtocols));
+                    logger.info("{} enabled TLS protocols: {}", contextDescription,
+                                enabledProtocols == null ? "system default" : String.join(", ", enabledProtocols));
+                    logger.debug("{} supported cipher suites: {}", contextDescription,
+                                 supportedCiphers == null ? "system default" : String.join(", ", supportedCiphers));
+                    logger.info("{} enabled cipher suites: {}", contextDescription,
+                                enabledCiphers == null ? "system default" : String.join(", ", enabledCiphers));
+                }
+
+                // Make sure it is possible to build the client context too
+                createNettySslContext(options, buildTrustStore, SocketType.CLIENT, openSslIsAvailable());
+            }
+            catch (Exception e)
+            {
+                throw new IOException("Failed to create SSL context using " + contextDescription, e);
+            }
         }
     }
 
+    /**
+     * Sanity checks all certificates to ensure we can actually load them
+     */
+    public static void validateSslCerts(EncryptionOptions.ServerEncryptionOptions serverOpts, EncryptionOptions clientOpts) throws IOException
+    {
+        validateSslContext("server_encryption_options", serverOpts, true, false);

Review comment:
       Instead of true/false values, can we use meaningful constants?

##########
File path: src/java/org/apache/cassandra/security/SSLFactory.java
##########
@@ -397,41 +390,121 @@ public static synchronized void initHotReloading(EncryptionOptions.ServerEncrypt
         isHotReloadingInitialized = true;
     }
 
-
-    /**
-     * Sanity checks all certificates to ensure we can actually load them
+    // Non-logging
+    /*
+     * This class will filter all requested ciphers out that are not supported by the current {@link SSLEngine},
+     * logging messages for all dropped ciphers, and throws an exception if no ciphers are supported
      */
-    public static void validateSslCerts(EncryptionOptions.ServerEncryptionOptions serverOpts, EncryptionOptions clientOpts) throws IOException
+    public static final class LoggingCipherSuiteFilter implements CipherSuiteFilter
     {
-        try
+        // Version without logging the ciphers, make sure same filtering logic is used
+        // all the time, regardless of user output.
+        public static final CipherSuiteFilter QUIET_FILTER = new LoggingCipherSuiteFilter();
+        final String settingDescription;
+
+        private LoggingCipherSuiteFilter()
         {
-            // Ensure we're able to create both server & client SslContexts if they might ever be needed
-            if (serverOpts != null && serverOpts.tlsEncryptionPolicy() != EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED)
-            {
-                createNettySslContext(serverOpts, true, SocketType.SERVER, openSslIsAvailable());
-                createNettySslContext(serverOpts, true, SocketType.CLIENT, openSslIsAvailable());
-            }
+            this.settingDescription = null;
         }
-        catch (Exception e)
+
+        public LoggingCipherSuiteFilter(String settingDescription)
         {
-            throw new IOException("Failed to create SSL context using server_encryption_options!", e);
+            this.settingDescription = settingDescription;
         }
 
-        try
+
+        @Override
+        public String[] filterCipherSuites(Iterable<String> ciphers, List<String> defaultCiphers,
+                                           Set<String> supportedCiphers)
         {
-            // Ensure we're able to create both server & client SslContexts if they might ever be needed
-            if (clientOpts != null && clientOpts.tlsEncryptionPolicy() != EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED)
+            Objects.requireNonNull(defaultCiphers, "defaultCiphers");
+            Objects.requireNonNull(supportedCiphers, "supportedCiphers");
+
+            final List<String> newCiphers;
+            if (ciphers == null)
+            {
+                newCiphers = new ArrayList<>(defaultCiphers.size());
+                ciphers = defaultCiphers;
+            }
+            else
+            {
+                newCiphers = new ArrayList<>(supportedCiphers.size());
+            }
+            for (String c : ciphers)
             {
-                createNettySslContext(clientOpts, clientOpts.require_client_auth, SocketType.SERVER, openSslIsAvailable());
-                createNettySslContext(clientOpts, clientOpts.require_client_auth, SocketType.CLIENT, openSslIsAvailable());
+                if (c == null)
+                {
+                    break;
+                }
+                if (supportedCiphers.contains(c))
+                {
+                    newCiphers.add(c);
+                }
+                else
+                {
+                    if (settingDescription != null)
+                    {
+                        logger.warn("Dropping unsupported cipher_suite {} from {} configuration",
+                                    c, settingDescription.toLowerCase());
+                    }
+                }
             }
+            if (newCiphers.isEmpty())
+            {
+                throw new IllegalStateException("No ciphers left after filtering supported cipher suite");
+            }
+
+            return newCiphers.toArray(new String[0]);
         }
-        catch (Exception e)
+    }
+
+    public static void validateSslContext(String contextDescription, EncryptionOptions options, boolean buildTrustStore, boolean logProtocolAndCiphers) throws IOException
+    {
+        if (options != null && options.tlsEncryptionPolicy() != EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED)
         {
-            throw new IOException("Failed to create SSL context using client_encryption_options!", e);
+            try
+            {
+                CipherSuiteFilter loggingCipherSuiteFilter =logProtocolAndCiphers ?  new LoggingCipherSuiteFilter(contextDescription)

Review comment:
       nit: extra space : `?  new` and missing space between `=logProtocol...`.

##########
File path: src/java/org/apache/cassandra/security/SSLFactory.java
##########
@@ -175,7 +173,7 @@ public static SSLContext createSSLContext(EncryptionOptions options, boolean bui
 
         try
         {
-            SSLContext ctx = SSLContext.getInstance(options.protocol);
+            SSLContext ctx = SSLContext.getInstance("TLS"); //TODO: Make sure all uses restrict protocol to configured.

Review comment:
       Is this TODO resolved? If so, please remove this comment?




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



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org