You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Fabien Rousseau (JIRA)" <ji...@apache.org> on 2013/10/23 15:55:43 UTC

[jira] [Comment Edited] (CASSANDRA-6181) Replaying a commit led to java.lang.StackOverflowError and node crash

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

Fabien Rousseau edited comment on CASSANDRA-6181 at 10/23/13 1:54 PM:
----------------------------------------------------------------------

After reviewing patch, here are some changes in RangeTombstone.java:

L143: '(pos >= 0 ? pos : -pos-1)' should probably be '(pos >= 0 ? pos+1 : -pos-1)'
if pos is positive, it means that start == ends[pos], and the interval should be inserted at (pos+1)


Below is a simple test to reproduce this behaviour:

{noformat}
    @Test
    public void overlappingPreviousEndEqualsStartTest1()
    {
        RangeTombstoneList l = new RangeTombstoneList(cmp, 0);
        // add a RangeTombstone, so, last insert is not in insertion order
        l.add(rt(11, 12, 2));
        l.add(rt(1, 4, 2));
        l.add(rt(4, 10, 5));

        assertEquals(2, l.search(b(3)).markedForDeleteAt);
        assertEquals(5, l.search(b(4)).markedForDeleteAt);
        assertEquals(5, l.search(b(8)).markedForDeleteAt);
        assertEquals(3, l.size());
    }
{noformat}

Some very minor changes in comments:
L200: should replace 'insertFrom' to 'addInternal'
L394: should the commented line be : setInternal(i, start, ends[i], markedAt, delTime) ? (it seems more in the spirit)
L422: TODO can be removed, because implemented a few lines above

Otherwise, LGTM



was (Author: frousseau):
After reviewing patch, here are some changes in RangeTombstone.java:

L143: '(pos >= 0 ? pos : -pos-1)' should probably be '(pos >= 0 ? pos+1 : -pos-1)'
if pos is positive, it means that start == ends[pos], and the interval should be inserted at (pos+1)

Below is a simple test to reproduce this behaviour:

    @Test
    public void overlappingPreviousEndEqualsStartTest1()
    {
        RangeTombstoneList l = new RangeTombstoneList(cmp, 0);
        // add a RangeTombstone, so, last insert is not in insertion order
        l.add(rt(11, 12, 2));
        l.add(rt(1, 4, 2));
        l.add(rt(4, 10, 5));

        assertEquals(2, l.search(b(3)).markedForDeleteAt);
        assertEquals(5, l.search(b(4)).markedForDeleteAt);
        assertEquals(5, l.search(b(8)).markedForDeleteAt);
        assertEquals(3, l.size());
    }


Some very minor changes in comments:
L200: should replace 'insertFrom' to 'addInternal'
L394: should the commented line be : setInternal(i, start, ends[i], markedAt, delTime) ? (it seems more in the spirit)
L422: TODO can be removed, because implemented a few lines above

Otherwise, LGTM

> 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)