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

Rick Branson commented on CASSANDRA-6285:
-----------------------------------------

I did some more digging around on our cluster that was running 2.0.6 when it 
saw the corruption: it took anywhere from a few hours to 48 hours for the first 
compaction with the out of order key exception to throw. These nodes are 
receiving thousands of writes per second, so it's not going to be trivially 
reproducible. We've been running one of the nodes with 2.0.8-tenative + 
enable_reallocate_buffers.txt and will report back once we've reached 72 hours 
and are comfortable rolling this out wide to our own clusters.

> 2.0 HSHA server introduces corrupt data
> ---------------------------------------
>
>                 Key: CASSANDRA-6285
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>            Reporter: David Sauer
>            Assignee: Pavel Yaskevich
>            Priority: Critical
>             Fix For: 2.0.8
>
>         Attachments: 6285_testnotes1.txt, 
> CASSANDRA-6285-disruptor-heap.patch, cassandra-attack-src.zip, 
> compaction_test.py, disruptor-high-cpu.patch, 
> disruptor-memory-corruption.patch, enable_reallocate_buffers.txt
>
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
> java.lang.RuntimeException: Last written key 
> DecoratedKey(1326283851463420237, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
>  >= current key DecoratedKey(954210699457429663, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
>  writing into 
> /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
>       at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
>       at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
>       at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
>       at 
> org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
>       at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>       at 
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
>       at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
>       at 
> org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296)
>       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.run(FutureTask.java:262)
>       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)
> Moving back to STC worked to keep the compactions running.
> Especialy my own Table i would like to move to LCS.
> After a major compaction with STC the move to LCS fails with the same 
> Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to