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

Sean McCarthy commented on CASSANDRA-12444:
-------------------------------------------

Seeing this failure on trunk as well : 
http://cassci.datastax.com/job/trunk_large_dtest/29/testReport/upgrade_tests.upgrade_through_versions_test/ProtoV4Upgrade_AllVersions_RandomPartitioner_EndsAt_Trunk_HEAD/rolling_upgrade_test/

> dtest failure in 
> upgrade_tests.upgrade_through_versions_test.ProtoV4Upgrade_AllVersions_RandomPartitioner_EndsAt_Trunk_HEAD.rolling_upgrade_test
> ------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-12444
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12444
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Sean McCarthy
>             Fix For: 3.x
>
>
> example failure: 
> http://cassci.datastax.com/job/cassandra-3.9_large_dtest/7/testReport/upgrade_tests.upgrade_through_versions_test/ProtoV4Upgrade_AllVersions_RandomPartitioner_EndsAt_Trunk_HEAD/rolling_upgrade_test/
> {code}
> Standard Output
> Error details: 
> Errors seen in logs for: node2
> node2: ERROR [SharedPool-Worker-1] 2016-08-06 17:24:26,794 Message.java:611 - 
> Unexpected exception during request; channel = [id: 0x9140a192, 
> /127.0.0.1:34121 => /127.0.0.2:9042]
> java.lang.AssertionError: null
>       at 
> org.apache.cassandra.db.ReadCommand$Serializer.serializedSize(ReadCommand.java:632)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.db.ReadCommand$Serializer.serializedSize(ReadCommand.java:536)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at org.apache.cassandra.net.MessageOut.payloadSize(MessageOut.java:166) 
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.net.OutboundTcpConnectionPool.getConnection(OutboundTcpConnectionPool.java:72)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.net.MessagingService.getConnection(MessagingService.java:609)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.net.MessagingService.sendOneWay(MessagingService.java:758)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.net.MessagingService.sendRR(MessagingService.java:701) 
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.net.MessagingService.sendRRWithFailure(MessagingService.java:684)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.service.AbstractReadExecutor.makeRequests(AbstractReadExecutor.java:110)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.service.AbstractReadExecutor.makeDigestRequests(AbstractReadExecutor.java:91)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.service.AbstractReadExecutor$AlwaysSpeculatingReadExecutor.executeAsync(AbstractReadExecutor.java:332)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.service.StorageProxy$SinglePartitionReadLifecycle.doInitialQueries(StorageProxy.java:1703)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:1658) 
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.service.StorageProxy.readRegular(StorageProxy.java:1605) 
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:1524) 
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.db.SinglePartitionReadCommand.execute(SinglePartitionReadCommand.java:335)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.service.pager.AbstractQueryPager.fetchPage(AbstractQueryPager.java:67)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.service.pager.SinglePartitionPager.fetchPage(SinglePartitionPager.java:34)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement$Pager$NormalPager.fetchPage(SelectStatement.java:315)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:351)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:227)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:76)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:487)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:464)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:130)
>  ~[apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:507)
>  [apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:401)
>  [apache-cassandra-3.0.8.jar:3.0.8]
>       at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>  [netty-all-4.0.23.Final.jar:4.0.23.Final]
>       at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333)
>  [netty-all-4.0.23.Final.jar:4.0.23.Final]
>       at 
> io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:32)
>  [netty-all-4.0.23.Final.jar:4.0.23.Final]
>       at 
> io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:324)
>  [netty-all-4.0.23.Final.jar:4.0.23.Final]
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_51]
>       at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164)
>  [apache-cassandra-3.0.8.jar:3.0.8]
>       at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
> [apache-cassandra-3.0.8.jar:3.0.8]
>       at java.lang.Thread.run(Thread.java:745) [na:1.8.0_51]
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to