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

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

I ran the scrub in online mode.

I just took down a node and am now running it in offline mode.  Will report 
back.

BTW, the default "sstablescrub" does not respect the memory limits set in 
cassandra.in.sh, so it failed for me with:
{code}
Exception in thread "main" java.lang.OutOfMemoryError: Java heap space
        at sun.security.provider.DigestBase.engineDigest(DigestBase.java:146)
        at 
java.security.MessageDigest$Delegate.engineDigest(MessageDigest.java:546)
        at java.security.MessageDigest.digest(MessageDigest.java:323)
        at org.apache.cassandra.utils.FBUtilities.hash(FBUtilities.java:229)
        at 
org.apache.cassandra.utils.FBUtilities.hashToBigInteger(FBUtilities.java:213)
        at 
org.apache.cassandra.dht.RandomPartitioner.getToken(RandomPartitioner.java:154)
        at 
org.apache.cassandra.dht.RandomPartitioner.decorateKey(RandomPartitioner.java:47)
        at 
org.apache.cassandra.cache.AutoSavingCache.readSaved(AutoSavingCache.java:118)
        at 
org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:230)
        at 
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:341)
        at 
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:313)
        at org.apache.cassandra.db.Table.initCf(Table.java:371)
        at org.apache.cassandra.db.Table.<init>(Table.java:304)
        at org.apache.cassandra.db.Table.open(Table.java:119)
        at org.apache.cassandra.db.Table.openWithoutSSTables(Table.java:102)
        at 
org.apache.cassandra.tools.StandaloneScrubber.main(StandaloneScrubber.java:65)
{code}

I edited it to update the hardocded limit of 256MB to a more reasonable value 
(the same as my cassandra.in.sh) to allow it to run without crashing.


                
> 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