You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ad...@apache.org on 2021/10/12 10:56:40 UTC

[cassandra] branch trunk updated: Add support for type casting in WHERE clause components and in the values of INSERT/UPDATE statements

This is an automated email from the ASF dual-hosted git repository.

adelapena pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 0d4cc2e  Add support for type casting in WHERE clause components and in the values of INSERT/UPDATE statements
0d4cc2e is described below

commit 0d4cc2ef0d246df34d534ef2f0df8ad9bb043206
Author: Andrés de la Peña <a....@gmail.com>
AuthorDate: Tue Oct 12 11:55:31 2021 +0100

    Add support for type casting in WHERE clause components and in the values of INSERT/UPDATE statements
    
    patch by Andrés de la Peña; reviewed by Benjamin Lerer for CASSANDRA-14337
---
 CHANGES.txt                                        |   1 +
 NEWS.txt                                           |   1 +
 doc/cql3/CQL.textile                               |   3 +-
 src/antlr/Parser.g                                 |   9 +-
 .../cassandra/cql3/functions/FunctionCall.java     |   6 +
 test/unit/org/apache/cassandra/cql3/CQLTester.java |  10 +
 .../cassandra/cql3/functions/CastFctsTest.java     | 271 +++++++++++++++++++--
 7 files changed, 278 insertions(+), 23 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index a98d6e0..127e555 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.1
+ * Add support for type casting in WHERE clause components and in the values of INSERT/UPDATE statements (CASSANDRA-14337)
  * add credentials file support to CQLSH (CASSANDRA-16983)
  * Skip remaining bytes in the Envelope buffer when a ProtocolException is thrown to avoid double decoding (CASSANDRA-17026)
  * Allow reverse iteration of resources during permissions checking (CASSANDRA-17016)
diff --git a/NEWS.txt b/NEWS.txt
index e8ae8e1..6121525 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -38,6 +38,7 @@ using the provided 'sstableupgrade' tool.
 
 New features
 ------------
+    - Added support for type casting in the WHERE clause components and in the values of INSERT and UPDATE statements.
     - Warn/abort thresholds added to read queries notifying clients when these thresholds trigger (by
       emitting a client warning or aborting the query).  This feature is disabled by default, scheduled
       to be enabled in 4.2; it is controlled with the configuration track_warnings.enabled,
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 46bf8a2..c60800e 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -122,6 +122,7 @@ bc(syntax)..
                | <collection-literal>
                | <variable>
                | <function> '(' (<term> (',' <term>)*)? ')'
+               | CAST '(' <term> AS <type> ')'
 
   <collection-literal> ::= <map-literal>
                          | <set-literal>
@@ -1951,7 +1952,7 @@ CQL3 distinguishes between built-in functions (so called 'native functions') and
 
 h3(#castFun). Cast
 
-The @cast@ function can be used to converts one native datatype to another.
+The @cast@ function can be used to convert one native datatype to another.
 
 The following table describes the conversions supported by the @cast@ function. Cassandra will silently ignore any cast converting a datatype into its own datatype.
 
diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index b3ba7b3..39b3d9f 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -1554,9 +1554,10 @@ termGroup returns [Term.Raw term]
     ;
 
 simpleTerm returns [Term.Raw term]
-    : v=value                                 { $term = v; }
-    | f=function                              { $term = f; }
-    | '(' c=comparatorType ')' t=simpleTerm   { $term = new TypeCast(c, t); }
+    : v=value                                        { $term = v; }
+    | f=function                                     { $term = f; }
+    | '(' c=comparatorType ')' t=simpleTerm          { $term = new TypeCast(c, t); }
+    | K_CAST '(' t=simpleTerm K_AS n=native_type ')' { $term = FunctionCall.Raw.newCast(t, n); }
     ;
 
 columnOperation[List<Pair<ColumnIdentifier, Operation.RawUpdate>> operations]
@@ -1776,7 +1777,7 @@ native_type returns [CQL3Type t]
     | K_COUNTER   { $t = CQL3Type.Native.COUNTER; }
     | K_DECIMAL   { $t = CQL3Type.Native.DECIMAL; }
     | K_DOUBLE    { $t = CQL3Type.Native.DOUBLE; }
-    | K_DURATION    { $t = CQL3Type.Native.DURATION; }
+    | K_DURATION  { $t = CQL3Type.Native.DURATION; }
     | K_FLOAT     { $t = CQL3Type.Native.FLOAT; }
     | K_INET      { $t = CQL3Type.Native.INET;}
     | K_INT       { $t = CQL3Type.Native.INT; }
diff --git a/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java b/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
index 0083a31..4761d66 100644
--- a/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
+++ b/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
@@ -148,6 +148,12 @@ public class FunctionCall extends Term.NonTerminal
             return new Raw(name, Collections.singletonList(raw));
         }
 
+        public static Raw newCast(Term.Raw raw, CQL3Type type)
+        {
+            FunctionName name = FunctionName.nativeFunction(CastFcts.getFunctionName(type));
+            return new Raw(name, Collections.singletonList(raw));
+        }
+
         public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
             Function fun = FunctionResolver.get(keyspace, name, terms, receiver.ksName, receiver.cfName, receiver.type);
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index af95352..167ad57 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -45,6 +45,8 @@ import javax.management.remote.rmi.RMIConnectorServer;
 import com.google.common.base.Objects;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.util.File;
 import org.junit.*;
 import org.slf4j.Logger;
@@ -89,6 +91,7 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JMXServerUtils;
+import org.assertj.core.api.Assertions;
 
 import static com.datastax.driver.core.SocketOptions.DEFAULT_CONNECT_TIMEOUT_MILLIS;
 import static com.datastax.driver.core.SocketOptions.DEFAULT_READ_TIMEOUT_MILLIS;
@@ -1528,6 +1531,13 @@ public abstract class CQLTester
         }
     }
 
+    protected void assertInvalidRequestMessage(String errorMessage, String query, Object... values)
+    {
+        Assertions.assertThatThrownBy(() -> execute(query, values))
+                  .isInstanceOf(InvalidRequestException.class)
+                  .hasMessageContaining(errorMessage);
+    }
+
     /**
      * Asserts that the message of the specified exception contains the specified text.
      *
diff --git a/test/unit/org/apache/cassandra/cql3/functions/CastFctsTest.java b/test/unit/org/apache/cassandra/cql3/functions/CastFctsTest.java
index 10f06b2..4278fb0 100644
--- a/test/unit/org/apache/cassandra/cql3/functions/CastFctsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/functions/CastFctsTest.java
@@ -27,6 +27,7 @@ import java.util.Date;
 
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.serializers.SimpleDateSerializer;
+import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.UUIDGen;
 
 import org.junit.Test;
@@ -38,24 +39,6 @@ public class CastFctsTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int primary key, b text, c double)");
 
-        assertInvalidSyntaxMessage("no viable alternative at input '(' (... b, c) VALUES ([CAST](...)",
-                                   "INSERT INTO %s (a, b, c) VALUES (CAST(? AS int), ?, ?)", 1.6, "test", 6.3);
-
-        assertInvalidSyntaxMessage("no viable alternative at input '(' (..." + KEYSPACE + "." + currentTable()
-                + " SET c = [cast](...)",
-                                   "UPDATE %s SET c = cast(? as double) WHERE a = ?", 1, 1);
-
-        assertInvalidSyntaxMessage("no viable alternative at input '(' (...= ? WHERE a = [CAST] (...)",
-                                   "UPDATE %s SET c = ? WHERE a = CAST (? AS INT)", 1, 2.0);
-
-        assertInvalidSyntaxMessage("no viable alternative at input '(' (..." + KEYSPACE + "." + currentTable()
-                + " WHERE a = [CAST] (...)",
-                                   "DELETE FROM %s WHERE a = CAST (? AS INT)", 1, 2.0);
-
-        assertInvalidSyntaxMessage("no viable alternative at input '(' (..." + KEYSPACE + "." + currentTable()
-                + " WHERE a = [CAST] (...)",
-                                   "SELECT * FROM %s WHERE a = CAST (? AS INT)", 1, 2.0);
-
         assertInvalidMessage("a cannot be cast to boolean", "SELECT CAST(a AS boolean) FROM %s");
     }
 
@@ -321,4 +304,256 @@ public class CastFctsTest extends CQLTester
                 "CAST(b AS text) FROM %s"),
                    row((byte) 2, (short) 2, 2, 2L, 2.0F, 2.0, BigDecimal.valueOf(2), "2", "2"));
     }
+
+    /**
+     * Verifies that the {@code CAST} function can be used in the values of {@code INSERT INTO} statements.
+     */
+    @Test
+    public void testCastsInInsertIntoValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        // Simple cast
+        execute("INSERT INTO %s (k, v) VALUES (1, CAST(1.3 AS int))");
+        assertRows(execute("SELECT v FROM %s"), row(1));
+
+        // Nested casts
+        execute("INSERT INTO %s (k, v) VALUES (1, CAST(CAST(CAST(2.3 AS int) AS float) AS int))");
+        assertRows(execute("SELECT v FROM %s"), row(2));
+
+        // Cast of placeholder with type hint
+        execute("INSERT INTO %s (k, v) VALUES (1, CAST((float) ? AS int))", 3.4f);
+        assertRows(execute("SELECT v FROM %s"), row(3));
+
+        // Cast of placeholder without type hint
+        assertInvalidRequestMessage("Ambiguous call to function system.castAsInt",
+                                    "INSERT INTO %s (k, v) VALUES (1, CAST(? AS int))", 3.4f);
+
+        // Type hint of cast
+        execute("INSERT INTO %s (k, v) VALUES (1, (int) CAST(4.9 AS int))");
+        assertRows(execute("SELECT v FROM %s"), row(4));
+
+        // Function of cast
+        execute(String.format("INSERT INTO %%s (k, v) VALUES (1, %s(CAST(5 AS float)))", floatToInt()));
+        assertRows(execute("SELECT v FROM %s"), row(5));
+
+        // Cast of function
+        execute(String.format("INSERT INTO %%s (k, v) VALUES (1, CAST(%s(6) AS int))", intToFloat()));
+        assertRows(execute("SELECT v FROM %s"), row(6));
+    }
+
+    /**
+     * Verifies that the {@code CAST} function can be used in the values of {@code UPDATE} statements.
+     */
+    @Test
+    public void testCastsInUpdateValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        // Simple cast
+        execute("UPDATE %s SET v = CAST(1.3 AS int) WHERE k = 1");
+        assertRows(execute("SELECT v FROM %s"), row(1));
+
+        // Nested casts
+        execute("UPDATE %s SET v = CAST(CAST(CAST(2.3 AS int) AS float) AS int) WHERE k = 1");
+        assertRows(execute("SELECT v FROM %s"), row(2));
+
+        // Cast of placeholder with type hint
+        execute("UPDATE %s SET v = CAST((float) ? AS int) WHERE k = 1", 3.4f);
+        assertRows(execute("SELECT v FROM %s"), row(3));
+
+        // Cast of placeholder without type hint
+        assertInvalidRequestMessage("Ambiguous call to function system.castAsInt",
+                                    "UPDATE %s SET v = CAST(? AS int) WHERE k = 1", 3.4f);
+
+        // Type hint of cast
+        execute("UPDATE %s SET v = (int) CAST(4.9 AS int) WHERE k = 1");
+        assertRows(execute("SELECT v FROM %s"), row(4));
+
+        // Function of cast
+        execute(String.format("UPDATE %%s SET v = %s(CAST(5 AS float)) WHERE k = 1", floatToInt()));
+        assertRows(execute("SELECT v FROM %s"), row(5));
+
+        // Cast of function
+        execute(String.format("UPDATE %%s SET v = CAST(%s(6) AS int) WHERE k = 1", intToFloat()));
+        assertRows(execute("SELECT v FROM %s"), row(6));
+    }
+
+    /**
+     * Verifies that the {@code CAST} function can be used in the {@code WHERE} clause of {@code UPDATE} statements.
+     */
+    @Test
+    public void testCastsInUpdateWhereClause() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        for (int i = 1; i <= 6; i++)
+        {
+            execute("INSERT INTO %s (k) VALUES (?)", i);
+        }
+
+        // Simple cast
+        execute("UPDATE %s SET v = ? WHERE k = CAST(1.3 AS int)", 1);
+        assertRows(execute("SELECT v FROM %s WHERE k = ?", 1), row(1));
+
+        // Nested casts
+        execute("UPDATE %s SET v = ? WHERE k = CAST(CAST(CAST(2.3 AS int) AS float) AS int)", 2);
+        assertRows(execute("SELECT v FROM %s WHERE k = ?", 2), row(2));
+
+        // Cast of placeholder with type hint
+        execute("UPDATE %s SET v = ? WHERE k = CAST((float) ? AS int)", 3, 3.4f);
+        assertRows(execute("SELECT v FROM %s WHERE k = ?", 3), row(3));
+
+        // Cast of placeholder without type hint
+        assertInvalidRequestMessage("Ambiguous call to function system.castAsInt",
+                                    "UPDATE %s SET v = ? WHERE k = CAST(? AS int)", 3, 3.4f);
+
+        // Type hint of cast
+        execute("UPDATE %s SET v = ? WHERE k = (int) CAST(4.9 AS int)", 4);
+        assertRows(execute("SELECT v FROM %s WHERE k = ?", 4), row(4));
+
+        // Function of cast
+        execute(String.format("UPDATE %%s SET v = ? WHERE k = %s(CAST(5 AS float))", floatToInt()), 5);
+        assertRows(execute("SELECT v FROM %s WHERE k = ?", 5), row(5));
+
+        // Cast of function
+        execute(String.format("UPDATE %%s SET v = ? WHERE k = CAST(%s(6) AS int)", intToFloat()), 6);
+        assertRows(execute("SELECT v FROM %s WHERE k = ?", 6), row(6));
+    }
+
+    /**
+     * Verifies that the {@code CAST} function can be used in the {@code WHERE} clause of {@code SELECT} statements.
+     */
+    @Test
+    public void testCastsInSelectWhereClause() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY)");
+
+        for (int i = 1; i <= 6; i++)
+        {
+            execute("INSERT INTO %s (k) VALUES (?)", i);
+        }
+
+        // Simple cast
+        assertRows(execute("SELECT k FROM %s WHERE k = CAST(1.3 AS int)"), row(1));
+
+        // Nested casts
+        assertRows(execute("SELECT k FROM %s WHERE k = CAST(CAST(CAST(2.3 AS int) AS float) AS int)"), row(2));
+
+        // Cast of placeholder with type hint
+        assertRows(execute("SELECT k FROM %s WHERE k = CAST((float) ? AS int)", 3.4f), row(3));
+
+        // Cast of placeholder without type hint
+        assertInvalidRequestMessage("Ambiguous call to function system.castAsInt",
+                                    "SELECT k FROM %s WHERE k = CAST(? AS int)", 3.4f);
+
+        // Type hint of cast
+        assertRows(execute("SELECT k FROM %s WHERE k = (int) CAST(4.9 AS int)"), row(4));
+
+        // Function of cast
+        assertRows(execute(String.format("SELECT k FROM %%s WHERE k = %s(CAST(5 AS float))", floatToInt())), row(5));
+
+        // Cast of function
+        assertRows(execute(String.format("SELECT k FROM %%s WHERE k = CAST(%s(6) AS int)", intToFloat())), row(6));
+    }
+
+    /**
+     * Verifies that the {@code CAST} function can be used in the {@code WHERE} clause of {@code DELETE} statements.
+     */
+    @Test
+    public void testCastsInDeleteWhereClause() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY)");
+
+        for (int i = 1; i <= 6; i++)
+        {
+            execute("INSERT INTO %s (k) VALUES (?)", i);
+        }
+
+        // Simple cast
+        execute("DELETE FROM %s WHERE k = CAST(1.3 AS int)");
+        assertEmpty(execute("SELECT * FROM %s WHERE k = ?", 1));
+
+        // Nested casts
+        execute("DELETE FROM %s WHERE k = CAST(CAST(CAST(2.3 AS int) AS float) AS int)");
+        assertEmpty(execute("SELECT * FROM %s WHERE k = ?", 2));
+
+        // Cast of placeholder with type hint
+        execute("DELETE FROM %s WHERE k = CAST((float) ? AS int)", 3.4f);
+        assertEmpty(execute("SELECT * FROM %s WHERE k = ?", 3));
+
+        // Cast of placeholder without type hint
+        assertInvalidRequestMessage("Ambiguous call to function system.castAsInt",
+                                    "DELETE FROM %s WHERE k = CAST(? AS int)", 3.4f);
+
+        // Type hint of cast
+        execute("DELETE FROM %s WHERE k = (int) CAST(4.9 AS int)");
+        assertEmpty(execute("SELECT * FROM %s WHERE k = ?", 4));
+
+        // Function of cast
+        execute(String.format("DELETE FROM %%s WHERE k = %s(CAST(5 AS float))", floatToInt()));
+        assertEmpty(execute("SELECT * FROM %s WHERE k = ?", 5));
+
+        // Cast of function
+        execute(String.format("DELETE FROM %%s WHERE k = CAST(%s(6) AS int)", intToFloat()));
+        assertEmpty(execute("SELECT * FROM %s WHERE k = ?", 6));
+    }
+
+    /**
+     * Creates a CQL function that casts an {@code int} argument into a {@code float}.
+     *
+     * @return the name of the created function
+     */
+    private String floatToInt() throws Throwable
+    {
+        return createFunction(KEYSPACE,
+                              "int, int",
+                              "CREATE FUNCTION IF NOT EXISTS %s (x float) " +
+                              "CALLED ON NULL INPUT " +
+                              "RETURNS int " +
+                              "LANGUAGE java " +
+                              "AS 'return Float.valueOf(x).intValue();'");
+    }
+
+    /**
+     * Creates a CQL function that casts a {@code float} argument into an {@code int}.
+     *
+     * @return the name of the created function
+     */
+    private String intToFloat() throws Throwable
+    {
+        return createFunction(KEYSPACE,
+                              "int, int",
+                              "CREATE FUNCTION IF NOT EXISTS %s (x int) " +
+                              "CALLED ON NULL INPUT " +
+                              "RETURNS float " +
+                              "LANGUAGE java " +
+                              "AS 'return (float) x;'");
+    }
+
+    /**
+     * Verifies that the {@code CAST} function can be used in the {@code WHERE} clause of {@code CREATE MATERIALIZED
+     * VIEW} statements.
+     */
+    @Test
+    public void testCastsInCreateViewWhereClause() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
+
+        String viewName = keyspace() + ".mv_with_cast";
+        execute(String.format("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s" +
+                              "   WHERE k < CAST(3.14 AS int) AND v IS NOT NULL" +
+                              "   PRIMARY KEY (v, k)", viewName));
+
+        // start storage service so MV writes are applied
+        StorageService.instance.initServer();
+
+        execute("INSERT INTO %s (k, v) VALUES (1, 10)");
+        execute("INSERT INTO %s (k, v) VALUES (2, 20)");
+        execute("INSERT INTO %s (k, v) VALUES (3, 30)");
+
+        assertRows(execute(String.format("SELECT * FROM %s", viewName)), row(10, 1), row(20, 2));
+
+        execute("DROP MATERIALIZED VIEW " + viewName);
+    }
 }

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org