[ https://issues.apache.org/jira/browse/CASSANDRA-11299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15199235#comment-15199235 ]
Julien Anguenot commented on CASSANDRA-11299: --------------------------------------------- I am actually experiencing the same issue after a 2.2.5 to 3.0.4 migration Note, I have not yet ran any repairs since my cluster, after migration of one single DC to 3.0.4, is still in a mixed state. I cannot finish the actual migration until this error is gone. I am currently running sstablesupgrade on the 3.0.4 nodes and will try a scrub of these tables causing problems. Will report if it is helping any as soon as I can. [~mmatloka] did you get around that bug somehow? {code} WARN [SharedPool-Worker-3] 2016-03-17 09:40:25,648 AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread Thread[SharedPool-Worker-3,5,main]: {} java.lang.AssertionError: null at org.apache.cassandra.index.internal.composites.CompositesSearcher$1Transform.findEntry(CompositesSearcher.java:225) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.index.internal.composites.CompositesSearcher$1Transform.applyToRow(CompositesSearcher.java:215) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:116) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:131) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:87) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:77) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:300) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:134) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:127) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:123) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:65) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:289) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:47) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:67) ~[apache-cassandra-3.0.4.jar:3.0.4] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_74] at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) ~[apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136) [apache-cassandra-3.0.4.jar:3.0.4] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [apache-cassandra-3.0.4.jar:3.0.4] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_74] {code} > AssertionError when quering by secondary index > ---------------------------------------------- > > Key: CASSANDRA-11299 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11299 > Project: Cassandra > Issue Type: Bug > Environment: Cassandra 3.3 > Reporter: Michał Matłoka > > Hi, > Recently we have upgraded from Cassandra 2.2.4 to 3.3. I have issues with one > table. When I try to query using any secondary index I get e.g. in cqlsh > {code} > Traceback (most recent call last): > File "/usr/bin/cqlsh.py", line 1249, in perform_simple_statement > result = future.result() > File > "/usr/share/cassandra/lib/cassandra-driver-internal-only-3.0.0-6af642d.zip/cassandra-driver-3.0.0-6af642d/cassandra/cluster.py", > line 3122, in result > raise self._final_exception > ReadFailure: code=1300 [Replica(s) failed to execute read] message="Operation > failed - received 0 responses and 1 failures" info={'failures': 1, > 'received_responses': 0, 'required_responses': 1, 'consistency': 'ONE'} > {code} > Node logs shows then: > {code} > [[AWARN [SharedPool-Worker-2] 2016-03-03 00:47:01,679 > AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread > Thread[SharedPool-Worker-2,5,main]: {} > java.lang.AssertionError: null > at > org.apache.cassandra.index.internal.composites.CompositesSearcher$1Transform.findEntry(CompositesSearcher.java:225) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.index.internal.composites.CompositesSearcher$1Transform.applyToRow(CompositesSearcher.java:215) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:116) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:133) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:89) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:79) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:294) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:134) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:127) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.db.ReadResponse$LocalDataResponse.<init>(ReadResponse.java:123) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:65) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:292) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1789) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2457) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_66] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) > ~[apache-cassandra-3.3.0.jar:3.3.0] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136) > [apache-cassandra-3.3.0.jar:3.3.0] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) > [apache-cassandra-3.3.0.jar:3.3.0] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66] > {code} > SStables are upgraded, I have tried repair and scrub. I have tried to rebuild > indexes, and even remove them and re-add them.It occurs on every cluster node. > Additionally I had seen in this table case where PRIMARY KEY was > duplicated!!! (there were two rows with same primary key, by seeing what > columns were set I can say one was older, and second was from newer query > which sets only a subset of columns) -- This message was sent by Atlassian JIRA (v6.3.4#6332)