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 2022/05/19 15:58:25 UTC

[GitHub] [cassandra] jonmeredith commented on a diff in pull request #1535: CASSANDRA-17513 Adding new property to server encryption options for creating outbound keystore for internode mTLS

jonmeredith commented on code in PR #1535:
URL: https://github.com/apache/cassandra/pull/1535#discussion_r877213769


##########
src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java:
##########
@@ -174,6 +170,30 @@ protected TrustManagerFactory buildTrustManagerFactory() throws SSLException
         }
     }
 
+    private KeyManagerFactory getKeyManagerFactory(final boolean isOutboundKeystore) throws SSLException

Review Comment:
   Why not pass in the keystore context directly rather than go through the indirection of the `isOutboundKeystore` boolean? I don't see it used for anything else.



##########
src/java/org/apache/cassandra/security/FileBasedSslContextFactory.java:
##########
@@ -102,14 +105,19 @@ private boolean hasTruststore()
     public synchronized void initHotReloading()
     {
         boolean hasKeystore = hasKeystore();
+        boolean hasOutboundKeystore = hasOutboundKeystore();
         boolean hasTruststore = hasTruststore();
 
-        if (hasKeystore || hasTruststore)
+        if (hasKeystore || hasOutboundKeystore|| hasTruststore)

Review Comment:
   missing a space



##########
src/java/org/apache/cassandra/security/PEMBasedSslContextFactory.java:
##########
@@ -355,6 +413,8 @@ private void enforceSingleTurstedCertificatesSource()
     {
         ENCODED_KEY("private_key"),
         KEY_PASSWORD("private_key_password"),
+        OUTBOUND_KEY("outbound_key"),

Review Comment:
   Should this be `OUTBOUND_ENCODED_KEY`. `outbound_private_key`?



##########
test/unit/org/apache/cassandra/security/DefaultSslContextFactoryTest.java:
##########
@@ -53,15 +53,23 @@ private void addKeystoreOptions(Map<String,Object> config)
         config.put("keystore_password", "cassandra");
     }
 
+    private void addOutboundKeystoreOptions(Map<String, Object> config)
+    {
+        config.put("outbound_keystore", "test/conf/cassandra_ssl_test.keystore");
+        config.put("outbound_keystore_password", "cassandra");
+    }
+
     @Test
     public void getSslContextOpenSSL() throws IOException
     {
-        EncryptionOptions options = new EncryptionOptions().withTrustStore("test/conf/cassandra_ssl_test.truststore")
-                                                           .withTrustStorePassword("cassandra")
-                                                           .withKeyStore("test/conf/cassandra_ssl_test.keystore")
-                                                           .withKeyStorePassword("cassandra")
-                                                           .withRequireClientAuth(false)
-                                                           .withCipherSuites("TLS_RSA_WITH_AES_128_CBC_SHA");
+        EncryptionOptions.ServerEncryptionOptions options = new EncryptionOptions.ServerEncryptionOptions().withTrustStore("test/conf/cassandra_ssl_test.truststore")
+                                                                                                           .withTrustStorePassword("cassandra")
+                                                                                                           .withKeyStore("test/conf/cassandra_ssl_test.keystore")
+                                                                                                           .withKeyStorePassword("cassandra")
+                                                                                                           .withOutboundKeystore("test/conf/cassandra_ssl_test.keystore")
+                                                                                                           .withOutboundKeystorePassword("cassandra")
+                                                                                                           .withRequireClientAuth(false)
+                                                                                                           .withCipherSuites("TLS_RSA_WITH_AES_128_CBC_SHA");
         SslContext sslContext = SSLFactory.getOrCreateSslContext(options, true, ISslContextFactory.SocketType.CLIENT);
         Assert.assertNotNull(sslContext);

Review Comment:
   How do you know if you got the right certificate here? From the regular keystore or the outbound keystore?



##########
src/java/org/apache/cassandra/security/PEMBasedSslContextFactory.java:
##########
@@ -188,7 +205,11 @@ public synchronized void initHotReloading()
         List<HotReloadableFile> fileList = new ArrayList<>();
         if (maybeFileBasedPrivateKey && hasKeystore())
         {
-            fileList.add(new HotReloadableFile(keystore));
+            fileList.add(new HotReloadableFile(keystoreContext.keystore));
+        }
+        if (maybeFileBasedOutboundPrivateKey && hasKeystore())

Review Comment:
   Should this be `hasOutboundKeystore()`?



##########
src/java/org/apache/cassandra/security/PEMBasedSslContextFactory.java:
##########
@@ -355,6 +413,8 @@ private void enforceSingleTurstedCertificatesSource()
     {
         ENCODED_KEY("private_key"),
         KEY_PASSWORD("private_key_password"),
+        OUTBOUND_KEY("outbound_key"),
+        OUTBOUND_KEY_PASSWORD("outbound_key_password"),

Review Comment:
   `outbound_private_key_password`?



##########
src/java/org/apache/cassandra/security/ISslContextFactory.java:
##########
@@ -99,6 +99,13 @@ default boolean hasKeystore()
         return true;
     }
 
+    /**
+     * Returns if this factory uses outbound keystore.
+     *
+     * @return {@code true} by default unless the implementation overrides this
+     */
+    boolean hasOutboundKeystore();

Review Comment:
   Can this default to false to avoid breaking existing implementations?



##########
src/java/org/apache/cassandra/security/PEMBasedSslContextFactory.java:
##########
@@ -92,33 +92,45 @@ public final class PEMBasedSslContextFactory extends FileBasedSslContextFactory
     private static final Logger logger = LoggerFactory.getLogger(PEMBasedSslContextFactory.class);
     private String pemEncodedKey;
     private String keyPassword;
+    private String pemEncodedOutboundKey;
+    private String outboundKeyPassword;

Review Comment:
   Could you have created a PEMBasedKeyStoreContext to hold this in? Then you could avoid duplication in `buildKeyManagerFactory` and `buildOutboundKeyManagerFactory`



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

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

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