[ https://issues.apache.org/jira/browse/CASSANDRA-9136?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14497030#comment-14497030 ]
Tyler Hobbs commented on CASSANDRA-9136: ---------------------------------------- Linking to CASSANDRA-8996 because this occasionally causes dtest failures when the default role setup triggers the NPE. > Improve error handling when table is queried before the schema has fully > propagated > ----------------------------------------------------------------------------------- > > 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)