You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Joshua McKenzie (JIRA)" <ji...@apache.org> on 2014/05/19 17:19:38 UTC

[jira] [Commented] (CASSANDRA-7262) During streaming: java.lang.AssertionError: Reference counter -1

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

Joshua McKenzie commented on CASSANDRA-7262:
--------------------------------------------

I ran into this while working on CASSANDRA-3569 and included the fix in the patch for that ticket.  We had no protection against multiple attempts to release a reference from within the StreamTransferTask.

{code:title=fix}
@@ -75,11 +77,15 @@ public class StreamTransferTask extends StreamTask
 
     public void abort()
     {
-        for (OutgoingFileMessage file : files.values())
+        // Prevent releasing reference multiple times
+        if (aborted.compareAndSet(false, true))
         {
-            file.sstable.releaseReference();
+            for (OutgoingFileMessage file : files.values())
+            {
+                file.sstable.releaseReference();
+            }
+            timeoutExecutor.shutdownNow();
         }
-        timeoutExecutor.shutdownNow();
     }
{code}

> During streaming: java.lang.AssertionError: Reference counter -1
> ----------------------------------------------------------------
>
>                 Key: CASSANDRA-7262
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7262
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Cassandra 2.07, x86-64 Ubuntu 12.04.4, Oracle java 1.7.0_45
>            Reporter: Duncan Sands
>            Priority: Minor
>         Attachments: system.log.gz
>
>
> Got this assertion failure this weekend during repair:
> ERROR [STREAM-IN-/192.168.21.14] 2014-05-17 01:17:52,332 StreamSession.java (line 420) [Stream #3a3ac8a2-dd50-11e3-b3c1-6bf6dccd6457] Streaming error occurred
> java.lang.RuntimeException: Outgoing stream handler has been closed
>         at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:170)
>         at org.apache.cassandra.streaming.StreamSession.receive(StreamSession.java:483)
>         at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:372)
>         at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:289)
>         at java.lang.Thread.run(Thread.java:744)
> ERROR [STREAM-IN-/192.168.21.14] 2014-05-17 01:17:52,350 CassandraDaemon.java (line 198) Exception in thread Thread[STREAM-IN-/192.168.21.14,5,RMI Runtime]
> java.lang.AssertionError: Reference counter -1 for /mnt/ssd1/cassandra/data/ldn_production/historical_accounts/ldn_production-historical_accounts-jb-79827-Data.db
>         at org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1107)
>         at org.apache.cassandra.streaming.StreamTransferTask.abort(StreamTransferTask.java:80)
>         at org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:322)
>         at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:425)
>         at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:300)
>         at java.lang.Thread.run(Thread.java:744)
> followed by a few more (the reference counter got down to -3).  Got the same kind of assertion failure on one other node (in a different data centre; there are 21 nodes altogether distributed over 4 data centres).
> I've attached the relevant part of the log.  It starts quite a bit before the assertion failure at the first exception on this node ("Cannot proceed on repair because a neighbor ... is dead"), and finishes a few hours afterwards when the node was restarted.



--
This message was sent by Atlassian JIRA
(v6.2#6252)