cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Tyler Hobbs (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-9136) Mixed 2.0.14 - 2.1.4 Cluster Error Deserializing RangeSliceCommand
Date Thu, 09 Apr 2015 20:20:14 GMT

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

Tyler Hobbs commented on CASSANDRA-9136:
----------------------------------------

bq. Am I missing something or this is not at all related to mixed cluster/upgrade? It's just
a case of querying a table before schema agreement has been reached, which well, you shouldn't
do (and we explicitly let that be the client job).

Well, it's only related because a table was created on a 2.1 node in a mixed cluster (and
a background process started querying it).  I agree that clients shouldn't query the table
until schema agreement is reached, but it would be good to make the consequences for bad behavior
a little less drastic.

bq. Are you sure that was the ticket you meant to link to? That doesn't seem related to closing
connections or not from what I can tell.

Bah, I meant CASSANDRA-5725

bq. At least for 2.1, I don't think that's trivial because we depend on the CFMetaData to
deserialize the remainder of the message. \[...\] In practice, we do ship the payload size
with messages, so in theory it shouldn't be too hard to generally skip to the end of the message
on error

Yeah, I think taking advantage of the payload size is the simplest option (with a little refactoring).

bq. Yes, though we still have to make sure we deserialize the remainder of the message (or
skip it somehow).

Right.



> Mixed 2.0.14 - 2.1.4 Cluster Error Deserializing RangeSliceCommand
> ------------------------------------------------------------------
>
>                 Key: CASSANDRA-9136
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-9136
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: 3 Nodes GCE, N1-Standard-2, Ubuntu 12, 1 Node on 2.1.4, 2 on 2.0.14
>            Reporter: Russell Alexander Spitzer
>            Assignee: Tyler Hobbs
>             Fix For: 2.1.5
>
>
> This error occurs during a rolling upgrade between 2.0.14 and 2.1.4.
> h3. Repo
> With all the nodes on 2.0.14 make the following tables
> {code}
> CREATE KEYSPACE test WITH replication = {
>   'class': 'SimpleStrategy',
>   'replication_factor': '2'
> };
> USE test;
> CREATE TABLE compact (
>   k int,
>   c int,
>   d int,
>   PRIMARY KEY ((k), c)
> ) WITH COMPACT STORAGE;
> CREATE TABLE norm (
>   k int,
>   c int,
>   d int,
>   PRIMARY KEY ((k), c)
> ) ;
> {code}
> Then load some data into these tables. I used the python driver
> {code}
> from cassandra.cluster import Cluster
> s = Cluster().connect()
> for x in range (1000):
>     for y in range (1000):
>        s.execute_async("INSERT INTO test.compact (k,c,d) VALUES (%d,%d,%d)"%(x,y,y))
>        s.execute_async("INSERT INTO test.norm (k,c,d) VALUES (%d,%d,%d)"%(x,y,y))
> {code}
> Upgrade one node from 2.0.14 -> 2.1.4
> From the 2.1.4 node, create a new table.
> Query that table
> On the 2.0.14 nodes you get these exceptions because the schema didn't propagate there.
 This exception kills the TCP connection between the nodes.
> {code}
> ERROR [Thread-19] 2015-04-08 18:48:45,337 CassandraDaemon.java (line 258) Exception in
thread Thread[Thread-19,5,main]
> java.lang.NullPointerException
> 	at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:247)
> 	at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:156)
> 	at org.apache.cassandra.net.MessageIn.read(MessageIn.java:99)
> 	at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:149)
> 	at org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:131)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:74)
> {code}
> Run cqlsh on the upgraded node and queries will fail until the TCP connection is established
again, easiest to repo with CL = ALL
> {code}
> cqlsh> SELECT count(*) FROM test.norm where k = 22 ;
> ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses]
message="Operation timed out - received only 1 responses." info={'received_responses': 1,
'required_responses': 2, 'consistency': 'ALL'}
> cqlsh> SELECT count(*) FROM test.norm where k = 21 ;
> ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses]
message="Operation timed out - received only 1 responses." info={'received_responses': 1,
'required_responses': 2, 'consistency': 'ALL'}
> {code}
> So connection made:
> {code}
> DEBUG [Thread-227] 2015-04-09 05:09:02,718 IncomingTcpConnection.java (line 107) Set
version for /10.240.14.115 to 8 (will use 7)
> {code}
> Connection broken by query of table before schema propagated:
> {code}
> ERROR [Thread-227] 2015-04-09 05:10:24,015 CassandraDaemon.java (line 258) Exception
in thread Thread[Thread-227,5,main]
> java.lang.NullPointerException
> 	at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:247)
> 	at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:156)
> 	at org.apache.cassandra.net.MessageIn.read(MessageIn.java:99)
> 	at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:149)
> 	at org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:131)
> 	at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:74)
> {code}
> All query to that node will fail with timeouts now until...
> Connection re-established
> {code}
> DEBUG [Thread-228] 2015-04-09 05:11:00,323 IncomingTcpConnection.java (line 107) Set
version for /10.240.14.115 to 8 (will use 7)
> {code}
> Now queries work again.



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

Mime
View raw message