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

Stefania edited comment on CASSANDRA-11749 at 5/27/16 5:00 AM:
---------------------------------------------------------------

Thanks for the pointer [~norman]: it's not a race amongst multiple threads but 
something similar. There is an out-of-band connection that the clients create 
so that the server can push notifications for things like schema changes and so 
forth. The forked copy processes are inheriting the socket handle from the 
parent cqlsh process. Releasing these socket handles in the forked processes 
fixes it _provided it is done as soon as possible_.

[~aholmber]: We can fix this by calling {{cluster.shutdown()}} as soon as the 
child processes start running but, I think we could still have a small window 
if a notification is received before the sockets have been closed. I'm not sure 
I understood the driver debug messages entirely, I've attached them in this 
file here [^driver_debug.txt]. We can see {{\[control connection\] Waiting for 
schema agreement}} logged by the parent process and then the bad record MAC 
received by both parent and child process. Is there anything else that could be 
done to avoid this race totally?

The fix in cqlsh copyutil is 
[here|https://github.com/stef1927/cassandra/commit/3fb2e0c286f5f4ae42e1e877d7635604b1896a09].


was (Author: stefania):
Thanks for the pointer [~norman]: it's not a race amongst multiple threads but 
something similar. There is an out-of-band connection that the clients create 
so that the server can push notifications for things like schema changes and so 
forth. The forked copy processes are inheriting the socket handle from the 
parent cqlsh process. Releasing these socket handles in the forked processes 
fixes it _provided it is done as soon as possible_.

[~aholmber]: I've fixed by calling {{cluster.shutdown()}} as soon as the child 
processes start running but, I think we could still have a small window if a 
notification is received before the sockets have been closed. I'm not sure I 
understood the driver debug messages entirely, I've attached them in this file 
here [^driver_debug.txt]. We can see {{\[control connection\] Waiting for 
schema agreement}} logged by the parent process and then the bad record MAC 
received by both parent and child process. Is there anything else that could be 
done to avoid this race totally?

The fix in cqlsh copyutil is 
[here|https://github.com/stef1927/cassandra/commit/3fb2e0c286f5f4ae42e1e877d7635604b1896a09].

> 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: driver_debug.txt, 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)

Reply via email to