You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "adelapena (via GitHub)" <gi...@apache.org> on 2023/06/22 18:36:36 UTC

[GitHub] [cassandra] adelapena opened a new pull request, #2436: CASSANDRA-18613 trunk: Add support for vectors on UDFs

adelapena opened a new pull request, #2436:
URL: https://github.com/apache/cassandra/pull/2436

   (no comment)


-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1259604005


##########
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 the CQL type {@code vector}. Vectors are represented as {@link List}s for convenience, since
+ * it's probably easier for UDFs trying to return a newly created vector to create it as a standard Java list, rather
+ * than using a custom, not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;
+
+        ImmutableList.Builder<E> values = ImmutableList.builder();
+        for (String element : Splitter.on(", ").split(value.substring(1, value.length() - 1)))
+        {
+            values.add(subtypeCodec.parse(element));
+        }
+
+        return values.build();
+    }
+
+    @Override
+    public String format(List<E> value) throws InvalidTypeException
+    {
+        return value == null ? "NULL" : Iterables.toString(value);
+    }
+
+    /**
+     * {@link VectorCodec} for vectors of elements using a fixed-length encoding.
+     */
+    private static class FixedLength<E> extends VectorCodec<E>
+    {
+        public FixedLength(VectorType type, TypeCodec<E> subtypeCodec)
+        {
+            super(type, subtypeCodec);
+        }
+
+        @Override
+        public boolean isValueLengthFixed()
+        {
+            return true;
+        }
+
+        @Override
+        public ByteBuffer serialize(List<E> value, ProtocolVersion protocolVersion) throws InvalidTypeException
+        {
+            if (value == null || type.getDimensions() <= 0)
+                return null;
+
+            ByteBuffer[] valueBuffs = new ByteBuffer[type.getDimensions()];
+            Iterator<E> values = value.iterator();
+            int allValueBuffsSize = 0;
+            for (int i = 0; i < type.getDimensions(); ++i)
+            {
+                ByteBuffer valueBuff = subtypeCodec.serialize(values.next(), protocolVersion);
+                allValueBuffsSize += valueBuff.limit();
+                valueBuff.rewind();
+                valueBuffs[i] = valueBuff;
+            }
+
+            // Since we already did an early return for <= 0 dimensions above
+            assert valueBuffs.length > 0;
+
+            ByteBuffer rv = ByteBuffer.allocate(allValueBuffsSize);
+            for (int i = 0; i < type.getDimensions(); ++i)
+            {
+                rv.put(valueBuffs[i]);
+            }
+            rv.flip();
+            return rv;
+        }
+
+        @Override
+        public List<E> deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion) throws InvalidTypeException
+        {
+            if (bytes == null || bytes.remaining() == 0)
+                return null;
+
+            // Determine element size by dividing count of remaining bytes by number of elements.
+            // This should have a remainder of zero since all elements are of the same fixed size.
+            int elementSize = Math.floorDiv(bytes.remaining(), type.getDimensions());
+            assert bytes.remaining() % type.getDimensions() == 0
+            : String.format("Expected elements of uniform size, observed %d elements with total bytes %d",
+                            type.getDimensions(), bytes.remaining());
+
+            ImmutableList.Builder<E> values = ImmutableList.builder();
+            for (int i = 0; i < type.getDimensions(); ++i)
+            {
+                ByteBuffer slice = bytes.slice();
+                slice.limit(elementSize);
+                values.add(subtypeCodec.deserialize(slice, protocolVersion));
+                bytes.position(bytes.position() + elementSize);
+            }
+
+            // Restore the input ByteBuffer to its original state
+            bytes.rewind();
+
+            return values.build();
+        }
+    }
+
+    /**
+     * {@link VectorCodec} for vectors of elements using a varaible-length encoding.
+     */
+    private static class VariableLength<E> extends VectorCodec<E>
+    {
+        public VariableLength(VectorType type, TypeCodec<E> subtypeCodec)
+        {
+            super(type, subtypeCodec);
+        }
+
+        @Override
+        public ByteBuffer serialize(List<E> values, ProtocolVersion version) throws InvalidTypeException

Review Comment:
   Any improvement on how vectors are encoded should be done in `VectorType`, the `VectorCodec` proposed here is just the driver codec mimicking it. That kind of improvement, if it's worth it, would also apply to collections.



-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1259539805


##########
src/java/org/apache/cassandra/cql3/functions/types/TypeCodec.java:
##########
@@ -488,6 +501,16 @@ public DataType getCqlType()
         return cqlType;
     }
 
+    /**
+     * Checks if all values are of fixed length.
+     *
+     * @return {@code true} if all values are of fixed length, {@code false} otherwise.
+     */
+    public boolean isValueLengthFixed()

Review Comment:
   I think that would create a multiple inheritance problem. For example, `PrimitiveLongCodec` extends `TypeCodec` and has multiple subclasses with both fixed and variable encoding. Those subclasses can't inherit from both `PrimitiveLongCodec` and the proposed Fixed/Variable classes. 
   
   Alternatively, we could make a separate interface just for the `isValueLengthFixed ` method and make `TypeCodec` implement it. That interface would be extended by two Fixed/Variable interfaces with default implementations, and the terminal subclasses of `TypeCodec` would implement them. 
   
   However, I find that adds too much complexity for a single boolean method. I prefer the current approach, which is identical to the one used in `AbstractType#valueLengthIfFixed` et al.



-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1241976206


##########
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:
   +1, I missed that while copy-pasting the driver's code, which uses a different code style.



##########
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:
   Done



-- 
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


[GitHub] [cassandra] bereng commented on pull request #2436: CASSANDRA-18613 trunk: Add support for vectors on UDFs

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#issuecomment-1630722299

   Ok +1 on green CI


-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1241975561


##########
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:
   Here `VECTOR` follows the style of the previous checks for `LIST`, `SET` and `MAP`. The differences start on the checks for `TUPLE` and `UDT`. I have changed the previous code to use the static import in all cases.



-- 
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


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

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#issuecomment-1607437880

   +1 


-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1259600344


##########
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 the CQL type {@code vector}. Vectors are represented as {@link List}s for convenience, since
+ * it's probably easier for UDFs trying to return a newly created vector to create it as a standard Java list, rather
+ * than using a custom, not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;
+
+        ImmutableList.Builder<E> values = ImmutableList.builder();
+        for (String element : Splitter.on(", ").split(value.substring(1, value.length() - 1)))
+        {
+            values.add(subtypeCodec.parse(element));
+        }
+
+        return values.build();
+    }
+
+    @Override
+    public String format(List<E> value) throws InvalidTypeException
+    {
+        return value == null ? "NULL" : Iterables.toString(value);
+    }
+
+    /**
+     * {@link VectorCodec} for vectors of elements using a fixed-length encoding.
+     */
+    private static class FixedLength<E> extends VectorCodec<E>
+    {
+        public FixedLength(VectorType type, TypeCodec<E> subtypeCodec)
+        {
+            super(type, subtypeCodec);
+        }
+
+        @Override
+        public boolean isValueLengthFixed()
+        {
+            return true;
+        }
+
+        @Override
+        public ByteBuffer serialize(List<E> value, ProtocolVersion protocolVersion) throws InvalidTypeException
+        {
+            if (value == null || type.getDimensions() <= 0)

Review Comment:
   I guess we can do that by adding a new `TypeCodec#valueLengthIfFixed` method.



-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1258109165


##########
test/unit/org/apache/cassandra/cql3/validation/operations/CQLVectorTest.java:
##########
@@ -248,25 +250,155 @@ 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 variable 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));
+
+        // Test wrong arguments on function call
+        assertInvalidThrowMessage("cannot be passed as argument 0 of function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s((int) 0, [3, 4]) FROM %%s", f));
+        assertInvalidThrowMessage("cannot be passed as argument 1 of function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2], (int) 0) FROM %%s", f));
+        assertInvalidThrowMessage("Invalid number of arguments in call to function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2]) FROM %%s", f));
+        assertInvalidThrowMessage("Invalid number of arguments in call to function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2], [3, 4], [5, 6]) FROM %%s", f));
+        assertInvalidThrowMessage("Unable to create a vector selector of type vector<int, 2> from 3 elements",
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2, 3], [4, 5, 6]) FROM %%s", f));
+
+        // Test wrong types on function creation
+        assertInvalidThrowMessage("vectors may only have positive dimentions; given 0",
+                                  InvalidRequestException.class,
+                                  "CREATE FUNCTION %s (x vector<int, 0>) " +
+                                  "CALLED ON NULL INPUT " +
+                                  "RETURNS vector<int, 2> " +
+                                  "LANGUAGE java " +
+                                  "AS 'return x;'");
+        assertInvalidThrowMessage("vectors may only have positive dimentions; given 0",
+                                  InvalidRequestException.class,
+                                  "CREATE FUNCTION %s (x vector<int, 2>) " +
+                                  "CALLED ON NULL INPUT " +
+                                  "RETURNS vector<int, 0> " +
+                                  "LANGUAGE java " +
+                                  "AS 'return x;'");
+
+        // function reading and writing a udt vector field
+        String udt = createType("CREATE TYPE %s (v vector<int,2>)");

Review Comment:
   The repeated runs of the test pass: https://app.circleci.com/pipelines/github/adelapena/cassandra/3010/workflows/cfcbe3c8-ec11-46db-96cd-a0b5ef9f6493/jobs/57489/steps
   
   It seems there is some defect in how `CQLTester` does the cleaning up of functions. `CQLTester#afterTest` correctly drops first the function and then the type. But somehow the dropping of the type ends up happening after the dropping of the function. Indeed, if we add a short sleep after dropping the function the error doesn't happen. That's something worth investigating in a separate ticket. For this one, I have just made the test drop the function.



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1257867109


##########
test/unit/org/apache/cassandra/cql3/validation/entities/UFSecurityTest.java:
##########
@@ -118,7 +118,13 @@ public void testSecurityPermissions() throws Throwable
                                     "     java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); return 0d;" +
                                     "} catch (Exception t) {" +
                                     "     throw new RuntimeException(t);" +
-                                    '}'}
+                                    '}'},

Review Comment:
   Formatting?



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1257850291


##########
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 the CQL type {@code vector}. Vectors are represented as {@link List}s for convenience, since
+ * it's probably easier for UDFs trying to return a newly created vector to create it as a standard Java list, rather
+ * than using a custom, not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;
+
+        ImmutableList.Builder<E> values = ImmutableList.builder();
+        for (String element : Splitter.on(", ").split(value.substring(1, value.length() - 1)))
+        {
+            values.add(subtypeCodec.parse(element));
+        }
+
+        return values.build();
+    }
+
+    @Override
+    public String format(List<E> value) throws InvalidTypeException
+    {
+        return value == null ? "NULL" : Iterables.toString(value);
+    }
+
+    /**
+     * {@link VectorCodec} for vectors of elements using a fixed-length encoding.
+     */
+    private static class FixedLength<E> extends VectorCodec<E>
+    {
+        public FixedLength(VectorType type, TypeCodec<E> subtypeCodec)
+        {
+            super(type, subtypeCodec);
+        }
+
+        @Override
+        public boolean isValueLengthFixed()
+        {
+            return true;
+        }
+
+        @Override
+        public ByteBuffer serialize(List<E> value, ProtocolVersion protocolVersion) throws InvalidTypeException
+        {
+            if (value == null || type.getDimensions() <= 0)

Review Comment:
   Given you know in advance the number of elements and their sizes can't you collapse both loops into a single one? Similar to what you do in deserialize?



-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1241976426


##########
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:
   Done



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1258096572


##########
test/unit/org/apache/cassandra/cql3/validation/entities/UFSecurityTest.java:
##########
@@ -118,7 +118,13 @@ public void testSecurityPermissions() throws Throwable
                                     "     java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); return 0d;" +
                                     "} catch (Exception t) {" +
                                     "     throw new RuntimeException(t);" +
-                                    '}'}
+                                    '}'},

Review Comment:
   It's all misaligned. The try-catch isn't indented to align with the ones above yes?



-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1259605343


##########
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 the CQL type {@code vector}. Vectors are represented as {@link List}s for convenience, since
+ * it's probably easier for UDFs trying to return a newly created vector to create it as a standard Java list, rather
+ * than using a custom, not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;

Review Comment:
   I've put the `return` in a new line, if it's what you meant.



-- 
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


[GitHub] [cassandra] adelapena commented on pull request #2436: CASSANDRA-18613 trunk: Add support for vectors on UDFs

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#issuecomment-1630662880

   @bereng I've tried to address the rest of the comments. CI is running for j8: https://app.circleci.com/pipelines/github/adelapena/cassandra/3015/workflows/00c4b26c-2cf3-48c3-a46a-5553be9d45c7
   
   I'll run j11 if j8 is ok and you agree with the changes.


-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1258072684


##########
test/unit/org/apache/cassandra/cql3/validation/entities/UFSecurityTest.java:
##########
@@ -118,7 +118,13 @@ public void testSecurityPermissions() throws Throwable
                                     "     java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); return 0d;" +
                                     "} catch (Exception t) {" +
                                     "     throw new RuntimeException(t);" +
-                                    '}'}
+                                    '}'},

Review Comment:
   What do you want to do with formatting here? I have tried to follow the style of the test. Specifically, what is done for long class names like `org.apache.cassandra.service.StorageService` above.



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1258097738


##########
test/unit/org/apache/cassandra/cql3/validation/entities/UFSecurityTest.java:
##########
@@ -118,7 +118,13 @@ public void testSecurityPermissions() throws Throwable
                                     "     java.lang.Runtime.getRuntime().exec(\"/tmp/foo\"); return 0d;" +
                                     "} catch (Exception t) {" +
                                     "     throw new RuntimeException(t);" +
-                                    '}'}
+                                    '}'},

Review Comment:
   Oh sorry, ICWYM with the long class name now. Ok I guess that's it then, it can't be improved...



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1258131559


##########
test/unit/org/apache/cassandra/cql3/validation/operations/CQLVectorTest.java:
##########
@@ -248,25 +250,155 @@ 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 variable 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));
+
+        // Test wrong arguments on function call
+        assertInvalidThrowMessage("cannot be passed as argument 0 of function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s((int) 0, [3, 4]) FROM %%s", f));
+        assertInvalidThrowMessage("cannot be passed as argument 1 of function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2], (int) 0) FROM %%s", f));
+        assertInvalidThrowMessage("Invalid number of arguments in call to function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2]) FROM %%s", f));
+        assertInvalidThrowMessage("Invalid number of arguments in call to function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2], [3, 4], [5, 6]) FROM %%s", f));
+        assertInvalidThrowMessage("Unable to create a vector selector of type vector<int, 2> from 3 elements",
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2, 3], [4, 5, 6]) FROM %%s", f));
+
+        // Test wrong types on function creation
+        assertInvalidThrowMessage("vectors may only have positive dimentions; given 0",
+                                  InvalidRequestException.class,
+                                  "CREATE FUNCTION %s (x vector<int, 0>) " +
+                                  "CALLED ON NULL INPUT " +
+                                  "RETURNS vector<int, 2> " +
+                                  "LANGUAGE java " +
+                                  "AS 'return x;'");
+        assertInvalidThrowMessage("vectors may only have positive dimentions; given 0",
+                                  InvalidRequestException.class,
+                                  "CREATE FUNCTION %s (x vector<int, 2>) " +
+                                  "CALLED ON NULL INPUT " +
+                                  "RETURNS vector<int, 0> " +
+                                  "LANGUAGE java " +
+                                  "AS 'return x;'");
+
+        // function reading and writing a udt vector field
+        String udt = createType("CREATE TYPE %s (v vector<int,2>)");

Review Comment:
   I think it's good you fixed it for the time being :-)



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1257845193


##########
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 the CQL type {@code vector}. Vectors are represented as {@link List}s for convenience, since
+ * it's probably easier for UDFs trying to return a newly created vector to create it as a standard Java list, rather
+ * than using a custom, not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;

Review Comment:
   formatting?



-- 
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


[GitHub] [cassandra] bereng commented on pull request #2436: CASSANDRA-18613 trunk: Add support for vectors on UDFs

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#issuecomment-1630231843

   @adelapena you're looking into the other things I mentioned? Asking just in case we're not both waiting on each other lol


-- 
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


[GitHub] [cassandra] adelapena closed pull request #2436: CASSANDRA-18613 trunk: Add support for vectors on UDFs

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena closed pull request #2436: CASSANDRA-18613 trunk: Add support for vectors on UDFs
URL: https://github.com/apache/cassandra/pull/2436


-- 
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


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

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
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


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

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1241051785


##########
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
+ * for UDFs trying to return a newly created vector to create it as a standard Java list, rather than using a custom,
+ * not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;
+
+        ImmutableList.Builder<E> values = ImmutableList.builder();
+        for (String element : Splitter.on(", ").split(value.substring(1, value.length() - 1)))

Review Comment:
   Sure  to use  hardcode like this ", " ?  and  it seems that there must be a space after the comma?



-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1241981589


##########
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:
   I'd say those checks are more related to CQL parsing than to UDF execution in particular, but I have added a few tests for completeness.



-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1241980026


##########
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:
   I'd say that plural is correct here since this is a codec for vector instances (vectors). But I have rephrased it to refer to the (singular) vector type instead. So it's `{@link TypeCodec} for the CQL type {@code vector}`, which is the format used by other codec builders.



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1257887712


##########
test/unit/org/apache/cassandra/cql3/validation/operations/CQLVectorTest.java:
##########
@@ -248,25 +250,155 @@ 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 variable 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));
+
+        // Test wrong arguments on function call
+        assertInvalidThrowMessage("cannot be passed as argument 0 of function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s((int) 0, [3, 4]) FROM %%s", f));
+        assertInvalidThrowMessage("cannot be passed as argument 1 of function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2], (int) 0) FROM %%s", f));
+        assertInvalidThrowMessage("Invalid number of arguments in call to function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2]) FROM %%s", f));
+        assertInvalidThrowMessage("Invalid number of arguments in call to function " + f,
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2], [3, 4], [5, 6]) FROM %%s", f));
+        assertInvalidThrowMessage("Unable to create a vector selector of type vector<int, 2> from 3 elements",
+                                  InvalidRequestException.class,
+                                  format("SELECT %s([1, 2, 3], [4, 5, 6]) FROM %%s", f));
+
+        // Test wrong types on function creation
+        assertInvalidThrowMessage("vectors may only have positive dimentions; given 0",
+                                  InvalidRequestException.class,
+                                  "CREATE FUNCTION %s (x vector<int, 0>) " +
+                                  "CALLED ON NULL INPUT " +
+                                  "RETURNS vector<int, 2> " +
+                                  "LANGUAGE java " +
+                                  "AS 'return x;'");
+        assertInvalidThrowMessage("vectors may only have positive dimentions; given 0",
+                                  InvalidRequestException.class,
+                                  "CREATE FUNCTION %s (x vector<int, 2>) " +
+                                  "CALLED ON NULL INPUT " +
+                                  "RETURNS vector<int, 0> " +
+                                  "LANGUAGE java " +
+                                  "AS 'return x;'");
+
+        // function reading and writing a udt vector field
+        String udt = createType("CREATE TYPE %s (v vector<int,2>)");

Review Comment:
   There's some cleanup error on the test if you run it locally, presumably related to this. It'd be good to cleanup in case we ever need to run the test repeatedly.



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1257864560


##########
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 the CQL type {@code vector}. Vectors are represented as {@link List}s for convenience, since
+ * it's probably easier for UDFs trying to return a newly created vector to create it as a standard Java list, rather
+ * than using a custom, not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;
+
+        ImmutableList.Builder<E> values = ImmutableList.builder();
+        for (String element : Splitter.on(", ").split(value.substring(1, value.length() - 1)))
+        {
+            values.add(subtypeCodec.parse(element));
+        }
+
+        return values.build();
+    }
+
+    @Override
+    public String format(List<E> value) throws InvalidTypeException
+    {
+        return value == null ? "NULL" : Iterables.toString(value);
+    }
+
+    /**
+     * {@link VectorCodec} for vectors of elements using a fixed-length encoding.
+     */
+    private static class FixedLength<E> extends VectorCodec<E>
+    {
+        public FixedLength(VectorType type, TypeCodec<E> subtypeCodec)
+        {
+            super(type, subtypeCodec);
+        }
+
+        @Override
+        public boolean isValueLengthFixed()
+        {
+            return true;
+        }
+
+        @Override
+        public ByteBuffer serialize(List<E> value, ProtocolVersion protocolVersion) throws InvalidTypeException
+        {
+            if (value == null || type.getDimensions() <= 0)
+                return null;
+
+            ByteBuffer[] valueBuffs = new ByteBuffer[type.getDimensions()];
+            Iterator<E> values = value.iterator();
+            int allValueBuffsSize = 0;
+            for (int i = 0; i < type.getDimensions(); ++i)
+            {
+                ByteBuffer valueBuff = subtypeCodec.serialize(values.next(), protocolVersion);
+                allValueBuffsSize += valueBuff.limit();
+                valueBuff.rewind();
+                valueBuffs[i] = valueBuff;
+            }
+
+            // Since we already did an early return for <= 0 dimensions above
+            assert valueBuffs.length > 0;
+
+            ByteBuffer rv = ByteBuffer.allocate(allValueBuffsSize);
+            for (int i = 0; i < type.getDimensions(); ++i)
+            {
+                rv.put(valueBuffs[i]);
+            }
+            rv.flip();
+            return rv;
+        }
+
+        @Override
+        public List<E> deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion) throws InvalidTypeException
+        {
+            if (bytes == null || bytes.remaining() == 0)
+                return null;
+
+            // Determine element size by dividing count of remaining bytes by number of elements.
+            // This should have a remainder of zero since all elements are of the same fixed size.
+            int elementSize = Math.floorDiv(bytes.remaining(), type.getDimensions());
+            assert bytes.remaining() % type.getDimensions() == 0
+            : String.format("Expected elements of uniform size, observed %d elements with total bytes %d",
+                            type.getDimensions(), bytes.remaining());
+
+            ImmutableList.Builder<E> values = ImmutableList.builder();
+            for (int i = 0; i < type.getDimensions(); ++i)
+            {
+                ByteBuffer slice = bytes.slice();
+                slice.limit(elementSize);
+                values.add(subtypeCodec.deserialize(slice, protocolVersion));
+                bytes.position(bytes.position() + elementSize);
+            }
+
+            // Restore the input ByteBuffer to its original state
+            bytes.rewind();
+
+            return values.build();
+        }
+    }
+
+    /**
+     * {@link VectorCodec} for vectors of elements using a varaible-length encoding.
+     */
+    private static class VariableLength<E> extends VectorCodec<E>
+    {
+        public VariableLength(VectorType type, TypeCodec<E> subtypeCodec)
+        {
+            super(type, subtypeCodec);
+        }
+
+        @Override
+        public ByteBuffer serialize(List<E> values, ProtocolVersion version) throws InvalidTypeException

Review Comment:
   I was trying to think of a way to encode all sizes upfront, but only the deltas so save some bytes. But we don't support negative VInts...



-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1255969143


##########
src/java/org/apache/cassandra/cql3/functions/types/GettableByIndexData.java:
##########
@@ -465,6 +465,21 @@ public interface GettableByIndexData
      */
     public <K, V> Map<K, V> getMap(int i, TypeToken<K> keysType, TypeToken<V> valuesType);
 
+    /**
+     * Returns the {@code i}th value as a vector.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@link List} (if the CQL type is a UDT, the registry will generate a codec

Review Comment:
   I've wrongly copied that comment from the getter for UDTs. Revisiting this has made me realise that the getters/setters are incorrect for usage with nested types. I have just fixed them and added a new test case covering them.



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1255496680


##########
src/java/org/apache/cassandra/cql3/functions/types/GettableByIndexData.java:
##########
@@ -465,6 +465,21 @@ public interface GettableByIndexData
      */
     public <K, V> Map<K, V> getMap(int i, TypeToken<K> keysType, TypeToken<V> valuesType);
 
+    /**
+     * Returns the {@code i}th value as a vector.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@link List} (if the CQL type is a UDT, the registry will generate a codec

Review Comment:
   You mention UDT here?



-- 
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


[GitHub] [cassandra] adelapena commented on pull request #2436: CASSANDRA-18613 trunk: Add support for vectors on UDFs

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#issuecomment-1630780343

   Thanks, I've rebased and squashed. CI results in a bit.


-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1241986954


##########
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
+ * for UDFs trying to return a newly created vector to create it as a standard Java list, rather than using a custom,
+ * not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;
+
+        ImmutableList.Builder<E> values = ImmutableList.builder();
+        for (String element : Splitter.on(", ").split(value.substring(1, value.length() - 1)))

Review Comment:
   The driver's `parse` method complements the `format` method. The latter uses Guava's `Iterables#toString`, which also uses `", "`: https://github.com/google/guava/blob/master/guava/src/com/google/common/collect/Iterators.java#L292



-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1241977214


##########
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:
   Indeed this is checked before getting here. I have added a defensive assertion.



-- 
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


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

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1243070743


##########
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
+ * for UDFs trying to return a newly created vector to create it as a standard Java list, rather than using a custom,
+ * not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;
+
+        ImmutableList.Builder<E> values = ImmutableList.builder();
+        for (String element : Splitter.on(", ").split(value.substring(1, value.length() - 1)))

Review Comment:
   ok



##########
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:
   thanks



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1257709700


##########
src/java/org/apache/cassandra/cql3/functions/types/GettableByIndexData.java:
##########
@@ -465,6 +465,21 @@ public interface GettableByIndexData
      */
     public <K, V> Map<K, V> getMap(int i, TypeToken<K> keysType, TypeToken<V> valuesType);
 
+    /**
+     * Returns the {@code i}th value as a vector.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@link List} (if the CQL type is a UDT, the registry will generate a codec

Review Comment:
   Nice catch! I hadn't gone that deep yet. Glad you caught it :-)



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1257844457


##########
src/java/org/apache/cassandra/cql3/functions/types/TypeCodec.java:
##########
@@ -488,6 +501,16 @@ public DataType getCqlType()
         return cqlType;
     }
 
+    /**
+     * Checks if all values are of fixed length.
+     *
+     * @return {@code true} if all values are of fixed length, {@code false} otherwise.
+     */
+    public boolean isValueLengthFixed()

Review Comment:
   I wonder if you could create Fixed/Variable classes to avoid all those isValueLengthFixed() methods 



-- 
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


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

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1259624498


##########
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 the CQL type {@code vector}. Vectors are represented as {@link List}s for convenience, since
+ * it's probably easier for UDFs trying to return a newly created vector to create it as a standard Java list, rather
+ * than using a custom, not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;
+
+        ImmutableList.Builder<E> values = ImmutableList.builder();
+        for (String element : Splitter.on(", ").split(value.substring(1, value.length() - 1)))
+        {
+            values.add(subtypeCodec.parse(element));
+        }
+
+        return values.build();
+    }
+
+    @Override
+    public String format(List<E> value) throws InvalidTypeException
+    {
+        return value == null ? "NULL" : Iterables.toString(value);
+    }
+
+    /**
+     * {@link VectorCodec} for vectors of elements using a fixed-length encoding.
+     */
+    private static class FixedLength<E> extends VectorCodec<E>
+    {
+        public FixedLength(VectorType type, TypeCodec<E> subtypeCodec)
+        {
+            super(type, subtypeCodec);
+        }
+
+        @Override
+        public boolean isValueLengthFixed()
+        {
+            return true;
+        }
+
+        @Override
+        public ByteBuffer serialize(List<E> value, ProtocolVersion protocolVersion) throws InvalidTypeException
+        {
+            if (value == null || type.getDimensions() <= 0)

Review Comment:
   The last commit can be greatly simplified imo, just serialize the first element and that tells you the size you need. No need for the `TypeCodec#valueLengthIfFixed` method.
   
   If you prefer this approach though I would rename it to `serializedSize` which seems to be the name used in other serializers.
   
   Wdyt?



-- 
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


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

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2436:
URL: https://github.com/apache/cassandra/pull/2436#discussion_r1259643298


##########
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 the CQL type {@code vector}. Vectors are represented as {@link List}s for convenience, since
+ * it's probably easier for UDFs trying to return a newly created vector to create it as a standard Java list, rather
+ * than using a custom, not-standard vector class.
+ *
+ * @param <E> The type of the vector elements.
+ */
+public abstract class VectorCodec<E> extends TypeCodec<List<E>>
+{
+    protected final VectorType type;
+    protected final TypeCodec<E> subtypeCodec;
+
+    private VectorCodec(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        super(type, TypeTokens.vectorOf(subtypeCodec.getJavaType()));
+        this.type = type;
+        this.subtypeCodec = subtypeCodec;
+    }
+
+    public static <E> VectorCodec<E> of(VectorType type, TypeCodec<E> subtypeCodec)
+    {
+        return subtypeCodec.isValueLengthFixed()
+               ? new FixedLength<>(type, subtypeCodec)
+               : new VariableLength<>(type, subtypeCodec);
+    }
+
+    @Override
+    public List<E> parse(String value) throws InvalidTypeException
+    {
+        if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;
+
+        ImmutableList.Builder<E> values = ImmutableList.builder();
+        for (String element : Splitter.on(", ").split(value.substring(1, value.length() - 1)))
+        {
+            values.add(subtypeCodec.parse(element));
+        }
+
+        return values.build();
+    }
+
+    @Override
+    public String format(List<E> value) throws InvalidTypeException
+    {
+        return value == null ? "NULL" : Iterables.toString(value);
+    }
+
+    /**
+     * {@link VectorCodec} for vectors of elements using a fixed-length encoding.
+     */
+    private static class FixedLength<E> extends VectorCodec<E>
+    {
+        public FixedLength(VectorType type, TypeCodec<E> subtypeCodec)
+        {
+            super(type, subtypeCodec);
+        }
+
+        @Override
+        public boolean isValueLengthFixed()
+        {
+            return true;
+        }
+
+        @Override
+        public ByteBuffer serialize(List<E> value, ProtocolVersion protocolVersion) throws InvalidTypeException
+        {
+            if (value == null || type.getDimensions() <= 0)

Review Comment:
   `valueLengthIfFixed` is the name used in `AbstractType`, but I guess the names can diverge based on preferences. I would however keep the fixed size method rather than trying to infer it from the first value.



-- 
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