Cassandra
  1. Cassandra
  2. CASSANDRA-6285

2.0 HSHA server introduces corrupt data

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Critical Critical
    • Resolution: Fixed
    • Fix Version/s: 2.0.8
    • Component/s: Core
    • Labels:
      None
    • Environment:

      4 nodes, shortly updated from 1.2.11 to 2.0.2

      Description

      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.

      1. compaction_test.py
        3 kB
        Russ Hatch
      2. CASSANDRA-6285-disruptor-heap.patch
        1 kB
        Pavel Yaskevich
      3. 6285_testnotes1.txt
        115 kB
        Michael Shuler
      4. disruptor-high-cpu.patch
        2 kB
        Viktor Kuzmin
      5. disruptor-memory-corruption.patch
        0.6 kB
        Viktor Kuzmin
      6. cassandra-attack-src.zip
        8.56 MB
        Viktor Kuzmin
      7. enable_reallocate_buffers.txt
        1 kB
        Brandon Williams

        Activity

        Hide
        David Sauer added a comment -

        After removing all the Data from the OpsCenters Keyspace (and using LCS) and collectiong new Data for a night, the command nodetool compact OpsCenter rollups60 failed with this Exception:

        Error occurred during compaction
        java.util.concurrent.ExecutionException: java.lang.RuntimeException: Last written key DecoratedKey(-6663228376520744598, 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f6c6f6767696e672d706572666f726d616e63655f67726f757065642d67657452656164436f756e740b0f0000000100000009726f6c6c75707336) >= current key DecoratedKey(-6896470603826733036, 37382e34362e3132382e3139382d6d65646970726569735f7365617263685f696e6465785f323031335f31305f30382d6d756c7469776f7264735f70686f6e656d732d6765744c69766553535461626c65436f756e74) writing into /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-14-Data.db
        at java.util.concurrent.FutureTask.report(FutureTask.java:122)
        at java.util.concurrent.FutureTask.get(FutureTask.java:188)
        at org.apache.cassandra.db.compaction.CompactionManager.performMaximal(CompactionManager.java:281)
        at org.apache.cassandra.db.ColumnFamilyStore.forceMajorCompaction(ColumnFamilyStore.java:1845)
        at org.apache.cassandra.service.StorageService.forceKeyspaceCompaction(StorageService.java:2167)
        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:606)
        at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:75)
        at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:279)
        at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112)
        at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46)
        at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237)
        at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138)
        at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252)
        at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819)
        at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:801)
        at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1487)
        at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:97)
        at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1328)
        at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1420)
        at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:848)
        at sun.reflect.GeneratedMethodAccessor35.invoke(Unknown Source)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322)
        at sun.rmi.transport.Transport$1.run(Transport.java:177)
        at sun.rmi.transport.Transport$1.run(Transport.java:174)
        at java.security.AccessController.doPrivileged(Native Method)
        at sun.rmi.transport.Transport.serviceCall(Transport.java:173)
        at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:556)
        at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:811)
        at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:670)
        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)
        Caused by: java.lang.RuntimeException: Last written key DecoratedKey(-6663228376520744598, 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f6c6f6767696e672d706572666f726d616e63655f67726f757065642d67657452656164436f756e740b0f0000000100000009726f6c6c75707336) >= current key DecoratedKey(-6896470603826733036, 37382e34362e3132382e3139382d6d65646970726569735f7365617263685f696e6465785f323031335f31305f30382d6d756c7469776f7264735f70686f6e656d732d6765744c69766553535461626c65436f756e74) writing into /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-14-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)
        ... 3 more

        Show
        David Sauer added a comment - After removing all the Data from the OpsCenters Keyspace (and using LCS) and collectiong new Data for a night, the command nodetool compact OpsCenter rollups60 failed with this Exception: Error occurred during compaction java.util.concurrent.ExecutionException: java.lang.RuntimeException: Last written key DecoratedKey(-6663228376520744598, 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f6c6f6767696e672d706572666f726d616e63655f67726f757065642d67657452656164436f756e740b0f0000000100000009726f6c6c75707336) >= current key DecoratedKey(-6896470603826733036, 37382e34362e3132382e3139382d6d65646970726569735f7365617263685f696e6465785f323031335f31305f30382d6d756c7469776f7264735f70686f6e656d732d6765744c69766553535461626c65436f756e74) writing into /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-14-Data.db at java.util.concurrent.FutureTask.report(FutureTask.java:122) at java.util.concurrent.FutureTask.get(FutureTask.java:188) at org.apache.cassandra.db.compaction.CompactionManager.performMaximal(CompactionManager.java:281) at org.apache.cassandra.db.ColumnFamilyStore.forceMajorCompaction(ColumnFamilyStore.java:1845) at org.apache.cassandra.service.StorageService.forceKeyspaceCompaction(StorageService.java:2167) 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:606) at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:75) at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:279) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46) at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237) at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138) at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252) at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819) at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:801) at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1487) at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:97) at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1328) at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1420) at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:848) at sun.reflect.GeneratedMethodAccessor35.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322) at sun.rmi.transport.Transport$1.run(Transport.java:177) at sun.rmi.transport.Transport$1.run(Transport.java:174) at java.security.AccessController.doPrivileged(Native Method) at sun.rmi.transport.Transport.serviceCall(Transport.java:173) at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:556) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:811) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:670) 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) Caused by: java.lang.RuntimeException: Last written key DecoratedKey(-6663228376520744598, 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f6c6f6767696e672d706572666f726d616e63655f67726f757065642d67657452656164436f756e740b0f0000000100000009726f6c6c75707336) >= current key DecoratedKey(-6896470603826733036, 37382e34362e3132382e3139382d6d65646970726569735f7365617263685f696e6465785f323031335f31305f30382d6d756c7469776f7264735f70686f6e656d732d6765744c69766553535461626c65436f756e74) writing into /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-14-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) ... 3 more
        Hide
        Oliver Bock added a comment - - edited

        Just in case it helps: I'm getting almost identical exceptions while running a single node stress test using cassandra-stress with Cassandra 2.0.2 (DSC) on Debian Wheezy with 2 GB RAM. I'm running 10e7 write ops on a single HDD, using (more or less) Cassandra's default configuration, specifically STC.

        ERROR [CompactionExecutor:14] 2013-11-29 15:33:39,978 CassandraDaemon.java (line 187) Exception in thread Thread[CompactionExecutor:14,1,main]
        java.lang.RuntimeException: Last written key DecoratedKey(-3658992336117051287, 3033353732383438) >= current key DecoratedKey(-4078405136366838408, 3033353634323236) writing into /srv3/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-tmp-jb-106-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$BackgroundCompactionTask.run(CompactionManager.java:197)
        	at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
        	at java.util.concurrent.FutureTask.run(Unknown Source)
        	at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
        	at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
        	at java.lang.Thread.run(Unknown Source)
        
        Show
        Oliver Bock added a comment - - edited Just in case it helps: I'm getting almost identical exceptions while running a single node stress test using cassandra-stress with Cassandra 2.0.2 (DSC) on Debian Wheezy with 2 GB RAM. I'm running 10e7 write ops on a single HDD, using (more or less) Cassandra's default configuration, specifically STC. ERROR [CompactionExecutor:14] 2013-11-29 15:33:39,978 CassandraDaemon.java (line 187) Exception in thread Thread[CompactionExecutor:14,1,main] java.lang.RuntimeException: Last written key DecoratedKey(-3658992336117051287, 3033353732383438) >= current key DecoratedKey(-4078405136366838408, 3033353634323236) writing into /srv3/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-tmp-jb-106-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$BackgroundCompactionTask.run(CompactionManager.java:197) at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) at java.util.concurrent.FutureTask.run(Unknown Source) at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) at java.lang.Thread.run(Unknown Source)
        Hide
        Brandon Kearby added a comment -

        I'm getting it as well on 2.0.4. I'm testing a new cluster. I don't get the error with one node, but when I add two or more I get the same error.

        ERROR [CompactionExecutor:6] 2014-01-15 17:13:13,395 CassandraDaemon.java (line 187) Exception in thread Thread[CompactionExecutor:6,1,main]
        java.lang.RuntimeException: Last written key DecoratedKey(-1983406872803353678, 545749545445523a333535383030333439353835353830303334) >= current key DecoratedKey(-7683510718755081698, 545749545445523a333639333235363931383339333238323537) writing into /BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-tmp-jb-121-Data.db
        	at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142)
        	at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165)
        	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$BackgroundCompactionTask.run(CompactionManager.java:197)
        	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:744)
        

        Here's the schema I'm testing with:

        create column family signal
          with column_type = 'Standard'
          and comparator = 'UTF8Type'
          and default_validation_class = 'BytesType'
          and key_validation_class = 'UTF8Type'
          and read_repair_chance = 0.1
          and dclocal_read_repair_chance = 0.0
          and gc_grace = 432000
          and min_compaction_threshold = 4
          and max_compaction_threshold = 32
          and replicate_on_write = true
          and compaction_strategy = 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'
          and caching = 'ALL'
          and compaction_strategy_options = {'sstable_size_in_mb' : '160'}
          and comment = 'A store of information about each individual signal.'
          and column_metadata = [
            {column_name : 'type',
            validation_class : UTF8Type},
            {column_name : 'foo_id',
            validation_class : LongType},
            validation_class : UTF8Type}]
          and compression_options = {'sstable_compression' : 'org.apache.cassandra.io.compress.LZ4Compressor'};
        
        Show
        Brandon Kearby added a comment - I'm getting it as well on 2.0.4. I'm testing a new cluster. I don't get the error with one node, but when I add two or more I get the same error. ERROR [CompactionExecutor:6] 2014-01-15 17:13:13,395 CassandraDaemon.java (line 187) Exception in thread Thread [CompactionExecutor:6,1,main] java.lang.RuntimeException: Last written key DecoratedKey(-1983406872803353678, 545749545445523a333535383030333439353835353830303334) >= current key DecoratedKey(-7683510718755081698, 545749545445523a333639333235363931383339333238323537) writing into /BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-tmp-jb-121-Data.db at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165) 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$BackgroundCompactionTask.run(CompactionManager.java:197) 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:744) Here's the schema I'm testing with: create column family signal with column_type = 'Standard' and comparator = 'UTF8Type' and default_validation_class = 'BytesType' and key_validation_class = 'UTF8Type' and read_repair_chance = 0.1 and dclocal_read_repair_chance = 0.0 and gc_grace = 432000 and min_compaction_threshold = 4 and max_compaction_threshold = 32 and replicate_on_write = true and compaction_strategy = 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy' and caching = 'ALL' and compaction_strategy_options = {'sstable_size_in_mb' : '160'} and comment = 'A store of information about each individual signal.' and column_metadata = [ {column_name : 'type', validation_class : UTF8Type}, {column_name : 'foo_id', validation_class : LongType}, validation_class : UTF8Type}] and compression_options = {'sstable_compression' : 'org.apache.cassandra.io.compress.LZ4Compressor'};
        Hide
        Jonathan Ellis added a comment -

        Can you enable snapshot_before_compaction and post the sstables that it's trying to compact? I can get you a private upload place if necessary.

        Show
        Jonathan Ellis added a comment - Can you enable snapshot_before_compaction and post the sstables that it's trying to compact? I can get you a private upload place if necessary.
        Hide
        Brandon Kearby added a comment -

        Sure,

        BTW, I tried changing to SizeTieredCompactionStrategy and got the same error. I'll enable snapshot_before_compaction.

        Show
        Brandon Kearby added a comment - Sure, BTW, I tried changing to SizeTieredCompactionStrategy and got the same error. I'll enable snapshot_before_compaction.
        Hide
        Brandon Kearby added a comment - - edited

        Snapshot of compaction before failing
        Added attachment: system-compactions_in_progress-jb-25-Data.db

        Logs before failing

        INFO [CompactionExecutor:6] 2014-01-15 18:38:47,690 ColumnFamilyStore.java (line 740) Enqueuing flush of Memtable-compactions_in_progress@856586691(847/8470 serialized/live bytes, 35 ops)
         INFO [FlushWriter:3] 2014-01-15 18:38:47,691 Memtable.java (line 333) Writing Memtable-compactions_in_progress@856586691(847/8470 serialized/live bytes, 35 ops)
         INFO [FlushWriter:3] 2014-01-15 18:38:47,700 Memtable.java (line 373) Completed flushing /BigData/lib/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-jb-24-Data.db (304 bytes) for commitlog position ReplayPosition(segmentId=1389810508756, position=33429429)
         INFO [CompactionExecutor:6] 2014-01-15 18:38:47,703 CompactionTask.java (line 115) Compacting [SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-45-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-46-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-67-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-72-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-78-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-75-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-56-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-62-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-66-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-49-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-47-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-57-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-61-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-79-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-65-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-50-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-58-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-77-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-54-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-53-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-48-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-64-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-68-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-76-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-55-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-74-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-60-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-52-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-69-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-71-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-73-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-63-Data.db')]
         INFO [CompactionExecutor:6] 2014-01-15 18:39:05,208 ColumnFamilyStore.java (line 740) Enqueuing flush of Memtable-compactions_in_progress@2046507929(0/0 serialized/live bytes, 1 ops)
         INFO [FlushWriter:3] 2014-01-15 18:39:05,208 Memtable.java (line 333) Writing Memtable-compactions_in_progress@2046507929(0/0 serialized/live bytes, 1 ops)
         INFO [FlushWriter:3] 2014-01-15 18:39:05,218 Memtable.java (line 373) Completed flushing /BigData/lib/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-jb-25-Data.db (42 bytes) for commitlog position ReplayPosition(segmentId=1389810508756, position=33430117)
        ERROR [CompactionExecutor:6] 2014-01-15 18:39:05,220 CassandraDaemon.java (line 187) Exception in thread Thread[CompactionExecutor:6,1,main]
        java.lang.RuntimeException: Last written key DecoratedKey(-5705444534806265577, 0000000000000000000000000000000000000000000000000000) >= current key DecoratedKey(-7490754936938484492, 00ab1b0000000000000000000000000000000000000000000000) writing into /BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-tmp-jb-80-Data.db
                at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142)
                at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165)
                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$BackgroundCompactionTask.run(CompactionManager.java:197)
                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:744)
         INFO [MemoryMeter:1] 2014-01-15 18:39:08,865 Memtable.java (line 451) CFS(Keyspace='system', ColumnFamily='sstable_activity') liveRatio is 14.596825396825396 (just-counted was 14.596825396825396).  calculation took 1ms for 84 cells
         INFO [MemoryMeter:1] 2014-01-15 18:43:44,575 Memtable.java (line 451) CFS(Keyspace='system', ColumnFamily='sstable_activity') liveRatio is 14.591111111111111 (just-counted was 14.585396825396826).  calculation took 4ms for 210 cells
        (END) 
        
        Show
        Brandon Kearby added a comment - - edited Snapshot of compaction before failing Added attachment: system-compactions_in_progress-jb-25-Data.db Logs before failing INFO [CompactionExecutor:6] 2014-01-15 18:38:47,690 ColumnFamilyStore.java (line 740) Enqueuing flush of Memtable-compactions_in_progress@856586691(847/8470 serialized/live bytes, 35 ops) INFO [FlushWriter:3] 2014-01-15 18:38:47,691 Memtable.java (line 333) Writing Memtable-compactions_in_progress@856586691(847/8470 serialized/live bytes, 35 ops) INFO [FlushWriter:3] 2014-01-15 18:38:47,700 Memtable.java (line 373) Completed flushing /BigData/lib/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-jb-24-Data.db (304 bytes) for commitlog position ReplayPosition(segmentId=1389810508756, position=33429429) INFO [CompactionExecutor:6] 2014-01-15 18:38:47,703 CompactionTask.java (line 115) Compacting [SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-45-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-46-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-67-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-72-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-78-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-75-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-56-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-62-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-66-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-49-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-47-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-57-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-61-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-79-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-65-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-50-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-58-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-77-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-54-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-53-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-48-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-64-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-68-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-76-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-55-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-74-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-60-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-52-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-69-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-71-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-73-Data.db'), SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-63-Data.db')] INFO [CompactionExecutor:6] 2014-01-15 18:39:05,208 ColumnFamilyStore.java (line 740) Enqueuing flush of Memtable-compactions_in_progress@2046507929(0/0 serialized/live bytes, 1 ops) INFO [FlushWriter:3] 2014-01-15 18:39:05,208 Memtable.java (line 333) Writing Memtable-compactions_in_progress@2046507929(0/0 serialized/live bytes, 1 ops) INFO [FlushWriter:3] 2014-01-15 18:39:05,218 Memtable.java (line 373) Completed flushing /BigData/lib/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-jb-25-Data.db (42 bytes) for commitlog position ReplayPosition(segmentId=1389810508756, position=33430117) ERROR [CompactionExecutor:6] 2014-01-15 18:39:05,220 CassandraDaemon.java (line 187) Exception in thread Thread [CompactionExecutor:6,1,main] java.lang.RuntimeException: Last written key DecoratedKey(-5705444534806265577, 0000000000000000000000000000000000000000000000000000) >= current key DecoratedKey(-7490754936938484492, 00ab1b0000000000000000000000000000000000000000000000) writing into /BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-tmp-jb-80-Data.db at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165) 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$BackgroundCompactionTask.run(CompactionManager.java:197) 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:744) INFO [MemoryMeter:1] 2014-01-15 18:39:08,865 Memtable.java (line 451) CFS(Keyspace='system', ColumnFamily='sstable_activity') liveRatio is 14.596825396825396 (just-counted was 14.596825396825396). calculation took 1ms for 84 cells INFO [MemoryMeter:1] 2014-01-15 18:43:44,575 Memtable.java (line 451) CFS(Keyspace='system', ColumnFamily='sstable_activity') liveRatio is 14.591111111111111 (just-counted was 14.585396825396826). calculation took 4ms for 210 cells (END)
        Hide
        Jonathan Ellis added a comment -

        can you tar up all the components, not just .db?

        Show
        Jonathan Ellis added a comment - can you tar up all the components, not just .db?
        Hide
        Jonathan Ellis added a comment -

        ... for all the sstables in the "Compacting" list

        Show
        Jonathan Ellis added a comment - ... for all the sstables in the "Compacting" list
        Hide
        Brandon Kearby added a comment -

        Hi Jonathan,

        Here's a link to what you need: http://bhorne.test.s3.amazonaws.com/cassandra.tar.gz

        Show
        Brandon Kearby added a comment - Hi Jonathan, Here's a link to what you need: http://bhorne.test.s3.amazonaws.com/cassandra.tar.gz
        Hide
        Brandon Kearby added a comment -

        So it seems like it might be related to the hsa server. BTW, I was getting https://issues.apache.org/jira/browse/CASSANDRA-6373 where it would hang describing the ring. So I upgraded to thrift-server-0.3.3.jar. When running with the sync server, I don't get the error above.

        A little more context, I'm using pig and the CassandraStorage class to drive the writes. Running as a map task with 12 concurrent mappers creates 2773 connections!

        lsof -i tcp:9160 | wc -l
        2773

        Show
        Brandon Kearby added a comment - So it seems like it might be related to the hsa server. BTW, I was getting https://issues.apache.org/jira/browse/CASSANDRA-6373 where it would hang describing the ring. So I upgraded to thrift-server-0.3.3.jar. When running with the sync server, I don't get the error above. A little more context, I'm using pig and the CassandraStorage class to drive the writes. Running as a map task with 12 concurrent mappers creates 2773 connections! lsof -i tcp:9160 | wc -l 2773
        Hide
        Jonathan Ellis added a comment -

        The .tar.gz does not contain the sstables mentioned in the error message

        Show
        Jonathan Ellis added a comment - The .tar.gz does not contain the sstables mentioned in the error message
        Hide
        Brandon Kearby added a comment -

        Correct. The tar contains a full log file with another example of the error.

        Show
        Brandon Kearby added a comment - Correct. The tar contains a full log file with another example of the error.
        Hide
        Jonathan Ellis added a comment -

        Ryan McGuire Can your team reproduce w/ the schema above and the sstables from the tarball?

        Compacting [SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-2-Data.db'), STableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-1-Data.db')]
        
        Show
        Jonathan Ellis added a comment - Ryan McGuire Can your team reproduce w/ the schema above and the sstables from the tarball? Compacting [SSTableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-2-Data.db'), STableReader(path='/BigData/lib/cassandra/data/SocialData/signal/SocialData-signal-jb-1-Data.db')]
        Hide
        Brandon Kearby added a comment - - edited

        BTW, It happens when we use hsha. The schema above is abbreviated as I've left out a lot of the payload from the signal table. Ping me if you need the full signal table definition.

        Show
        Brandon Kearby added a comment - - edited BTW, It happens when we use hsha. The schema above is abbreviated as I've left out a lot of the payload from the signal table. Ping me if you need the full signal table definition.
        Hide
        Brandon Kearby added a comment -

        When I cranked up the number of rpc_max_threads and changed to sync, it stopped happening.

        Show
        Brandon Kearby added a comment - When I cranked up the number of rpc_max_threads and changed to sync, it stopped happening.
        Hide
        Brandon Kearby added a comment -

        After doing some more digging, looks like my issue is the same as https://issues.apache.org/jira/browse/CASSANDRA-4687

        Show
        Brandon Kearby added a comment - After doing some more digging, looks like my issue is the same as https://issues.apache.org/jira/browse/CASSANDRA-4687
        Hide
        Jonathan Ellis added a comment -

        Russ Hatch would still be useful to try to repro w/ Brandon's instructions since we don't have a way to repro 4687 yet.

        Show
        Jonathan Ellis added a comment - Russ Hatch would still be useful to try to repro w/ Brandon's instructions since we don't have a way to repro 4687 yet.
        Hide
        Russ Hatch added a comment -

        Jonathan Ellis – I was able to get the exception to occur by doing the following:

        create a new cluster with ccm, and populate with 3 nodes

        create keyspace SocialData with placement_strategy='org.apache.cassandra.locator.SimpleStrategy' and strategy_options = {replication_factor:3};
        

        create signal Column Family (I had to modify schema above a little bit to make it work):

        create column family signal
          with column_type = 'Standard'
          and comparator = 'UTF8Type'
          and default_validation_class = 'BytesType'
          and key_validation_class = 'UTF8Type'
          and read_repair_chance = 0.1
          and dclocal_read_repair_chance = 0.0
          and gc_grace = 432000
          and min_compaction_threshold = 4
          and max_compaction_threshold = 32
          and replicate_on_write = true
          and compaction_strategy = 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'
          and caching = 'ALL'
          and compaction_strategy_options = {'sstable_size_in_mb' : '160'}
          and comment = 'A store of information about each individual signal.'
          and column_metadata = [
            {column_name : 'type', validation_class : UTF8Type},
            {column_name : 'foo_id', validation_class : LongType}]
          and compression_options = {'sstable_compression' : 'org.apache.cassandra.io.compress.LZ4Compressor'}; 
        

        stopped the nodes
        copied all the files from the provided tar's /data/SocialData/ directory to one of my nodes
        started the nodes up again
        At this point I didn't find any data in the signal column family (using 'list signal;')
        The exception appeared in the node's log

        ERROR [CompactionExecutor:10] 2014-01-27 12:45:26,734 CassandraDaemon.java (line 187) Exception in thread Thread[CompactionExecutor:10,1,main]
        java.lang.RuntimeException: Last written key DecoratedKey(4322717900587903123, 706f737431353834373031323038270903ae0022076d9f) >= current key DecoratedKey(-7009815163526224622, 545749545445523a333533343836323333393032363439333437) writing into /home/rhatch/.ccm/test_cluster_1390845354/node1/data/SocialData/signal/SocialData-signal-tmp-jb-7-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$BackgroundCompactionTask.run(CompactionManager.java:197)
        	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:744)
        

        I was curious if repair would have any bearing, so I ran repair on one node (after which I can see data in the signal table), then I stopped and started the nodes again – a similar exception appears in the log for all 3 nodes ('Last written key DecoratedKey ...').

        I'm not 100% certain if my procedure for using the provided tar's test data was correct, so let me know if there's anything obvious I missed and I'll run through it again.

        Show
        Russ Hatch added a comment - Jonathan Ellis – I was able to get the exception to occur by doing the following: create a new cluster with ccm, and populate with 3 nodes create keyspace SocialData with placement_strategy='org.apache.cassandra.locator.SimpleStrategy' and strategy_options = {replication_factor:3}; create signal Column Family (I had to modify schema above a little bit to make it work): create column family signal with column_type = 'Standard' and comparator = 'UTF8Type' and default_validation_class = 'BytesType' and key_validation_class = 'UTF8Type' and read_repair_chance = 0.1 and dclocal_read_repair_chance = 0.0 and gc_grace = 432000 and min_compaction_threshold = 4 and max_compaction_threshold = 32 and replicate_on_write = true and compaction_strategy = 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy' and caching = 'ALL' and compaction_strategy_options = {'sstable_size_in_mb' : '160'} and comment = 'A store of information about each individual signal.' and column_metadata = [ {column_name : 'type', validation_class : UTF8Type}, {column_name : 'foo_id', validation_class : LongType}] and compression_options = {'sstable_compression' : 'org.apache.cassandra.io.compress.LZ4Compressor'}; stopped the nodes copied all the files from the provided tar's /data/SocialData/ directory to one of my nodes started the nodes up again At this point I didn't find any data in the signal column family (using 'list signal;') The exception appeared in the node's log ERROR [CompactionExecutor:10] 2014-01-27 12:45:26,734 CassandraDaemon.java (line 187) Exception in thread Thread[CompactionExecutor:10,1,main] java.lang.RuntimeException: Last written key DecoratedKey(4322717900587903123, 706f737431353834373031323038270903ae0022076d9f) >= current key DecoratedKey(-7009815163526224622, 545749545445523a333533343836323333393032363439333437) writing into /home/rhatch/.ccm/test_cluster_1390845354/node1/data/SocialData/signal/SocialData-signal-tmp-jb-7-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$BackgroundCompactionTask.run(CompactionManager.java:197) 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:744) I was curious if repair would have any bearing, so I ran repair on one node (after which I can see data in the signal table), then I stopped and started the nodes again – a similar exception appears in the log for all 3 nodes ('Last written key DecoratedKey ...'). I'm not 100% certain if my procedure for using the provided tar's test data was correct, so let me know if there's anything obvious I missed and I'll run through it again.
        Hide
        Jonathan Ellis added a comment -

        Was that 2.0 HEAD or 2.0.4?

        Show
        Jonathan Ellis added a comment - Was that 2.0 HEAD or 2.0.4?
        Hide
        Russ Hatch added a comment -

        oh sorry, forgot that detail. I reproduced from the cassandra-2.0.2 tag.

        Show
        Russ Hatch added a comment - oh sorry, forgot that detail. I reproduced from the cassandra-2.0.2 tag.
        Hide
        Jonathan Ellis added a comment -

        Can you try 2.0 HEAD as well just to be sure?

        Show
        Jonathan Ellis added a comment - Can you try 2.0 HEAD as well just to be sure?
        Hide
        Russ Hatch added a comment -

        OK, appears we have the same issue on 2.0 HEAD as well (8bbb6e...) – exception appears on startup using the procedure I included earlier.

        Show
        Russ Hatch added a comment - OK, appears we have the same issue on 2.0 HEAD as well (8bbb6e...) – exception appears on startup using the procedure I included earlier.
        Hide
        Russ Hatch added a comment -

        I'm going to attempt to condense this down to a simple dtest as well.

        Show
        Russ Hatch added a comment - I'm going to attempt to condense this down to a simple dtest as well.
        Hide
        Brandon Kearby added a comment -

        Hi Russ Hatch,

        Here's the full schema I'm using to test with:

        create keyspace SocialData
        with placement_strategy = 'NetworkTopologyStrategy'
        and strategy_options =

        {DC-Analytics : 3}

        and durable_writes = true;

        use SocialData;

        create column family signal
        with column_type = 'Standard'
        and comparator = 'UTF8Type'
        and default_validation_class = 'BytesType'
        and key_validation_class = 'UTF8Type'
        and read_repair_chance = 0.1
        and dclocal_read_repair_chance = 0.0
        and gc_grace = 432000
        and min_compaction_threshold = 4
        and max_compaction_threshold = 32
        and replicate_on_write = true
        and compaction_strategy = 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'
        and caching = 'NONE'
        and compaction_strategy_options =

        {'sstable_size_in_mb' : '160'}

        and comment = 'A store of information about each individual signal.'
        and column_metadata = [

        {column_name : 'type', validation_class : UTF8Type}

        ,

        {column_name : 'department_id', validation_class : LongType}

        ,

        {column_name : 'ecosystem_account_id', validation_class : UTF8Type}

        ,

        {column_name : 'content_type', validation_class : UTF8Type}

        ,

        {column_name : 'rating_count', validation_class : LongType}

        ,

        {column_name : 'service_account_id', validation_class : UTF8Type}

        ,

        {column_name : 'time', validation_class : LongType}

        ,

        {column_name : 'organization_id', validation_class : LongType}

        ,

        {column_name : 'conversation_id', validation_class : UTF8Type}

        ,

        {column_name : 'favorites_count', validation_class : LongType}

        ,

        {column_name : 'dislike_count', validation_class : LongType}

        ,

        {column_name : 'url', validation_class : UTF8Type}

        ,

        {column_name : 'impressions', validation_class : LongType}

        ,

        {column_name : 'network_strength', validation_class : LongType}

        ,

        {column_name : 'parent_signal_id', validation_class : UTF8Type}

        ,

        {column_name : 'account_snapshot_id', validation_class : UTF8Type}

        ,

        {column_name : 'region_id', validation_class : LongType}

        ,

        {column_name : 'time_bucket', validation_class : LongType}

        ,

        {column_name : 'enriched_on', validation_class : LongType}

        ,

        {column_name : 'dachis_account_id', validation_class : UTF8Type}

        ,

        {column_name : 'text', validation_class : UTF8Type}

        ,

        {column_name : 'sentiment', validation_class : LongType}

        ,

        {column_name : 'like_count', validation_class : LongType}

        ,

        {column_name : 'industry_id', validation_class : LongType}

        ,

        {column_name : 'service', validation_class : UTF8Type}

        ,

        {column_name : 'cloned_from', validation_class : UTF8Type}

        ,

        {column_name : 'constituent_type', validation_class : UTF8Type}

        ,

        {column_name : 'listings_count', validation_class : LongType}

        ,

        {column_name : 'network_size', validation_class : LongType}

        ,

        {column_name : 'analyzed', validation_class : Int32Type}

        ,

        {column_name : 'username', validation_class : UTF8Type}

        ,

        {column_name : 'service_signal_id', validation_class : UTF8Type}

        ,

        {column_name : 'language', validation_class : UTF8Type}

        ,

        {column_name : 'brand_id', validation_class : LongType}

        ,

        {column_name : 'rating', validation_class : LongType}

        ,

        {column_name : 'relationship_id', validation_class : UTF8Type}

        ]
        and compression_options =

        {'sstable_compression' : 'org.apache.cassandra.io.compress.LZ4Compressor'}

        ;

        Show
        Brandon Kearby added a comment - Hi Russ Hatch , Here's the full schema I'm using to test with: create keyspace SocialData with placement_strategy = 'NetworkTopologyStrategy' and strategy_options = {DC-Analytics : 3} and durable_writes = true; use SocialData; create column family signal with column_type = 'Standard' and comparator = 'UTF8Type' and default_validation_class = 'BytesType' and key_validation_class = 'UTF8Type' and read_repair_chance = 0.1 and dclocal_read_repair_chance = 0.0 and gc_grace = 432000 and min_compaction_threshold = 4 and max_compaction_threshold = 32 and replicate_on_write = true and compaction_strategy = 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy' and caching = 'NONE' and compaction_strategy_options = {'sstable_size_in_mb' : '160'} and comment = 'A store of information about each individual signal.' and column_metadata = [ {column_name : 'type', validation_class : UTF8Type} , {column_name : 'department_id', validation_class : LongType} , {column_name : 'ecosystem_account_id', validation_class : UTF8Type} , {column_name : 'content_type', validation_class : UTF8Type} , {column_name : 'rating_count', validation_class : LongType} , {column_name : 'service_account_id', validation_class : UTF8Type} , {column_name : 'time', validation_class : LongType} , {column_name : 'organization_id', validation_class : LongType} , {column_name : 'conversation_id', validation_class : UTF8Type} , {column_name : 'favorites_count', validation_class : LongType} , {column_name : 'dislike_count', validation_class : LongType} , {column_name : 'url', validation_class : UTF8Type} , {column_name : 'impressions', validation_class : LongType} , {column_name : 'network_strength', validation_class : LongType} , {column_name : 'parent_signal_id', validation_class : UTF8Type} , {column_name : 'account_snapshot_id', validation_class : UTF8Type} , {column_name : 'region_id', validation_class : LongType} , {column_name : 'time_bucket', validation_class : LongType} , {column_name : 'enriched_on', validation_class : LongType} , {column_name : 'dachis_account_id', validation_class : UTF8Type} , {column_name : 'text', validation_class : UTF8Type} , {column_name : 'sentiment', validation_class : LongType} , {column_name : 'like_count', validation_class : LongType} , {column_name : 'industry_id', validation_class : LongType} , {column_name : 'service', validation_class : UTF8Type} , {column_name : 'cloned_from', validation_class : UTF8Type} , {column_name : 'constituent_type', validation_class : UTF8Type} , {column_name : 'listings_count', validation_class : LongType} , {column_name : 'network_size', validation_class : LongType} , {column_name : 'analyzed', validation_class : Int32Type} , {column_name : 'username', validation_class : UTF8Type} , {column_name : 'service_signal_id', validation_class : UTF8Type} , {column_name : 'language', validation_class : UTF8Type} , {column_name : 'brand_id', validation_class : LongType} , {column_name : 'rating', validation_class : LongType} , {column_name : 'relationship_id', validation_class : UTF8Type} ] and compression_options = {'sstable_compression' : 'org.apache.cassandra.io.compress.LZ4Compressor'} ;
        Hide
        Russ Hatch added a comment -

        I was able to repro with the data provided in the tar (as noted above). Unfortunately my attempts to use dtest to reproduce the issue "from scratch" haven't been successful. First I tried testing with dtest w/CQL, but had no luck, though I may have missed something when trying to translate everything into CQL.

        Next I tried dtest w/thrift but similarly wasn't able to trigger the issue in this way. I tried using the hsha rpc_server and was getting what appeared to be an unrelated error:

        ERROR [Thrift-Selector_0] 2014-01-28 11:09:16,762 Message.java (line 153) Read an invalid frame size of 0. Are you using TFramedTransport on the client side?
        

        I'll attach my basic dtest here in case it's useful later, but as for now I can't repro the issue without the provided db.

        Show
        Russ Hatch added a comment - I was able to repro with the data provided in the tar (as noted above). Unfortunately my attempts to use dtest to reproduce the issue "from scratch" haven't been successful. First I tried testing with dtest w/CQL, but had no luck, though I may have missed something when trying to translate everything into CQL. Next I tried dtest w/thrift but similarly wasn't able to trigger the issue in this way. I tried using the hsha rpc_server and was getting what appeared to be an unrelated error: ERROR [Thrift-Selector_0] 2014-01-28 11:09:16,762 Message.java (line 153) Read an invalid frame size of 0. Are you using TFramedTransport on the client side? I'll attach my basic dtest here in case it's useful later, but as for now I can't repro the issue without the provided db.
        Hide
        Tyler Hobbs added a comment -

        Brandon Kearby It looks like there may be a few things going on here.

        The first is that some of your column names are not valid UTF-8. I'm not terribly familiar with the UTF-8 specs, but they seem to fail validation in different ways, and Python seems to agree that they are not valid UTF-8, so I don't think it's a problem with our validation code. Did you change the comparator from BytesType to UTF8Type at some point? It might not be relevant to this ticket, but you may want to check on that on your end.

        The second problem is that SocialData-signal-jb-2-Data.db has some out-of-order rows. It looks like about 9 rows are randomly out of place in the sstable. Running scrub would fix this, but I think it's erroring on UTF8 validation. If I change the comparator to BytesType, the scrub completes and the rows are written in order. So the problem is not necessarily with compaction itself but with out-of-order rows being written to sstables.

        Given that switching from hsha to sync seemed to fix the problem, I wonder if that's part of the original cause.

        Show
        Tyler Hobbs added a comment - Brandon Kearby It looks like there may be a few things going on here. The first is that some of your column names are not valid UTF-8. I'm not terribly familiar with the UTF-8 specs, but they seem to fail validation in different ways, and Python seems to agree that they are not valid UTF-8, so I don't think it's a problem with our validation code. Did you change the comparator from BytesType to UTF8Type at some point? It might not be relevant to this ticket, but you may want to check on that on your end. The second problem is that SocialData-signal-jb-2-Data.db has some out-of-order rows. It looks like about 9 rows are randomly out of place in the sstable. Running scrub would fix this, but I think it's erroring on UTF8 validation. If I change the comparator to BytesType, the scrub completes and the rows are written in order. So the problem is not necessarily with compaction itself but with out-of-order rows being written to sstables. Given that switching from hsha to sync seemed to fix the problem, I wonder if that's part of the original cause.
        Hide
        Brandon Kearby added a comment -

        Tyler Hobbs, The odd thing is that with hsha, it works with one node. When we have two or more nodes in the cluster, it starts getting these errors.

        Show
        Brandon Kearby added a comment - Tyler Hobbs , The odd thing is that with hsha, it works with one node. When we have two or more nodes in the cluster, it starts getting these errors.
        Hide
        Ravi Prasad added a comment -

        cc Pavel Yaskevich
        we were also seeing such random out of place partitions/rows in sstables (rows not hashing to the node) while using disruptor based hsha thrift server, causing compaction to fail with out of order keys. this used to happen on freshly flushed sstables in L0. We also used to see thrift validation failing on some columns while reading back. We don't see these after switching back to sync server.

        Show
        Ravi Prasad added a comment - cc Pavel Yaskevich we were also seeing such random out of place partitions/rows in sstables (rows not hashing to the node) while using disruptor based hsha thrift server, causing compaction to fail with out of order keys. this used to happen on freshly flushed sstables in L0. We also used to see thrift validation failing on some columns while reading back. We don't see these after switching back to sync server.
        Hide
        Pavel Yaskevich added a comment - - edited

        Ravi Prasad Can you try with the most recent release of hsha, version 0.3.3? Just remove the old jar and drop in new one, that should be sufficient.

        Show
        Pavel Yaskevich added a comment - - edited Ravi Prasad Can you try with the most recent release of hsha, version 0.3.3? Just remove the old jar and drop in new one, that should be sufficient.
        Hide
        Brandon Kearby added a comment -

        Pavel Yaskevich, I was running with 0.3.3. The previous version would hang on describe ring for me.

        Show
        Brandon Kearby added a comment - Pavel Yaskevich , I was running with 0.3.3. The previous version would hang on describe ring for me.
        Hide
        Pavel Yaskevich added a comment -

        Russ Hatch The 0 frame size you are seeing is a known Thrift problem which happens even with stock server implementations, they are working on it but it shouldn't cause any problems as such frames are ignored (it could also happen if something does e.g. telnet to the thrift port). I'm not sure that this is a problem with HsHa directly but might be unveiled by the increased throughput you can get with HsHa comparing to sync, it looks exactly like https://issues.apache.org/jira/browse/CASSANDRA-4687 (as Brandon Kearby mentioned) so can you try disabling key_cache and try uploading again with hsha?

        Show
        Pavel Yaskevich added a comment - Russ Hatch The 0 frame size you are seeing is a known Thrift problem which happens even with stock server implementations, they are working on it but it shouldn't cause any problems as such frames are ignored (it could also happen if something does e.g. telnet to the thrift port). I'm not sure that this is a problem with HsHa directly but might be unveiled by the increased throughput you can get with HsHa comparing to sync, it looks exactly like https://issues.apache.org/jira/browse/CASSANDRA-4687 (as Brandon Kearby mentioned) so can you try disabling key_cache and try uploading again with hsha?
        Hide
        Brandon Kearby added a comment -

        Pavel Yaskevich, I've tried disabling the key_cache and it didn't help. That was the last thing I tried.

        Show
        Brandon Kearby added a comment - Pavel Yaskevich , I've tried disabling the key_cache and it didn't help. That was the last thing I tried.
        Hide
        Pavel Yaskevich added a comment -

        I see that you have it set to NONE in CF schema but have you also tried disabling it all together in yaml? I'm not saying that it would help but trying to eliminate all possibilities. It's just not obvious to me if it's a hsha problem how Thrift could actually be correctly interpreting erroneous data from the socket, dispatching it the right Thrift handler and deserializing whole mutation (and meta information) to insert it into storage...

        Show
        Pavel Yaskevich added a comment - I see that you have it set to NONE in CF schema but have you also tried disabling it all together in yaml? I'm not saying that it would help but trying to eliminate all possibilities. It's just not obvious to me if it's a hsha problem how Thrift could actually be correctly interpreting erroneous data from the socket, dispatching it the right Thrift handler and deserializing whole mutation (and meta information) to insert it into storage...
        Hide
        Ravi Prasad added a comment -

        Also, one more factor with disruptor based hsha is direct memory/Unsafe versus heap-based message buffers. When we encountered this issue, we were running with jna, hence was using direct memory buffers. I didn't test with heap-based message buffers.

        Show
        Ravi Prasad added a comment - Also, one more factor with disruptor based hsha is direct memory/Unsafe versus heap-based message buffers. When we encountered this issue, we were running with jna, hence was using direct memory buffers. I didn't test with heap-based message buffers.
        Hide
        Nikolai Grigoriev added a comment -

        I have started seeing these too. Surprisingly...after adding OpsCenter CE to my cluster. I do not see these associated with my own data.

        java.lang.RuntimeException: Last written key DecoratedKey(3542937286762954312, 31302e332e34352e3135382d676574466c757368657350656e64696e67) >= current
        key DecoratedKey(-2152912038130700738, 31302e332e34352e3135362d77696e7465726d7574655f6a6d657465722d776d5f6170706c69636174696f6e732d676574526563656e744
        26c6f6f6d46) writing into /hadoop/disk1/cassandra/data/OpsCenter/rollups300/OpsCenter-rollups300-tmp-jb-5055-Data.db
                at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142)
                at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165)
                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$BackgroundCompactionTask.run(CompactionManager.java:197)
                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:1145)
                at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
                at java.lang.Thread.run(Thread.java:724)
        
        Show
        Nikolai Grigoriev added a comment - I have started seeing these too. Surprisingly...after adding OpsCenter CE to my cluster. I do not see these associated with my own data. java.lang.RuntimeException: Last written key DecoratedKey(3542937286762954312, 31302e332e34352e3135382d676574466c757368657350656e64696e67) >= current key DecoratedKey(-2152912038130700738, 31302e332e34352e3135362d77696e7465726d7574655f6a6d657465722d776d5f6170706c69636174696f6e732d676574526563656e744 26c6f6f6d46) writing into /hadoop/disk1/cassandra/data/OpsCenter/rollups300/OpsCenter-rollups300-tmp-jb-5055-Data.db at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165) 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$BackgroundCompactionTask.run(CompactionManager.java:197) 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:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang. Thread .run( Thread .java:724)
        Hide
        sankalp kohli added a comment -

        This issue is also in logs attached in CASSANDRA-6716.

        Show
        sankalp kohli added a comment - This issue is also in logs attached in CASSANDRA-6716 .
        Hide
        Nikolai Grigoriev added a comment -

        Can confirm on my side. I have switched to "sync" RPC server and after few scrubs/restarts I am running my load tests on a 6-node 2.0.5 cluster without a single exception in last ~8 hours.

        I tried to correlate the moment I started getting large number of FileNotFoundException's with other events in my cluster....realized that it was not exactly 2.0.5 upgrade. It seems to correlate mostly with a moment when my jmeter server went out of free space and a bunch of tests crashed. Obviously, these crashes have terminated a few hundreds of client connections to Cassandra.

        Not sure if it is related but it seems that from that moment it was some sort of snowball effect.

        Show
        Nikolai Grigoriev added a comment - Can confirm on my side. I have switched to "sync" RPC server and after few scrubs/restarts I am running my load tests on a 6-node 2.0.5 cluster without a single exception in last ~8 hours. I tried to correlate the moment I started getting large number of FileNotFoundException's with other events in my cluster....realized that it was not exactly 2.0.5 upgrade. It seems to correlate mostly with a moment when my jmeter server went out of free space and a bunch of tests crashed. Obviously, these crashes have terminated a few hundreds of client connections to Cassandra. Not sure if it is related but it seems that from that moment it was some sort of snowball effect.
        Hide
        Marcus Eriksson added a comment -

        I think we can conclude that this only happens with HSHA

        I brought back TThreadedSelectorServer instead of the Disruptor based server and have been running it for a few hours without the bug happening.

        Could someone (Viktor Kuzmin ?) try out https://github.com/krummas/cassandra/commits/marcuse/hsha and see if you can break it?

        Note that i had to make a change in thrift 0.9.1 to get it to build and work, I'll follow up on that if this seems to solve the issue.

        Show
        Marcus Eriksson added a comment - I think we can conclude that this only happens with HSHA I brought back TThreadedSelectorServer instead of the Disruptor based server and have been running it for a few hours without the bug happening. Could someone ( Viktor Kuzmin ?) try out https://github.com/krummas/cassandra/commits/marcuse/hsha and see if you can break it? Note that i had to make a change in thrift 0.9.1 to get it to build and work, I'll follow up on that if this seems to solve the issue.
        Hide
        Viktor Kuzmin added a comment -

        https://www.dropbox.com/s/3ldg10zh7qvva27/cassandra-attack.jar

        Schema is located inside jar file - cassandra.txt

        1. Start cassandra
        2. java -jar cassandra-attack.jar
        3. Stop cassandra
        4. Start cassandra - commit logs will be corrupted.

        Show
        Viktor Kuzmin added a comment - https://www.dropbox.com/s/3ldg10zh7qvva27/cassandra-attack.jar Schema is located inside jar file - cassandra.txt 1. Start cassandra 2. java -jar cassandra-attack.jar 3. Stop cassandra 4. Start cassandra - commit logs will be corrupted.
        Hide
        Nikolai Grigoriev added a comment -

        Marcus Eriksson

        I think using HSHA makes it easier to reproduce but...I am running SYNC for over a week now and recently I have experienced the same issue again.

        We had another unclean shutdown (hrrr...some people are smarter than the UPSes ) and after bringing the nodes back I have found that on one node my compactions constantly fail with FileNotFoundException. Even worse, I can't scrub the keyspace/CF in question because "scrub" fails instantly with "RuntimeException: Tried to hard link to file that does not exist...". I have reported that one too. It is impossible to scrub. The only way to fix that issue I have found so far is to restart Cassandra on that node, stop compactions as soon as it starts (well, I could disable them differently, I assume) and then scrub. Sometimes I have to do it in several iterations to complete the process. Once I scrub all problematic KS/CFs I see no more exceptions.

        Show
        Nikolai Grigoriev added a comment - Marcus Eriksson I think using HSHA makes it easier to reproduce but...I am running SYNC for over a week now and recently I have experienced the same issue again. We had another unclean shutdown (hrrr...some people are smarter than the UPSes ) and after bringing the nodes back I have found that on one node my compactions constantly fail with FileNotFoundException. Even worse, I can't scrub the keyspace/CF in question because "scrub" fails instantly with "RuntimeException: Tried to hard link to file that does not exist...". I have reported that one too. It is impossible to scrub. The only way to fix that issue I have found so far is to restart Cassandra on that node, stop compactions as soon as it starts (well, I could disable them differently, I assume) and then scrub. Sometimes I have to do it in several iterations to complete the process. Once I scrub all problematic KS/CFs I see no more exceptions.
        Hide
        Viktor Kuzmin added a comment -

        Bug with FileNotFoundException is not related to HsHa problem.
        And about several iterations for scrub:
        https://issues.apache.org/jira/browse/CASSANDRA-6791

        Show
        Viktor Kuzmin added a comment - Bug with FileNotFoundException is not related to HsHa problem. And about several iterations for scrub: https://issues.apache.org/jira/browse/CASSANDRA-6791
        Hide
        Jonathan Ellis added a comment -

        According to http://mail-archives.apache.org/mod_mbox/cassandra-user/201402.mbox/%3C038601cf28ea$a2e504d0$e8af0e70$@struq.com%3E the 0.9 TThreadSelectorServer works well, although I'm not sure if he means that it performs better than 0.8 TTSS or just that it doesn't cause corruption.

        Show
        Jonathan Ellis added a comment - According to http://mail-archives.apache.org/mod_mbox/cassandra-user/201402.mbox/%3C038601cf28ea$a2e504d0$e8af0e70$@struq.com%3E the 0.9 TThreadSelectorServer works well, although I'm not sure if he means that it performs better than 0.8 TTSS or just that it doesn't cause corruption.
        Hide
        Viktor Kuzmin added a comment -

        I've tried to investigate problem with HsHaDistruptorServer, but with no luck. Telling the truth, I see no reason for that server to corrupt data.
        Also HsHaDistruptor do not corrupt data in case useHeapBasedAllocation is turned on.
        More over if you look at disruptor-thrift-server code - Message.reallocateDataBuffer and turn on heap based allocation only for dataBuffer then you will not see corruption.

        Show
        Viktor Kuzmin added a comment - I've tried to investigate problem with HsHaDistruptorServer, but with no luck. Telling the truth, I see no reason for that server to corrupt data. Also HsHaDistruptor do not corrupt data in case useHeapBasedAllocation is turned on. More over if you look at disruptor-thrift-server code - Message.reallocateDataBuffer and turn on heap based allocation only for dataBuffer then you will not see corruption.
        Hide
        Pavel Yaskevich added a comment -

        Nikolai Grigoriev and Brandon Kearby can you try setting useHeapBasedAllocation to "true" ? I'm fine with switch back to TThreadedSelectorServer if that helps.

        Show
        Pavel Yaskevich added a comment - Nikolai Grigoriev and Brandon Kearby can you try setting useHeapBasedAllocation to "true" ? I'm fine with switch back to TThreadedSelectorServer if that helps.
        Hide
        Nikolai Grigoriev added a comment -

        Pavel Yaskevich

        That seems to be a parameter of the Thrift server...How do I control this parameter? Or I should just disable JNA?

        Show
        Nikolai Grigoriev added a comment - Pavel Yaskevich That seems to be a parameter of the Thrift server...How do I control this parameter? Or I should just disable JNA?
        Hide
        Pavel Yaskevich added a comment -

        You can do it via JMX or disable JNA, I can also make a patch with would set it explicitly in Cassandra code.

        Show
        Pavel Yaskevich added a comment - You can do it via JMX or disable JNA, I can also make a patch with would set it explicitly in Cassandra code.
        Hide
        Christopher Wirt added a comment -

        [~Jonathan Ellis] [~Marcus Eriksson] That was my post from the user mail list.
        After our 1.2.14 -> 2.0.5 upgrade and failure to get the new HsHa stable in our system, we moved to using the thrift 0.9.1 TTSS with reasonable success. We've now been running for two weeks under a relatively high read load.
        We haven't seen any "DecoratedKey != ..." errors.
        We have seen some warnings on start up about SSTable rows being out of order.
        We have seen commit logs starting to build up with "All time blocked" on the FlushWriter incrementing.

        Performance comparisons might be a little unfair, but certainly our p95, p99 have overall improved.

        Obviously very keen to not be running a custom build of C*.

        Show
        Christopher Wirt added a comment - [~Jonathan Ellis] [~Marcus Eriksson] That was my post from the user mail list. After our 1.2.14 -> 2.0.5 upgrade and failure to get the new HsHa stable in our system, we moved to using the thrift 0.9.1 TTSS with reasonable success. We've now been running for two weeks under a relatively high read load. We haven't seen any "DecoratedKey != ..." errors. We have seen some warnings on start up about SSTable rows being out of order. We have seen commit logs starting to build up with "All time blocked" on the FlushWriter incrementing. Performance comparisons might be a little unfair, but certainly our p95, p99 have overall improved. Obviously very keen to not be running a custom build of C*.
        Hide
        Marcus Eriksson added a comment -

        Christopher Wirt could you paste those startup log lines?

        Show
        Marcus Eriksson added a comment - Christopher Wirt could you paste those startup log lines?
        Hide
        Christopher Wirt added a comment -

        These have since disappeared. I just restarted this node just now to check. We haven't run a scrub.

        WARN [main] 2014-02-16 23:23:02,032 LeveledManifest.java (line 171) At level 1, SSTableReader(path='/disk2/cassandra/data/struqrealtime/impressionstorev2/struqrealtime-impressionstorev2-jb-118905-Data.db') [DecoratedKey(-1513272878957942943, c41c955b40274acfa466ccb6079a21e5), DecoratedKey(6301362410765453237, 43ae61aacbc446be92c8bdea1d43e342)] overlaps SSTableReader(path='/disk2/cassandra/data/struqrealtime/impressionstorev2/struqrealtime-impressionstorev2-jb-116400-Data.db') [DecoratedKey(3953001739649874864, 5811ce41b7014917ab82eb32e8861ca5), DecoratedKey(9190609424240623933, 4e5b00a5a7594289924674974f44a995)]. This could be caused by a bug in Cassandra 1.1.0 .. 1.1.3 or due to the fact that you have dropped sstables from another node into the data directory. Sending back to L0. If you didn't drop in sstables, and have not yet run scrub, you should do so since you may also have rows out-of-order within an sstable

        Show
        Christopher Wirt added a comment - These have since disappeared. I just restarted this node just now to check. We haven't run a scrub. WARN [main] 2014-02-16 23:23:02,032 LeveledManifest.java (line 171) At level 1, SSTableReader(path='/disk2/cassandra/data/struqrealtime/impressionstorev2/struqrealtime-impressionstorev2-jb-118905-Data.db') [DecoratedKey(-1513272878957942943, c41c955b40274acfa466ccb6079a21e5), DecoratedKey(6301362410765453237, 43ae61aacbc446be92c8bdea1d43e342)] overlaps SSTableReader(path='/disk2/cassandra/data/struqrealtime/impressionstorev2/struqrealtime-impressionstorev2-jb-116400-Data.db') [DecoratedKey(3953001739649874864, 5811ce41b7014917ab82eb32e8861ca5), DecoratedKey(9190609424240623933, 4e5b00a5a7594289924674974f44a995)] . This could be caused by a bug in Cassandra 1.1.0 .. 1.1.3 or due to the fact that you have dropped sstables from another node into the data directory. Sending back to L0. If you didn't drop in sstables, and have not yet run scrub, you should do so since you may also have rows out-of-order within an sstable
        Hide
        Marcus Eriksson added a comment -

        Christopher Wirt those entries are unrelated (and fixed in CASSANDRA-6688)

        Show
        Marcus Eriksson added a comment - Christopher Wirt those entries are unrelated (and fixed in CASSANDRA-6688 )
        Hide
        Pavel Yaskevich added a comment -

        The patch sets heap based allocation by default in disruptor server, should make it easier for people to test that scenario...

        Show
        Pavel Yaskevich added a comment - The patch sets heap based allocation by default in disruptor server, should make it easier for people to test that scenario...
        Hide
        Jonathan Ellis added a comment -

        Michael Shuler Can you test hsha with Viktor's jar above? (https://issues.apache.org/jira/browse/CASSANDRA-6285?focusedCommentId=13917950&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13917950)

        I want to know if

        1. you can reproduce with a single node
        2. if not, if you can reproduce with multiple nodes
        3. assuming either 1 or 2, if you can still reproduce after applying Pavel's heap allocation path
        Show
        Jonathan Ellis added a comment - Michael Shuler Can you test hsha with Viktor's jar above? ( https://issues.apache.org/jira/browse/CASSANDRA-6285?focusedCommentId=13917950&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13917950 ) I want to know if you can reproduce with a single node if not, if you can reproduce with multiple nodes assuming either 1 or 2, if you can still reproduce after applying Pavel's heap allocation path
        Hide
        Michael Shuler added a comment -

        Sure - let me see what I can find out.

        Show
        Michael Shuler added a comment - Sure - let me see what I can find out.
        Hide
        Michael Shuler added a comment -

        6285_testnotes1.txt attached.

        Neither a single node with hsha, nor a 3 node ccm cluster with hsha gave me any interesting errors with the attack jar. Should I go back and try some of the previous repro steps and check yay/nay on the patch fixing this for those?

        Show
        Michael Shuler added a comment - 6285_testnotes1.txt attached. Neither a single node with hsha, nor a 3 node ccm cluster with hsha gave me any interesting errors with the attack jar. Should I go back and try some of the previous repro steps and check yay/nay on the patch fixing this for those?
        Hide
        Pavel Yaskevich added a comment - - edited

        Michael Shuler Can you try the same on the machine running Linux (if you haven't done that yet)?

        Edit: from the log it looks like Disruptor wasn't using the off-heap memory because JNA is disabled, "Off-heap allocation couldn't be used as JNA is not present in classpath or broken, using on-heap instead." So it would be great if you could test this on Linux with jna enabled.

        Thanks!

        Show
        Pavel Yaskevich added a comment - - edited Michael Shuler Can you try the same on the machine running Linux (if you haven't done that yet)? Edit: from the log it looks like Disruptor wasn't using the off-heap memory because JNA is disabled, "Off-heap allocation couldn't be used as JNA is not present in classpath or broken, using on-heap instead." So it would be great if you could test this on Linux with jna enabled. Thanks!
        Hide
        Michael Shuler added a comment - - edited

        I'm using a linux machine - and will link in JNA - good suggestion.

        Show
        Michael Shuler added a comment - - edited I'm using a linux machine - and will link in JNA - good suggestion.
        Hide
        Michael Shuler added a comment - - edited

        With jna enabled, yes, on a single node, after running the attack jar and restarting c*, I get:

         INFO [main] 2014-03-06 14:46:51,272 ColumnFamilyStore.java (line 254) Initializing tmp.CF
         INFO [main] 2014-03-06 14:46:51,277 ColumnFamilyStore.java (line 254) Initializing system_traces.sessions
         INFO [main] 2014-03-06 14:46:51,280 ColumnFamilyStore.java (line 254) Initializing system_traces.events
         INFO [main] 2014-03-06 14:46:51,281 CassandraDaemon.java (line 291) completed pre-loading (5 keys) key cache.
         INFO [main] 2014-03-06 14:46:51,288 CommitLog.java (line 130) Replaying /var/lib/cassandra/commitlog/CommitLog-3-1394138577628.log, /var/lib/
        cassandra/commitlog/CommitLog-3-1394138577629.log
         INFO [main] 2014-03-06 14:46:51,311 CommitLogReplayer.java (line 184) Replaying /var/lib/cassandra/commitlog/CommitLog-3-1394138577628.log (C
        L version 3, messaging version 7)
        ERROR [main] 2014-03-06 14:46:51,432 CommitLogReplayer.java (line 306) Unexpected error deserializing mutation; saved to /tmp/mutation77387084
        28696995512dat and ignored.  This may be caused by replaying a mutation against a table with the same name but incompatible schema.  Exception
         follows: 
        org.apache.cassandra.serializers.MarshalException: Invalid version for TimeUUID type.
                at org.apache.cassandra.serializers.TimeUUIDSerializer.validate(TimeUUIDSerializer.java:39)
                at org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:172)
                at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:276)
                at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:97)
                at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:151)
                at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:131)
                at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:312)
                at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:471)
                at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:560)
        

        I'll double-check a 3 node cluster, then patch and see where I get.

        (edit) this looks quite different than the previously posted errors - not sure if I'm on the right track, here..

        Show
        Michael Shuler added a comment - - edited With jna enabled, yes, on a single node, after running the attack jar and restarting c*, I get: INFO [main] 2014-03-06 14:46:51,272 ColumnFamilyStore.java (line 254) Initializing tmp.CF INFO [main] 2014-03-06 14:46:51,277 ColumnFamilyStore.java (line 254) Initializing system_traces.sessions INFO [main] 2014-03-06 14:46:51,280 ColumnFamilyStore.java (line 254) Initializing system_traces.events INFO [main] 2014-03-06 14:46:51,281 CassandraDaemon.java (line 291) completed pre-loading (5 keys) key cache. INFO [main] 2014-03-06 14:46:51,288 CommitLog.java (line 130) Replaying /var/lib/cassandra/commitlog/CommitLog-3-1394138577628.log, /var/lib/ cassandra/commitlog/CommitLog-3-1394138577629.log INFO [main] 2014-03-06 14:46:51,311 CommitLogReplayer.java (line 184) Replaying /var/lib/cassandra/commitlog/CommitLog-3-1394138577628.log (C L version 3, messaging version 7) ERROR [main] 2014-03-06 14:46:51,432 CommitLogReplayer.java (line 306) Unexpected error deserializing mutation; saved to /tmp/mutation77387084 28696995512dat and ignored. This may be caused by replaying a mutation against a table with the same name but incompatible schema. Exception follows: org.apache.cassandra.serializers.MarshalException: Invalid version for TimeUUID type. at org.apache.cassandra.serializers.TimeUUIDSerializer.validate(TimeUUIDSerializer.java:39) at org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:172) at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:276) at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:97) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:151) at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:131) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:312) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:471) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:560) I'll double-check a 3 node cluster, then patch and see where I get. (edit) this looks quite different than the previously posted errors - not sure if I'm on the right track, here..
        Hide
        Michael Shuler added a comment -

        Both a single local hsha node and 3x node ccm cluster with hsha (jna on both) throw the above errors after attack.jar run and restart. The patch does appear to fix both single and ccm cluster. My pre-patch ccm cluster never fully restarted, but do we need logs or anything from before/after?

        Show
        Michael Shuler added a comment - Both a single local hsha node and 3x node ccm cluster with hsha (jna on both) throw the above errors after attack.jar run and restart. The patch does appear to fix both single and ccm cluster. My pre-patch ccm cluster never fully restarted, but do we need logs or anything from before/after?
        Hide
        Pavel Yaskevich added a comment -

        I don't think we need logs, Jonathan Ellis I'm thinking of commiting attached patch which should help meanwhile I working on resolving off-heap problem, WDYT?

        Show
        Pavel Yaskevich added a comment - I don't think we need logs, Jonathan Ellis I'm thinking of commiting attached patch which should help meanwhile I working on resolving off-heap problem, WDYT?
        Hide
        Benedict added a comment - - edited

        So, I think there may potentially be at least two races in the off heap deallocation. I suspect that may not be everything, though, as these two races probably won't cause the problem often. These are predicated on the assumption that thrift doesn't copy data from the DirectByteBuffer that the hsha server provides to it, so could be wrong, but anyway:

        1) CL appends can be lagged behind the memtable update and, as a result, the acknowledgment to the client of success writing. If the CL record contains the ByteBuffer when it is freed, and that address is then reused in another allocation, it will write incorrect data to the commit log.
        2) I believe thrift calls are two stage. If this is the case, and the client disconnects in between sending the first stage and receiving the result in the second stage, the buffer could be freed whilst still in flight to the memtable/CL

        These are just quick ideas for where it might be, I haven't familiarised myself fully with thrift, the disruptor etc. to be certain if these are plausible, but it may turn out to be useful so thought I'd share.

        Show
        Benedict added a comment - - edited So, I think there may potentially be at least two races in the off heap deallocation. I suspect that may not be everything, though, as these two races probably won't cause the problem often. These are predicated on the assumption that thrift doesn't copy data from the DirectByteBuffer that the hsha server provides to it, so could be wrong, but anyway: 1) CL appends can be lagged behind the memtable update and, as a result, the acknowledgment to the client of success writing. If the CL record contains the ByteBuffer when it is freed, and that address is then reused in another allocation, it will write incorrect data to the commit log. 2) I believe thrift calls are two stage. If this is the case, and the client disconnects in between sending the first stage and receiving the result in the second stage, the buffer could be freed whilst still in flight to the memtable/CL These are just quick ideas for where it might be, I haven't familiarised myself fully with thrift, the disruptor etc. to be certain if these are plausible, but it may turn out to be useful so thought I'd share.
        Hide
        Rick Branson added a comment -

        Unfortunately this hit us during our 2.0.5 upgrade and 'sync' is not an option for the # of connections we have per node (tried this). We've been running Marcus' patch in prod and limping along on it, but it looks like the requestInvoke() override is causing the requests to get executed on the selector pool (which is limited by CPU #) instead of the executor service so our response times are pretty bad. The lack of anything showing up in the JMX for the executor service definitely points towards this.

        Show
        Rick Branson added a comment - Unfortunately this hit us during our 2.0.5 upgrade and 'sync' is not an option for the # of connections we have per node (tried this). We've been running Marcus' patch in prod and limping along on it, but it looks like the requestInvoke() override is causing the requests to get executed on the selector pool (which is limited by CPU #) instead of the executor service so our response times are pretty bad. The lack of anything showing up in the JMX for the executor service definitely points towards this.
        Hide
        Pavel Yaskevich added a comment -

        Rick Branson Which Marcus' patch are you talking about? Also I want to clarify one thing - disruptor server doesn't use requestInvoke(FrameBuffer) but dispatchInvoke(Message) which schedules message to executor pool based on ring buffer (WorkerPool) so actual execution is done in the separate thread. I can attach a patch which would switch back to TThreadedSelectorServer which is packed with Thrift (the only different between it and disruptor is that it schedules to classic thread pool), maybe disruptor server wasn't as good an idea for all of the real world use cases...

        Show
        Pavel Yaskevich added a comment - Rick Branson Which Marcus' patch are you talking about? Also I want to clarify one thing - disruptor server doesn't use requestInvoke(FrameBuffer) but dispatchInvoke(Message) which schedules message to executor pool based on ring buffer (WorkerPool) so actual execution is done in the separate thread. I can attach a patch which would switch back to TThreadedSelectorServer which is packed with Thrift (the only different between it and disruptor is that it schedules to classic thread pool), maybe disruptor server wasn't as good an idea for all of the real world use cases...
        Hide
        Benedict added a comment -

        It looks like thrift doesn't retain the DirectByteBuffer, just reads straight from them. The only possible window for corruption is during the construction of the thrift args object, which is a fairly narrow window.

        Show
        Benedict added a comment - It looks like thrift doesn't retain the DirectByteBuffer, just reads straight from them. The only possible window for corruption is during the construction of the thrift args object, which is a fairly narrow window.
        Hide
        Rick Branson added a comment -

        We put in the TThreadedSelectorServer patch from Marcus. On top of that, to get our response times down from the 10x what they should be, I rolled out a larger hard-coded selector thread pool size of 256 (instead of the # of processors – a measly 16). This is shaping up nicely.

        Show
        Rick Branson added a comment - We put in the TThreadedSelectorServer patch from Marcus. On top of that, to get our response times down from the 10x what they should be, I rolled out a larger hard-coded selector thread pool size of 256 (instead of the # of processors – a measly 16). This is shaping up nicely.
        Hide
        Pavel Yaskevich added a comment -

        Rick Branson So what you are saying is that after problem with disruptor you never tried it again with on-heap buffers but switched to TThreadedSelectorServer and increased selector pool size and the requestInvoke() is the problem with TThreadedSelectorServer?

        Show
        Pavel Yaskevich added a comment - Rick Branson So what you are saying is that after problem with disruptor you never tried it again with on-heap buffers but switched to TThreadedSelectorServer and increased selector pool size and the requestInvoke() is the problem with TThreadedSelectorServer?
        Hide
        Rick Branson added a comment -

        Pavel Yaskevich: We had perf issues with the disruptor as well (sudden spikes of CPU to 100%) + this so I just wanted to get production away from it ASAP.

        Show
        Rick Branson added a comment - Pavel Yaskevich : We had perf issues with the disruptor as well (sudden spikes of CPU to 100%) + this so I just wanted to get production away from it ASAP.
        Hide
        Pavel Yaskevich added a comment -

        Rick Branson But the most important question for this ticket at least is - did you run with on or off heap buffers? I can bring TThreadedSelectorServer back in this ticket or just go with on-heap buffers and disruptor. Michael Shuler do you have any performance tests related to Thrift server? Maybe there is a low hanging fruit in there to fix up the spikes that Rick mentioned if we can reproduce.

        Show
        Pavel Yaskevich added a comment - Rick Branson But the most important question for this ticket at least is - did you run with on or off heap buffers? I can bring TThreadedSelectorServer back in this ticket or just go with on-heap buffers and disruptor. Michael Shuler do you have any performance tests related to Thrift server? Maybe there is a low hanging fruit in there to fix up the spikes that Rick mentioned if we can reproduce.
        Hide
        Jonathan Ellis added a comment -

        I'm thinking of commiting attached patch which should help meanwhile I working on resolving off-heap problem

        Yeah, let's do this for now and roll 2.0.6 so we can stop the bleeding, then figure out whether doing more work on disruptor or TTSS is better for 2.0.7.

        Show
        Jonathan Ellis added a comment - I'm thinking of commiting attached patch which should help meanwhile I working on resolving off-heap problem Yeah, let's do this for now and roll 2.0.6 so we can stop the bleeding, then figure out whether doing more work on disruptor or TTSS is better for 2.0.7.
        Hide
        Rick Branson added a comment -

        Pavel Yaskevich: off-heap for disruptor.

        I think that we should really consider bringing back the old HSHA implementation from 1.2 as the "hsha" and allow switching to the disruptor implementation as another rpc_server_type for those that want to try it out.

        Show
        Rick Branson added a comment - Pavel Yaskevich : off-heap for disruptor. I think that we should really consider bringing back the old HSHA implementation from 1.2 as the "hsha" and allow switching to the disruptor implementation as another rpc_server_type for those that want to try it out.
        Hide
        Brandon Williams added a comment -

        I think that we should really consider bringing back the old HSHA implementation from 1.2 as the "hsha" and allow switching to the disruptor implementation as another rpc_server_type for those that want to try it out.

        I think I'm inclined to agree with this, aside from creating yaml creation problems in a minor. If we just give up and effectively revert, what have we lost? We need 2.0 stabilization sooner rather than later now, in all aspects. If we can't trust disruptor except with a small change, let's just not trust it yet and worry about that in a future release. With 2.1 beta already out, we can't tolerate much instability in the 2.0 branch.

        Show
        Brandon Williams added a comment - I think that we should really consider bringing back the old HSHA implementation from 1.2 as the "hsha" and allow switching to the disruptor implementation as another rpc_server_type for those that want to try it out. I think I'm inclined to agree with this, aside from creating yaml creation problems in a minor. If we just give up and effectively revert, what have we lost? We need 2.0 stabilization sooner rather than later now, in all aspects. If we can't trust disruptor except with a small change, let's just not trust it yet and worry about that in a future release. With 2.1 beta already out, we can't tolerate much instability in the 2.0 branch.
        Hide
        Pavel Yaskevich added a comment -

        I'm not sure if there is a point of going all the way back to original HsHa when there is TThrededSelectorServer, but I'm fine with going with disruptor as a separate option, something like "disruptor_hsha" and making "hsha" - TThrededSelectorServer from Thrift, that's how I wanted it originally. Also I just want to mention that people have reported that disruptor works for them with on-heap buffers, so I am not sure if we need to go all paranoid about this...

        Show
        Pavel Yaskevich added a comment - I'm not sure if there is a point of going all the way back to original HsHa when there is TThrededSelectorServer, but I'm fine with going with disruptor as a separate option, something like "disruptor_hsha" and making "hsha" - TThrededSelectorServer from Thrift, that's how I wanted it originally. Also I just want to mention that people have reported that disruptor works for them with on-heap buffers, so I am not sure if we need to go all paranoid about this...
        Hide
        Marcus Eriksson added a comment -

        My branch from above needed a tiny hack to thrift (https://github.com/krummas/thrift/commit/01ba2a3f3d386d0981371aab2494470e2a78e596), so if we want to roll with TTSS we should refactor our thrift usage a bit to avoid that hack

        Show
        Marcus Eriksson added a comment - My branch from above needed a tiny hack to thrift ( https://github.com/krummas/thrift/commit/01ba2a3f3d386d0981371aab2494470e2a78e596 ), so if we want to roll with TTSS we should refactor our thrift usage a bit to avoid that hack
        Hide
        Pavel Yaskevich added a comment -

        Ah, so they have finally made transport a protected field in FrameBuffer... Well, that considerably complicates things with switching back TThreadedSelectorServer.

        Show
        Pavel Yaskevich added a comment - Ah, so they have finally made transport a protected field in FrameBuffer... Well, that considerably complicates things with switching back TThreadedSelectorServer.
        Hide
        Sylvain Lebresne added a comment -

        Alright, I've committed Pavel's patch above as a stopgap solution as discussed above because I want to start a vote on 2.0.6 asap (the changelog is getting pretty big). I've created CASSANDRA-6815 to decide what we want the followup for that to be for 2.0.7.

        Show
        Sylvain Lebresne added a comment - Alright, I've committed Pavel's patch above as a stopgap solution as discussed above because I want to start a vote on 2.0.6 asap (the changelog is getting pretty big). I've created CASSANDRA-6815 to decide what we want the followup for that to be for 2.0.7.
        Hide
        Miles Shang added a comment - - edited

        To add to Rick Branson's input, we're also seeing the same stacktrace as Michael Shuler (TimeUUID MarshalException). I inspected the row mutations that caused it. Three ranges were nonsensical: the key, the column name, and the value. By nonsensical, I mean that they don't match my expectation of what we are inserting in production data. All other ranges seemed fine (timestamps, masks, sizes, cfid). The key, column name, and value were read successfully, so their length metadata was good. For our data, the column comparator is TimeUUID. Our client library is pycassa. Whereas pycassa generates tuuids like this: 913d7fea-a631-11e3-8080-808080808080, the nonsensical column names look like this: 22050aa4-de11-e380-8080-80808080800b and this: 10c326eb-86a4-e211-e380-808080808080. Most are of the first form. By shifting these nonsensical tuuids to the left or right by an octet, you get a reasonable tuuid. I don't have a similar insight into the nonsensical keys and values, but they could also be left or right shifted.

        Show
        Miles Shang added a comment - - edited To add to Rick Branson 's input, we're also seeing the same stacktrace as Michael Shuler (TimeUUID MarshalException). I inspected the row mutations that caused it. Three ranges were nonsensical: the key, the column name, and the value. By nonsensical, I mean that they don't match my expectation of what we are inserting in production data. All other ranges seemed fine (timestamps, masks, sizes, cfid). The key, column name, and value were read successfully, so their length metadata was good. For our data, the column comparator is TimeUUID. Our client library is pycassa. Whereas pycassa generates tuuids like this: 913d7fea-a631-11e3-8080-808080808080, the nonsensical column names look like this: 22050aa4-de11-e380-8080-80808080800b and this: 10c326eb-86a4-e211-e380-808080808080. Most are of the first form. By shifting these nonsensical tuuids to the left or right by an octet, you get a reasonable tuuid. I don't have a similar insight into the nonsensical keys and values, but they could also be left or right shifted.
        Hide
        Ryan McGuire added a comment -

        I'd like to be able to reproduce this in dtests to track this bug. Seeing as Russ Hatch's python test wasn't able to repro this issue, and a quick test I wrote doesn't either, does anyone have a simple way to reproduce this issue?

        Viktor Kuzmin would you mind sharing the source code for your attack jar?

        Show
        Ryan McGuire added a comment - I'd like to be able to reproduce this in dtests to track this bug. Seeing as Russ Hatch 's python test wasn't able to repro this issue, and a quick test I wrote doesn't either, does anyone have a simple way to reproduce this issue? Viktor Kuzmin would you mind sharing the source code for your attack jar?
        Hide
        Viktor Kuzmin added a comment - - edited

        Attached patches for on-heap disruptor.

        First pacth (disruptor-high-cpu.patch) turns off any key interests in case we're waiting for message to be processed. We need that cause processing may be delayed in case of high load and there may be something available to read from stream. In that case we'll have 100% cpu core usage.

        Second patch (disruptor-memory-corruption.patch) makes copy from off-heap ByteBuffer when reading binary data. This binary data may be stored inside cassandra as is even after message processing. And binary data can be corrupted - cause it's memory may be already deallocated.

        Show
        Viktor Kuzmin added a comment - - edited Attached patches for on-heap disruptor. First pacth (disruptor-high-cpu.patch) turns off any key interests in case we're waiting for message to be processed. We need that cause processing may be delayed in case of high load and there may be something available to read from stream. In that case we'll have 100% cpu core usage. Second patch (disruptor-memory-corruption.patch) makes copy from off-heap ByteBuffer when reading binary data. This binary data may be stored inside cassandra as is even after message processing. And binary data can be corrupted - cause it's memory may be already deallocated.
        Hide
        Viktor Kuzmin added a comment -

        Attached cassandra-attack-src.zip - eclipse project for making high load test on cassandra.
        This attack uses 100 threads to make writes, reads and deletes.

        Show
        Viktor Kuzmin added a comment - Attached cassandra-attack-src.zip - eclipse project for making high load test on cassandra. This attack uses 100 threads to make writes, reads and deletes.
        Hide
        Benedict added a comment - - edited

        Hmm. Just taking a look at Viktor's patch, I realised that my initial conclusions were actually quite plausible and probably (one of) the causes of the problem. When I dismissed them, I didn't realise we were using a custom TBinaryProtocol implementation. In particular (1) is definitely possible, and probably the cause of the issue, although the attack jar source would be helpful to figure out of there are any other potential causes. We should be able to force the problem to occur by artificially delaying the commit log write to prove this.

        Either way, I don't think Viktor's patch is the best way to deal with this problem, as it leaves cleaning up the direct buffers to GC. Since we could be creating a lot of these, we could create an awful lot of artificial memory pressure. Honestly, I think the best solution is to simply avoid using direct buffers with thrift, at least until 2.1, which should fix this problem by ensuring the CL write (if not commit) has happened before performing the memtable insertion.

        Show
        Benedict added a comment - - edited Hmm. Just taking a look at Viktor's patch, I realised that my initial conclusions were actually quite plausible and probably (one of) the causes of the problem. When I dismissed them, I didn't realise we were using a custom TBinaryProtocol implementation. In particular (1) is definitely possible, and probably the cause of the issue, although the attack jar source would be helpful to figure out of there are any other potential causes. We should be able to force the problem to occur by artificially delaying the commit log write to prove this. Either way, I don't think Viktor's patch is the best way to deal with this problem, as it leaves cleaning up the direct buffers to GC. Since we could be creating a lot of these, we could create an awful lot of artificial memory pressure. Honestly, I think the best solution is to simply avoid using direct buffers with thrift, at least until 2.1, which should fix this problem by ensuring the CL write (if not commit) has happened before performing the memtable insertion.
        Hide
        Viktor Kuzmin added a comment -

        My patch is defenetly NOT GOOD. Also for me that patch means anothe thing: it seems that we have 'success' answer before data is passed to commitlog... I don't think that this is good.

        Show
        Viktor Kuzmin added a comment - My patch is defenetly NOT GOOD. Also for me that patch means anothe thing: it seems that we have 'success' answer before data is passed to commitlog... I don't think that this is good.
        Hide
        Benedict added a comment -

        we have 'success' answer before data is passed to commitlog

        Yes, see my comment from a few days ago:

        1) CL appends can be lagged behind the memtable update and, as a result, the acknowledgment to the client of success writing. If the CL record contains the ByteBuffer when it is freed, and that address is then reused in another allocation, it will write incorrect data to the commit log.

        This is an absolutely plausible scenario since we do actually slice directly from the DirectByteBuffer, which I previously thought we did not.

        Show
        Benedict added a comment - we have 'success' answer before data is passed to commitlog Yes, see my comment from a few days ago: 1) CL appends can be lagged behind the memtable update and, as a result, the acknowledgment to the client of success writing. If the CL record contains the ByteBuffer when it is freed, and that address is then reused in another allocation, it will write incorrect data to the commit log. This is an absolutely plausible scenario since we do actually slice directly from the DirectByteBuffer, which I previously thought we did not.
        Hide
        Benedict added a comment -

        Has anybody tested this problem against 2.1? As if this is the only issue, it should be fixed there.

        Show
        Benedict added a comment - Has anybody tested this problem against 2.1? As if this is the only issue, it should be fixed there.
        Hide
        Ryan McGuire added a comment -

        Benedict I haven't yet been able to reproduce this with anything other than Viktor's attack jar. I'm thinking Java's threading is beating Python's threading here, so I created a dtest that just run's his jar directly. This test is currently passing on cassandra-2.0 and cassandra-2.1 HEAD.

        Show
        Ryan McGuire added a comment - Benedict I haven't yet been able to reproduce this with anything other than Viktor's attack jar. I'm thinking Java's threading is beating Python's threading here, so I created a dtest that just run's his jar directly. This test is currently passing on cassandra-2.0 and cassandra-2.1 HEAD.
        Hide
        Jonathan Ellis added a comment -

        You'd want to revert Pavel's patch from 2.1 to test Benedict's theory.

        Show
        Jonathan Ellis added a comment - You'd want to revert Pavel's patch from 2.1 to test Benedict's theory.
        Hide
        Viktor Kuzmin added a comment -

        You may set threads count to only one in cass-atack jar and you will be still able to reproduce error.

        Show
        Viktor Kuzmin added a comment - You may set threads count to only one in cass-atack jar and you will be still able to reproduce error.
        Hide
        Benedict added a comment -

        You'd want to revert Pavel's patch from 2.1

        beta1 should be fine to test against for this

        Show
        Benedict added a comment - You'd want to revert Pavel's patch from 2.1 beta1 should be fine to test against for this
        Hide
        Viktor Kuzmin added a comment -

        I've tried my test with beta1 and I can confirm that I was not able to reproduce bug.
        I think that it will be better to not use disruptor on 2.0.x even with on-heap allocation (we can still reuse buffer in case message will be of equal size when previous).
        And it should be safe to use disruptor on 2.1 branch.

        We'll be waiting for 2.1 release, cause it really impressed me over 2.0

        Show
        Viktor Kuzmin added a comment - I've tried my test with beta1 and I can confirm that I was not able to reproduce bug. I think that it will be better to not use disruptor on 2.0.x even with on-heap allocation (we can still reuse buffer in case message will be of equal size when previous). And it should be safe to use disruptor on 2.1 branch. We'll be waiting for 2.1 release, cause it really impressed me over 2.0
        Hide
        Viktor Kuzmin added a comment -

        Telling the truth, I don't think that this is really fixed in 2.0.6.
        It's not easy to reproduce bug right now, but I think it can be. thrift-disruptor server does not allocate new Buffer for new message in case new message is of equal size with previous. In that case bug can be reproduced even with on-heap allocation.

        Show
        Viktor Kuzmin added a comment - Telling the truth, I don't think that this is really fixed in 2.0.6. It's not easy to reproduce bug right now, but I think it can be. thrift-disruptor server does not allocate new Buffer for new message in case new message is of equal size with previous. In that case bug can be reproduced even with on-heap allocation.
        Hide
        Benedict added a comment -

        +1. That needs to be fixed as well.

        Show
        Benedict added a comment - +1. That needs to be fixed as well.
        Hide
        Robert Coli added a comment -

        ... Telling the truth, I don't think that this is really fixed in 2.0.6.

        If hsha is irrevocably broken with data corruption risk in 2.0 line, could we either get it wired off in the next point release, or some messaging in NEWS.txt that instructs people not to use it? My preference is the former to cover upgraders who are foolish enough to not read NEWS.txt; I am unable to see the benefit of leaving it usable if it is known broken.

        Show
        Robert Coli added a comment - ... Telling the truth, I don't think that this is really fixed in 2.0.6. If hsha is irrevocably broken with data corruption risk in 2.0 line, could we either get it wired off in the next point release, or some messaging in NEWS.txt that instructs people not to use it? My preference is the former to cover upgraders who are foolish enough to not read NEWS.txt; I am unable to see the benefit of leaving it usable if it is known broken.
        Hide
        Edward Capriolo added a comment -

        I read thought this. Does it make sense to call this HSHA2 and restore the old code and call it HSHA? I

        Show
        Edward Capriolo added a comment - I read thought this. Does it make sense to call this HSHA2 and restore the old code and call it HSHA? I
        Hide
        Rick Branson added a comment -

        This is not fixed. Still seeing the same exception running 2.0.6.

        ERROR [CompactionExecutor:7] 2014-05-09 17:59:58,640 CassandraDaemon.java (line 196) Exception in thread Thread[CompactionExecutor:7,1,main]
        java.lang.RuntimeException: Last written key DecoratedKey(132126721345628486111245439753727165857, 0f3b67f2) >= current key DecoratedKey(37424530135488872684523334498941679307, 196b70ab) writing into /data/cassandra/data/redacted/Redacted/redacted-Redacted-tmp-jb-156533-Data.db
        at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142)
        at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165)
        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$BackgroundCompactionTask.run(CompactionManager.java:197)
        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:744)

        Show
        Rick Branson added a comment - This is not fixed. Still seeing the same exception running 2.0.6. ERROR [CompactionExecutor:7] 2014-05-09 17:59:58,640 CassandraDaemon.java (line 196) Exception in thread Thread [CompactionExecutor:7,1,main] java.lang.RuntimeException: Last written key DecoratedKey(132126721345628486111245439753727165857, 0f3b67f2) >= current key DecoratedKey(37424530135488872684523334498941679307, 196b70ab) writing into /data/cassandra/data/redacted/Redacted/redacted-Redacted-tmp-jb-156533-Data.db at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165) 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$BackgroundCompactionTask.run(CompactionManager.java:197) 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:744)
        Show
        Brandon Williams added a comment - The line in question: https://github.com/xedin/disruptor_thrift_server/commit/77d6715af0eeba4c52f42fa6ba6549c8ae52ffa7#diff-18c889f19dc9fbeb73af99dcff152b6eR421
        Hide
        Brandon Williams added a comment -

        Patch to enable buffer reallocation.

        Show
        Brandon Williams added a comment - Patch to enable buffer reallocation.
        Hide
        Jonathan Ellis added a comment -

        I thought we were reallocating by default but I must have gotten that confused with on-heap buffers above. If Viktor is right, reusing buffers is always potentially dangerous and should just be removed. Can you comment, Pavel Yaskevich?

        Show
        Jonathan Ellis added a comment - I thought we were reallocating by default but I must have gotten that confused with on-heap buffers above. If Viktor is right, reusing buffers is always potentially dangerous and should just be removed. Can you comment, Pavel Yaskevich ?
        Hide
        Pavel Yaskevich added a comment -

        No, by default it's turned off, because Thrift side expectation is that once the invocation is complete nobody else holds the buffers, but it seems like the problem is that on Cassandra side we actually never copy the buffer for the commit log (or was it something else?). So we need to set thrift server to alwayReallocate explicitly.

        Rick Branson I can give you updated jar so you don't have to wait for the release of Cassandra which would have alwaysReallocate set to true by default.

        Show
        Pavel Yaskevich added a comment - No, by default it's turned off, because Thrift side expectation is that once the invocation is complete nobody else holds the buffers, but it seems like the problem is that on Cassandra side we actually never copy the buffer for the commit log (or was it something else?). So we need to set thrift server to alwayReallocate explicitly. Rick Branson I can give you updated jar so you don't have to wait for the release of Cassandra which would have alwaysReallocate set to true by default.
        Hide
        Pavel Yaskevich added a comment -

        So I can do two things, a). set alwaysReuse to true by default and release 0.3.5 today b). you can just switch to alwaysReallocate(true) in the configuration for 2.0.8, either works for me.

        Show
        Pavel Yaskevich added a comment - So I can do two things, a). set alwaysReuse to true by default and release 0.3.5 today b). you can just switch to alwaysReallocate(true) in the configuration for 2.0.8, either works for me.
        Hide
        Jonathan Ellis added a comment -

        Yeah, we do treat BB as immutable so CL would understandably not expect Thrift to pull the rug out from under it.

        I'm fine with calling alwaysReallocate on the Cassandra side in the interest of not changing things out from under any other users.

        Show
        Jonathan Ellis added a comment - Yeah, we do treat BB as immutable so CL would understandably not expect Thrift to pull the rug out from under it. I'm fine with calling alwaysReallocate on the Cassandra side in the interest of not changing things out from under any other users.
        Hide
        Brandon Williams added a comment -

        I have no issue with doing a) AND b), just to be extra safe, if we know this puts the nail in this ticket's coffin.

        Show
        Brandon Williams added a comment - I have no issue with doing a) AND b), just to be extra safe, if we know this puts the nail in this ticket's coffin.
        Hide
        Edward Capriolo added a comment -

        I was poking around the dependency a bit

        Running com.thinkaurelius.thrift.OffHeapMultiRequestTest
        Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 2.515 sec
        Running com.thinkaurelius.thrift.OnHeapMultiConnectionWithReallocateTest
        Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.481 sec
        Running com.thinkaurelius.thrift.OffheapMultiConnectionWithRellocateTest
        Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.478 sec
        Running com.thinkaurelius.thrift.OnHeapMultiConnectionTest
        Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.417 sec
        Running com.thinkaurelius.thrift.OnHeapMultiRequestWithReallocateTest
        Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.735 sec
        Running com.thinkaurelius.thrift.OffHeapMultiRequestWithReallocateTest
        Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.437 sec
        Running com.thinkaurelius.thrift.OffHeapMultiConnectionTest
        Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.282 sec
        Running com.thinkaurelius.thrift.OnHeapMultiRequestTest
        Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.491 sec

        Q. Are a few tests that run in roughly 20 seconds enough to prove that this component fit for production? These highly concurrent off heap systems can have very subtle bugs as this ticket shows.

        If I understand correct HSHA is not the default, the artifact has good test coverage, and only a handful of findbugs issues. Is there any piece that is going to run end-to-end or attempt to load/concurrently test these classes and be more rigorous then the previous system? Can that be made?

        Show
        Edward Capriolo added a comment - I was poking around the dependency a bit Running com.thinkaurelius.thrift.OffHeapMultiRequestTest Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 2.515 sec Running com.thinkaurelius.thrift.OnHeapMultiConnectionWithReallocateTest Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.481 sec Running com.thinkaurelius.thrift.OffheapMultiConnectionWithRellocateTest Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.478 sec Running com.thinkaurelius.thrift.OnHeapMultiConnectionTest Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.417 sec Running com.thinkaurelius.thrift.OnHeapMultiRequestWithReallocateTest Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.735 sec Running com.thinkaurelius.thrift.OffHeapMultiRequestWithReallocateTest Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.437 sec Running com.thinkaurelius.thrift.OffHeapMultiConnectionTest Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.282 sec Running com.thinkaurelius.thrift.OnHeapMultiRequestTest Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.491 sec Q. Are a few tests that run in roughly 20 seconds enough to prove that this component fit for production? These highly concurrent off heap systems can have very subtle bugs as this ticket shows. If I understand correct HSHA is not the default, the artifact has good test coverage, and only a handful of findbugs issues. Is there any piece that is going to run end-to-end or attempt to load/concurrently test these classes and be more rigorous then the previous system? Can that be made?
        Hide
        Pavel Yaskevich added a comment -

        Brandon Williams I have released 0.3.5 just now, with reallocation and on-heap buffers turned on by default. +1 on the change so it's either we commit 0.3.5 or your patch.

        Edward Capriolo Here is a definition of unit testing, in my tests I cover single/multi connection on-heap/off-heap +/- reallocation scenarios, basically everything to prove that server functions properly in all of the modes and returns correct results based on the operation being used. end-to-end tests are what systems which integrate project are supposed to do and that is done by stress and bpdlab testing. If you have been following discussion in this ticket you must have already realized that the problem is not caused by HsHa server directly but rather by the fact that Cassandra holds Thrift buffers even after blocking evaluation is finished.

        Show
        Pavel Yaskevich added a comment - Brandon Williams I have released 0.3.5 just now, with reallocation and on-heap buffers turned on by default. +1 on the change so it's either we commit 0.3.5 or your patch. Edward Capriolo Here is a definition of unit testing , in my tests I cover single/multi connection on-heap/off-heap +/- reallocation scenarios, basically everything to prove that server functions properly in all of the modes and returns correct results based on the operation being used. end-to-end tests are what systems which integrate project are supposed to do and that is done by stress and bpdlab testing. If you have been following discussion in this ticket you must have already realized that the problem is not caused by HsHa server directly but rather by the fact that Cassandra holds Thrift buffers even after blocking evaluation is finished.
        Hide
        Pavel Yaskevich added a comment -

        Brandon Williams 0.3.5 is already available on maven central so act as you think appropriate.

        Show
        Pavel Yaskevich added a comment - Brandon Williams 0.3.5 is already available on maven central so act as you think appropriate.
        Hide
        Edward Capriolo added a comment -

        @Pavel I understand what you are saying. I understand what unit test is. Whenever I submit patches to the ASF they come with tests. ::cough:: ::cough::. In any case, what I was saying is the external dependency does not do load testing. Cassandra does not default to hsha. I DO NOT see any reference in this conversation into how exactly the HSHA server is now load/correctness tested. If such a test exists great, if not potentially should be added.

        Show
        Edward Capriolo added a comment - @Pavel I understand what you are saying. I understand what unit test is. Whenever I submit patches to the ASF they come with tests. ::cough:: ::cough::. In any case, what I was saying is the external dependency does not do load testing. Cassandra does not default to hsha. I DO NOT see any reference in this conversation into how exactly the HSHA server is now load/correctness tested. If such a test exists great, if not potentially should be added.
        Hide
        Edward Capriolo added a comment -

        Also while we are on the topic.

        1. The default is sync because on Windows hsha is about 30% slower. On Linux,
        2. sync/hsha performance is about the same, with hsha of course using less memory.
          #
        3. Alternatively, can provide your own RPC server by providing the fully-qualified class name
        4. of an o.a.c.t.TServerFactory that can create an instance of it.
          rpc_server_type: sync

        The logic behind this default confuses me. The the vast majority of the cassandra user base is linux. We chose 'sync' so the uncommon case is not slowed down. Clearly anyone using linux should switch to hsha because it uses less memory and is wiched fast according to github tests. But not being the default does it really get performance/correctness evaluated in any meaningful way?

        Show
        Edward Capriolo added a comment - Also while we are on the topic. The default is sync because on Windows hsha is about 30% slower. On Linux, sync/hsha performance is about the same, with hsha of course using less memory. # Alternatively, can provide your own RPC server by providing the fully-qualified class name of an o.a.c.t.TServerFactory that can create an instance of it. rpc_server_type: sync The logic behind this default confuses me. The the vast majority of the cassandra user base is linux. We chose 'sync' so the uncommon case is not slowed down. Clearly anyone using linux should switch to hsha because it uses less memory and is wiched fast according to github tests. But not being the default does it really get performance/correctness evaluated in any meaningful way?
        Hide
        Benedict added a comment -

        For many workloads sync is faster than async on linux also (by a significant margin), so perhaps the docs should be updated.

        Show
        Benedict added a comment - For many workloads sync is faster than async on linux also (by a significant margin), so perhaps the docs should be updated.
        Hide
        Rick Branson added a comment -

        I think what might help this specific quality issue out is just moving to the new HSHA implementation entirely in a later version and removing the choice. The new HSHA supposedly eliminates the performance issues that made it not a good default choice, so it appears as if there's no advantage to having the other choices.

        Show
        Rick Branson added a comment - I think what might help this specific quality issue out is just moving to the new HSHA implementation entirely in a later version and removing the choice. The new HSHA supposedly eliminates the performance issues that made it not a good default choice, so it appears as if there's no advantage to having the other choices.
        Hide
        Brandon Williams added a comment -

        For 2.1, I can get behind that I think, especially calling it 'disruptor' or pretty much anything besides 'HSHA.' For 2.0 though it's hard to swallow in a minor.

        Show
        Brandon Williams added a comment - For 2.1, I can get behind that I think, especially calling it 'disruptor' or pretty much anything besides 'HSHA.' For 2.0 though it's hard to swallow in a minor.
        Hide
        Rick Branson added a comment -

        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.

        Show
        Rick Branson added a comment - 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.
        Hide
        Rick Branson added a comment -

        Haven't been able to repro in over 5 days. We're considering the enable_reallocate_buffers.txt patch fixed and production-ready.

        Show
        Rick Branson added a comment - Haven't been able to repro in over 5 days. We're considering the enable_reallocate_buffers.txt patch fixed and production-ready.
        Hide
        Brandon Williams added a comment - - edited

        I committed this patch to 2.0, but did not update the disruptor jar for fear of any further regressions, so the patch Rick tested is in there. For 2.1, I committed both this patch and disruptor 0.3.5.

        Show
        Brandon Williams added a comment - - edited I committed this patch to 2.0, but did not update the disruptor jar for fear of any further regressions, so the patch Rick tested is in there. For 2.1, I committed both this patch and disruptor 0.3.5.
        Hide
        Brandon Williams added a comment -

        Oops, wait, I only changed the maven dependency. Mikhail Stepura could you clean up the 2.1+ side of things?

        Show
        Brandon Williams added a comment - Oops, wait, I only changed the maven dependency. Mikhail Stepura could you clean up the 2.1+ side of things?
        Hide
        Mikhail Stepura added a comment -
        Show
        Mikhail Stepura added a comment - Brandon Williams done.
        Hide
        Brandon Williams added a comment -

        Thanks.

        Show
        Brandon Williams added a comment - Thanks.
        Hide
        Alexander Sterligov added a comment - - edited

        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.

        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]

        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?

        Show
        Alexander Sterligov added a comment - - edited 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. 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] 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?
        Hide
        Nikolai Grigoriev added a comment -

        Alexander Sterligov I was always wondering why did I always see these problems appearing for OpsCenter keyspace. My keyspace had much more traffic but when I had this problem - it always manifested itself with OpsCenter keyspace. Even when I was also using Thrift (we use native protocol now).

        I even remember disabling OpsCenter to prove the point

        Show
        Nikolai Grigoriev added a comment - Alexander Sterligov I was always wondering why did I always see these problems appearing for OpsCenter keyspace. My keyspace had much more traffic but when I had this problem - it always manifested itself with OpsCenter keyspace. Even when I was also using Thrift (we use native protocol now). I even remember disabling OpsCenter to prove the point
        Hide
        Alexander Sterligov added a comment - - edited

        Have you proven that it's really related to OpsCenter?

        We've switched to "sync", but still get corrupted sstables. Now we get exception not during compaction, but at start:

        ERROR [SSTableBatchOpen:10] 2014-10-22 02:47:48,762 CassandraDaemon.java:166 - Exception in thread Thread[SSTableBatchOpen:10,5,main]
        java.lang.IllegalStateException: SSTable first key DecoratedKey(4206305143314087741, 800100010000000c62617463685f6d7574617465000010250d00010b0d000000010000004e33372e3134302e3134312e3231322d6973732d736c6f745f636f6e66696775726174696f6e5f746172) > last key DecoratedKey(-4632241097675266745, 800100010000000c62617463685f6d7574617465000010260d00010b0d000000010000005133372e3134302e3134312e3231322d6973732d736c6f745f636f6e66696775726174696f6e5f746172676574)
        at org.apache.cassandra.io.sstable.SSTableReader.validate(SSTableReader.java:1083) ~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:398) ~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:294) ~[apache-cassandra-2.1.0.jar:2.1.0]
        at org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:430) ~[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]

        And nodetools scrub doesn't help. It finds no errors and after restart we get same exceptions.

        Show
        Alexander Sterligov added a comment - - edited Have you proven that it's really related to OpsCenter? We've switched to "sync", but still get corrupted sstables. Now we get exception not during compaction, but at start: ERROR [SSTableBatchOpen:10] 2014-10-22 02:47:48,762 CassandraDaemon.java:166 - Exception in thread Thread [SSTableBatchOpen:10,5,main] java.lang.IllegalStateException: SSTable first key DecoratedKey(4206305143314087741, 800100010000000c62617463685f6d7574617465000010250d00010b0d000000010000004e33372e3134302e3134312e3231322d6973732d736c6f745f636f6e66696775726174696f6e5f746172) > last key DecoratedKey(-4632241097675266745, 800100010000000c62617463685f6d7574617465000010260d00010b0d000000010000005133372e3134302e3134312e3231322d6973732d736c6f745f636f6e66696775726174696f6e5f746172676574) at org.apache.cassandra.io.sstable.SSTableReader.validate(SSTableReader.java:1083) ~ [apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:398) ~ [apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:294) ~ [apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:430) ~ [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] And nodetools scrub doesn't help. It finds no errors and after restart we get same exceptions.
        Hide
        Nikolai Grigoriev added a comment -

        I think this is the error that you cannot fix by scrubbing. Corrupted sstable. I was fixing those by deleting the sstables and doing repairs. Unfortunately, if that happens on many nodes there is a risk of data loss.

        As for the OpsCenter - do not get me wrong I did not want to say that OpsCenter was directly responsible for these troubles. But I do believe that OpsCenter does something particular that reveals the bug in hsha server. At least this was my impression. After disabling OpsCenter and fixing the outstanding problems I do not recall seeing those errors anymore. And I was also using Thrift and I was writing and reading 100x more data than OpsCenter.

        Show
        Nikolai Grigoriev added a comment - I think this is the error that you cannot fix by scrubbing. Corrupted sstable. I was fixing those by deleting the sstables and doing repairs. Unfortunately, if that happens on many nodes there is a risk of data loss. As for the OpsCenter - do not get me wrong I did not want to say that OpsCenter was directly responsible for these troubles. But I do believe that OpsCenter does something particular that reveals the bug in hsha server. At least this was my impression. After disabling OpsCenter and fixing the outstanding problems I do not recall seeing those errors anymore. And I was also using Thrift and I was writing and reading 100x more data than OpsCenter.
        Hide
        Pavel Yaskevich added a comment -

        Alexander Sterligov Did you get any WARN messages like this "N out of order rows found while scrubbing <file>; Those have been written (in order) to a new sstable <new-file>" while running scrub? Anyhow, you will have to delete affected files and repair from the neighbors, I'm also not sure how much of an involvement Thrift has in this because the only thing that could go wrong (shared buffers) was already fixed to be copied for every request and everything is allocated on-heap....

        Rick Branson Are you running HsHa with 2.1 or still on 2.0 ?

        Show
        Pavel Yaskevich added a comment - Alexander Sterligov Did you get any WARN messages like this "N out of order rows found while scrubbing <file>; Those have been written (in order) to a new sstable <new-file>" while running scrub? Anyhow, you will have to delete affected files and repair from the neighbors, I'm also not sure how much of an involvement Thrift has in this because the only thing that could go wrong (shared buffers) was already fixed to be copied for every request and everything is allocated on-heap.... Rick Branson Are you running HsHa with 2.1 or still on 2.0 ?
        Hide
        Alexander Sterligov added a comment - - edited

        Pavel Yaskevich No, I've not seen such messages. sstablescrub failed with NPE. sstables were corrupted on all 17 nodes. I removed them manually and there was no errors overnight. It seems sync really impacted the problem. Maybe there are some another problem which hides with sync server. I still have problems - validation hangs on one table on all nodes .

        Show
        Alexander Sterligov added a comment - - edited Pavel Yaskevich No, I've not seen such messages. sstablescrub failed with NPE. sstables were corrupted on all 17 nodes. I removed them manually and there was no errors overnight. It seems sync really impacted the problem. Maybe there are some another problem which hides with sync server. I still have problems - validation hangs on one table on all nodes .
        Hide
        Nikolai Grigoriev added a comment -

        By the way, I am getting

        ERROR [CompactionExecutor:2333] 2014-10-23 18:29:53,590 CassandraDaemon.java (line 199) Exception in thread Thread[Compactio
        nExecutor:2333,1,main]
        java.lang.RuntimeException: Last written key DecoratedKey(1156541975678546868, 001000000000111100000000000003bc510f000010000
        0000003bc510f00000000111100000000100000000000004000000000000000000100) >= current key DecoratedKey(36735936098318717, 001000
        0000001111000000000000015feb8a00001000000000015feb8a00000000111100000000100000000000004000000000000000000100) writing into /
        cassandra-data/disk2/myks/mytable/myks-mytable-tmp-jb-94445-Data.db
                at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142)
                at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165)
                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$BackgroundCompactionTask.run(CompactionManager.java:198)
                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:745)
        

        with 2.0.10 release. I am using native protocol. I believe native protocol handler is based on HSHA, am I right? Anyway, I am getting those too.

        Show
        Nikolai Grigoriev added a comment - By the way, I am getting ERROR [CompactionExecutor:2333] 2014-10-23 18:29:53,590 CassandraDaemon.java (line 199) Exception in thread Thread [Compactio nExecutor:2333,1,main] java.lang.RuntimeException: Last written key DecoratedKey(1156541975678546868, 001000000000111100000000000003bc510f000010000 0000003bc510f00000000111100000000100000000000004000000000000000000100) >= current key DecoratedKey(36735936098318717, 001000 0000001111000000000000015feb8a00001000000000015feb8a00000000111100000000100000000000004000000000000000000100) writing into / cassandra-data/disk2/myks/mytable/myks-mytable-tmp-jb-94445-Data.db at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165) 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$BackgroundCompactionTask.run(CompactionManager.java:198) 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:745) with 2.0.10 release. I am using native protocol. I believe native protocol handler is based on HSHA, am I right? Anyway, I am getting those too.
        Hide
        Pavel Yaskevich added a comment -

        Nikolai Grigoriev No, native protocol is not using Thrift, which further confirms that it's cross transport problem, I think we should create a separate ticket to handle it.
        Alexander Sterligov Do you still have stacktrace for the NPE you've got while scrubbing?

        Show
        Pavel Yaskevich added a comment - Nikolai Grigoriev No, native protocol is not using Thrift, which further confirms that it's cross transport problem, I think we should create a separate ticket to handle it. Alexander Sterligov Do you still have stacktrace for the NPE you've got while scrubbing?
        Hide
        Alexander Sterligov added a comment - - edited

        Pavel Yaskevich That NPE happend once and unfortunatelly I have not saved it. If I'll get it once more I'll save this sstable.
        I totally removed OpsCenter keyspace (with sstables) and recreated them. I don't get "Last written key DecoratedKey" any more. By the way, this error definetely causees streams to hang on 100%.

        I have several strange things happening now:

        • I've noticed that it takes about 30 minutes between "nodetool repair" and first pending AntiEntropySession. Is that ok?
        • Repair is already running for 24 hours (~13GB per node, 17 nodes). What's the number of AntiEntropySessions to finish single repair? Number of key ranges?

          Pool Name Active Pending Completed Blocked All time blocked
          CounterMutationStage 0 0 0 0 0
          ReadStage 0 0 392196 0 0
          RequestResponseStage 0 0 5271906 0 0
          MutationStage 0 0 19832506 0 0
          ReadRepairStage 0 0 2280 0 0
          GossipStage 0 0 453830 0 0
          CacheCleanupExecutor 0 0 0 0 0
          MigrationStage 0 0 0 0 0
          ValidationExecutor 0 0 39446 0 0
          MemtableReclaimMemory 0 0 29927 0 0
          InternalResponseStage 0 0 588279 0 0
          AntiEntropyStage 0 0 5325285 0 0
          MiscStage 0 0 0 0 0
          CommitLogArchiver 0 0 0 0 0
          MemtableFlushWriter 0 0 29927 0 0
          PendingRangeCalculator 0 0 30 0 0
          MemtablePostFlush 0 0 135734 0 0
          CompactionExecutor 31 31 502175 0 0
          AntiEntropySessions 3 3 3446 0 0
          HintedHandoff 0 0 44 0 0

          Message type Dropped
          RANGE_SLICE 0
          READ_REPAIR 0
          PAGED_RANGE 0
          BINARY 0
          READ 0
          MUTATION 2
          _TRACE 0
          REQUEST_RESPONSE 0
          COUNTER_MUTATION 0

        • Some validation compactions run for more than 100% (1923%). I thinks that it's CASSANDRA-7239, right?
        • the amount of sstables for some CFs is about 15 000 and continues to grow during repair.
        • There are several following exceptions during repair

          ERROR [RepairJobTask:80] 2014-10-24 13:27:31,717 RepairJob.java:127 - Error occurred during snapshot phase
          java.lang.RuntimeException: Could not create snapshot at /37.140.189.163
          at org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:77) ~[apache-cassandra-2.1.0.jar:2.1.0]
          at org.apache.cassandra.net.MessagingService$5$1.run(MessagingService.java:347) ~[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]
          ERROR [AntiEntropySessions:141] 2014-10-24 13:27:31,724 RepairSession.java:303 - repair #da2cb020-5b5f-11e4-a45e-d9cec1206f33 session completed with the following error
          java.io.IOException: Failed during snapshot creation.
          at org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344) ~[apache-cassandra-2.1.0.jar:2.1.0]
          at org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:128) ~[apache-cassandra-2.1.0.jar:2.1.0]
          at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) ~[guava-16.0.jar:na]
          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]
          ERROR [AntiEntropySessions:141] 2014-10-24 13:27:31,724 CassandraDaemon.java:166 - Exception in thread Thread[AntiEntropySessions:141,5,RMI Runtime]
          java.lang.RuntimeException: java.io.IOException: Failed during snapshot creation.
          at com.google.common.base.Throwables.propagate(Throwables.java:160) ~[guava-16.0.jar:na]
          at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) ~[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]
          Caused by: java.io.IOException: Failed during snapshot creation.
          at org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344) ~[apache-cassandra-2.1.0.jar:2.1.0]
          at org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:128) ~[apache-cassandra-2.1.0.jar:2.1.0]
          at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) ~[guava-16.0.jar:na]
          ... 3 common frames omitted

        Show
        Alexander Sterligov added a comment - - edited Pavel Yaskevich That NPE happend once and unfortunatelly I have not saved it. If I'll get it once more I'll save this sstable. I totally removed OpsCenter keyspace (with sstables) and recreated them. I don't get "Last written key DecoratedKey" any more. By the way, this error definetely causees streams to hang on 100%. I have several strange things happening now: I've noticed that it takes about 30 minutes between "nodetool repair" and first pending AntiEntropySession. Is that ok? Repair is already running for 24 hours (~13GB per node, 17 nodes). What's the number of AntiEntropySessions to finish single repair? Number of key ranges? Pool Name Active Pending Completed Blocked All time blocked CounterMutationStage 0 0 0 0 0 ReadStage 0 0 392196 0 0 RequestResponseStage 0 0 5271906 0 0 MutationStage 0 0 19832506 0 0 ReadRepairStage 0 0 2280 0 0 GossipStage 0 0 453830 0 0 CacheCleanupExecutor 0 0 0 0 0 MigrationStage 0 0 0 0 0 ValidationExecutor 0 0 39446 0 0 MemtableReclaimMemory 0 0 29927 0 0 InternalResponseStage 0 0 588279 0 0 AntiEntropyStage 0 0 5325285 0 0 MiscStage 0 0 0 0 0 CommitLogArchiver 0 0 0 0 0 MemtableFlushWriter 0 0 29927 0 0 PendingRangeCalculator 0 0 30 0 0 MemtablePostFlush 0 0 135734 0 0 CompactionExecutor 31 31 502175 0 0 AntiEntropySessions 3 3 3446 0 0 HintedHandoff 0 0 44 0 0 Message type Dropped RANGE_SLICE 0 READ_REPAIR 0 PAGED_RANGE 0 BINARY 0 READ 0 MUTATION 2 _TRACE 0 REQUEST_RESPONSE 0 COUNTER_MUTATION 0 Some validation compactions run for more than 100% (1923%). I thinks that it's CASSANDRA-7239 , right? the amount of sstables for some CFs is about 15 000 and continues to grow during repair. There are several following exceptions during repair ERROR [RepairJobTask:80] 2014-10-24 13:27:31,717 RepairJob.java:127 - Error occurred during snapshot phase java.lang.RuntimeException: Could not create snapshot at /37.140.189.163 at org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:77) ~ [apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.net.MessagingService$5$1.run(MessagingService.java:347) ~ [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] ERROR [AntiEntropySessions:141] 2014-10-24 13:27:31,724 RepairSession.java:303 - repair #da2cb020-5b5f-11e4-a45e-d9cec1206f33 session completed with the following error java.io.IOException: Failed during snapshot creation. at org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344) ~ [apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:128) ~ [apache-cassandra-2.1.0.jar:2.1.0] at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) ~ [guava-16.0.jar:na] 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] ERROR [AntiEntropySessions:141] 2014-10-24 13:27:31,724 CassandraDaemon.java:166 - Exception in thread Thread [AntiEntropySessions:141,5,RMI Runtime] java.lang.RuntimeException: java.io.IOException: Failed during snapshot creation. at com.google.common.base.Throwables.propagate(Throwables.java:160) ~ [guava-16.0.jar:na] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) ~ [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] Caused by: java.io.IOException: Failed during snapshot creation. at org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344) ~ [apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:128) ~ [apache-cassandra-2.1.0.jar:2.1.0] at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) ~ [guava-16.0.jar:na] ... 3 common frames omitted
        Hide
        Marcus Eriksson added a comment -

        I think the cause of the latest exceptions in this ticket is CASSANDRA-8211

        Show
        Marcus Eriksson added a comment - I think the cause of the latest exceptions in this ticket is CASSANDRA-8211
        Hide
        Randy Fradin added a comment - - edited

        I am getting this exception using Thrift HSHA in 2.1.0:

        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]

        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:

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

        EDIT: I copied my comment to a new issue (CASSANDRA-8719) since this issue one is long closed

        Show
        Randy Fradin added a comment - - edited I am getting this exception using Thrift HSHA in 2.1.0: 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] 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: 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... EDIT: I copied my comment to a new issue ( CASSANDRA-8719 ) since this issue one is long closed

          People

          • Assignee:
            Pavel Yaskevich
            Reporter:
            David Sauer
            Reviewer:
            Brandon Williams
            Tester:
            Rick Branson
          • Votes:
            4 Vote for this issue
            Watchers:
            34 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development