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

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

While the root cause of the reported issue is CASSANDRA-11840, as a consequence 
of that the stream session failed while an sstable was being transferred, 
making its reference be released by {{StreamTransferTask.abort()}} -> 
{{OutgoingFileMessage.complete()}} before the transfer was complete, 
potentially causing the subsequent {{Memory was freed}} error if the sstable is 
garbage collected.

The ideal solution is CASSANDRA-11956 (to interrupt the stream transfer task 
right away), but while that is not in place we should release the sstable 
reference only after the ongoing transfer is finished (even if the session is 
aborted). So the idea of the patch is to set a {{transferring}} flag on 
{{OutgoingFileMessage}} while the file is being transferred, and if the session 
is failed before that the reference is not released, only after the 
{{transferring}} flag is unset at the end of the file transfer. I added a unit 
test that verifies that an sstable is not unreferenced if it's being 
transferred and the stream session fails, only after the transfer is finished.

I also added a ninja fix to cache the {{FileMessageHeader}} size to avoid 
paying a high cost of recalculation of the size of large compressed files after 
the file is transferred on 
[StreamSession.fileSent(FileMessageHeader)|https://github.com/apache/cassandra/blob/9c8ee4c73f4e4e8d5b7693d34a0d6d6397418e90/src/java/org/apache/cassandra/streaming/StreamSession.java#L574]

Patch and tests available below:
||2.1||2.2||3.0||3.7||trunk||
|[branch|https://github.com/apache/cassandra/compare/cassandra-2.1...pauloricardomg:2.1-11345]|[branch|https://github.com/apache/cassandra/compare/cassandra-2.2...pauloricardomg:2.2-11345]|[branch|https://github.com/apache/cassandra/compare/cassandra-3.0...pauloricardomg:3.0-11345]|[branch|https://github.com/apache/cassandra/compare/cassandra-3.7...pauloricardomg:3.7-11345]|[branch|https://github.com/apache/cassandra/compare/trunk...pauloricardomg:trunk-11345]|
|[testall|http://cassci.datastax.com/view/Dev/view/paulomotta/job/pauloricardomg-2.1-11345-testall/lastCompletedBuild/testReport/]|[testall|http://cassci.datastax.com/view/Dev/view/paulomotta/job/pauloricardomg-2.2-11345-testall/lastCompletedBuild/testReport/]|[testall|http://cassci.datastax.com/view/Dev/view/paulomotta/job/pauloricardomg-3.0-11345-testall/lastCompletedBuild/testReport/]|[testall|http://cassci.datastax.com/view/Dev/view/paulomotta/job/pauloricardomg-3.7-11345-testall/lastCompletedBuild/testReport/]|[testall|http://cassci.datastax.com/view/Dev/view/paulomotta/job/pauloricardomg-trunk-11345-testall/lastCompletedBuild/testReport/]|
|[dtest|http://cassci.datastax.com/view/Dev/view/paulomotta/job/pauloricardomg-2.1-11345-dtest/lastCompletedBuild/testReport/]|[dtest|http://cassci.datastax.com/view/Dev/view/paulomotta/job/pauloricardomg-2.2-11345-dtest/lastCompletedBuild/testReport/]|[dtest|http://cassci.datastax.com/view/Dev/view/paulomotta/job/pauloricardomg-3.0-11345-dtest/lastCompletedBuild/testReport/]|[dtest|http://cassci.datastax.com/view/Dev/view/paulomotta/job/pauloricardomg-3.7-11345-dtest/lastCompletedBuild/testReport/]|[dtest|http://cassci.datastax.com/view/Dev/view/paulomotta/job/pauloricardomg-trunk-11345-dtest/lastCompletedBuild/testReport/]|

Would you mind reviewing [~yukim]? Thanks in advance!

commit info: minor conflicts all the way up to 3.7 that merges cleanly to trunk

> 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