[ https://issues.apache.org/jira/browse/CASSANDRA-12867?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Joshua McKenzie updated CASSANDRA-12867: ---------------------------------------- Reviewer: Alex Petrov > Batch with multiple conditional updates for the same partition causes > AssertionError > ------------------------------------------------------------------------------------ > > Key: CASSANDRA-12867 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12867 > Project: Cassandra > Issue Type: Bug > Components: CQL > Reporter: Kurt Greaves > Assignee: Kurt Greaves > Attachments: 12867-3.0.patch > > > Reproduced in 3.0.10 and 3.10. Used to work in 3.0.9 and earlier. Bug was > introduced in CASSANDRA-12060. > The following causes an AssertionError: > {code} > CREATE KEYSPACE test WITH replication = { 'class' : 'SimpleStrategy', > 'replication_factor' : 1 }; > create table test.test (id int PRIMARY KEY, val text); > BEGIN BATCH INSERT INTO test.test (id, val) VALUES (999, 'aaa') IF NOT > EXISTS; INSERT INTO test.test (id, val) VALUES (999, 'ccc') IF NOT EXISTS; > APPLY BATCH ; > {code} > Stack trace is as follows: > {code} > ERROR [Native-Transport-Requests-2] 2016-10-31 04:16:44,231 Message.java:622 > - Unexpected exception during request; channel = [id: 0x176e1c04, > L:/127.0.0.1:9042 - R:/127.0.0.1:59743] > java.lang.AssertionError: null > at > org.apache.cassandra.cql3.statements.CQL3CasRequest.setConditionsForRow(CQL3CasRequest.java:138) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.CQL3CasRequest.addExistsCondition(CQL3CasRequest.java:104) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.CQL3CasRequest.addNotExist(CQL3CasRequest.java:84) > ~[main/:na] > at > org.apache.cassandra.cql3.IfNotExistsCondition.addConditionsTo(IfNotExistsCondition.java:28) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.ModificationStatement.addConditions(ModificationStatement.java:482) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.BatchStatement.makeCasRequest(BatchStatement.java:434) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.BatchStatement.executeWithConditions(BatchStatement.java:379) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.BatchStatement.execute(BatchStatement.java:358) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.BatchStatement.execute(BatchStatement.java:346) > ~[main/:na] > at > org.apache.cassandra.cql3.statements.BatchStatement.execute(BatchStatement.java:341) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:218) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:249) > ~[main/:na] > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:234) > ~[main/:na] > at > org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:115) > ~[main/:na] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:516) > [main/:na] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:409) > [main/:na] > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > [netty-all-4.0.39.Final.jar:4.0.39.Final] > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:366) > [netty-all-4.0.39.Final.jar:4.0.39.Final] > at > io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:35) > [netty-all-4.0.39.Final.jar:4.0.39.Final] > at > io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:357) > [netty-all-4.0.39.Final.jar:4.0.39.Final] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > [na:1.8.0_102] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162) > [main/:na] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) > [main/:na] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_102] > {code} > The problem is that previous will receive a value after the first statement > in the batch is evaluated in BatchStatement.makeCasRequest. I can't see any > reason why we have this assertion, it seems to me that it's unnecessary. > Removing it fixes the problem (obviously) but I'm not sure if it breaks > something else, or if this is an intended failure case (in which case it > should be caught earlier on). > Relevant code is as follows: > {code:title=CQL3CasRequest.java} > private void setConditionsForRow(Clustering clustering, RowCondition > condition) > { > if (clustering == Clustering.STATIC_CLUSTERING) > { > assert staticConditions == null; > staticConditions = condition; > } > else > { > RowCondition previous = conditions.put(clustering, condition); > assert previous == null; > } > } > {code} > I've attached a patch that fixes the issue by removing the assert -- This message was sent by Atlassian JIRA (v6.3.4#6332)