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

Omid Aladini commented on CASSANDRA-4411:
-----------------------------------------

I can confirm that the problem is still there. I offline-scrubbed using 1.1.3 
(sstables were generated by 1.1.0) , but the scrubber did not report any 
out-of-order sstables, but sent some sstables back to L0. On compaction though, 
I get the exception:

{quote}
2012-08-08_18:15:41.85260 java.lang.RuntimeException: Last written key 
DecoratedKey(135076574692378869287086649376333921820, SOME_KEY_1) >= current 
key DecoratedKey(135076574692378869287086649376333921820, SOME_KEY_1) writing 
into /var/lib/cassandra/abcd/data/KSP/CF1/KSP-CF1-tmp-he-178793-Data.db
2012-08-08_18:15:41.85303       at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:134)
2012-08-08_18:15:41.85314       at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:153)
2012-08-08_18:15:41.85326       at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159)
2012-08-08_18:15:41.85338       at 
org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
2012-08-08_18:15:41.85351       at 
org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:288)
2012-08-08_18:15:41.85364       at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
2012-08-08_18:15:41.85375       at 
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-08-08_18:15:41.85385       at 
java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
2012-08-08_18:15:41.85395       at java.util.concurrent.FutureTask.run(Unknown 
Source)
2012-08-08_18:15:41.85403       at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
2012-08-08_18:15:41.85414       at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
2012-08-08_18:15:41.85424       at java.lang.Thread.run(Unknown Source)
{quote}


                
> 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