Details
-
Bug
-
Status: Resolved
-
Low
-
Resolution: Fixed
-
None
-
Cassandra 2.07, x86-64 Ubuntu 12.04.4, Oracle java 1.7.0_45
-
Low
Description
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)
...
Attachments
Attachments
Issue Links
- duplicates
-
CASSANDRA-3569 Failure detector downs should not break streams
- Resolved
- relates to
-
CASSANDRA-7705 Safer Resource Management
- Resolved