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

Paulo Motta commented on CASSANDRA-11345:
-----------------------------------------

[~vineus] What compaction strategy are you using? what is the average sstable 
size in the sending node?

I suspect the following race is happening:
- Node A sends SSTable X to bootstrapping node B
- SSTable X is obsoleted by compaction during streaming
- Node A finishes sending SSTable X to B and starts [calculating header 
size|https://github.com/apache/cassandra/blob/21448c50891642f95097a9e5ed0a3802bd90a877/src/java/org/apache/cassandra/streaming/StreamSession.java#L563]
 to update metrics (this became more expensive for compressed tables after 
CASSANDRA-10680)
- Meanwhile, node B finishes receiving SSTable X and sends "received" message 
to node A
- Node A receives "received" message for SSTable X and [releases its 
reference|https://github.com/apache/cassandra/blob/21448c50891642f95097a9e5ed0a3802bd90a877/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java#L99],
 causing it to be garbage collected since it was obsoleted
- Since {{CompressionMetadata}} was released, [header size 
calculation|https://github.com/apache/cassandra/blob/21448c50891642f95097a9e5ed0a3802bd90a877/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java#L133]
 fails with {{Memory was freed}}

Do you think this is plausible [~yukim] ? 

Simple solution would be to cache header size on first calculation, but we need 
to confirm first this is really what is happening because it could be hiding 
some other issue.

[~vineus] Can you [set log 
level|https://docs.datastax.com/en/cassandra/2.1/cassandra/tools/toolsSetLogLev.html]
 of package {{org.apache.cassandra.streaming}} on sending on receiving nodes 
and attach system logs after error happens again?

> Assertion Errors "Memory was freed" during streaming
> ----------------------------------------------------
>
>                 Key: CASSANDRA-11345
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11345
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Streaming and Messaging
>            Reporter: Jean-Francois Gosselin
>            Assignee: Paulo Motta
>
> We encountered the following AssertionError (twice on the same node) during a 
> repair :
> On node /172.16.63.41
> {noformat}
> INFO  [STREAM-IN-/10.174.216.160] 2016-03-09 02:38:13,900 
> StreamResultFuture.java:180 - [Stream #f6980580-e55f-11e5-8f08-ef9e099ce99e] 
> Session with /10.174.216.160 is complete                                      
>                                                                   
> WARN  [STREAM-IN-/10.174.216.160] 2016-03-09 02:38:13,900 
> StreamResultFuture.java:207 - [Stream #f6980580-e55f-11e5-8f08-ef9e099ce99e] 
> Stream failed                           
> ERROR [STREAM-OUT-/10.174.216.160] 2016-03-09 02:38:13,906 
> StreamSession.java:505 - [Stream #f6980580-e55f-11e5-8f08-ef9e099ce99e] 
> Streaming error occurred                    
> java.lang.AssertionError: Memory was freed                                    
>                                                                               
>                    
>         at 
> org.apache.cassandra.io.util.SafeMemory.checkBounds(SafeMemory.java:97) 
> ~[apache-cassandra-2.1.13.jar:2.1.13]                                         
>           
>         at org.apache.cassandra.io.util.Memory.getLong(Memory.java:249) 
> ~[apache-cassandra-2.1.13.jar:2.1.13]                                         
>                      
>         at 
> org.apache.cassandra.io.compress.CompressionMetadata.getTotalSizeForSections(CompressionMetadata.java:247)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]                
>         at 
> org.apache.cassandra.streaming.messages.FileMessageHeader.size(FileMessageHeader.java:112)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]                                
>         at 
> org.apache.cassandra.streaming.StreamSession.fileSent(StreamSession.java:546) 
> ~[apache-cassandra-2.1.13.jar:2.1.13]                                         
>     
>         at 
> org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:50)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]                      
>         at 
> org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:41)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]                      
>         at 
> org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:45)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]                                    
>         at 
> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:351)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]           
>         at 
> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:331)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]                   
>         at java.lang.Thread.run(Thread.java:745) [na:1.7.0_65]                
>                                                                               
>                    
> {noformat}     
> On node /10.174.216.160
>  
> {noformat}       
> ERROR [STREAM-OUT-/172.16.63.41] 2016-03-09 02:38:14,140 
> StreamSession.java:505 - [Stream #f6980580-e55f-11e5-8f08-ef9e099ce99e] 
> Streaming error occurred                          
> java.io.IOException: Connection reset by peer                                 
>                                                                               
>                
>         at sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.7.0_65] 
>                                                                               
>                
>         at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) 
> ~[na:1.7.0_65]                                                                
>                       
>         at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) 
> ~[na:1.7.0_65]                                                                
>                           
>         at sun.nio.ch.IOUtil.write(IOUtil.java:65) ~[na:1.7.0_65]             
>                                                                               
>                
>         at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:487) 
> ~[na:1.7.0_65]                                                                
>                    
>         at 
> org.apache.cassandra.io.util.DataOutputStreamAndChannel.write(DataOutputStreamAndChannel.java:48)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]                     
>         at 
> org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:44)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]                                
>         at 
> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:351)
>  [apache-cassandra-2.1.13.jar:2.1.13]        
>         at 
> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:323)
>  [apache-cassandra-2.1.13.jar:2.1.13]                
>         at java.lang.Thread.run(Thread.java:745) [na:1.7.0_65]                
>                                                                               
>                
> INFO  [STREAM-IN-/172.16.63.41] 2016-03-09 02:38:14,142 
> StreamResultFuture.java:180 - [Stream #f6980580-e55f-11e5-8f08-ef9e099ce99e] 
> Session with /172.16.63.41 is complete
> WARN  [STREAM-IN-/172.16.63.41] 2016-03-09 02:38:14,142 
> StreamResultFuture.java:207 - [Stream #f6980580-e55f-11e5-8f08-ef9e099ce99e] 
> Stream failed                         
> ERROR [STREAM-OUT-/172.16.63.41] 2016-03-09 02:38:14,143 
> StreamSession.java:505 - [Stream #f6980580-e55f-11e5-8f08-ef9e099ce99e] 
> Streaming error occurred                  
> java.io.IOException: Broken pipe                                              
>                                                                               
>                
>         at sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.7.0_65] 
>                                                                               
>                
>         at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) 
> ~[na:1.7.0_65]                                                                
>                       
>         at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) 
> ~[na:1.7.0_65]                                                                
>                           
>         at sun.nio.ch.IOUtil.write(IOUtil.java:65) ~[na:1.7.0_65]             
>                                                                               
>                
>         at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:487) 
> ~[na:1.7.0_65]                                                                
>                    
>         at 
> org.apache.cassandra.io.util.DataOutputStreamAndChannel.write(DataOutputStreamAndChannel.java:48)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]                     
>         at 
> org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:44)
>  ~[apache-cassandra-2.1.13.jar:2.1.13]                                
>         at 
> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:351)
>  [apache-cassandra-2.1.13.jar:2.1.13]        
>         at 
> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:331)
>  [apache-cassandra-2.1.13.jar:2.1.13]                
>         at java.lang.Thread.run(Thread.java:745) [na:1.7.0_65]     
> {noformat}



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

Reply via email to