Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 8B7BE200BC3 for ; Thu, 3 Nov 2016 09:29:01 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 8A23E160AFE; Thu, 3 Nov 2016 08:29:01 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id D29CF160AFF for ; Thu, 3 Nov 2016 09:29:00 +0100 (CET) Received: (qmail 40517 invoked by uid 500); 3 Nov 2016 08:29:00 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 40498 invoked by uid 99); 3 Nov 2016 08:29:00 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 03 Nov 2016 08:28:59 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id C45A52C2A6D for ; Thu, 3 Nov 2016 08:28:59 +0000 (UTC) Date: Thu, 3 Nov 2016 08:28:59 +0000 (UTC) From: "Benjamin Lerer (JIRA)" To: commits@cassandra.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (CASSANDRA-12867) Batch with multiple conditional updates for the same partition causes AssertionError MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 03 Nov 2016 08:29:01 -0000 [ https://issues.apache.org/jira/browse/CASSANDRA-12867?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15632053#comment-15632053 ] Benjamin Lerer commented on CASSANDRA-12867: -------------------------------------------- bq. No matter how nonsensical the statements, if we allow it we should ensure it works. 70 to 80% of the time I spend on ticket goes into writting tests and I can tell you one thing: No matter how hard you think about it you will often forget the nonsensical the statements. > 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: Sylvain Lebresne > 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)