[ https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14292451#comment-14292451 ]
Randy Fradin commented on CASSANDRA-6285: ----------------------------------------- I am getting this exception using Thrift HSHA in 2.1.0: {quote} INFO [CompactionExecutor:8] 2015-01-26 13:32:51,818 CompactionTask.java (line 138) Compacting [SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-2-Data.db'), SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-1-Data.db')] INFO [CompactionExecutor:8] 2015-01-26 13:32:51,890 ColumnFamilyStore.java (line 856) Enqueuing flush of compactions_in_progress: 212 (0%) on-heap, 20 (0%) off-heap INFO [MemtableFlushWriter:8] 2015-01-26 13:32:51,892 Memtable.java (line 326) Writing Memtable-compactions_in_progress@1155018639(0 serialized bytes, 1 ops, 0%/0% of on/off-heap limit) INFO [MemtableFlushWriter:8] 2015-01-26 13:32:51,896 Memtable.java (line 360) Completed flushing /tmp/cass_test/cassandra/TestCassandra/data/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/system-compactions_in_progress-ka-2-Data.db (42 bytes) for commitlog position ReplayPosition(segmentId=1422296630707, position=430226) ERROR [CompactionExecutor:8] 2015-01-26 13:32:51,906 CassandraDaemon.java (line 166) Exception in thread Thread[CompactionExecutor:8,1,RMI Runtime] java.lang.RuntimeException: Last written key DecoratedKey(131206587314004820534098544948237170809, 800100010000000c62617463685f6d7574617465000000) >= current key DecoratedKey(14775611966645399672119169777260659240, 726f776b65793030385f31343232323937313537353835) writing into /tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-tmp-ka-3-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_40] at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_40] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_40] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_40] at java.lang.Thread.run(Thread.java:724) [na:1.7.0_40] {quote} I don't think it's caused by CASSANDRA-8211, because it happens during the first compaction that takes place between the first 2 SSTables to get flushed from an initially empty column family. Also, I've only been able to reproduce it when using both *hsha* for the rpc server and *offheap_objects* for memtable allocation. If I switch either to sync or to offheap_buffers or heap_buffers then I cannot reproduce the problem. Also under the same circumstances I'm pretty sure I've seen incorrect data being returned to a client multiget_slice request before any SSTables had been flushed yet, so I presume this is corruption that happens before any flush/compaction takes place. nodetool scrub yielded these errors: {quote} INFO [CompactionExecutor:9] 2015-01-26 13:48:01,512 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-2-Data.db') (168780 bytes) INFO [CompactionExecutor:10] 2015-01-26 13:48:01,512 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-1-Data.db') (135024 bytes) WARN [CompactionExecutor:9] 2015-01-26 13:48:01,531 OutputHandler.java (line 52) Out of order row detected (DecoratedKey(14775611966645399672119169777260659240, 726f776b65793030385f31343232323937313537353835) found after DecoratedKey(131206587314004820534098544948237170809, 800100010000000c62617463685f6d7574617465000000)) WARN [CompactionExecutor:9] 2015-01-26 13:48:01,534 OutputHandler.java (line 57) Error reading row (stacktrace follows): java.lang.RuntimeException: Last written key DecoratedKey(131206587314004820534098544948237170809, 800100010000000c62617463685f6d7574617465000000) >= current key DecoratedKey(131206587314004820534098544948237170809, 800100010000000c62617463685f6d7574617465000000) writing into /tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-tmp-ka-4-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.io.sstable.SSTableRewriter.tryAppend(SSTableRewriter.java:141) ~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.Scrubber.scrub(Scrubber.java:186) ~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.CompactionManager.scrubOne(CompactionManager.java:592) [apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.CompactionManager.access$300(CompactionManager.java:100) [apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.CompactionManager$3.execute(CompactionManager.java:315) [apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:270) [apache-cassandra-2.1.0.jar:2.1.0] at java.util.concurrent.FutureTask.run(FutureTask.java:262) [na:1.7.0_40] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_40] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_40] at java.lang.Thread.run(Thread.java:724) [na:1.7.0_40] WARN [CompactionExecutor:9] 2015-01-26 13:48:01,534 OutputHandler.java (line 52) Row starting at position 25342 is unreadable; skipping to next WARN [CompactionExecutor:10] 2015-01-26 13:48:01,534 OutputHandler.java (line 52) Out of order row detected (DecoratedKey(29459452031265566667651334397450214244, 726f776b65793030355f31343232323936393033323837) found after DecoratedKey(131206587314004820534098544948237170809, 800100010000000c62617463685f6d7574617465000000)) etc... {quote} > 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)