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

Mina Naguib commented on CASSANDRA-4411:
----------------------------------------


Unfortunately the problem did not go away after scrubbing.

I scrubbed 2 of the problematic nodes.  Immediately after the scrub (5 hours) 
finished, a compaction was attempted and again failed:

{code}
ERROR [CompactionExecutor:47] 2012-07-23 19:48:52,500 
AbstractCassandraDaemon.java (line 134) Exception in thread 
Thread[CompactionExecutor:47,1,main]
java.lang.AssertionError
        at 
org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
        at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
        at 
org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
        at 
org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
        at 
org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
        at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
        at 
org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
        at 
org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
        at java.util.concurrent.FutureTask.run(FutureTask.java:138)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
{code}

I've verified (using ls and the timestamps of *-Data.db sstables) that there 
are no old sstables and all of the sstables in the CF are the ones generated 
during the 5 hours of scrubbing.

I've also stopped and restarted one of these nodes, and again shortly after 
restart the compaction failed with a different stack trace:

{code}
java.lang.RuntimeException: Last written key 
DecoratedKey(225595347341523546110318866012608496, 
64313635626665302d333764372d313165302d393933622d303032366239333763386531) >= 
current key DecoratedKey(221078382620949716286900834756484795, 
37303538643361662d616362662d343030312d313565382d633662303030303030336131) 
writing into /var/lib/cassandra/data/MYKS/MYCF/MYKS-MYCF-tmp-hd-520277-Data.db
        at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:134)
        at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:153)
        at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159)
        at 
org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
        at 
org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
        at java.util.concurrent.FutureTask.run(FutureTask.java:138)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
{code}

Is the fix for this ticket contained in some other code beyond the 2 posted 
patches (4411.txt, 4411-followup.txt) ?  That's what I'm running with on top of 
1.1.2.

                
> Assertion with LCS compaction
> -----------------------------
>
>                 Key: CASSANDRA-4411
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.1.2
>            Reporter: Anton Winter
>            Assignee: Sylvain Lebresne
>             Fix For: 1.1.3
>
>         Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
> 4411.txt, assertion-w-more-debugging-info-omid.log, 
> assertion.moreinfo.system.log, system.log
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of 
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2 
> upgrade until sstablescrub shows no errors.  The exceptions described in 
> CASSANDRA-4321 do not occur as frequently now but the integrity check still 
> throws exceptions on a number of nodes.  Once those exceptions occur 
> compactionstats shows a large number of pending tasks with no progression 
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
>         at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
>         at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
>         at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
>         at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
>         at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
>         at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
>         at 
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         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:1110)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>         at java.lang.Thread.run(Thread.java:636)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to