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

Marcus Eriksson commented on CASSANDRA-7262:
--------------------------------------------

I'm a bit worried about callers expecting the session to be actually closed 
after calling closeSession, with this it might not be (though, I looked and 
couldn't really find a case where they do). Making closeSession synchronized as 
well would fix that, wdyt?

> 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
>            Assignee: Joshua McKenzie
>            Priority: Minor
>             Fix For: 2.0.9
>
>         Attachments: 7262_v1.txt, 7262_v2.txt, 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.
> Edit: The following Reference counter assertion failures followed the 1st on 
> a different file and have a different stack trace:
> ERROR [CompactionExecutor:382] 2014-05-17 01:17:53,157 CassandraDaemon.java 
> (line 198) Exception in thread Thread[CompactionExecutor:382,1,main]
> java.lang.AssertionError: Reference counter -1 for 
> /mnt/ssd1/cassandra/data/ldn_production/historical_accounts/ldn_production-historical_accounts-jb-83888-Data.db
>    at 
> org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1107)
>    at 
> org.apache.cassandra.io.sstable.SSTableReader.releaseReferences(SSTableReader.java:1429)
>    at 
> org.apache.cassandra.db.compaction.CompactionController.close(CompactionController.java:207)
>    at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:220)
>    at 
> org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
>    at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>    at 
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
>    at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
>    at 
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:197)
>    ...



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

Reply via email to