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/01 23:39:16 UTC

[GitHub] [cassandra] jonmeredith opened a new pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

jonmeredith opened a new pull request #763:
URL: https://github.com/apache/cassandra/pull/763


   Fixes listening on optional native transport, checking SSL Keystores if only optional is enabled, disabling optional if no Keystore exists at the default path and improves logging on encryption status of listening sockets and established connections.
   
   Details in the individual commit descriptions.
   
   The first two commits are from CASSANDRA-16152 which is currently under review (but being applied to all versions, not just trunk).


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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r499940185



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionSettings.java
##########
@@ -83,8 +83,8 @@ public boolean authenticate(InetAddress address, int port)
 
     public String toString()
     {
-        return format("address: (%s), nic: %s, encryption: %s",
-                      bindAddress, FBUtilities.getNetworkInterface(bindAddress.address), SocketFactory.encryptionLogStatement(null, encryption));
+        return format("address: (%s), nic: %s, %s",

Review comment:
       Can do - I thought it was redundant with the descriptions 'unencrypted', 'optionally encrypted' or 'encrypted' 
   
   would go from
   ```
   Listening on address: (/127.0.0.1:7012), nic: lo0, unencrypted
   Listening on address: (/127.0.0.1:7012), nic: lo0, optionally encrypted(jdk)
   Listening on address: (/127.0.0.2:7012), nic: lo0, encrypted(jdk)
   ```
   to
   ```
   Listening on address: (/127.0.0.1:7012), nic: lo0, encryption: unencrypted
   Listening on address: (/127.0.0.1:7012), nic: lo0, encryption: optionally encrypted(jdk)
   Listening on address: (/127.0.0.2:7012), nic: lo0, encryption: encrypted(jdk)
   ```
   
   Still prefer it back?




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


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r502085602



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -503,4 +494,31 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) t
             }
         }
     }
+
+    private static class rejectSslHandler extends ByteToMessageDecoder

Review comment:
       Class names should begin with upper-case characters.

##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -98,17 +95,22 @@ public void initChannel(SocketChannel channel) throws Exception
 
             // order of handlers: ssl -> logger -> handshakeHandler
             // For either unencrypted or transitional modes, allow Ssl optionally.
-            if (settings.encryption.optional)
+            switch(settings.encryption.tlsEncryptionPolicy())
             {
-                pipeline.addFirst("ssl", new OptionalSslHandler(settings.encryption));
-            }
-            else
-            {
-                SslContext sslContext = SSLFactory.getOrCreateSslContext(settings.encryption, true, SSLFactory.SocketType.SERVER);
-                InetSocketAddress peer = settings.encryption.require_endpoint_verification ? channel.remoteAddress() : null;
-                SslHandler sslHandler = newSslHandler(channel, sslContext, peer);
-                logger.trace("creating inbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());
-                pipeline.addFirst("ssl", sslHandler);
+                case UNENCRYPTED:
+                    // Handler checks for SSL connection attempts and cleanly rejects them if encryption is disabled
+                    pipeline.addFirst("rejectssl", new rejectSslHandler());
+                    break;
+                case OPTIONAL:
+                    pipeline.addFirst("ssl", new OptionalSslHandler(settings.encryption));
+                    break;
+                case ENCRYPTED:
+                    SslContext sslContext = SSLFactory.getOrCreateSslContext(settings.encryption, true, SSLFactory.SocketType.SERVER);

Review comment:
       Nit: If its not a big deal, could you create a constant `BUILD_TRUST_STORE=true` and pass it in. Makes the code a bit more readable. If you feel too ambitious, make this into a builder.




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r501188428



##########
File path: src/java/org/apache/cassandra/transport/Server.java
##########
@@ -139,18 +139,19 @@ public synchronized void start()
         if (workerGroup != null)
             bootstrap = bootstrap.group(workerGroup);
 
+        final EncryptionOptions clientEnc = DatabaseDescriptor.getNativeProtocolEncryptionOptions();
+
         if (this.useSSL)
         {
-            final EncryptionOptions clientEnc = DatabaseDescriptor.getNativeProtocolEncryptionOptions();
-
-            if (clientEnc.optional)
+            if (clientEnc.tlsEncryptionPolicy() == EncryptionOptions.TlsEncryptionPolicy.optional)
             {
-                logger.info("Enabling optionally encrypted CQL connections between client and server");
+                logger.debug("Enabling optionally encrypted CQL connections between client and server");
                 bootstrap.childHandler(new OptionalSecureInitializer(this, clientEnc));
             }
             else
             {
-                logger.info("Enabling encrypted CQL connections between client and server");
+                logger.debug("Enabling encrypted CQL connections between client and server");
+                assert clientEnc.tlsEncryptionPolicy() == EncryptionOptions.TlsEncryptionPolicy.encrypted : "Unexpected policy: " + clientEnc.tlsEncryptionPolicy();

Review comment:
       Yes, I can tag it with an `@Nonnull`, as that seems used elsewhere, though I don't know what current best practice is there for static analysis tools.




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r509683311



##########
File path: test/distributed/org/apache/cassandra/distributed/test/AbstractEncryptionOptionsTest.java
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.distributed.test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslHandler;
+import io.netty.util.concurrent.FutureListener;
+import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.security.SSLFactory;
+import org.apache.cassandra.utils.concurrent.SimpleCondition;
+
+public class AbstractEncryptionOptionsTest extends TestBaseImpl

Review comment:
       will do

##########
File path: src/java/org/apache/cassandra/net/InboundSockets.java
##########
@@ -37,6 +37,7 @@
 import io.netty.util.concurrent.SucceededFuture;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.EncryptionOptions;

Review comment:
       ack




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


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r498988230



##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -27,6 +28,23 @@
 
 public class EncryptionOptions
 {
+    public enum TlsEncryptionPolicy
+    {
+        unencrypted("unencrypted"), optional("optionally encrypted"), encrypted("encrypted");

Review comment:
       Usual convention is to use upper case identifiers for enum elements.

##########
File path: src/java/org/apache/cassandra/net/SocketFactory.java
##########
@@ -228,39 +228,41 @@ static SslHandler newSslHandler(Channel channel, SslContext sslContext, @Nullabl
         return sslHandler;
     }
 
-    static String encryptionLogStatement(EncryptionOptions options)
+    /**
+     * Summarizes the intended encryption options, suitable for logging. Once a connection is established, use
+     * {@link SocketFactory#encryptionConnectionSummary} below.
+     * @param options options to summarize
+     * @return description of encryption options
+     */
+    static String encryptionOptionsSummary(EncryptionOptions options)

Review comment:
       Would it be better to simply change this to:
   ```java
   if (options == null)
       return "disabled";
   
   String encryptionType = SSLFactory.openSslIsAvailable() ? "openssl" : "jdk";
   return options.tlsEncryptionPolicy().description() + '(' + encryptionType + ')';
   ```

##########
File path: src/java/org/apache/cassandra/net/InboundConnectionSettings.java
##########
@@ -83,8 +83,8 @@ public boolean authenticate(InetAddress address, int port)
 
     public String toString()
     {
-        return format("address: (%s), nic: %s, encryption: %s",
-                      bindAddress, FBUtilities.getNetworkInterface(bindAddress.address), SocketFactory.encryptionLogStatement(null, encryption));
+        return format("address: (%s), nic: %s, %s",

Review comment:
       Could we add back `encryption: ` label?

##########
File path: src/java/org/apache/cassandra/transport/Server.java
##########
@@ -139,18 +139,19 @@ public synchronized void start()
         if (workerGroup != null)
             bootstrap = bootstrap.group(workerGroup);
 
+        final EncryptionOptions clientEnc = DatabaseDescriptor.getNativeProtocolEncryptionOptions();
+
         if (this.useSSL)
         {
-            final EncryptionOptions clientEnc = DatabaseDescriptor.getNativeProtocolEncryptionOptions();
-
-            if (clientEnc.optional)
+            if (clientEnc.tlsEncryptionPolicy() == EncryptionOptions.TlsEncryptionPolicy.optional)
             {
-                logger.info("Enabling optionally encrypted CQL connections between client and server");
+                logger.debug("Enabling optionally encrypted CQL connections between client and server");
                 bootstrap.childHandler(new OptionalSecureInitializer(this, clientEnc));
             }
             else
             {
-                logger.info("Enabling encrypted CQL connections between client and server");
+                logger.debug("Enabling encrypted CQL connections between client and server");
+                assert clientEnc.tlsEncryptionPolicy() == EncryptionOptions.TlsEncryptionPolicy.encrypted : "Unexpected policy: " + clientEnc.tlsEncryptionPolicy();

Review comment:
       Is `clientEnc.tlsEncryptionPolicy()` guaranteed to be non-null?




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


[GitHub] [cassandra] jonmeredith closed pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith closed pull request #763:
URL: https://github.com/apache/cassandra/pull/763


   


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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r504111946



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionSettings.java
##########
@@ -83,8 +83,8 @@ public boolean authenticate(InetAddress address, int port)
 
     public String toString()
     {
-        return format("address: (%s), nic: %s, encryption: %s",
-                      bindAddress, FBUtilities.getNetworkInterface(bindAddress.address), SocketFactory.encryptionLogStatement(null, encryption));
+        return format("address: (%s), nic: %s, encrypted: %s",

Review comment:
       bah, fail putting it back the way it was before a comment from Dinesh. Will 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



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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r509686211



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionSettings.java
##########
@@ -157,7 +157,8 @@ public InboundConnectionSettings withLegacyDefaults()
         ServerEncryptionOptions encryption = this.encryption;
         if (encryption == null)
             encryption = DatabaseDescriptor.getInternodeMessagingEncyptionOptions();
-        encryption = encryption.withOptional(false);
+        encryption = encryption.withOptional(false).withInternodeEncryption(ServerEncryptionOptions.InternodeEncryption.all);

Review comment:
       The `withLegacyDefaults` template is used if the legacy ssl storage port is enabled, and connections on that port should always be encrypted so I defaulted to `all` to make sure the SSL handler (and not the optional SSL handler) was installed.

##########
File path: src/java/org/apache/cassandra/config/DatabaseDescriptor.java
##########
@@ -757,9 +757,13 @@ else if (conf.commitlog_segment_size_in_mb * 1024 < 2 * conf.max_mutation_size_i
             throw new ConfigurationException("commitlog_segment_size_in_mb must be at least twice the size of max_mutation_size_in_kb / 1024", false);
 
         // native transport encryption options
+        if (conf.client_encryption_options != null)
+        {
+            conf.client_encryption_options.applyConfig();
+        }
         if (conf.native_transport_port_ssl != null

Review comment:
       ack, can't check the encryption policy if the client encryption options are null.

##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -98,14 +113,50 @@ public EncryptionOptions(EncryptionOptions options)
         require_client_auth = options.require_client_auth;
         require_endpoint_verification = options.require_endpoint_verification;
         enabled = options.enabled;
-        if (options.optional != null) {
-            optional = options.optional;
-        } else {
-            // If someone is asking for an _insecure_ connection and not explicitly telling us to refuse
-            // encrypted connections we assume they would like to be able to transition to encrypted connections
-            // in the future.
-            optional = !enabled;
+        this.optional = options.optional;
+    }
+
+    /* Computes enabled and optional before use. Because the configuration can be loaded
+     * through pluggable mechanisms this is the only safe way to make sure that
+     * enabled and optional are set correctly.
+     */
+    public EncryptionOptions applyConfig()
+    {
+        if (isEnabled != null || isOptional != null)

Review comment:
       heh, refactor fail. Yes it should.




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


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r500553772



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionSettings.java
##########
@@ -83,8 +83,8 @@ public boolean authenticate(InetAddress address, int port)
 
     public String toString()
     {
-        return format("address: (%s), nic: %s, encryption: %s",
-                      bindAddress, FBUtilities.getNetworkInterface(bindAddress.address), SocketFactory.encryptionLogStatement(null, encryption));
+        return format("address: (%s), nic: %s, %s",

Review comment:
       From a parsing standpoint, I prefer the second. It's easier to parse the log entries, for example, in Splunk. It is also consistent with what we print in `OutboundConnectionSettings` https://github.com/apache/cassandra/pull/763/files#diff-77d6e44359d621fcf54fef9e65dd0a10L173




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r504102639



##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -302,6 +363,30 @@ public boolean isEnabled() {
             return this.internode_encryption != InternodeEncryption.none;
         }
 
+        @Override
+        public TlsEncryptionPolicy tlsEncryptionPolicy()
+        {
+            if (this.enabled != null)
+            {
+                throw new ConfigurationException("enabled should not be configured for server_encryption_policy, must use internode_encryption");

Review comment:
       good eye, 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



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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r502109182



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -503,4 +494,31 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) t
             }
         }
     }
+
+    private static class rejectSslHandler extends ByteToMessageDecoder

Review comment:
       ack, thought I was copying existing convention for the Optional handle, but I'm obviously not.




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r501182729



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionSettings.java
##########
@@ -83,8 +83,8 @@ public boolean authenticate(InetAddress address, int port)
 
     public String toString()
     {
-        return format("address: (%s), nic: %s, encryption: %s",
-                      bindAddress, FBUtilities.getNetworkInterface(bindAddress.address), SocketFactory.encryptionLogStatement(null, encryption));
+        return format("address: (%s), nic: %s, %s",

Review comment:
       Ok, will keep the original `encryption:` tag.




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


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r502085602



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -503,4 +494,31 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) t
             }
         }
     }
+
+    private static class rejectSslHandler extends ByteToMessageDecoder

Review comment:
       Class names should begin with upper-case characters.

##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -98,17 +95,22 @@ public void initChannel(SocketChannel channel) throws Exception
 
             // order of handlers: ssl -> logger -> handshakeHandler
             // For either unencrypted or transitional modes, allow Ssl optionally.
-            if (settings.encryption.optional)
+            switch(settings.encryption.tlsEncryptionPolicy())
             {
-                pipeline.addFirst("ssl", new OptionalSslHandler(settings.encryption));
-            }
-            else
-            {
-                SslContext sslContext = SSLFactory.getOrCreateSslContext(settings.encryption, true, SSLFactory.SocketType.SERVER);
-                InetSocketAddress peer = settings.encryption.require_endpoint_verification ? channel.remoteAddress() : null;
-                SslHandler sslHandler = newSslHandler(channel, sslContext, peer);
-                logger.trace("creating inbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());
-                pipeline.addFirst("ssl", sslHandler);
+                case UNENCRYPTED:
+                    // Handler checks for SSL connection attempts and cleanly rejects them if encryption is disabled
+                    pipeline.addFirst("rejectssl", new rejectSslHandler());
+                    break;
+                case OPTIONAL:
+                    pipeline.addFirst("ssl", new OptionalSslHandler(settings.encryption));
+                    break;
+                case ENCRYPTED:
+                    SslContext sslContext = SSLFactory.getOrCreateSslContext(settings.encryption, true, SSLFactory.SocketType.SERVER);

Review comment:
       Nit: If its not a big deal, could you create a constant `BUILD_TRUST_STORE=true` and pass it in. Makes the code a bit more readable. If you feel too ambitious, make this into a builder.




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


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r501223597



##########
File path: src/java/org/apache/cassandra/transport/Server.java
##########
@@ -139,18 +139,19 @@ public synchronized void start()
         if (workerGroup != null)
             bootstrap = bootstrap.group(workerGroup);
 
+        final EncryptionOptions clientEnc = DatabaseDescriptor.getNativeProtocolEncryptionOptions();
+
         if (this.useSSL)
         {
-            final EncryptionOptions clientEnc = DatabaseDescriptor.getNativeProtocolEncryptionOptions();
-
-            if (clientEnc.optional)
+            if (clientEnc.tlsEncryptionPolicy() == EncryptionOptions.TlsEncryptionPolicy.optional)
             {
-                logger.info("Enabling optionally encrypted CQL connections between client and server");
+                logger.debug("Enabling optionally encrypted CQL connections between client and server");
                 bootstrap.childHandler(new OptionalSecureInitializer(this, clientEnc));
             }
             else
             {
-                logger.info("Enabling encrypted CQL connections between client and server");
+                logger.debug("Enabling encrypted CQL connections between client and server");
+                assert clientEnc.tlsEncryptionPolicy() == EncryptionOptions.TlsEncryptionPolicy.encrypted : "Unexpected policy: " + clientEnc.tlsEncryptionPolicy();

Review comment:
       `@NotNull` is the way to go.




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


[GitHub] [cassandra] dcapwell commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r503503981



##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -280,13 +340,14 @@ public int hashCode()
 
         public ServerEncryptionOptions()
         {
+            unsetEnabled(); // set to null DatabaseDescriptor can check if it was explicitly set.

Review comment:
       is this needed?  the default is null so setting null shouldn't be required.




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r509682394



##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -114,19 +165,61 @@ public EncryptionOptions(EncryptionOptions options)
      * @return if the channel should be encrypted
      */
     public boolean isEnabled() {
-        return this.enabled;
+        ensureConfigApplied();
+        return isEnabled;
     }
 
     /**
      * Sets if encryption should be enabled for this channel. Note that this should only be called by
      * the configuration parser or tests. It is public only for that purpose, mutating enabled state
      * is probably a bad idea.
-     * @param enabled
+     * @param enabled value to set
      */
     public void setEnabled(boolean enabled) {

Review comment:
       Digging into this, snakeyaml finds `isEnabled` when deciding how to read/write to EncryptionOptions and needs the isEnabled/isOptional types to match exactly.




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


[GitHub] [cassandra] dcapwell commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r502706371



##########
File path: src/java/org/apache/cassandra/db/virtual/SimpleDataSet.java
##########
@@ -84,7 +84,7 @@ private DecoratedKey makeDecoratedKey(Object... partitionKeyValues)
     {
         ByteBuffer partitionKey = partitionKeyValues.length == 1
                                 ? decompose(metadata.partitionKeyType, partitionKeyValues[0])
-                                : ((CompositeType) metadata.partitionKeyType).decompose(ByteBufferAccessor.instance, partitionKeyValues);

Review comment:
       teehee




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r504110164



##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -280,13 +340,14 @@ public int hashCode()
 
         public ServerEncryptionOptions()
         {
+            unsetEnabled(); // set to null DatabaseDescriptor can check if it was explicitly set.

Review comment:
       So removing in the current form causes test failures as the Yaml config loader I was originally using initializes the enabled field.
   
   ```
   Caused by: org.apache.cassandra.exceptions.ConfigurationException: enabled should not be configured for server_encryption_options, must use internode_encryption
   	at org.apache.cassandra.config.EncryptionOptions$ServerEncryptionOptions.tlsEncryptionPolicy(EncryptionOptions.java:365)
   	at org.apache.cassandra.security.SSLFactory.validateSslCerts(SSLFactory.java:409)
   	... 12 more
   ```
   
   I'll try with the improved version.
   




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r504105429



##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -280,13 +340,14 @@ public int hashCode()
 
         public ServerEncryptionOptions()
         {
+            unsetEnabled(); // set to null DatabaseDescriptor can check if it was explicitly set.

Review comment:
       Agreed -- not needed.




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r502109182



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -503,4 +494,31 @@ protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) t
             }
         }
     }
+
+    private static class rejectSslHandler extends ByteToMessageDecoder

Review comment:
       ack, thought I was copying existing convention for the Optional handle, but I'm obviously not.

##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -98,17 +95,22 @@ public void initChannel(SocketChannel channel) throws Exception
 
             // order of handlers: ssl -> logger -> handshakeHandler
             // For either unencrypted or transitional modes, allow Ssl optionally.
-            if (settings.encryption.optional)
+            switch(settings.encryption.tlsEncryptionPolicy())
             {
-                pipeline.addFirst("ssl", new OptionalSslHandler(settings.encryption));
-            }
-            else
-            {
-                SslContext sslContext = SSLFactory.getOrCreateSslContext(settings.encryption, true, SSLFactory.SocketType.SERVER);
-                InetSocketAddress peer = settings.encryption.require_endpoint_verification ? channel.remoteAddress() : null;
-                SslHandler sslHandler = newSslHandler(channel, sslContext, peer);
-                logger.trace("creating inbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());
-                pipeline.addFirst("ssl", sslHandler);
+                case UNENCRYPTED:
+                    // Handler checks for SSL connection attempts and cleanly rejects them if encryption is disabled
+                    pipeline.addFirst("rejectssl", new rejectSslHandler());
+                    break;
+                case OPTIONAL:
+                    pipeline.addFirst("ssl", new OptionalSslHandler(settings.encryption));
+                    break;
+                case ENCRYPTED:
+                    SslContext sslContext = SSLFactory.getOrCreateSslContext(settings.encryption, true, SSLFactory.SocketType.SERVER);

Review comment:
       Went with the low effort route as there's already more than I wanted to refactor in the beta.




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r504111775



##########
File path: src/java/org/apache/cassandra/net/SocketFactory.java
##########
@@ -228,39 +228,41 @@ static SslHandler newSslHandler(Channel channel, SslContext sslContext, @Nullabl
         return sslHandler;
     }
 
-    static String encryptionLogStatement(EncryptionOptions options)
+    /**
+     * Summarizes the intended encryption options, suitable for logging. Once a connection is established, use
+     * {@link SocketFactory#encryptionConnectionSummary} below.
+     * @param options options to summarize
+     * @return description of encryption options
+     */
+    static String encryptionOptionsSummary(EncryptionOptions options)
     {
-        if (options == null)
-            return "disabled";
+        if (options == null || options.tlsEncryptionPolicy() == EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED)
+            return EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED.description();
 
         String encryptionType = SSLFactory.openSslIsAvailable() ? "openssl" : "jdk";
-        return "enabled (" + encryptionType + ')';
+        return options.tlsEncryptionPolicy().description() + '(' + encryptionType + ')';
     }
 
-    static String encryptionLogStatement(Channel channel, EncryptionOptions options)
+    /**
+     * Summarizes the encryption status of a channel, suitable for logging.
+     * @return description of channel encryption
+     */
+    static String encryptionConnectionSummary(Channel channel)
     {
-        if (options == null || !options.isEnabled())
-            return "disabled";
-
-        StringBuilder sb = new StringBuilder(64);
-        if (options.optional)
-            sb.append("optional (factory=");
-        else
-            sb.append("enabled (factory=");
-        sb.append(SSLFactory.openSslIsAvailable() ? "openssl" : "jdk");
-
-        final SslHandler sslHandler = channel == null ? null : channel.pipeline().get(SslHandler.class);
-        if (sslHandler != null)
+        final SslHandler sslHandler = channel.pipeline().get(SslHandler.class);
+        if (sslHandler == null)
         {
-            SSLSession session = sslHandler.engine().getSession();
-            sb.append(";protocol=")
-              .append(session.getProtocol())
-              .append(";cipher=")
-              .append(session.getCipherSuite());
+            return EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED.description();
         }
-
-        sb.append(')');
-        return sb.toString();
+        SSLSession session = sslHandler.engine().getSession();
+
+        return  "encrypted(factory=" +
+                (SSLFactory.openSslIsAvailable() ? "openssl" : "jdk") +
+                ";protocol=" +
+                (session != null ? session.getProtocol() : "MISSING SESSION") +

Review comment:
       Pure paranoia. I didn't want to lose diagnostic information from the log statements in the event of a crash building the string used there.




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


[GitHub] [cassandra] dineshjoshi commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dineshjoshi commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r505001534



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -460,14 +447,24 @@ void setupMessagingPipeline(InetAddressAndPort from, int useMessagingVersion, in
                         handler.id(true),
                         useMessagingVersion,
                         initiate.framing,
-                        pipeline.get("ssl") != null ? encryptionLogStatement(pipeline.channel(), settings.encryption) : "disabled");
+                        SocketFactory.encryptionConnectionSummary(pipeline.channel()));
 
             pipeline.addLast("deserialize", handler);
 
             pipeline.remove(this);
         }
     }
 
+    private static SslHandler getSslHandler(String description, Channel channel, EncryptionOptions.ServerEncryptionOptions encryptionOptions) throws IOException
+    {
+        final boolean BUILD_TRUST_STORE = true;

Review comment:
       I'd prefer to keep it as a constant. Not everybody is going to open the code in IntelliJ idea. We follow the Sun Java coding standards with slight modifications. [Constants are to be named all in upper case with underscores as a separator](https://www.oracle.com/java/technologies/javase/codeconventions-namingconventions.html).




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r502110687



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -98,17 +95,22 @@ public void initChannel(SocketChannel channel) throws Exception
 
             // order of handlers: ssl -> logger -> handshakeHandler
             // For either unencrypted or transitional modes, allow Ssl optionally.
-            if (settings.encryption.optional)
+            switch(settings.encryption.tlsEncryptionPolicy())
             {
-                pipeline.addFirst("ssl", new OptionalSslHandler(settings.encryption));
-            }
-            else
-            {
-                SslContext sslContext = SSLFactory.getOrCreateSslContext(settings.encryption, true, SSLFactory.SocketType.SERVER);
-                InetSocketAddress peer = settings.encryption.require_endpoint_verification ? channel.remoteAddress() : null;
-                SslHandler sslHandler = newSslHandler(channel, sslContext, peer);
-                logger.trace("creating inbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());
-                pipeline.addFirst("ssl", sslHandler);
+                case UNENCRYPTED:
+                    // Handler checks for SSL connection attempts and cleanly rejects them if encryption is disabled
+                    pipeline.addFirst("rejectssl", new rejectSslHandler());
+                    break;
+                case OPTIONAL:
+                    pipeline.addFirst("ssl", new OptionalSslHandler(settings.encryption));
+                    break;
+                case ENCRYPTED:
+                    SslContext sslContext = SSLFactory.getOrCreateSslContext(settings.encryption, true, SSLFactory.SocketType.SERVER);

Review comment:
       Went with the low effort route as there's already more than I wanted to refactor in the beta.




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r509682740



##########
File path: src/java/org/apache/cassandra/config/DatabaseDescriptor.java
##########
@@ -757,9 +757,13 @@ else if (conf.commitlog_segment_size_in_mb * 1024 < 2 * conf.max_mutation_size_i
             throw new ConfigurationException("commitlog_segment_size_in_mb must be at least twice the size of max_mutation_size_in_kb / 1024", false);
 
         // native transport encryption options
+        if (conf.client_encryption_options != null)
+        {
+            conf.client_encryption_options.applyConfig();
+        }
         if (conf.native_transport_port_ssl != null

Review comment:
       will do.

##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -98,14 +113,50 @@ public EncryptionOptions(EncryptionOptions options)
         require_client_auth = options.require_client_auth;
         require_endpoint_verification = options.require_endpoint_verification;
         enabled = options.enabled;
-        if (options.optional != null) {
-            optional = options.optional;
-        } else {
-            // If someone is asking for an _insecure_ connection and not explicitly telling us to refuse
-            // encrypted connections we assume they would like to be able to transition to encrypted connections
-            // in the future.
-            optional = !enabled;
+        this.optional = options.optional;
+    }
+
+    /* Computes enabled and optional before use. Because the configuration can be loaded
+     * through pluggable mechanisms this is the only safe way to make sure that
+     * enabled and optional are set correctly.
+     */
+    public EncryptionOptions applyConfig()
+    {
+        if (isEnabled != null || isOptional != null)

Review comment:
       absoutely, 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



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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r504095689



##########
File path: src/java/org/apache/cassandra/transport/Server.java
##########
@@ -139,29 +137,27 @@ public synchronized void start()
         if (workerGroup != null)
             bootstrap = bootstrap.group(workerGroup);
 
-        if (this.useSSL)
-        {
-            final EncryptionOptions clientEnc = DatabaseDescriptor.getNativeProtocolEncryptionOptions();
+        final EncryptionOptions clientEnc = DatabaseDescriptor.getNativeProtocolEncryptionOptions();
 
-            if (clientEnc.optional)
-            {
-                logger.info("Enabling optionally encrypted CQL connections between client and server");
+        switch (this.tlsEncryptionPolicy)
+        {
+            case UNENCRYPTED:
+                bootstrap.childHandler(new Initializer(this));
+                break;
+            case OPTIONAL:
+                logger.debug("Enabling optionally encrypted CQL connections between client and server");
                 bootstrap.childHandler(new OptionalSecureInitializer(this, clientEnc));
-            }
-            else
-            {
-                logger.info("Enabling encrypted CQL connections between client and server");
+                break;
+            case ENCRYPTED:
+                logger.debug("Enabling encrypted CQL connections between client and server");
                 bootstrap.childHandler(new SecureInitializer(this, clientEnc));
-            }
-        }
-        else
-        {
-            bootstrap.childHandler(new Initializer(this));
+                break;
+            default:
+                throw new IllegalStateException("Unrecognized TLS encryption policy: " + this.tlsEncryptionPolicy);
         }
 
         // Bind and start to accept incoming connections.
-        logger.info("Using Netty Version: {}", Version.identify().entrySet());

Review comment:
       It prevents it being logged twice if there is a separate SSL port specified. We only need Netty version information once.




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r501183954



##########
File path: src/java/org/apache/cassandra/net/SocketFactory.java
##########
@@ -228,39 +228,41 @@ static SslHandler newSslHandler(Channel channel, SslContext sslContext, @Nullabl
         return sslHandler;
     }
 
-    static String encryptionLogStatement(EncryptionOptions options)
+    /**
+     * Summarizes the intended encryption options, suitable for logging. Once a connection is established, use
+     * {@link SocketFactory#encryptionConnectionSummary} below.
+     * @param options options to summarize
+     * @return description of encryption options
+     */
+    static String encryptionOptionsSummary(EncryptionOptions options)

Review comment:
       Outputing `unencrypted(openssl)` read badly to me, and I wanted to enforce the same description of encrypted/optional/unencrypted from the enum so I was also getting rid of `disabled`.




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r504102063



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -460,14 +447,24 @@ void setupMessagingPipeline(InetAddressAndPort from, int useMessagingVersion, in
                         handler.id(true),
                         useMessagingVersion,
                         initiate.framing,
-                        pipeline.get("ssl") != null ? encryptionLogStatement(pipeline.channel(), settings.encryption) : "disabled");
+                        SocketFactory.encryptionConnectionSummary(pipeline.channel()));
 
             pipeline.addLast("deserialize", handler);
 
             pipeline.remove(this);
         }
     }
 
+    private static SslHandler getSslHandler(String description, Channel channel, EncryptionOptions.ServerEncryptionOptions encryptionOptions) throws IOException
+    {
+        final boolean BUILD_TRUST_STORE = true;

Review comment:
       it was a review comment from Dinesh that I agreed with, I think it's about conveying more meaning than a simple boolean rather than rely on the IDE to provide hints what they mean.




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


[GitHub] [cassandra] dcapwell commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r503575716



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -460,14 +447,24 @@ void setupMessagingPipeline(InetAddressAndPort from, int useMessagingVersion, in
                         handler.id(true),
                         useMessagingVersion,
                         initiate.framing,
-                        pipeline.get("ssl") != null ? encryptionLogStatement(pipeline.channel(), settings.encryption) : "disabled");
+                        SocketFactory.encryptionConnectionSummary(pipeline.channel()));
 
             pipeline.addLast("deserialize", handler);
 
             pipeline.remove(this);
         }
     }
 
+    private static SslHandler getSslHandler(String description, Channel channel, EncryptionOptions.ServerEncryptionOptions encryptionOptions) throws IOException
+    {
+        final boolean BUILD_TRUST_STORE = true;
+        SslContext sslContext = SSLFactory.getOrCreateSslContext(encryptionOptions, BUILD_TRUST_STORE, SSLFactory.SocketType.SERVER);
+        InetSocketAddress peer = encryptionOptions.require_endpoint_verification ? (InetSocketAddress) channel.remoteAddress() : null;
+        SslHandler sslHandler = newSslHandler(channel, sslContext, peer);
+        logger.trace(description + " inbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());

Review comment:
       nit, `logger.trace("{} inbound netty...", description, sslContext.getClass().getName(), sslHandler.engine().getClass().getName());`

##########
File path: src/java/org/apache/cassandra/net/InboundConnectionSettings.java
##########
@@ -83,8 +83,8 @@ public boolean authenticate(InetAddress address, int port)
 
     public String toString()
     {
-        return format("address: (%s), nic: %s, encryption: %s",
-                      bindAddress, FBUtilities.getNetworkInterface(bindAddress.address), SocketFactory.encryptionLogStatement(null, encryption));
+        return format("address: (%s), nic: %s, encrypted: %s",

Review comment:
       wasn't encryption correct before?  you are doing a summary and not a boolean; the following feels weird
   
   ```
   encrypted: unencrypted
   ```

##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -460,14 +447,24 @@ void setupMessagingPipeline(InetAddressAndPort from, int useMessagingVersion, in
                         handler.id(true),
                         useMessagingVersion,
                         initiate.framing,
-                        pipeline.get("ssl") != null ? encryptionLogStatement(pipeline.channel(), settings.encryption) : "disabled");
+                        SocketFactory.encryptionConnectionSummary(pipeline.channel()));
 
             pipeline.addLast("deserialize", handler);
 
             pipeline.remove(this);
         }
     }
 
+    private static SslHandler getSslHandler(String description, Channel channel, EncryptionOptions.ServerEncryptionOptions encryptionOptions) throws IOException
+    {
+        final boolean BUILD_TRUST_STORE = true;

Review comment:
       also only looks like its used in one place, so kinda prefer the code before that just added true into the method params

##########
File path: src/java/org/apache/cassandra/net/SocketFactory.java
##########
@@ -228,39 +228,41 @@ static SslHandler newSslHandler(Channel channel, SslContext sslContext, @Nullabl
         return sslHandler;
     }
 
-    static String encryptionLogStatement(EncryptionOptions options)
+    /**
+     * Summarizes the intended encryption options, suitable for logging. Once a connection is established, use
+     * {@link SocketFactory#encryptionConnectionSummary} below.
+     * @param options options to summarize
+     * @return description of encryption options
+     */
+    static String encryptionOptionsSummary(EncryptionOptions options)
     {
-        if (options == null)
-            return "disabled";
+        if (options == null || options.tlsEncryptionPolicy() == EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED)
+            return EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED.description();
 
         String encryptionType = SSLFactory.openSslIsAvailable() ? "openssl" : "jdk";
-        return "enabled (" + encryptionType + ')';
+        return options.tlsEncryptionPolicy().description() + '(' + encryptionType + ')';
     }
 
-    static String encryptionLogStatement(Channel channel, EncryptionOptions options)
+    /**
+     * Summarizes the encryption status of a channel, suitable for logging.
+     * @return description of channel encryption
+     */
+    static String encryptionConnectionSummary(Channel channel)
     {
-        if (options == null || !options.isEnabled())
-            return "disabled";
-
-        StringBuilder sb = new StringBuilder(64);
-        if (options.optional)
-            sb.append("optional (factory=");
-        else
-            sb.append("enabled (factory=");
-        sb.append(SSLFactory.openSslIsAvailable() ? "openssl" : "jdk");
-
-        final SslHandler sslHandler = channel == null ? null : channel.pipeline().get(SslHandler.class);
-        if (sslHandler != null)
+        final SslHandler sslHandler = channel.pipeline().get(SslHandler.class);
+        if (sslHandler == null)
         {
-            SSLSession session = sslHandler.engine().getSession();
-            sb.append(";protocol=")
-              .append(session.getProtocol())
-              .append(";cipher=")
-              .append(session.getCipherSuite());
+            return EncryptionOptions.TlsEncryptionPolicy.UNENCRYPTED.description();
         }
-
-        sb.append(')');
-        return sb.toString();
+        SSLSession session = sslHandler.engine().getSession();
+
+        return  "encrypted(factory=" +
+                (SSLFactory.openSslIsAvailable() ? "openssl" : "jdk") +
+                ";protocol=" +
+                (session != null ? session.getProtocol() : "MISSING SESSION") +

Review comment:
       why is session nullable now but wasn't before the patch?  

##########
File path: src/java/org/apache/cassandra/net/InboundSockets.java
##########
@@ -202,7 +203,8 @@ private static void addBindings(InboundConnectionSettings template, ImmutableLis
         InboundConnectionSettings       settings = template.withDefaults();
         InboundConnectionSettings legacySettings = template.withLegacyDefaults();
 
-        if (settings.encryption.enable_legacy_ssl_storage_port)
+        if (settings.encryption.enable_legacy_ssl_storage_port &&

Review comment:
       I am wondering if it would be better to push this logic into `org.apache.cassandra.net.InboundConnectionSettings` rather than have here? or DatabaseDescriptor (only allow true if encryptions != none

##########
File path: src/java/org/apache/cassandra/transport/Server.java
##########
@@ -139,29 +137,27 @@ public synchronized void start()
         if (workerGroup != null)
             bootstrap = bootstrap.group(workerGroup);
 
-        if (this.useSSL)
-        {
-            final EncryptionOptions clientEnc = DatabaseDescriptor.getNativeProtocolEncryptionOptions();
+        final EncryptionOptions clientEnc = DatabaseDescriptor.getNativeProtocolEncryptionOptions();
 
-            if (clientEnc.optional)
-            {
-                logger.info("Enabling optionally encrypted CQL connections between client and server");
+        switch (this.tlsEncryptionPolicy)
+        {
+            case UNENCRYPTED:
+                bootstrap.childHandler(new Initializer(this));
+                break;
+            case OPTIONAL:
+                logger.debug("Enabling optionally encrypted CQL connections between client and server");
                 bootstrap.childHandler(new OptionalSecureInitializer(this, clientEnc));
-            }
-            else
-            {
-                logger.info("Enabling encrypted CQL connections between client and server");
+                break;
+            case ENCRYPTED:
+                logger.debug("Enabling encrypted CQL connections between client and server");
                 bootstrap.childHandler(new SecureInitializer(this, clientEnc));
-            }
-        }
-        else
-        {
-            bootstrap.childHandler(new Initializer(this));
+                break;
+            default:
+                throw new IllegalStateException("Unrecognized TLS encryption policy: " + this.tlsEncryptionPolicy);
         }
 
         // Bind and start to accept incoming connections.
-        logger.info("Using Netty Version: {}", Version.identify().entrySet());

Review comment:
       why move this log to a different location?




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


[GitHub] [cassandra] jonmeredith commented on pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on pull request #763:
URL: https://github.com/apache/cassandra/pull/763#issuecomment-778314165


   Merged.


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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r504111207



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -460,14 +447,24 @@ void setupMessagingPipeline(InetAddressAndPort from, int useMessagingVersion, in
                         handler.id(true),
                         useMessagingVersion,
                         initiate.framing,
-                        pipeline.get("ssl") != null ? encryptionLogStatement(pipeline.channel(), settings.encryption) : "disabled");
+                        SocketFactory.encryptionConnectionSummary(pipeline.channel()));
 
             pipeline.addLast("deserialize", handler);
 
             pipeline.remove(this);
         }
     }
 
+    private static SslHandler getSslHandler(String description, Channel channel, EncryptionOptions.ServerEncryptionOptions encryptionOptions) throws IOException
+    {
+        final boolean BUILD_TRUST_STORE = true;
+        SslContext sslContext = SSLFactory.getOrCreateSslContext(encryptionOptions, BUILD_TRUST_STORE, SSLFactory.SocketType.SERVER);
+        InetSocketAddress peer = encryptionOptions.require_endpoint_verification ? (InetSocketAddress) channel.remoteAddress() : null;
+        SslHandler sslHandler = newSslHandler(channel, sslContext, peer);
+        logger.trace(description + " inbound netty SslContext: context={}, engine={}", sslContext.getClass().getName(), sslHandler.engine().getClass().getName());

Review comment:
       ack




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


[GitHub] [cassandra] dcapwell commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r508131591



##########
File path: src/java/org/apache/cassandra/config/DatabaseDescriptor.java
##########
@@ -757,9 +757,13 @@ else if (conf.commitlog_segment_size_in_mb * 1024 < 2 * conf.max_mutation_size_i
             throw new ConfigurationException("commitlog_segment_size_in_mb must be at least twice the size of max_mutation_size_in_kb / 1024", false);
 
         // native transport encryption options
+        if (conf.client_encryption_options != null)
+        {
+            conf.client_encryption_options.applyConfig();
+        }
         if (conf.native_transport_port_ssl != null

Review comment:
       maybe move this into the null check if statement like you did for server_encryption_options?

##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -98,14 +113,50 @@ public EncryptionOptions(EncryptionOptions options)
         require_client_auth = options.require_client_auth;
         require_endpoint_verification = options.require_endpoint_verification;
         enabled = options.enabled;
-        if (options.optional != null) {
-            optional = options.optional;
-        } else {
-            // If someone is asking for an _insecure_ connection and not explicitly telling us to refuse
-            // encrypted connections we assume they would like to be able to transition to encrypted connections
-            // in the future.
-            optional = !enabled;
+        this.optional = options.optional;
+    }
+
+    /* Computes enabled and optional before use. Because the configuration can be loaded
+     * through pluggable mechanisms this is the only safe way to make sure that
+     * enabled and optional are set correctly.
+     */
+    public EncryptionOptions applyConfig()
+    {
+        if (isEnabled != null || isOptional != null)

Review comment:
       should this call `ensureConfigNotApplied`?

##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -114,19 +165,61 @@ public EncryptionOptions(EncryptionOptions options)
      * @return if the channel should be encrypted
      */
     public boolean isEnabled() {
-        return this.enabled;
+        ensureConfigApplied();
+        return isEnabled;
     }
 
     /**
      * Sets if encryption should be enabled for this channel. Note that this should only be called by
      * the configuration parser or tests. It is public only for that purpose, mutating enabled state
      * is probably a bad idea.
-     * @param enabled
+     * @param enabled value to set
      */
     public void setEnabled(boolean enabled) {

Review comment:
       I am not sure why the test doesn't seem to pick this up, but since the field is no longer public the test you added is failing.

##########
File path: test/distributed/org/apache/cassandra/distributed/test/AbstractEncryptionOptionsTest.java
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.distributed.test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Assert;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import io.netty.handler.ssl.SslContext;
+import io.netty.handler.ssl.SslHandler;
+import io.netty.util.concurrent.FutureListener;
+import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.security.SSLFactory;
+import org.apache.cassandra.utils.concurrent.SimpleCondition;
+
+public class AbstractEncryptionOptionsTest extends TestBaseImpl

Review comment:
       this is causing CI to fail as CI searches for files that match this pattern: `*Test.java`, can you rename this to remove `Test`?

##########
File path: src/java/org/apache/cassandra/net/InboundSockets.java
##########
@@ -37,6 +37,7 @@
 import io.netty.util.concurrent.SucceededFuture;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.EncryptionOptions;

Review comment:
       can you remove?  unused import

##########
File path: src/java/org/apache/cassandra/net/InboundConnectionSettings.java
##########
@@ -157,7 +157,8 @@ public InboundConnectionSettings withLegacyDefaults()
         ServerEncryptionOptions encryption = this.encryption;
         if (encryption == null)
             encryption = DatabaseDescriptor.getInternodeMessagingEncyptionOptions();
-        encryption = encryption.withOptional(false);
+        encryption = encryption.withOptional(false).withInternodeEncryption(ServerEncryptionOptions.InternodeEncryption.all);

Review comment:
       in 3.0 I see the default is `none` and not `all`

##########
File path: src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
##########
@@ -135,6 +128,22 @@ public Config loadConfig(URL url) throws ConfigurationException
         }
     }
 
+    public static <T> T parseYamlString(Class<T> klass, String yamlString)
+    {
+        return parseYamlBytes(klass, yamlString.getBytes());
+    }
+
+    private static <T> T parseYamlBytes(Class<T> klass, byte[] configBytes)
+    {
+        Constructor constructor = new CustomConstructor(klass, Thread.currentThread().getContextClassLoader());

Review comment:
       should this be `Yaml.class.getClassLoader()` or `klass.getClassLoader()`?

##########
File path: src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
##########
@@ -135,6 +128,22 @@ public Config loadConfig(URL url) throws ConfigurationException
         }
     }
 
+    public static <T> T parseYamlString(Class<T> klass, String yamlString)
+    {
+        return parseYamlBytes(klass, yamlString.getBytes());

Review comment:
       should be `yamlString.getBytes(StandardCharsets.UTF_8)`; `getBytes()` is host specific, so some hosts may be utf-8, others may be ascii.




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


[GitHub] [cassandra] jonmeredith commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
jonmeredith commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r509684525



##########
File path: src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
##########
@@ -135,6 +128,22 @@ public Config loadConfig(URL url) throws ConfigurationException
         }
     }
 
+    public static <T> T parseYamlString(Class<T> klass, String yamlString)
+    {
+        return parseYamlBytes(klass, yamlString.getBytes());

Review comment:
       should be handled by updates to CASSANDRA-16152 now.

##########
File path: src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
##########
@@ -135,6 +128,22 @@ public Config loadConfig(URL url) throws ConfigurationException
         }
     }
 
+    public static <T> T parseYamlString(Class<T> klass, String yamlString)
+    {
+        return parseYamlBytes(klass, yamlString.getBytes());
+    }
+
+    private static <T> T parseYamlBytes(Class<T> klass, byte[] configBytes)
+    {
+        Constructor constructor = new CustomConstructor(klass, Thread.currentThread().getContextClassLoader());

Review comment:
       should be handled by updates to CASSANDRA-16152 now.




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


[GitHub] [cassandra] dcapwell commented on a change in pull request #763: CASSANDRA-16144 TLS connections to the storage port on a node without server encryption configured causes java.io.IOException accessing missing keystore

Posted by GitBox <gi...@apache.org>.
dcapwell commented on a change in pull request #763:
URL: https://github.com/apache/cassandra/pull/763#discussion_r502706774



##########
File path: src/java/org/apache/cassandra/net/InboundConnectionInitiator.java
##########
@@ -460,14 +447,24 @@ void setupMessagingPipeline(InetAddressAndPort from, int useMessagingVersion, in
                         handler.id(true),
                         useMessagingVersion,
                         initiate.framing,
-                        pipeline.get("ssl") != null ? encryptionLogStatement(pipeline.channel(), settings.encryption) : "disabled");
+                        SocketFactory.encryptionConnectionSummary(pipeline.channel()));
 
             pipeline.addLast("deserialize", handler);
 
             pipeline.remove(this);
         }
     }
 
+    private static SslHandler getSslHandler(String description, Channel channel, EncryptionOptions.ServerEncryptionOptions encryptionOptions) throws IOException
+    {
+        final boolean BUILD_TRUST_STORE = true;

Review comment:
       `s/BUILD_TRUST_STORE/buildTrustStore/g`

##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -302,6 +363,30 @@ public boolean isEnabled() {
             return this.internode_encryption != InternodeEncryption.none;
         }
 
+        @Override
+        public TlsEncryptionPolicy tlsEncryptionPolicy()
+        {
+            if (this.enabled != null)
+            {
+                throw new ConfigurationException("enabled should not be configured for server_encryption_policy, must use internode_encryption");

Review comment:
       do you mean `server_encryption_options` instead of `server_encryption_policy`?

##########
File path: src/java/org/apache/cassandra/config/EncryptionOptions.java
##########
@@ -114,19 +119,74 @@ public EncryptionOptions(EncryptionOptions options)
      * @return if the channel should be encrypted
      */
     public boolean isEnabled() {
-        return this.enabled;
+        return this.enabled != null && enabled;
     }
 
     /**
      * Sets if encryption should be enabled for this channel. Note that this should only be called by
      * the configuration parser or tests. It is public only for that purpose, mutating enabled state
      * is probably a bad idea.
-     * @param enabled
+     * @param enabled value to set
      */
     public void setEnabled(boolean enabled) {
         this.enabled = enabled;
     }
 
+    protected void unsetEnabled()
+    {
+        this.enabled = null;
+    }
+
+    /**
+     * Indicates if the channel may be encrypted (but is not requried to be).
+     * Explicitly providing a value in the configuration take precedent.
+     * If no optional value is set and !isEnabled(), then optional connections are allowed
+     * if a keystore exists. Without it, it would be impossible to establish the connections.
+     * @return if the channel may be encrypted
+     */
+    public boolean isOptional()
+    {
+        if (optional != null)
+            return optional;
+
+        // If someone is asking for an _insecure_ connection and not explicitly telling us to refuse
+        // encrypted connections AND they have a keystore file, we assume they would like to be able
+        // to transition to encrypted connections in the future.
+        if (new File(keystore).exists())

Review comment:
       can we move this into the constructor?  would be good to avoid touching the filesystem every time a new connection is opened;  do we allow updating keystore at runtime, if so how do we detect this?




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