Jon Meredith created CASSANDRA-16144:
----------------------------------------

             Summary: TLS connections to the storage port on a node without 
server encryption configured causes java.io.IOException accessing missing 
keystore
                 Key: CASSANDRA-16144
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-16144
             Project: Cassandra
          Issue Type: Bug
          Components: Messaging/Internode
            Reporter: Jon Meredith
            Assignee: Jon Meredith


If a TLS connection is requested against a node with all encryption disabled by 
configuration,
configured with

{code}
server_encryption_options: {optional:false, internode_encryption: none}
{code}

it logs the following error if no keystore exists for the node.

{code}
INFO  [Messaging-EventLoop-3-3] 2020-09-15T14:30:02,952 : - 
127.0.0.1:7000->127.0.1.1:7000-URGENT_MESSAGES-[no-channel] failed to connect
io.netty.channel.AbstractChannel$AnnotatedConnectException: Connection refused: 
local1-i1/127.0.1.1:7000
Caused by: java.net.ConnectException: Connection refused
       at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method) ~[?:?]
       at 
sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:779) ~[?:?]
       at 
io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:330)
 ~[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:334)
 ~[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:702) 
~[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:650)
 ~[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:576) 
~[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:493) 
[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) 
[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at java.lang.Thread.run(Thread.java:834) [?:?]
WARN  [Messaging-EventLoop-3-9] 2020-09-15T14:30:06,375 : - Failed to 
initialize a channel. Closing: [id: 0x0746c157, L:/127.0.0.1:7000 - 
R:/127.0.0.1:59623]
java.io.IOException: failed to build trust manager store for secure connections
       at 
org.apache.cassandra.security.SSLFactory.buildKeyManagerFactory(SSLFactory.java:232)
 ~[apache-cassandra-4.0-beta1-SNAPSHOT.jar:4.0-beta1-SNAPSHOT]
       at 
org.apache.cassandra.security.SSLFactory.createNettySslContext(SSLFactory.java:300)
 ~[apache-cassandra-4.0-beta1-SNAPSHOT.jar:4.0-beta1-SNAPSHOT]
       at 
org.apache.cassandra.security.SSLFactory.getOrCreateSslContext(SSLFactory.java:276)
 ~[apache-cassandra-4.0-beta1-SNAPSHOT.jar:4.0-beta1-SNAPSHOT]
       at 
org.apache.cassandra.security.SSLFactory.getOrCreateSslContext(SSLFactory.java:257)
 ~[apache-cassandra-4.0-beta1-SNAPSHOT.jar:4.0-beta1-SNAPSHOT]
       at 
org.apache.cassandra.net.InboundConnectionInitiator$Initializer.initChannel(InboundConnectionInitiator.java:107)
 ~[apache-cassandra-4.0-beta1-SNAPSHOT.jar:4.0-beta1-SNAPSHOT]
       at 
org.apache.cassandra.net.InboundConnectionInitiator$Initializer.initChannel(InboundConnectionInitiator.java:71)
 ~[apache-cassandra-4.0-beta1-SNAPSHOT.jar:4.0-beta1-SNAPSHOT]
       at 
io.netty.channel.ChannelInitializer.initChannel(ChannelInitializer.java:129) 
[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.ChannelInitializer.handlerAdded(ChannelInitializer.java:112) 
[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.AbstractChannelHandlerContext.callHandlerAdded(AbstractChannelHandlerContext.java:938)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.DefaultChannelPipeline.callHandlerAdded0(DefaultChannelPipeline.java:609)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.DefaultChannelPipeline.access$100(DefaultChannelPipeline.java:46)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.DefaultChannelPipeline$PendingHandlerAddedTask.execute(DefaultChannelPipeline.java:1463)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.DefaultChannelPipeline.callHandlerAddedForAllHandlers(DefaultChannelPipeline.java:1115)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.DefaultChannelPipeline.invokeHandlerAddedIfNeeded(DefaultChannelPipeline.java:650)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.AbstractChannel$AbstractUnsafe.register0(AbstractChannel.java:502)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.AbstractChannel$AbstractUnsafe.access$200(AbstractChannel.java:417)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.channel.AbstractChannel$AbstractUnsafe$1.run(AbstractChannel.java:474) 
[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:164)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:472)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:500) 
[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) 
[netty-all-4.1.50.Final.jar:4.1.50.Final]
       at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 [netty-all-4.1.50.Final.jar:4.1.50.Final]
       at java.lang.Thread.run(Thread.java:834) [?:?]
Caused by: java.nio.file.NoSuchFileException: conf/.keystore
       at 
sun.nio.fs.UnixException.translateToIOException(UnixException.java:92) ~[?:?]
       at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:111) 
~[?:?]
       at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:116) 
~[?:?]
       at 
sun.nio.fs.UnixFileSystemProvider.newByteChannel(UnixFileSystemProvider.java:219)
 ~[?:?]
       at java.nio.file.Files.newByteChannel(Files.java:370) ~[?:?]
       at java.nio.file.Files.newByteChannel(Files.java:421) ~[?:?]
       at 
java.nio.file.spi.FileSystemProvider.newInputStream(FileSystemProvider.java:420)
 ~[?:?]
       at java.nio.file.Files.newInputStream(Files.java:155) ~[?:?]
       at 
org.apache.cassandra.security.SSLFactory.buildKeyManagerFactory(SSLFactory.java:207)
 ~[apache-cassandra-4.0-beta1-SNAPSHOT.jar:4.0-beta1-SNAPSHOT]
       ... 23 more
{code}

This happens regardless of the settings of server_encryption_options.optional, 
as the OptionalSslHandler is installed if optional is true, and if optional is 
not true, an SSL handler is always installed.

https://github.com/apache/cassandra/blob/674b6cc1a5e905a9c234c649adaad2de79cfa560/src/java/org/apache/cassandra/net/InboundConnectionInitiator.java#L101


CASSANDRA-15262 improved backward 4.0 backward compatability with 3.11 by 
defaulting to allowing optional encrypted connections if optional was not 
explicitly configured, however if an operator has not installed a keystore and 
an secure connection cannot possibly be established, neither the optional or 
required SSL  handlers should be installed.

Similarly if the operator has explicitly disabled SSL, then neither SSL 
handlers should be installed.

Temporarily commenting out both handlers and testing a TLS connection logs, an 
incoming TLS connection logs an ERROR

{code}
ERROR [Messaging-EventLoop-3-1] 2020-09-15 15:09:48,898 
InboundConnectionInitiator.java:355 - Failed to properly handshake with peer 
/127.0.0.1:60525. Closing the channel.
io.netty.handler.codec.DecoderException: 
org.apache.cassandra.net.Message$InvalidLegacyProtocolMagic: Read 369295616, 
Expected -900387334
       at 
io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:471)
       at 
io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:276)
       at 
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
       at 
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
       at 
io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
       at 
io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
       at 
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
       at 
io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
       at 
io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
       at 
io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:163)
       at 
io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:714)
       at 
io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:650)
       at 
io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:576)
       at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:493)
       at 
io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
       at 
io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
       at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
       at java.base/java.lang.Thread.run(Thread.java:834)
Caused by: org.apache.cassandra.net.Message$InvalidLegacyProtocolMagic: Read 
369295616, Expected -900387334
       at 
org.apache.cassandra.net.Message.validateLegacyProtocolMagic(Message.java:333)
       at 
org.apache.cassandra.net.HandshakeProtocol$Initiate.maybeDecode(HandshakeProtocol.java:167)
       at 
org.apache.cassandra.net.InboundConnectionInitiator$Handler.initiate(InboundConnectionInitiator.java:255)
       at 
org.apache.cassandra.net.InboundConnectionInitiator$Handler.decode(InboundConnectionInitiator.java:248)
       at 
io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:501)
       at 
io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:440)
       ... 17 common frames omitted
{code}

I think it would be helpful to an include a {{DisabledSslHandler}} that uses a 
similar method to the {{OptionalSSLHandler}} of waiting for enough bytes to be 
received, and then logging a warning about an SSL connection attempt on the 
port to help the operator spot misconfigurations or security scans.

There are another couple of related, but minor additional issues.

The listening message does not currently indicate whether optional secure 
connections are permitted, it should be explicit about whether secure 
connections are required, optional and possibly include the internode mode.

{code}
 INFO  [main] 2020-09-15 15:09:33,045 InboundConnectionInitiator.java:128 - 
Listening on address: (localhost/127.0.0.1:7000), nic: lo0, encryption: disabled
{code}

Also {{org.apache.cassandra.security.SSLFactory#buildKeyManagerFactory}} throws 
{{throw new IOException("failed to build trust manager store for secure 
connections", e);}} when it should be {{throw new IOException("failed to build 
key manager store for secure connections", e);}} (probably a copy/paste from 
{{buildTrustManagerFactory}})

Proposed changes to resolve this issue and improve behavior:

* Introduce a DisabledSslHandler that logs a WARN if an SSL connection is 
attempted when administratively disabled.
* Add a check that the keystore exists before defaulting optional to false (as 
a path to a keystore is hard-coded and simply not supplying it does not work)
* Update the listening message to be explicit about what types of encrypted 
connections will be accepted.
* Fix the Keystone IOException to correctly identify the Keystore as the culprit



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to