Uploaded image for project: 'Cassandra'
  1. Cassandra
  2. CASSANDRA-6181

Replaying a commit led to java.lang.StackOverflowError and node crash

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Urgent
    • Resolution: Fixed
    • 1.2.12, 2.0.2
    • None
    • None
    • 1.2.8 & 1.2.10 - ubuntu 12.04

    • Critical

    Description

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

       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)
      

      Attachments

        1. 6181.txt
          14 kB
          Sylvain Lebresne

        Issue Links

          Activity

            People

              slebresne Sylvain Lebresne
              jdamick Jeffrey Damick
              Sylvain Lebresne
              Fabien Rousseau
              Votes:
              1 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: