You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2016/02/29 18:20:47 UTC

[3/6] cassandra git commit: Preserve order for preferred SSL cipher suites

Preserve order for preferred SSL cipher suites

Patch by Stefan Podkowinski and Tom Petracca; reviewed by Stefania for CASSANDRA-11164


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/a24bd6c6
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/a24bd6c6
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/a24bd6c6

Branch: refs/heads/trunk
Commit: a24bd6c6a554415ab0cb173e0a3ffb96510f7a1c
Parents: ecbeb08
Author: Stefan Podkowinski <s....@gmail.com>
Authored: Tue Feb 16 11:46:11 2016 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Mon Feb 29 17:17:47 2016 +0000

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 .../apache/cassandra/security/SSLFactory.java   |  41 ++++++----
 .../thrift/CustomTThreadPoolServer.java         |   4 +-
 .../org/apache/cassandra/transport/Server.java  |   3 +-
 .../cassandra/transport/SimpleClient.java       |   3 +-
 test/conf/keystore.jks                          | Bin 0 -> 2191 bytes
 .../cassandra/security/SSLFactoryTest.java      |  75 +++++++++++++++++++
 7 files changed, 109 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a24bd6c6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index aa3adf5..103ac16 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.6
+ * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
  * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
  * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
  * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)
@@ -33,6 +34,7 @@ Merged from 2.1:
  * (cqlsh) Support timezone conversion using pytz (CASSANDRA-10397)
  * cqlsh: change default encoding to UTF-8 (CASSANDRA-11124)
 
+
 2.2.5
  * maxPurgeableTimestamp needs to check memtables too (CASSANDRA-9949)
  * Apply change to compaction throughput in real time (CASSANDRA-10025)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a24bd6c6/src/java/org/apache/cassandra/security/SSLFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/security/SSLFactory.java b/src/java/org/apache/cassandra/security/SSLFactory.java
index e9aa07d..a327de9 100644
--- a/src/java/org/apache/cassandra/security/SSLFactory.java
+++ b/src/java/org/apache/cassandra/security/SSLFactory.java
@@ -24,9 +24,10 @@ import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.security.KeyStore;
 import java.security.cert.X509Certificate;
+import java.util.Arrays;
 import java.util.Date;
 import java.util.Enumeration;
-import java.util.Set;
+import java.util.List;
 
 import javax.net.ssl.KeyManagerFactory;
 import javax.net.ssl.SSLContext;
@@ -37,10 +38,12 @@ import javax.net.ssl.TrustManagerFactory;
 
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 
 /**
@@ -58,8 +61,8 @@ public final class SSLFactory
         SSLContext ctx = createSSLContext(options, true);
         SSLServerSocket serverSocket = (SSLServerSocket)ctx.getServerSocketFactory().createServerSocket();
         serverSocket.setReuseAddress(true);
-        String[] suits = filterCipherSuites(serverSocket.getSupportedCipherSuites(), options.cipher_suites);
-        serverSocket.setEnabledCipherSuites(suits);
+        String[] suites = filterCipherSuites(serverSocket.getSupportedCipherSuites(), options.cipher_suites);
+        serverSocket.setEnabledCipherSuites(suites);
         serverSocket.setNeedClientAuth(options.require_client_auth);
         serverSocket.setEnabledProtocols(ACCEPTED_PROTOCOLS);
         serverSocket.bind(new InetSocketAddress(address, port), 500);
@@ -71,8 +74,8 @@ public final class SSLFactory
     {
         SSLContext ctx = createSSLContext(options, true);
         SSLSocket socket = (SSLSocket) ctx.getSocketFactory().createSocket(address, port, localAddress, localPort);
-        String[] suits = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
-        socket.setEnabledCipherSuites(suits);
+        String[] suites = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
+        socket.setEnabledCipherSuites(suites);
         socket.setEnabledProtocols(ACCEPTED_PROTOCOLS);
         return socket;
     }
@@ -82,8 +85,8 @@ public final class SSLFactory
     {
         SSLContext ctx = createSSLContext(options, true);
         SSLSocket socket = (SSLSocket) ctx.getSocketFactory().createSocket(address, port);
-        String[] suits = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
-        socket.setEnabledCipherSuites(suits);
+        String[] suites = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
+        socket.setEnabledCipherSuites(suites);
         socket.setEnabledProtocols(ACCEPTED_PROTOCOLS);
         return socket;
     }
@@ -93,8 +96,8 @@ public final class SSLFactory
     {
         SSLContext ctx = createSSLContext(options, true);
         SSLSocket socket = (SSLSocket) ctx.getSocketFactory().createSocket();
-        String[] suits = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
-        socket.setEnabledCipherSuites(suits);
+        String[] suites = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
+        socket.setEnabledCipherSuites(suites);
         socket.setEnabledProtocols(ACCEPTED_PROTOCOLS);
         return socket;
     }
@@ -155,12 +158,18 @@ public final class SSLFactory
         return ctx;
     }
 
-    private static String[] filterCipherSuites(String[] supported, String[] desired)
+    public static String[] filterCipherSuites(String[] supported, String[] desired)
     {
-        Set<String> des = Sets.newHashSet(desired);
-        Set<String> toReturn = Sets.intersection(Sets.newHashSet(supported), des);
-        if (des.size() > toReturn.size())
-            logger.warn("Filtering out {} as it isnt supported by the socket", StringUtils.join(Sets.difference(des, toReturn), ","));
-        return toReturn.toArray(new String[toReturn.size()]);
+        if (Arrays.equals(supported, desired))
+            return desired;
+        List<String> ldesired = Arrays.asList(desired);
+        ImmutableSet<String> ssupported = ImmutableSet.copyOf(supported);
+        String[] ret = Iterables.toArray(Iterables.filter(ldesired, Predicates.in(ssupported)), String.class);
+        if (desired.length > ret.length && logger.isWarnEnabled())
+        {
+            Iterable<String> missing = Iterables.filter(ldesired, Predicates.not(Predicates.in(Sets.newHashSet(ret))));
+            logger.warn("Filtering out {} as it isn't supported by the socket", Iterables.toString(missing));
+        }
+        return ret;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a24bd6c6/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index bde5310..c5f34ae 100644
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@ -245,7 +245,7 @@ public class CustomTThreadPoolServer extends TServer
                 if (clientEnc.enabled)
                 {
                     logger.info("enabling encrypted thrift connections between client and server");
-                    TSSLTransportParameters params = new TSSLTransportParameters(clientEnc.protocol, clientEnc.cipher_suites);
+                    TSSLTransportParameters params = new TSSLTransportParameters(clientEnc.protocol, new String[0]);
                     params.setKeyStore(clientEnc.keystore, clientEnc.keystore_password);
                     if (clientEnc.require_client_auth)
                     {
@@ -254,6 +254,8 @@ public class CustomTThreadPoolServer extends TServer
                     }
                     TServerSocket sslServer = TSSLTransportFactory.getServerSocket(addr.getPort(), 0, addr.getAddress(), params);
                     SSLServerSocket sslServerSocket = (SSLServerSocket) sslServer.getServerSocket();
+                    String[] suites = SSLFactory.filterCipherSuites(sslServerSocket.getSupportedCipherSuites(), clientEnc.cipher_suites);
+                    sslServerSocket.setEnabledCipherSuites(suites);
                     sslServerSocket.setEnabledProtocols(SSLFactory.ACCEPTED_PROTOCOLS);
                     serverTransport = new TCustomServerSocket(sslServer.getServerSocket(), args.keepAlive, args.sendBufferSize, args.recvBufferSize);
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a24bd6c6/src/java/org/apache/cassandra/transport/Server.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java
index c56564c..43d07fc 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -331,7 +331,8 @@ public class Server implements CassandraDaemon.Server
         protected final SslHandler createSslHandler() {
             SSLEngine sslEngine = sslContext.createSSLEngine();
             sslEngine.setUseClientMode(false);
-            sslEngine.setEnabledCipherSuites(encryptionOptions.cipher_suites);
+            String[] suites = SSLFactory.filterCipherSuites(sslEngine.getSupportedCipherSuites(), encryptionOptions.cipher_suites);
+            sslEngine.setEnabledCipherSuites(suites);
             sslEngine.setNeedClientAuth(encryptionOptions.require_client_auth);
             sslEngine.setEnabledProtocols(SSLFactory.ACCEPTED_PROTOCOLS);
             return new SslHandler(sslEngine);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a24bd6c6/src/java/org/apache/cassandra/transport/SimpleClient.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/SimpleClient.java b/src/java/org/apache/cassandra/transport/SimpleClient.java
index 701a24c..4759c2a 100644
--- a/src/java/org/apache/cassandra/transport/SimpleClient.java
+++ b/src/java/org/apache/cassandra/transport/SimpleClient.java
@@ -291,7 +291,8 @@ public class SimpleClient implements Closeable
             super.initChannel(channel);
             SSLEngine sslEngine = sslContext.createSSLEngine();
             sslEngine.setUseClientMode(true);
-            sslEngine.setEnabledCipherSuites(encryptionOptions.cipher_suites);
+            String[] suites = SSLFactory.filterCipherSuites(sslEngine.getSupportedCipherSuites(), encryptionOptions.cipher_suites);
+            sslEngine.setEnabledCipherSuites(suites);
             sslEngine.setEnabledProtocols(SSLFactory.ACCEPTED_PROTOCOLS);
             channel.pipeline().addFirst("ssl", new SslHandler(sslEngine));
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a24bd6c6/test/conf/keystore.jks
----------------------------------------------------------------------
diff --git a/test/conf/keystore.jks b/test/conf/keystore.jks
new file mode 100644
index 0000000..334025d
Binary files /dev/null and b/test/conf/keystore.jks differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a24bd6c6/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/security/SSLFactoryTest.java b/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
new file mode 100644
index 0000000..b3510bd
--- /dev/null
+++ b/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
@@ -0,0 +1,75 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.security;
+
+import static org.junit.Assert.assertArrayEquals;
+
+import java.io.IOException;
+import java.net.InetAddress;
+
+import javax.net.ssl.SSLServerSocket;
+
+import com.google.common.base.Predicates;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
+import org.junit.Test;
+
+public class SSLFactoryTest
+{
+
+    @Test
+    public void testFilterCipherSuites()
+    {
+        String[] supported = new String[] {"x", "b", "c", "f"};
+        String[] desired = new String[] { "k", "a", "b", "c" };
+        assertArrayEquals(new String[] { "b", "c" }, SSLFactory.filterCipherSuites(supported, desired));
+
+        desired = new String[] { "c", "b", "x" };
+        assertArrayEquals(desired, SSLFactory.filterCipherSuites(supported, desired));
+    }
+
+    @Test
+    public void testServerSocketCiphers() throws IOException
+    {
+        ServerEncryptionOptions options = new EncryptionOptions.ServerEncryptionOptions();
+        options.keystore = "test/conf/keystore.jks";
+        options.keystore_password = "cassandra";
+        options.truststore = options.keystore;
+        options.truststore_password = options.keystore_password;
+        options.cipher_suites = new String[] {
+            "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA",
+            "TLS_DHE_RSA_WITH_AES_128_CBC_SHA", "TLS_DHE_RSA_WITH_AES_256_CBC_SHA",
+            "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA", "TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA"
+        };
+
+        // enabled ciphers must be a subset of configured ciphers with identical order
+        try (SSLServerSocket socket = SSLFactory.getServerSocket(options, InetAddress.getLocalHost(), 55123))
+        {
+            String[] enabled = socket.getEnabledCipherSuites();
+            String[] wanted = Iterables.toArray(Iterables.filter(Lists.newArrayList(options.cipher_suites),
+                                                                 Predicates.in(Lists.newArrayList(enabled))),
+                                                String.class);
+            assertArrayEquals(wanted, enabled);
+        }
+    }
+
+}