cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Alexander Sterligov (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data
Date Tue, 21 Oct 2014 18:42:41 GMT

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

Alexander Sterligov edited comment on CASSANDRA-6285 at 10/21/14 6:42 PM:
--------------------------------------------------------------------------

It looks like this is not fixed in 2.1.0. We have cassandra under heavy load through binary
interface and only OpsCenter by thrift. OpsCenter rollups are corrupted in about an hour after
scrub.

{quote}
ERROR [CompactionExecutor:71] 2014-10-21 22:16:39,950 CassandraDaemon.java:166 - Exception
in thread Thread[CompactionExecutor:71,1,main]
java.lang.RuntimeException: Last written key DecoratedKey(-7581200918995348250, 39352e3130382e3234322e32302d6973732d73686172645f696e666f2d676574426c6f6f6d46696c74657246616c7365506f73697469766573)
>= current key DecoratedKey(-8301289422298317140, 800100010000000c62617463685f6d75746174650006d04a0d00010b0d0000000100000025)
writing into /ssd/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-ka-9128-Data.db
        at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:172)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:196) ~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:110)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:177)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:74)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:235)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_51]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_51]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
~[na:1.7.0_51]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[na:1.7.0_51]
        at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
{quote}

We'll try to switch to sync and see what will happen.

Is it possible that streaming hangs because of that exception? Is it possible that this exception
affect minor compactions of other keyspaces?


was (Author: sterligovak):
It looks like this is not fixed in 2.1.0. We have cassandra under heavy load through binary
interface and only OpsCenter by thrift. OpsCenter rollups are corrupted in about an hour minutes
after scrub.

{quote}
ERROR [CompactionExecutor:71] 2014-10-21 22:16:39,950 CassandraDaemon.java:166 - Exception
in thread Thread[CompactionExecutor:71,1,main]
java.lang.RuntimeException: Last written key DecoratedKey(-7581200918995348250, 39352e3130382e3234322e32302d6973732d73686172645f696e666f2d676574426c6f6f6d46696c74657246616c7365506f73697469766573)
>= current key DecoratedKey(-8301289422298317140, 800100010000000c62617463685f6d75746174650006d04a0d00010b0d0000000100000025)
writing into /ssd/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-ka-9128-Data.db
        at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:172)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:196) ~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:110)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:177)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:74)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:235)
~[apache-cassandra-2.1.0.jar:2.1.0]
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_51]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_51]
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
~[na:1.7.0_51]
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[na:1.7.0_51]
        at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
{quote}

We'll try to switch to sync and see what will happen.

Is it possible that streaming hangs because of that exception? Is it possible that this exception
affect minor compactions of other keyspaces?

> 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.3.4#6332)

Mime
View raw message