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

Sylvain Lebresne commented on CASSANDRA-4321:
---------------------------------------------

bq. which leads me to suspect if it's due to the "Range" (vs. Bounds) used in 
LeveledCompactionStrategy::getScanners. Any ideas?

No, that Range is correct because this correspond to repair ranges and is 
correctly interpreted. And in fact, when doing compaction that range is 
actually null so that cannot be the problem.


So Anton sent me some sstables that triggered an out-of-order exception when 
compacting them. What I did with that is:
1) apply the last version of the v5 patch on this issue on top of the current 
git cassandra-1.1 branch (using the release of CASSANDRA 1.1.1 would probably 
work too because I don't think there is any other related fix since 1.1.1 that 
went into the git branch but anyway, that's what I used).
2) I ran the offline scrub *while the node was stopped*. I insist on that last 
part because that having the node run during the offline scrub would mess 
things up. I'd actually like to make the offline scrub check if the node is 
running and refuse to work in that case but I'm not sure of what is the best 
way to do that.
3) I restarted the node once the scrub was done

I was then able to compact the node fully (i.e, I ran compaction until there 
was nothing more to compact) and this without hitting any more error.

Was I lucky? Are you guys able to reproduce those steps and still get more 
errors?
                
> stackoverflow building interval tree & possible sstable corruptions
> -------------------------------------------------------------------
>
>                 Key: CASSANDRA-4321
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.1.1
>            Reporter: Anton Winter
>            Assignee: Sylvain Lebresne
>             Fix For: 1.1.2
>
>         Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v5.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows-v5.txt, 
> 0003-Create-standalone-scrub-v5.txt, ooyala-hastur-stacktrace.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
>         at java.util.Arrays.mergeSort(Arrays.java:1157)
>         at java.util.Arrays.sort(Arrays.java:1092)
>         at java.util.Collections.sort(Collections.java:134)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:49)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
>         at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
>         at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
>         at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
>         at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
>         at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:45)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalTree.<init>(IntervalTree.java:39)
>         at 
> org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTracker.java:560)
>         at 
> org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:617)
>         at org.apache.cassandra.db.DataTracker.replace(DataTracker.java:320)
>         at 
> org.apache.cassandra.db.DataTracker.addInitialSSTables(DataTracker.java:259)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:234)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:331)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:309)
>         at org.apache.cassandra.db.Table.initCf(Table.java:367)
>         at org.apache.cassandra.db.Table.<init>(Table.java:299)
>         at org.apache.cassandra.db.Table.open(Table.java:114)
>         at org.apache.cassandra.db.Table.open(Table.java:97)
>         at org.apache.cassandra.db.Table$2.apply(Table.java:574)
>         at org.apache.cassandra.db.Table$2.apply(Table.java:571)
>         at com.google.common.collect.Iterators$8.next(Iterators.java:751)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.all(ColumnFamilyStore.java:1625)
>         at 
> org.apache.cassandra.db.MeteredFlusher.countFlushingBytes(MeteredFlusher.java:118)
>         at org.apache.cassandra.db.MeteredFlusher.run(MeteredFlusher.java:45)
>         at 
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:79)
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>         at 
> java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:351)
>         at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:178)
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:165)
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:267)
>         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)
> DEBUG 11:37:51,052 Initializing ksU.cfS
> And then finally fails with the following:
> DEBUG 11:49:03,752 Creating IntervalNode from 
> [Interval(DecoratedKey(104860264640932324846851821824650966808, 
> 4fcc88eb0218216164673394), 
> DecoratedKey(93975306025956344620001177071135439009, 
> 4fc8fb042c98458c7a58bc3b)), 
> Interval(DecoratedKey(104860264640932324846851821824650966808, 
> 4fcc88eb0218216164673394), 
> DecoratedKey(93975306025956344620001177071135439009, 
> 4fc8fb042c98458c7a58bc3b)), 
> Interval(DecoratedKey(104860264640932324846851821824650966808, 
> 4fcc88eb0218216164673394), 
> DecoratedKey(93975306025956344620001177071135439009, 
> 4fc8fb042c98458c7a58bc3b)), 
> Interval(DecoratedKey(104860264640932324846851821824650966808, 
> 4fcc88eb0218216164673394), 
> DecoratedKey(93975306025956344620001177071135439009, 
> 4fc8fb042c98458c7a58bc3b)), 
> Interval(DecoratedKey(104860264640932324846851821824650966808, 
> 4fcc88eb0218216164673394), 
> DecoratedKey(93975306025956344620001177071135439009, 
> 4fc8fb042c98458c7a58bc3b)), 
> Interval(DecoratedKey(104860264640932324846851821824650966808, 
> 4fcc88eb0218216164673394), 
> DecoratedKey(93975306025956344620001177071135439009, 
> 4fc8fb042c98458c7a58bc3b))]
> java.lang.reflect.InvocationTargetException
> DEBUG 11:49:03,753 Configured datacenter replicas are dc1:2, dc2:2, dc3:2, 
> dc4:2, dc5:0, dc6:2, dc7:0, dc8:0, dc9:2
>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>         at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:616)
>         at 
> org.apache.commons.daemon.support.DaemonLoader.load(DaemonLoader.java:160)
> Caused by: java.lang.StackOverflowError
>         at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
>         at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
>         at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
>         at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
>         at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:45)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:64)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.<init>(IntervalNode.java:62)
>         at 
> org.apache.cassandra.utils.IntervalTree.IntervalTree.<init>(IntervalTree.java:39)
>         at 
> org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTracker.java:560)
>         at 
> org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:617)
>         at org.apache.cassandra.db.DataTracker.replace(DataTracker.java:320)
>         at 
> org.apache.cassandra.db.DataTracker.addInitialSSTables(DataTracker.java:259)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:234)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:331)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:309)
>         at org.apache.cassandra.db.Table.initCf(Table.java:367)
>         at org.apache.cassandra.db.Table.<init>(Table.java:299)
>         at org.apache.cassandra.db.Table.open(Table.java:114)
>         at org.apache.cassandra.db.Table.open(Table.java:97)
>         at 
> org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:204)
>         at 
> org.apache.cassandra.service.AbstractCassandraDaemon.init(AbstractCassandraDaemon.java:254)
>         ... 5 more
> Cannot load daemon
> Service exit with a return value of 3
> Running with assertions enabled allows me to start the instance but when 
> doing so I get errors such as:
> ERROR 01:22:22,753 Exception in thread 
> Thread[SSTableBatchOpen:2,5,main]java.lang.AssertionError: SSTable first key 
> DecoratedKey(100294972947100949193477090306072672386, 
> 4fcf051ef5067d7f17d9fc35) > last key 
> DecoratedKey(90250429663386465697464050082134975058, 4fce996e3c1eed8c4b17dd66)
> at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:412)
> at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:187)
> at org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:225)
> 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)
> and:
> ERROR 01:27:58,946 Exception in thread Thread[CompactionExecutor:9,1,main]
> java.lang.AssertionError: Last written key 
> DecoratedKey(81958437188197992567937826278457419048, 
> 4fa1aebad23f81e4321d344d) >= current key 
> DecoratedKey(64546479828744423263742604083767363606, 
> 4fcafc0f19f6a8092d4d4f94) writing into 
> /var/lib/XX/data/cassandra/ks1/cf1/ks1-cf1-tmp-hd-657317-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: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)
> Just like the initial errors compactions appear to stop occurring after this 
> point.  
> Given the above this looks like sstables are getting corrupted.  By 
> restarting nodes I am able to identify several hundred sstables exhibiting 
> the same problem and this appears to be growing.
> I have tried scrubbing those affected nodes but the problem continues to 
> occur.  If this is due to sstable corruptions is there another way of 
> validating sstables for correctness?  Given that it has spread to various 
> servers in other DC's it looks like this is directly related to the 1.1.1 
> upgrade recently performed on the ring.

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