Details
-
Bug
-
Status: Resolved
-
Normal
-
Resolution: Fixed
-
Normal
Description
The batch CAS feature introduced in Cassandra 2.0.6 does not support the LOCAL_SERIAL consistency level, and always uses SERIAL.
Create a cluster with 4 nodes with the following topology:
Datacenter: DC2 =============== Status=Up/Down |/ State=Normal/Leaving/Joining/Moving -- Address Load Tokens Owns Host ID Rack UN 127.0.0.3 269 KB 256 26.3% ae92d997-6042-42d9-b447-943080569742 RAC1 UN 127.0.0.4 197.81 KB 256 25.1% 3edc92d7-9d1b-472a-8452-24dddbc4502c RAC1 Datacenter: DC1 =============== Status=Up/Down |/ State=Normal/Leaving/Joining/Moving -- Address Load Tokens Owns Host ID Rack UN 127.0.0.1 226.92 KB 256 24.8% dbc17bd7-1ede-47a2-9b31-6063752d6eb3 RAC1 UN 127.0.0.2 179.27 KB 256 23.7% bb0ad285-34d2-4989-a664-b068986ab6fa RAC1
In cqlsh:
cqlsh> CREATE KEYSPACE foo WITH replication = {'class': 'NetworkTopologyStrategy', 'DC1': 2, 'DC2': 2}; cqlsh> USE foo; cqlsh:foo> CREATE TABLE bar (x text, y bigint, z bigint, t bigint, PRIMARY KEY(x,y));
Kill nodes 127.0.0.3 and 127.0.0.4:
Datacenter: DC2 =============== Status=Up/Down |/ State=Normal/Leaving/Joining/Moving -- Address Load Tokens Owns Host ID Rack DN 127.0.0.3 262.37 KB 256 26.3% ae92d997-6042-42d9-b447-943080569742 RAC1 DN 127.0.0.4 208.04 KB 256 25.1% 3edc92d7-9d1b-472a-8452-24dddbc4502c RAC1 Datacenter: DC1 =============== Status=Up/Down |/ State=Normal/Leaving/Joining/Moving -- Address Load Tokens Owns Host ID Rack UN 127.0.0.1 214.82 KB 256 24.8% dbc17bd7-1ede-47a2-9b31-6063752d6eb3 RAC1 UN 127.0.0.2 178.23 KB 256 23.7% bb0ad285-34d2-4989-a664-b068986ab6fa RAC1
Connect to 127.0.0.1 in DC1 and run a CAS batch at CL.LOCAL_SERIAL+LOCAL_QUORUM:
final Cluster cluster = new Cluster.Builder() .addContactPoint("127.0.0.1") .withLoadBalancingPolicy(new DCAwareRoundRobinPolicy("DC1")) .build(); final Session session = cluster.connect("foo"); Batch batch = QueryBuilder.batch(); batch.add(new SimpleStatement("INSERT INTO bar (x,y,z) VALUES ('abc', 123, 1) IF NOT EXISTS")); batch.add(new SimpleStatement("UPDATE bar SET t=2 WHERE x='abc' AND y=123")); batch.setConsistencyLevel(ConsistencyLevel.LOCAL_QUORUM); batch.setSerialConsistencyLevel(ConsistencyLevel.LOCAL_SERIAL); session.execute(batch);
The batch fails with:
Caused by: com.datastax.driver.core.exceptions.UnavailableException: Not enough replica available for query at consistency SERIAL (3 required but only 2 alive)
at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:44)
at com.datastax.driver.core.Responses$Error$1.decode(Responses.java:33)
at com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:182)
at org.jboss.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:66)
... 21 more