[ https://issues.apache.org/jira/browse/CASSANDRA-7465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14049092#comment-14049092 ]
Jason Brown commented on CASSANDRA-7465: ---------------------------------------- applies [~tjake]'s patch, reran stress on my cluster, and no longer see the DecoratedKey error +1 on the patch > DecoratedKey assertion error on reads > ------------------------------------- > > Key: CASSANDRA-7465 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7465 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: 3 nodes > Oracle Linux Server 6.3 > kernel ver 2.6.39 > Reporter: Jason Brown > Assignee: T Jake Luciani > Priority: Blocker > Fix For: 2.1.0 > > Attachments: 7465.txt > > > Getting the following exception when running read stress: > {code}WARN [SharedPool-Worker-31] 2014-06-27 21:25:51,391 > AbstractTracingAwareExecutorService.java:166 - Uncaught exception on thread > Thread[SharedPool-Worker-31,10,main]: {} > java.lang.AssertionError: DecoratedKey(-5397116645141815707, > 30303031393143364639) != DecoratedKey(-5397116645141815707, > 30303031343439443233) in > /u/sdd/cassandra-jasobrown/data/Keyspace1/Standard1-6ab9bd90fe3b11e385edff96c2ef2fd6/Keyspace1-Standard1-ka-73-Data.db > at > org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:111) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init>(SSTableNamesIterator.java:59) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:89) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > org.apache.cassandra.db.CollationController.collectTimeOrderedData(CollationController.java:125) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:59) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1873) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1681) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:341) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.java:55) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1393) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2060) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) > ~[na:1.7.0_13] > at > org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:162) > ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) > [apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] > at java.lang.Thread.run(Thread.java:722) [na:1.7.0_13] > {code} > I have a three node cluster, which I populate with the following stress > command: > {code} > cassandra-stress write n=60000000 -schema replication\(factor\=2\) -key > populate=1..60000000 -rate threads=42 -mode native prepared cql3 -port > native=9043 thrift=9161 -node athena06-a,athena06-b,athena06-c -col > n=fixed\(21\) size=exp\(11..42\) > {code} > Then I run the read stress: > {code} > cassandra-stress read n=100000000 -key dist=extr\(1..600000000,2\) -mode > native prepared cql3 -port native=9043 thrift=9161 -node > athena06-b,athena06-c,athena06-a -col n=fixed\(21\) -rate threads=64 > {code} > The above exception occurs semi-frequently (several to ~50 times a minute, > but seems to depend on amount of data in cluster - anecdotal evidence only). -- This message was sent by Atlassian JIRA (v6.2#6252)