cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Russ Hatch (JIRA)" <j...@apache.org>
Subject [jira] [Reopened] (CASSANDRA-10122) AssertionError after upgrade to 3.0
Date Tue, 01 Dec 2015 18:35:11 GMT

     [ https://issues.apache.org/jira/browse/CASSANDRA-10122?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Russ Hatch reopened CASSANDRA-10122:
------------------------------------

Looks like this is still happening, at least on 2.1->3.0 upgrades. I suspect it's probably
also happening on 2.2->3.0 upgrades and will confirm shortly.

to repro on 2.1->3.0 using cassandra-dtest:
{noformat}
export UPGRADE_PATH=2_1:3_0
nosetests -xvs upgrade_through_versions_test.py:TestUpgrade_from_cassandra_2_1_HEAD_to_cassandra_3_0_HEAD.rolling_upgrade_test
{noformat}

Here's how it looks in the logs when repro'd today:
{noformat}
ERROR [SharedPool-Worker-5] 2015-12-01 11:31:30,067 Message.java:611 - Unexpected exception
during request; channel = [id: 0xa7623e8b, /127.0.0.1:48657 => /127.0.0.1:9042]
java.lang.AssertionError: null
        at org.apache.cassandra.db.ReadCommand$LegacyReadCommandSerializer.serializedSize(ReadCommand.java:1188)
~[main/:na]
        at org.apache.cassandra.db.ReadCommand$LegacyReadCommandSerializer.serializedSize(ReadCommand.java:1135)
~[main/:na]
        at org.apache.cassandra.net.MessageOut.payloadSize(MessageOut.java:166) ~[main/:na]
        at org.apache.cassandra.net.OutboundTcpConnectionPool.getConnection(OutboundTcpConnectionPool.java:72)
~[main/:na]
        at org.apache.cassandra.net.MessagingService.getConnection(MessagingService.java:595)
~[main/:na]
        at org.apache.cassandra.net.MessagingService.sendOneWay(MessagingService.java:744)
~[main/:na]
        at org.apache.cassandra.net.MessagingService.sendRR(MessagingService.java:687) ~[main/:na]
        at org.apache.cassandra.net.MessagingService.sendRRWithFailure(MessagingService.java:670)
~[main/:na]
        at org.apache.cassandra.service.AbstractReadExecutor.makeRequests(AbstractReadExecutor.java:112)
~[main/:na]
        at org.apache.cassandra.service.AbstractReadExecutor.makeDataRequests(AbstractReadExecutor.java:85)
~[main/:na]
        at org.apache.cassandra.service.AbstractReadExecutor$AlwaysSpeculatingReadExecutor.executeAsync(AbstractReadExecutor.java:332)
~[main/:na]
        at org.apache.cassandra.service.StorageProxy$SinglePartitionReadLifecycle.doInitialQueries(StorageProxy.java:1599)
~[main/:na]
        at org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:1554) ~[main/:na]
        at org.apache.cassandra.service.StorageProxy.readRegular(StorageProxy.java:1501) ~[main/:na]
        at org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:1420) ~[main/:na]
        at org.apache.cassandra.db.SinglePartitionReadCommand.execute(SinglePartitionReadCommand.java:302)
~[main/:na]
        at org.apache.cassandra.service.pager.AbstractQueryPager.fetchPage(AbstractQueryPager.java:67)
~[main/:na]
        at org.apache.cassandra.service.pager.SinglePartitionPager.fetchPage(SinglePartitionPager.java:34)
~[main/:na]
        at org.apache.cassandra.cql3.statements.SelectStatement$Pager$NormalPager.fetchPage(SelectStatement.java:302)
~[main/:na]
        at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:338)
~[main/:na]
        at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:214)
~[main/:na]
        at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:76)
~[main/:na]
        at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:206)
~[main/:na]
        at org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:472)
~[main/:na]
        at org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:449)
~[main/:na]
        at org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:130)
~[main/:na]
        at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:507)
[main/:na]
        at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:401)
[main/:na]
        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_66]
        at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
[main/:na]
        at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na]
        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66]
{noformat}

> AssertionError after upgrade to 3.0
> -----------------------------------
>
>                 Key: CASSANDRA-10122
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10122
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Russ Hatch
>            Assignee: Sylvain Lebresne
>             Fix For: 3.0.1, 3.1
>
>         Attachments: node1.log, node2.log, node3.log
>
>
> Upgrade tests are encountering this exception after upgrade from 2.2 HEAD to 3.0 HEAD:
> {noformat}
> ERROR [SharedPool-Worker-4] 2015-08-18 12:33:57,858 Message.java:611 - Unexpected exception
during request; channel = [id: 0xa5ba2c7a, /127.0.0.1:55048 => /127.0.0.1:9042]
> java.lang.AssertionError: null
>         at org.apache.cassandra.db.ReadCommand$Serializer.serializedSize(ReadCommand.java:520)
~[main/:na]
>         at org.apache.cassandra.db.ReadCommand$Serializer.serializedSize(ReadCommand.java:461)
~[main/:na]
>         at org.apache.cassandra.net.MessageOut.payloadSize(MessageOut.java:166) ~[main/:na]
>         at org.apache.cassandra.net.OutboundTcpConnectionPool.getConnection(OutboundTcpConnectionPool.java:72)
~[main/:na]
>         at org.apache.cassandra.net.MessagingService.getConnection(MessagingService.java:583)
~[main/:na]
>         at org.apache.cassandra.net.MessagingService.sendOneWay(MessagingService.java:733)
~[main/:na]
>         at org.apache.cassandra.net.MessagingService.sendRR(MessagingService.java:676)
~[main/:na]
>         at org.apache.cassandra.net.MessagingService.sendRRWithFailure(MessagingService.java:659)
~[main/:na]
>         at org.apache.cassandra.service.AbstractReadExecutor.makeRequests(AbstractReadExecutor.java:103)
~[main/:na]
>         at org.apache.cassandra.service.AbstractReadExecutor.makeDataRequests(AbstractReadExecutor.java:76)
~[main/:na]
>         at org.apache.cassandra.service.AbstractReadExecutor$AlwaysSpeculatingReadExecutor.executeAsync(AbstractReadExecutor.java:323)
~[main/:na]
>         at org.apache.cassandra.service.StorageProxy$SinglePartitionReadLifecycle.doInitialQueries(StorageProxy.java:1599)
~[main/:na]
>         at org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:1554)
~[main/:na]
>         at org.apache.cassandra.service.StorageProxy.readRegular(StorageProxy.java:1501)
~[main/:na]
>         at org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:1420) ~[main/:na]
>         at org.apache.cassandra.db.SinglePartitionReadCommand$Group.execute(SinglePartitionReadCommand.java:457)
~[main/:na]
>         at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:232)
~[main/:na]
>         at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:202)
~[main/:na]
>         at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:72)
~[main/:na]
>         at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:204)
~[main/:na]
>         at org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:470)
~[main/:na]
>         at org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:447)
~[main/:na]
>         at org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:139)
~[main/:na]
>         at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:507)
[main/:na]
>         at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:401)
[main/:na]
>         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_45]
>         at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
[main/:na]
>         at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
> {noformat}
> This occurs while the cluster is in a mixed version state, with the first node upgraded
to 3.0, and the remaining two nodes still on 2.2.



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

Mime
View raw message