You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Stefania (JIRA)" <ji...@apache.org> on 2016/05/12 04:49:12 UTC

[jira] [Comment Edited] (CASSANDRA-11749) CQLSH gets SSL exception following a COPY FROM

    [ https://issues.apache.org/jira/browse/CASSANDRA-11749?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15281206#comment-15281206 ] 

Stefania edited comment on CASSANDRA-11749 at 5/12/16 4:48 AM:
---------------------------------------------------------------

Thank you for the information Sam. 

The Android netty issue is similar but, if there is a problem with Netty, it must be for an entirely different reason. So far I've reproduced this exception with Netty 4.0.23 and 4.0.36 using JDK 1.8.0_91 and various versions of Python/OpenSSL (Python 2.7.6 with openssl 1.0.1f-1ubuntu2.19, which is the default on Ubuntu Trusty, Python 2.7.10 custom built with openssl 1.0.2h or 1.0.2g). I know that Datastax have also reproduced it with JDK 1.8.0_40 and netty 4.0.34. I've also verified that the exception exists with SSLv3 as well, so it is not a TLS bug.

Adding a pause of 1 second between the end of the COPY FROM and the next SELECT statement makes the exception disappear. 

Reducing the pause makes the exception gradually easier to reproduce. Without any pause the exception occurs once every 10 times approximately. Therefore I suspect a concurrency bug server side, possibly in Netty. I think closing the SSL connections from the COPY FROM Python processes somehow overlaps with the main session from the cqlsh SELECT. A single COPY FROM process is sufficient to reproduce the exception, and in fact makes it easier to debug server side (attached [^stdout_single_process.txt.zip]). 

If [~norman] has any suggestions I can test them or I can provide further debug information by adding debug statements to the Netty SSLHandler. 

If we cannot understand what the problem is, as a workaround we can introduce a pause after a COPY FROM when using SSL.



was (Author: stefania):
Thank you for the information Sam. 

The Android netty issue is similar but, if there is a problem with Netty, it must be for an entirely different reason. So far I've reproduced this exception with Netty 4.0.23 and 4.0.36 using JDK 1.8.0_91 and various versions of Python/OpenSSL (Python 2.7.6 with openssl 1.0.1f-1ubuntu2.19, which is the default on Ubuntu Trusty, Python 2.7.10 custom built with openssl 1.0.2h or 1.0.2g). I know that Datastax have also reproduced it with JDK 1.8.0_40 and netty 4.0.34. I've also verified that the exception exists with SSLv3 as well, so it is not a TLS bug.

Adding a pause of 1 second between the end of the COPY FROM and the next SELECT statement makes the exception disappear. 

Reducing the pause makes the exception gradually easier to reproduce. Without any pause the exception occurs once every 10 times approximately. Therefore I suspect a concurrency bug server side, possibly in Netty. I think the closing of the SSL connections from the COPY FROM Python processes somehow overlap with the main session from the cqlsh SELECT. A single COPY FROM process is sufficient to reproduce the exception, and in fact makes it easier to debug server side (attached [^stdout_single_process.txt.zip]). 

If [~norman] has any suggestions I can test them or I can provide further debug information by adding debug statements to the Netty SSLHandler. 

If we cannot understand what the problem is, as a workaround we can introduce a pause after a COPY FROM when using SSL.


> CQLSH gets SSL exception following a COPY FROM
> ----------------------------------------------
>
>                 Key: CASSANDRA-11749
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11749
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Tools
>            Reporter: Stefania
>            Assignee: Stefania
>             Fix For: 2.1.x
>
>         Attachments: stdout.txt.zip, stdout_single_process.txt.zip
>
>
> When running Cassandra and cqlsh with SSL, the following command occasionally results in the exception below:
> {code}
> cqlsh --ssl -f kv.cql
> {code}
> {code}
> ERROR [SharedPool-Worker-2] 2016-05-11 12:41:03,583 Message.java:538 - Unexpected exception during request; channel = [id: 0xeb75e05d, /127.0.0.1:51083 => /127.0.0.1:9042]
> io.netty.handler.codec.DecoderException: javax.net.ssl.SSLException: bad record MAC
>         at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:280) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:149) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:787) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.channel.epoll.EpollSocketChannel$EpollSocketUnsafe.epollInReady(EpollSocketChannel.java:722) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:326) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:264) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]
> Caused by: javax.net.ssl.SSLException: bad record MAC
>         at sun.security.ssl.Alerts.getSSLException(Alerts.java:208) ~[na:1.8.0_91]
>         at sun.security.ssl.SSLEngineImpl.fatal(SSLEngineImpl.java:1728) ~[na:1.8.0_91]
>         at sun.security.ssl.SSLEngineImpl.readRecord(SSLEngineImpl.java:981) ~[na:1.8.0_91]
>         at sun.security.ssl.SSLEngineImpl.readNetRecord(SSLEngineImpl.java:907) ~[na:1.8.0_91]
>         at sun.security.ssl.SSLEngineImpl.unwrap(SSLEngineImpl.java:781) ~[na:1.8.0_91]
>         at javax.net.ssl.SSLEngine.unwrap(SSLEngine.java:624) ~[na:1.8.0_91]
>         at io.netty.handler.ssl.SslHandler.unwrap(SslHandler.java:982) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.handler.ssl.SslHandler.unwrap(SslHandler.java:908) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.handler.ssl.SslHandler.decode(SslHandler.java:854) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:249) ~[netty-all-4.0.23.Final.jar:4.0.23.Final]
>         ... 10 common frames omitted
> Caused by: javax.crypto.BadPaddingException: bad record MAC
>         at sun.security.ssl.InputRecord.decrypt(InputRecord.java:219) ~[na:1.8.0_91]
>         at sun.security.ssl.EngineInputRecord.decrypt(EngineInputRecord.java:177) ~[na:1.8.0_91]
>         at sun.security.ssl.SSLEngineImpl.readRecord(SSLEngineImpl.java:974) ~[na:1.8.0_91]
>         ... 17 common frames omitted
> {code}
> where
> {code}
> cat kv.cql 
> create keyspace if not exists cvs_copy_ks with replication = {'class': 'SimpleStrategy', 'replication_factor':1};
> create table if not exists cvs_copy_ks.kv (key int primary key, value text);
> truncate cvs_copy_ks.kv;
> copy cvs_copy_ks.kv (key, value) from 'kv.csv' with header='true';
> select * from cvs_copy_ks.kv;
> drop keyspace cvs_copy_ks;
> stefi@cuoricina:~/git/cstar/cassandra$ cat kv.c
> kv.cql  kv.csv  
> cat kv.csv 
> key,value
> 1,'a'
> 2,'b'
> 3,'c'
> {code}
> The COPY FROM succeeds, however the following select does not. 
> The easiest way to reproduce this is to restart the Cassandra process, it seems to happen in preference after a restart.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)