Details
Description
This error occurs during a rolling upgrade between 2.0.14 and 2.1.4.
Repo
With all the nodes on 2.0.14 make the following tables
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) ) ;
Then load some data into these tables. I used the python driver
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))
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.
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)
Run cqlsh on the upgraded node and queries will fail until the TCP connection is established again, easiest to repo with CL = ALL
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'}
So connection made:
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)
Connection broken by query of table before schema propagated:
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)
All query to that node will fail with timeouts now until...
Connection re-established
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)
Now queries work again.
Attachments
Attachments
Issue Links
- is related to
-
CASSANDRA-9289 Recover from unknown table when deserializing internode messages
- Resolved
- relates to
-
CASSANDRA-8996 dtests should pass on trunk
- Resolved