You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "Maxwell-Guo (via GitHub)" <gi...@apache.org> on 2023/06/25 03:23:53 UTC

[GitHub] [cassandra] Maxwell-Guo commented on a diff in pull request #2436: CASSANDRA-18613 trunk: Add support for vectors on UDFs

Maxwell-Guo commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1241005977


##########
src/java/org/apache/cassandra/cql3/functions/types/VectorType.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.functions.types;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A vector type.
+ */
+public class VectorType extends DataType
+{
+    private final DataType subtype;
+    private final int dimensions;
+
+    VectorType(DataType subtype, int dimensions)
+    {
+        super(Name.VECTOR);
+        this.subtype = subtype;
+        this.dimensions = dimensions;
+    }
+
+    public DataType getSubtype() {
+        return this.subtype;
+    }
+
+    public int getDimensions() {
+        return this.dimensions;
+    }
+
+    @Override
+    public List<DataType> getTypeArguments()
+    {
+        return Arrays.asList(subtype, DataType.cint());
+    }
+
+    @Override
+    public boolean isFrozen()
+    {
+        return true;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        VectorType that = (VectorType) o;
+
+        if (dimensions != that.dimensions) return false;
+        if (!subtype.equals(that.subtype)) return false;
+        return name == that.name;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int result = name.hashCode();
+        result = 31 * result + dimensions;
+        result = 31 * result + subtype.hashCode();
+        return result;
+    }
+
+    @Override
+    public String toString() {

Review Comment:
   I think “{”  should start from a new line



##########
src/java/org/apache/cassandra/cql3/functions/types/VectorType.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.functions.types;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A vector type.
+ */
+public class VectorType extends DataType
+{
+    private final DataType subtype;
+    private final int dimensions;
+
+    VectorType(DataType subtype, int dimensions)
+    {
+        super(Name.VECTOR);
+        this.subtype = subtype;
+        this.dimensions = dimensions;
+    }
+
+    public DataType getSubtype() {
+        return this.subtype;
+    }
+
+    public int getDimensions() {

Review Comment:
   I think “{”  should start from a new line



##########
test/unit/org/apache/cassandra/cql3/validation/operations/CQLVectorTest.java:
##########
@@ -248,25 +250,103 @@ public void token()
     @Test
     public void udf() throws Throwable
     {
-        // For future authors, if this test starts to fail as vectors become supported in UDFs, please update this test
-        // to test the integration and remove the requirement that we reject UDFs all together
         createTable(KEYSPACE, "CREATE TABLE %s (pk int primary key, value vector<int, 2>)");
-        Assertions.assertThatThrownBy(() -> createFunction(KEYSPACE,
-                                                           "",
-                                                           "CREATE FUNCTION %s (x vector<int, 2>) " +
-                                                           "CALLED ON NULL INPUT " +
-                                                           "RETURNS vector<int, 2> " +
-                                                           "LANGUAGE java " +
-                                                           "AS 'return x;'"))
-                  .hasRootCauseMessage("Vectors are not supported on UDFs; given vector<int, 2>");
-
-        Assertions.assertThatThrownBy(() -> createFunction(KEYSPACE,
-                                                           "",
-                                                           "CREATE FUNCTION %s (x list<vector<int, 2>>) " +
-                                                           "CALLED ON NULL INPUT " +
-                                                           "RETURNS list<vector<int, 2>> " +
-                                                           "LANGUAGE java " +
-                                                           "AS 'return x;'"))
-                  .hasRootCauseMessage("Vectors are not supported on UDFs; given list<vector<int, 2>>");
+        Vector<Integer> vector = vector(1, 2);
+        execute("INSERT INTO %s (pk, value) VALUES (0, ?)", vector);
+
+        // identitiy function
+        String f =  createFunction(KEYSPACE,
+                                   "",
+                                   "CREATE FUNCTION %s (x vector<int, 2>) " +
+                                   "CALLED ON NULL INPUT " +
+                                   "RETURNS vector<int, 2> " +
+                                   "LANGUAGE java " +
+                                   "AS 'return x;'");
+        assertRows(execute(format("SELECT %s(value) FROM %%s", f)), row(vector));
+        assertRows(execute(format("SELECT %s([2, 3]) FROM %%s", f)), row(vector(2, 3)));
+        assertRows(execute(format("SELECT %s(null) FROM %%s", f)), row((Vector<Integer>) null));
+
+        // identitiy function with nested type
+        f = createFunction(KEYSPACE,
+                           "",
+                           "CREATE FUNCTION %s (x list<vector<int, 2>>) " +
+                           "CALLED ON NULL INPUT " +
+                           "RETURNS list<vector<int, 2>> " +
+                           "LANGUAGE java " +
+                           "AS 'return x;'");
+        assertRows(execute(format("SELECT %s([value]) FROM %%s", f)), row(list(vector)));
+        assertRows(execute(format("SELECT %s([[2, 3]]) FROM %%s", f)), row(list(vector(2, 3))));
+        assertRows(execute(format("SELECT %s(null) FROM %%s", f)), row((Vector<Integer>) null));
+
+        // identitiy function with elements of variable length
+        f =  createFunction(KEYSPACE,
+                            "",
+                            "CREATE FUNCTION %s (x vector<text, 2>) " +
+                            "CALLED ON NULL INPUT " +
+                            "RETURNS vector<text, 2> " +
+                            "LANGUAGE java " +
+                            "AS 'return x;'");
+        assertRows(execute(format("SELECT %s(['abc', 'defghij']) FROM %%s", f)), row(vector("abc", "defghij")));
+        assertRows(execute(format("SELECT %s(null) FROM %%s", f)), row((Vector<Integer>) null));
+
+        // function accessing vector argument elements
+        f = createFunction(KEYSPACE,
+                           "",
+                           "CREATE FUNCTION %s (x vector<int, 2>, i int) " +
+                           "CALLED ON NULL INPUT " +
+                           "RETURNS int " +
+                           "LANGUAGE java " +
+                           "AS 'return x == null ? null : x.get(i);'");
+        assertRows(execute(format("SELECT %s(value, 0), %<s(value, 1) FROM %%s", f)), row(1, 2));
+        assertRows(execute(format("SELECT %s([2, 3], 0), %<s([2, 3], 1) FROM %%s", f)), row(2, 3));
+        assertRows(execute(format("SELECT %s(null, 0) FROM %%s", f)), row((Integer) null));
+
+        // function accessing vector argument dimensions
+        f = createFunction(KEYSPACE,
+                           "",
+                           "CREATE FUNCTION %s (x vector<int, 2>) " +
+                           "CALLED ON NULL INPUT " +
+                           "RETURNS int " +
+                           "LANGUAGE java " +
+                           "AS 'return x == null ? 0 : x.size();'");
+        assertRows(execute(format("SELECT %s(value) FROM %%s", f)), row(2));
+        assertRows(execute(format("SELECT %s([2, 3]) FROM %%s", f)), row(2));
+        assertRows(execute(format("SELECT %s(null) FROM %%s", f)), row(0));
+
+        // build vector with elements of fixed length
+        f = createFunction(KEYSPACE,
+                           "",
+                           "CREATE FUNCTION %s () " +
+                           "CALLED ON NULL INPUT " +
+                           "RETURNS vector<double, 3> " +
+                           "LANGUAGE java " +
+                           "AS 'return Arrays.asList(1.3, 2.2, 3.1);'");
+        assertRows(execute(format("SELECT %s() FROM %%s", f)), row(vector(1.3, 2.2, 3.1)));
+
+        // build vector with elements of varaible length
+        f = createFunction(KEYSPACE,
+                           "",
+                           "CREATE FUNCTION %s () " +
+                           "CALLED ON NULL INPUT " +
+                           "RETURNS vector<text, 3> " +
+                           "LANGUAGE java " +
+                           "AS 'return Arrays.asList(\"a\", \"bc\", \"def\");'");
+        assertRows(execute(format("SELECT %s() FROM %%s", f)), row(vector("a", "bc", "def")));
+
+        // concat vectors, just to put it all together
+        f =  createFunction(KEYSPACE,
+                            "",
+                            "CREATE FUNCTION %s (x vector<int, 2>, y vector<int, 2>) " +
+                            "CALLED ON NULL INPUT " +
+                            "RETURNS vector<int, 4> " +
+                            "LANGUAGE java " +
+                            "AS '" +
+                            "if (x == null || y == null) return null;" +
+                            "List<Integer> l = new ArrayList<Integer>(x); " +
+                            "l.addAll(y); " +
+                            "return l;'");
+        assertRows(execute(format("SELECT %s(value, [3, 4]) FROM %%s", f)), row(vector(1, 2, 3, 4)));
+        assertRows(execute(format("SELECT %s([2, 3], value) FROM %%s", f)), row(vector(2, 3, 1, 2)));
+        assertRows(execute(format("SELECT %s(null, null) FROM %%s", f)), row((Vector<Integer>) null));

Review Comment:
   Should some exception cases be added?
   like: vector<int, element1> (element1 is not number), vector<int, element2> (element is equal or less than 0).
   I know this usage is not right.But what we want to see is if the wrong usage  in line with our expectations



##########
src/java/org/apache/cassandra/cql3/functions/types/CodecRegistry.java:
##########
@@ -847,6 +864,12 @@ private <T> TypeCodec<T> maybeCreateCodec(DataType cqlType, T value)
             }
         }
 
+        if ((cqlType == null || cqlType.getName() == VECTOR) && value instanceof List)

Review Comment:
   I know we have imported the DataType type class, so it is ok to set "cqlType.getName() == VECTOR"
   I just have a little bit of cleanliness, I think it is better to align with the similar logic below, either remove their ”DataType.Name.“, or align with them. change to :
   
   `
       
           if ((cqlType == null || cqlType.getName() == TUPLE)
               && value instanceof TupleValue)
           {
               return (TypeCodec<T>)
                      TypeCodec.tuple(cqlType == null ? ((TupleValue) value).getType() : (TupleType) cqlType);
           }
   
           if ((cqlType == null || cqlType.getName() == UDT) && value instanceof UDTValue)
           {
               return (TypeCodec<T>)
                      TypeCodec.userType(cqlType == null ? ((UDTValue) value).getType() : (UserType) cqlType);
           }
   `
   
   or `if ((cqlType == null || cqlType.getName() == DataType.Name.VECTOR) && value instanceof List)`



##########
src/java/org/apache/cassandra/cql3/functions/types/VectorType.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.functions.types;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A vector type.
+ */
+public class VectorType extends DataType
+{
+    private final DataType subtype;
+    private final int dimensions;
+
+    VectorType(DataType subtype, int dimensions)
+    {
+        super(Name.VECTOR);
+        this.subtype = subtype;
+        this.dimensions = dimensions;
+    }
+
+    public DataType getSubtype() {

Review Comment:
   I think “{”  should start from a new line



##########
src/java/org/apache/cassandra/cql3/functions/types/VectorCodec.java:
##########
@@ -0,0 +1,211 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.functions.types;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.cql3.functions.types.exceptions.InvalidTypeException;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.vint.VIntCoding;
+
+/**
+ * {@link TypeCodec} for vectors. Vectors are represented as {@link List}s for convenience, since it's probably easier

Review Comment:
   "{@link TypeCodec} for vectors. Vectors"  , should be vector instead of vectors ?



##########
src/java/org/apache/cassandra/cql3/functions/types/VectorType.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.functions.types;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A vector type.
+ */
+public class VectorType extends DataType
+{
+    private final DataType subtype;
+    private final int dimensions;
+
+    VectorType(DataType subtype, int dimensions)
+    {
+        super(Name.VECTOR);
+        this.subtype = subtype;
+        this.dimensions = dimensions;

Review Comment:
   Do we need to do some validation for dimensions here ? if some one set a dimensions that less than 0 ? Though I know org.apache.cassandra.db.marshal.VectorType 's construct function will do that .



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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