You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2016/12/15 17:02:37 UTC

cassandra git commit: Fix DELETE and UPDATE queries with empty IN restrictions

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 0fe82be83 -> 9fc1ffb63


Fix DELETE and UPDATE queries with empty IN restrictions

patch by Alex Petrov; reviewed by Benjamin Lerer for CASSANDRA-12829


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/9fc1ffb6
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/9fc1ffb6
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/9fc1ffb6

Branch: refs/heads/cassandra-3.0
Commit: 9fc1ffb631a5a44bc11b55061288758e39a9d20a
Parents: 0fe82be
Author: Alex Petrov <ol...@gmail.com>
Authored: Thu Dec 15 17:56:41 2016 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Thu Dec 15 18:01:32 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../restrictions/StatementRestrictions.java     | 11 ++++
 .../cql3/statements/BatchStatement.java         | 11 ++--
 .../cql3/statements/ModificationStatement.java  | 15 +++--
 .../org/apache/cassandra/cql3/CQLTester.java    |  7 ++
 .../cql3/validation/operations/BatchTest.java   | 40 ++++++++++++
 .../cql3/validation/operations/DeleteTest.java  | 63 ++++++++++++++++--
 .../cql3/validation/operations/InsertTest.java  |  6 --
 .../operations/InsertUpdateIfConditionTest.java |  6 ++
 .../cql3/validation/operations/UpdateTest.java  | 69 +++++++++++++++++++-
 10 files changed, 202 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9fc1ffb6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 145afb9..e69bf08 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.11
+ * Fix DELETE and UPDATE queries with empty IN restrictions (CASSANDRA-12829)
  * Mark MVs as built after successful bootstrap (CASSANDRA-12984)
  * Estimated TS drop-time histogram updated with Cell.NO_DELETION_TIME (CASSANDRA-13040)
  * Nodetool compactionstats fails with NullPointerException (CASSANDRA-13021)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9fc1ffb6/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
index 647d22f..542dec9 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@ -430,6 +430,17 @@ public final class StatementRestrictions
     }
 
     /**
+     * Checks if restrictions on the clustering key have IN restrictions.
+     *
+     * @return <code>true</code> if the restrictions on the clustering key have IN restrictions,
+     * <code>false</code> otherwise.
+     */
+    public boolean clusteringKeyRestrictionsHasIN()
+    {
+        return clusteringColumnsRestrictions.isIN();
+    }
+
+    /**
      * Processes the clustering column restrictions.
      *
      * @param hasQueriableIndex <code>true</code> if some of the queried data are indexed, <code>false</code> otherwise

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9fc1ffb6/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index f0aa835..76a6460 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -401,7 +401,7 @@ public class BatchStatement implements CQLStatement
             QueryOptions statementOptions = options.forStatement(i);
             long timestamp = attrs.getTimestamp(now, statementOptions);
             List<ByteBuffer> pks = statement.buildPartitionKeyNames(statementOptions);
-            if (pks.size() > 1)
+            if (statement.getRestrictions().keyIsInRelation())
                 throw new IllegalArgumentException("Batch with conditions cannot span multiple partitions (you cannot use IN on the partition key)");
             if (key == null)
             {
@@ -413,12 +413,11 @@ public class BatchStatement implements CQLStatement
                 throw new InvalidRequestException("Batch with conditions cannot span multiple partitions");
             }
 
-            SortedSet<Clustering> clusterings = statement.createClustering(statementOptions);
+            checkFalse(statement.getRestrictions().clusteringKeyRestrictionsHasIN(),
+                       "IN on the clustering key columns is not supported with conditional %s",
+                       statement.type.isUpdate()? "updates" : "deletions");
 
-            checkFalse(clusterings.size() > 1,
-                       "IN on the clustering key columns is not supported with conditional updates");
-
-            Clustering clustering = Iterables.getOnlyElement(clusterings);
+            Clustering clustering = Iterables.getOnlyElement(statement.createClustering(statementOptions));
 
             if (statement.hasConditions())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9fc1ffb6/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 01c2ad1..acfa16b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -139,6 +139,11 @@ public abstract class ModificationStatement implements CQLStatement
         this.requiresRead = requiresReadBuilder.build();
     }
 
+    public StatementRestrictions getRestrictions()
+    {
+        return restrictions;
+    }
+
     public Iterable<Function> getFunctions()
     {
         List<Function> functions = new ArrayList<>();
@@ -445,20 +450,18 @@ public abstract class ModificationStatement implements CQLStatement
     {
         List<ByteBuffer> keys = buildPartitionKeyNames(options);
         // We don't support IN for CAS operation so far
-        checkFalse(keys.size() > 1,
+        checkFalse(restrictions.keyIsInRelation(),
                    "IN on the partition key is not supported with conditional %s",
                    type.isUpdate()? "updates" : "deletions");
 
         DecoratedKey key = cfm.decorateKey(keys.get(0));
         long now = options.getTimestamp(queryState);
-        SortedSet<Clustering> clusterings = createClustering(options);
 
-        checkFalse(clusterings.size() > 1,
+        checkFalse(restrictions.clusteringKeyRestrictionsHasIN(),
                    "IN on the clustering key columns is not supported with conditional %s",
                     type.isUpdate()? "updates" : "deletions");
 
-        Clustering clustering = Iterables.getOnlyElement(clusterings);
-
+        Clustering clustering = Iterables.getOnlyElement(createClustering(options));
         CQL3CasRequest request = new CQL3CasRequest(cfm, key, false, conditionColumns(), updatesRegularRows(), updatesStaticRow());
 
         addConditions(clustering, request, options);
@@ -663,7 +666,7 @@ public abstract class ModificationStatement implements CQLStatement
 
                 PartitionUpdate upd = collector.getPartitionUpdate(cfm, dk, options.getConsistency());
 
-                if (clusterings.isEmpty())
+                if (!restrictions.hasClusteringColumnsRestriction())
                 {
                     addUpdateForKey(upd, Clustering.EMPTY, params);
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9fc1ffb6/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 3d8d03b..4732ed3 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -382,6 +382,13 @@ public abstract class CQLTester
             store.forceBlockingFlush();
     }
 
+
+    public void flush(boolean forceFlush)
+    {
+        if (forceFlush)
+            flush();
+    }
+
     @FunctionalInterface
     public interface CheckedFunction {
         void apply() throws Throwable;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9fc1ffb6/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
index 66226eb..e8f169d 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
@@ -199,4 +199,44 @@ public class BatchTest extends CQLTester
         assertRows(execute(String.format("SELECT * FROM %s", tbl1)), row(0, 1, 2));
         assertRows(execute(String.format("SELECT * FROM %s", tbl2)), row(0, 3, 4));
     }
+
+    @Test
+    public void testBatchWithInRestriction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a,b))");
+        execute("INSERT INTO %s (a,b,c) VALUES (?,?,?)",1,1,1);
+        execute("INSERT INTO %s (a,b,c) VALUES (?,?,?)",1,2,2);
+        execute("INSERT INTO %s (a,b,c) VALUES (?,?,?)",1,3,3);
+
+        for (String inClause : new String[] { "()", "(1, 2)"})
+        {
+            assertInvalidMessage("IN on the clustering key columns is not supported with conditional updates",
+                                 "BEGIN BATCH " +
+                                 "UPDATE %1$s SET c = 100 WHERE a = 1 AND b = 1 IF c = 1;" +
+                                 "UPDATE %1$s SET c = 200 WHERE a = 1 AND b IN " + inClause + " IF c = 1;" +
+                                 "APPLY BATCH");
+
+            assertInvalidMessage("IN on the clustering key columns is not supported with conditional deletions",
+                                 "BEGIN BATCH " +
+                                 "UPDATE %1$s SET c = 100 WHERE a = 1 AND b = 1 IF c = 1;" +
+                                 "DELETE FROM %1$s WHERE a = 1 AND b IN " + inClause + " IF c = 1;" +
+                                 "APPLY BATCH");
+
+            assertInvalidMessage("Batch with conditions cannot span multiple partitions (you cannot use IN on the partition key)",
+                                 "BEGIN BATCH " +
+                                 "UPDATE %1$s SET c = 100 WHERE a = 1 AND b = 1 IF c = 1;" +
+                                 "UPDATE %1$s SET c = 200 WHERE a IN " + inClause + " AND b = 1 IF c = 1;" +
+                                 "APPLY BATCH");
+
+            assertInvalidMessage("Batch with conditions cannot span multiple partitions (you cannot use IN on the partition key)",
+                                 "BEGIN BATCH " +
+                                 "UPDATE %1$s SET c = 100 WHERE a = 1 AND b = 1 IF c = 1;" +
+                                 "DELETE FROM %1$s WHERE a IN " + inClause + " AND b = 1 IF c = 1;" +
+                                 "APPLY BATCH");
+        }
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1,1,1),
+                   row(1,2,2),
+                   row(1,3,3));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9fc1ffb6/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index b9ccafb..18a6ca3 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@ -1165,12 +1165,6 @@ public class DeleteTest extends CQLTester
         }
     }
 
-    private void flush(boolean forceFlush)
-    {
-        if (forceFlush)
-            flush();
-    }
-
     @Test
     public void testDeleteAndReverseQueries() throws Throwable
     {
@@ -1196,4 +1190,61 @@ public class DeleteTest extends CQLTester
             row(9), row(8), row(1), row(0)
         );
     }
+
+    /**
+     * Test for CASSANDRA-12829
+     */
+    @Test
+    public void testDeleteWithEmptyInRestriction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a,b))");
+        execute("INSERT INTO %s (a,b,c) VALUES (?,?,?)", 1, 1, 1);
+        execute("INSERT INTO %s (a,b,c) VALUES (?,?,?)", 1, 2, 2);
+        execute("INSERT INTO %s (a,b,c) VALUES (?,?,?)", 1, 3, 3);
+
+        execute("DELETE FROM %s WHERE a IN ();");
+        execute("DELETE FROM %s WHERE a IN () AND b IN ();");
+        execute("DELETE FROM %s WHERE a IN () AND b = 1;");
+        execute("DELETE FROM %s WHERE a = 1 AND b IN ();");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1),
+                   row(1, 2, 2),
+                   row(1, 3, 3));
+
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, s int static, PRIMARY KEY ((a,b), c))");
+        execute("INSERT INTO %s (a,b,c,d,s) VALUES (?,?,?,?,?)", 1, 1, 1, 1, 1);
+        execute("INSERT INTO %s (a,b,c,d,s) VALUES (?,?,?,?,?)", 1, 1, 2, 2, 1);
+        execute("INSERT INTO %s (a,b,c,d,s) VALUES (?,?,?,?,?)", 1, 1, 3, 3, 1);
+
+        execute("DELETE FROM %s WHERE a = 1 AND b = 1 AND c IN ();");
+        execute("DELETE FROM %s WHERE a = 1 AND b IN () AND c IN ();");
+        execute("DELETE FROM %s WHERE a IN () AND b IN () AND c IN ();");
+        execute("DELETE FROM %s WHERE a IN () AND b = 1 AND c IN ();");
+        execute("DELETE FROM %s WHERE a IN () AND b IN () AND c = 1;");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1, 1, 1),
+                   row(1, 1, 2, 1, 2),
+                   row(1, 1, 3, 1, 3));
+
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY ((a,b), c, d))");
+        execute("INSERT INTO %s (a,b,c,d,e) VALUES (?,?,?,?,?)", 1, 1, 1, 1, 1);
+        execute("INSERT INTO %s (a,b,c,d,e) VALUES (?,?,?,?,?)", 1, 1, 1, 2, 2);
+        execute("INSERT INTO %s (a,b,c,d,e) VALUES (?,?,?,?,?)", 1, 1, 1, 3, 3);
+        execute("INSERT INTO %s (a,b,c,d,e) VALUES (?,?,?,?,?)", 1, 1, 1, 4, 4);
+
+        execute("DELETE FROM %s WHERE a = 1 AND b = 1 AND c IN ();");
+        execute("DELETE FROM %s WHERE a = 1 AND b = 1 AND c = 1 AND d IN ();");
+        execute("DELETE FROM %s WHERE a = 1 AND b = 1 AND c IN () AND d IN ();");
+        execute("DELETE FROM %s WHERE a = 1 AND b IN () AND c IN () AND d IN ();");
+        execute("DELETE FROM %s WHERE a IN () AND b IN () AND c IN () AND d IN ();");
+        execute("DELETE FROM %s WHERE a IN () AND b IN () AND c IN () AND d = 1;");
+        execute("DELETE FROM %s WHERE a IN () AND b IN () AND c = 1 AND d = 1;");
+        execute("DELETE FROM %s WHERE a IN () AND b IN () AND c = 1 AND d IN ();");
+        execute("DELETE FROM %s WHERE a IN () AND b = 1");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1, 1, 1, 1, 1),
+                   row(1, 1, 1, 2, 2),
+                   row(1, 1, 1, 3, 3),
+                   row(1, 1, 1, 4, 4));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9fc1ffb6/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
index a030613..c9a9faf 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
@@ -285,12 +285,6 @@ public class InsertTest extends CQLTester
                              "INSERT INTO %s (partitionKey, clustering_2, staticValue) VALUES (0, 0, 'A')");
     }
 
-    private void flush(boolean forceFlush)
-    {
-        if (forceFlush)
-            flush();
-    }
-
     @Test
     public void testPKInsertWithValueOver64K() throws Throwable
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9fc1ffb6/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
index f1dc9a2..fd02a69 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
@@ -119,6 +119,8 @@ public class InsertUpdateIfConditionTest extends CQLTester
 
         createTable(" CREATE TABLE %s (k int, c int, v1 text, PRIMARY KEY(k, c))");
         assertInvalidMessage("IN on the clustering key columns is not supported with conditional updates",
+                             "UPDATE %s SET v1 = 'A' WHERE k = 0 AND c IN () IF EXISTS");
+        assertInvalidMessage("IN on the clustering key columns is not supported with conditional updates",
                              "UPDATE %s SET v1 = 'A' WHERE k = 0 AND c IN (1, 2) IF EXISTS");
     }
 
@@ -227,7 +229,11 @@ public class InsertUpdateIfConditionTest extends CQLTester
         assertInvalidMessage("IN on the clustering key columns is not supported with conditional deletions",
                              "DELETE FROM %s WHERE k = 'k' AND i IN (0, 1) IF v = 'foo'");
         assertInvalidMessage("IN on the clustering key columns is not supported with conditional deletions",
+                             "DELETE FROM %s WHERE k = 'k' AND i IN () IF v = 'foo'");
+        assertInvalidMessage("IN on the clustering key columns is not supported with conditional deletions",
                              "DELETE FROM %s WHERE k = 'k' AND i IN (0, 1) IF EXISTS");
+        assertInvalidMessage("IN on the clustering key columns is not supported with conditional deletions",
+                             "DELETE FROM %s WHERE k = 'k' AND i IN () IF EXISTS");
 
         assertInvalidMessage("Invalid 'unset' value in condition",
                              "DELETE FROM %s WHERE k = 'k' AND i = 0 IF v = ?", unset());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9fc1ffb6/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
index 0170ed2..690d4f9 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
@@ -521,9 +521,72 @@ public class UpdateTest extends CQLTester
         assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v4", "v3")));
     }
 
-    private void flush(boolean forceFlush)
+    /**
+     * Test for CASSANDRA-12829
+     */
+    @Test
+    public void testUpdateWithEmptyInRestriction() throws Throwable
     {
-        if (forceFlush)
-            flush();
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a,b))");
+        execute("INSERT INTO %s (a,b,c) VALUES (?,?,?)",1,1,1);
+        execute("INSERT INTO %s (a,b,c) VALUES (?,?,?)",1,2,2);
+        execute("INSERT INTO %s (a,b,c) VALUES (?,?,?)",1,3,3);
+
+        assertInvalidMessage("Some clustering keys are missing: b",
+                             "UPDATE %s SET c = 100 WHERE a IN ();");
+        execute("UPDATE %s SET c = 100 WHERE a IN () AND b IN ();");
+        execute("UPDATE %s SET c = 100 WHERE a IN () AND b = 1;");
+        execute("UPDATE %s SET c = 100 WHERE a = 1 AND b IN ();");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1,1,1),
+                   row(1,2,2),
+                   row(1,3,3));
+
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, s int static, PRIMARY KEY ((a,b), c))");
+        execute("INSERT INTO %s (a,b,c,d,s) VALUES (?,?,?,?,?)",1,1,1,1,1);
+        execute("INSERT INTO %s (a,b,c,d,s) VALUES (?,?,?,?,?)",1,1,2,2,1);
+        execute("INSERT INTO %s (a,b,c,d,s) VALUES (?,?,?,?,?)",1,1,3,3,1);
+
+        execute("UPDATE %s SET d = 100 WHERE a = 1 AND b = 1 AND c IN ();");
+        execute("UPDATE %s SET d = 100 WHERE a = 1 AND b IN () AND c IN ();");
+        execute("UPDATE %s SET d = 100 WHERE a IN () AND b IN () AND c IN ();");
+        execute("UPDATE %s SET d = 100 WHERE a IN () AND b IN () AND c = 1;");
+        execute("UPDATE %s SET d = 100 WHERE a IN () AND b = 1 AND c IN ();");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1,1,1,1,1),
+                   row(1,1,2,1,2),
+                   row(1,1,3,1,3));
+
+        // No clustering keys restricted, update whole partition
+        execute("UPDATE %s set s = 100 where a = 1 AND b = 1;");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1,1,1,100,1),
+                   row(1,1,2,100,2),
+                   row(1,1,3,100,3));
+
+        execute("UPDATE %s set s = 200 where a = 1 AND b IN ();");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1,1,1,100,1),
+                   row(1,1,2,100,2),
+                   row(1,1,3,100,3));
+
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY ((a,b), c, d))");
+        execute("INSERT INTO %s (a,b,c,d,e) VALUES (?,?,?,?,?)",1,1,1,1,1);
+        execute("INSERT INTO %s (a,b,c,d,e) VALUES (?,?,?,?,?)",1,1,1,2,2);
+        execute("INSERT INTO %s (a,b,c,d,e) VALUES (?,?,?,?,?)",1,1,1,3,3);
+        execute("INSERT INTO %s (a,b,c,d,e) VALUES (?,?,?,?,?)",1,1,1,4,4);
+
+        execute("UPDATE %s SET e = 100 WHERE a = 1 AND b = 1 AND c = 1 AND d IN ();");
+        execute("UPDATE %s SET e = 100 WHERE a = 1 AND b = 1 AND c IN () AND d IN ();");
+        execute("UPDATE %s SET e = 100 WHERE a = 1 AND b IN () AND c IN () AND d IN ();");
+        execute("UPDATE %s SET e = 100 WHERE a IN () AND b IN () AND c IN () AND d IN ();");
+        execute("UPDATE %s SET e = 100 WHERE a IN () AND b IN () AND c IN () AND d = 1;");
+        execute("UPDATE %s SET e = 100 WHERE a IN () AND b IN () AND c = 1 AND d = 1;");
+        execute("UPDATE %s SET e = 100 WHERE a IN () AND b IN () AND c = 1 AND d IN ();");
+        assertRows(execute("SELECT * FROM %s"),
+                   row(1,1,1,1,1),
+                   row(1,1,1,2,2),
+                   row(1,1,1,3,3),
+                   row(1,1,1,4,4));
     }
 }