Andy Tolbert created CASSANDRA-14638:
----------------------------------------
Summary: Column result order can change in 'SELECT *' results when
upgrading from 2.1 to 3.0 causing response corruption for queries using
prepared statements when static columns are used
Key: CASSANDRA-14638
URL: https://issues.apache.org/jira/browse/CASSANDRA-14638
Project: Cassandra
Issue Type: Bug
Environment: Single C* node ccm cluster upgraded from C* 2.1.20 to
3.0.17
Reporter: Andy Tolbert
When performing an upgrade from C* 2.1.20 to 3.1.17 I observed that the order
of columns returned from a 'SELECT *' query changes, particularly when static
columns are involved.
This may not seem like that much of a problem, however if using Prepared
Statements, any clients that remain connected during the upgrade may encounter
issues consuming results from these queries, as data is reordered and the
client not aware of it. The result definition is sent in the original prepared
statement response, so if order changes the client has no way of knowing (until
C* 4.0 via CASSANDRA-10786) without re-preparing, which is non-trivial as most
client drivers cache prepared statements.
This could lead to reading the wrong values for columns, which could result in
some kind of deserialization exception or if the data types of the switched
columns are compatible, the wrong values. This happens even if the client
attempts to retrieve a column value by name (i.e. row.getInt("colx")).
Unfortunately I don't think there is an easy fix for this. If the order was
changed back to the previous format, you risk issues for users upgrading from
older 3.0 version. I think it would be nice to add a note in the NEWS file in
the 3.0 upgrade section that describes this issue, and how to work around it
(specify all column names of interest explicitly in query).
Example schema and code to reproduce:
{noformat}
create keyspace ks with replication = {'class': 'SimpleStrategy',
'replication_factor': 1};
create table ks.tbl (p0 text,
p1 text,
m map<text, text> static,
t text,
u text static,
primary key (p0, p1)
);
insert into ks.tbl (p0, p1, m, t, u) values ('p0', 'p1', { 'm0' : 'm1' }, 't',
'u');{noformat}
When querying with 2.1 you'll observe the following order via cqlsh:
{noformat}
p0 | p1 | m | u | t
----+----+--------------+---+---
p0 | p1 | {'m0': 'm1'} | u | t{noformat}
With 3.0, observe that u and m are transposed:
{noformat}
p0 | p1 | u | m | t
----+----+---+--------------+---
p0 | p1 | u | {'m0': 'm1'} | t{noformat}
{code:java}
import com.datastax.driver.core.BoundStatement;
import com.datastax.driver.core.Cluster;
import com.datastax.driver.core.ColumnDefinitions;
import com.datastax.driver.core.PreparedStatement;
import com.datastax.driver.core.ResultSet;
import com.datastax.driver.core.Row;
import com.datastax.driver.core.Session;
import com.google.common.util.concurrent.Uninterruptibles;
import java.util.concurrent.TimeUnit;
public class LiveUpgradeTest {
public static void main(String args[]) {
Cluster cluster = Cluster.builder().addContactPoints("127.0.0.1").build();
try {
Session session = cluster.connect();
PreparedStatement p = session.prepare("SELECT * from ks.tbl");
BoundStatement bs = p.bind();
// continually query every 30 seconds
while (true) {
try {
ResultSet r = session.execute(bs);
Row row = r.one();
int i = 0;
// iterate over the result metadata in order printing the
// index, name, type, and length of the first row of data.
for (ColumnDefinitions.Definition d : r.getColumnDefinitions()) {
System.out.println(
i++
+ ": "
+ d.getName()
+ " -> "
+ d.getType()
+ " -> val = "
+ row.getBytesUnsafe(d.getName()).array().length);
}
} catch (Throwable t) {
t.printStackTrace();
} finally {
Uninterruptibles.sleepUninterruptibly(30, TimeUnit.SECONDS);
}
}
} finally {
cluster.close();
}
}
}
{code}
To reproduce, set up a cluster, the schema, and run this script. Then upgrade
the cluster to 3.0.17 (with ccm, ccm stop; ccm node1 setdir -v 3.0.17; ccm
start works) and observe after the client is able to reconnect that the results
are in a different order. i.e.:
With 2.x:
{noformat}
0: p0 -> varchar -> val = 2
1: p1 -> varchar -> val = 2
2: m -> map<varchar, varchar> -> val = 16
3: u -> varchar -> val = 1
4: t -> varchar -> val = 1{noformat}
With 3.x:
{noformat}
0: p0 -> varchar -> val = 2
1: p1 -> varchar -> val = 2
2: m -> map<varchar, varchar> -> val = 1
3: u -> varchar -> val = 16 (<-- the data for 'm' is now at index 3)
4: t -> varchar -> val = 1{noformat}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]