[ 
https://issues.apache.org/jira/browse/CASSANDRA-6181?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sylvain Lebresne updated CASSANDRA-6181:
----------------------------------------

    Attachment: 6181.txt

I unfortunately haven't been to reproduce with the commit log from Jeffrey.

That being said, looking at the stacktrace more closely, I don't think that 
this is an infinite loop. Rather, in some insertion cases, we have to iterate 
over all (or a large part) of the range tombstones and that is currently done 
recursively so this can blow up the stack. The blow-up does reproduce rather 
easily in a unit test (with 3K range tombstone, which is not small, but not all 
that much). I though we would be unlikely to run into that case with the way 
range tombstones are used in practice, but I suppose that's still possible if 
you have multiple clustering columns so maybe that's just that.

Anyway, I don't really another fix than to rewrite the logic non-recursively.  
Attaching a patch for this. This is probably a little bit more involved that 
what I'd like to push in 1.2 at this point, but at same I don't think there is 
any simpler way to fix this. On the bright side, RangeTombstoneList is 
relatively well covered by unit tests.

[~exabytes18], [~jdamick]: If you guys could check that the attached patch does 
fix this for you, that would be awesome.


> Replaying a commit led to java.lang.StackOverflowError and node crash
> ---------------------------------------------------------------------
>
>                 Key: CASSANDRA-6181
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6181
>             Project: Cassandra
>          Issue Type: Bug
>         Environment: 1.2.8 & 1.2.10 - ubuntu 12.04
>            Reporter: Jeffrey Damick
>            Assignee: Sylvain Lebresne
>            Priority: Critical
>             Fix For: 1.2.12
>
>         Attachments: 6181.txt
>
>
> 2 of our nodes died after attempting to replay a commit.  I can attach the 
> commit log file if that helps.
> It was occurring on 1.2.8, after several failed attempts to start, we 
> attempted startup with 1.2.10.  This also yielded the same issue (below).  
> The only resolution was to physically move the commit log file out of the way 
> and then the nodes were able to start...  
> The replication factor was 3 so I'm hoping there was no data loss...
> {code}
>  INFO [main] 2013-10-11 14:50:35,891 CommitLogReplayer.java (line 119) 
> Replaying /ebs/cassandra/commitlog/CommitLog-2-1377542389560.log
> ERROR [MutationStage:18] 2013-10-11 14:50:37,387 CassandraDaemon.java (line 
> 191) Exception in thread Thread[MutationStage:18,5,main]
> java.lang.StackOverflowError
>         at 
> org.apache.cassandra.db.marshal.TimeUUIDType.compareTimestampBytes(TimeUUIDType.java:68)
>         at 
> org.apache.cassandra.db.marshal.TimeUUIDType.compare(TimeUUIDType.java:57)
>         at 
> org.apache.cassandra.db.marshal.TimeUUIDType.compare(TimeUUIDType.java:29)
>         at 
> org.apache.cassandra.db.marshal.AbstractType.compareCollectionMembers(AbstractType.java:229)
>         at 
> org.apache.cassandra.db.marshal.AbstractCompositeType.compare(AbstractCompositeType.java:81)
>         at 
> org.apache.cassandra.db.marshal.AbstractCompositeType.compare(AbstractCompositeType.java:31)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.insertAfter(RangeTombstoneList.java:439)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:405)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.weakInsertFrom(RangeTombstoneList.java:472)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.insertAfter(RangeTombstoneList.java:456)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:405)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.weakInsertFrom(RangeTombstoneList.java:472)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.insertAfter(RangeTombstoneList.java:456)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:405)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.weakInsertFrom(RangeTombstoneList.java:472)
> .... etc.... over and over until ....
>         at 
> org.apache.cassandra.db.RangeTombstoneList.weakInsertFrom(RangeTombstoneList.java:472)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.insertAfter(RangeTombstoneList.java:456)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:405)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:144)
>         at 
> org.apache.cassandra.db.RangeTombstoneList.addAll(RangeTombstoneList.java:186)
>         at org.apache.cassandra.db.DeletionInfo.add(DeletionInfo.java:180)
>         at 
> org.apache.cassandra.db.AtomicSortedColumns.addAllWithSizeDelta(AtomicSortedColumns.java:197)
>         at 
> org.apache.cassandra.db.AbstractColumnContainer.addAllWithSizeDelta(AbstractColumnContainer.java:99)
>         at org.apache.cassandra.db.Memtable.resolve(Memtable.java:207)
>         at org.apache.cassandra.db.Memtable.put(Memtable.java:170)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:745)
>         at org.apache.cassandra.db.Table.apply(Table.java:388)
>         at org.apache.cassandra.db.Table.apply(Table.java:353)
>         at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer$1.runMayThrow(CommitLogReplayer.java:258)
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>         at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:166)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:724)
> {code}



--
This message was sent by Atlassian JIRA
(v6.1#6144)

Reply via email to