adelapena commented on code in PR #2655:
URL: https://github.com/apache/cassandra/pull/2655#discussion_r1327980575


##########
test/unit/org/apache/cassandra/db/CellTest.java:
##########
@@ -370,7 +373,13 @@ private static ByteBuffer bb(String str)
 
     private static ByteBuffer udt(ByteBuffer...buffers)
     {
-        return UserType.buildValue(buffers);
+        List<AbstractType<?>> types = new ArrayList<>(buffers.length);
+        for (int i = 0, m = buffers.length; i < m; i++)
+        {
+            types.add(BytesType.instance);
+        }
+        TupleType tupleType = new TupleType(types);
+        return tupleType.pack(Arrays.asList(buffers));

Review Comment:
   We don't need to use `Arrays.asList`, this can use the `pack` overload that 
does the same.



##########
test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java:
##########
@@ -308,13 +307,13 @@ private void tupleCkReadWrite(Order order)
             for (ByteBuffer value : testcase.uniqueRows)
             {
                 map.put(value, count);
-                ByteBuffer[] tupleBuffers = 
tupleType.split(ByteBufferAccessor.instance, value);
+                Object[] tupleBuffers = tupleType.unpack(value).toArray();
 
                 // use cast to avoid warning
-                execute("INSERT INTO %s (pk, ck, value) VALUES (?, ?, ?)", 1, 
tuple((Object[]) tupleBuffers), count);
+                execute("INSERT INTO %s (pk, ck, value) VALUES (?, ?, ?)", 1, 
tuple(tupleBuffers), count);

Review Comment:
   The comment `use cast to avoid warning` seems unnecessary now.



##########
src/java/org/apache/cassandra/service/paxos/PaxosRepairHistory.java:
##########
@@ -171,7 +170,7 @@ public List<ByteBuffer> toTupleBufferList()
     {
         List<ByteBuffer> tuples = new ArrayList<>(size() + 1);
         for (int i = 0 ; i < 1 + size() ; ++i)
-            tuples.add(TupleType.buildValue(new ByteBuffer[] { 
TOKEN_FACTORY.toByteArray(tokenInclusiveUpperBound(i)), 
ballotLowBound[i].toBytes() }));
+            
tuples.add(TYPE.pack(Arrays.asList(TOKEN_FACTORY.toByteArray(tokenInclusiveUpperBound(i)),
 ballotLowBound[i].toBytes())));

Review Comment:
   We don't need to use `Arrays.asList`, this can use the `pack` overload that 
does the same.



##########
test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java:
##########
@@ -270,13 +269,13 @@ public void tuplePartitionReadWrite()
             for (ByteBuffer value : testcase.uniqueRows)
             {
                 map.put(value, count);
-                ByteBuffer[] tupleBuffers = 
tupleType.split(ByteBufferAccessor.instance, value);
+                Object[] tupleBuffers = tupleType.unpack(value).toArray();
 
                 // use cast to avoid warning
-                execute("INSERT INTO %s (id, value) VALUES (?, ?)", 
tuple((Object[]) tupleBuffers), count);
+                execute("INSERT INTO %s (id, value) VALUES (?, ?)", 
tuple(tupleBuffers), count);

Review Comment:
   The comment `use cast to avoid warning` seems unnecessary now.



##########
test/unit/org/apache/cassandra/schema/TableMetadataTest.java:
##########
@@ -52,13 +52,14 @@ public void testPartitionKeyAsCQLLiteral()
                      
metadata1.partitionKeyAsCQLLiteral(type1.decompose("test:", "composite!", 
"type)")));
 
         // composite type with tuple
-        CompositeType type2 = CompositeType.getInstance(new 
TupleType(Arrays.asList(FloatType.instance, UTF8Type.instance)),
+        TupleType tupleType = new TupleType(Arrays.asList(FloatType.instance, 
UTF8Type.instance));
+        CompositeType type2 = CompositeType.getInstance(tupleType,
                                                         IntegerType.instance);
         TableMetadata metadata2 = TableMetadata.builder(keyspaceName, 
tableName)
                                                .addPartitionKeyColumn("key", 
type2)
                                                .build();
-        ByteBuffer tupleValue = TupleType.buildValue(new ByteBuffer[]{ 
FloatType.instance.decompose(0.33f),
-                                                                       
UTF8Type.instance.decompose("tuple test") });
+        ByteBuffer tupleValue = 
tupleType.pack(Arrays.asList(FloatType.instance.decompose(0.33f),

Review Comment:
   We don't need to use `Arrays.asList`, this can use the `pack` overload that 
does the same.



##########
src/java/org/apache/cassandra/cql3/MultiElements.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.MultiElementType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.MarshalException;
+
+/**
+ * Static classes for MultiElementTypes.
+ */
+public final class MultiElements

Review Comment:
   Since we are refactoring, I wonder whether it would make sense to make a new 
 `org.apache.cassandra.cql3.terms` package and put all the terms there, instead 
of having them spread over the generic `cql` package. wdyt?
   



##########
src/java/org/apache/cassandra/db/marshal/MultiElementType.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Base type for the types being composed of multi-elements like Collections, 
Tuples, UDTs or Vectors.
+ * This class unifies the methods used by the CQL layer to work with those 
types.
+ */
+public abstract class MultiElementType<T> extends AbstractType<T>
+{
+    protected MultiElementType(ComparisonType comparisonType)
+    {
+        super(comparisonType);
+    }
+
+    /**
+     * Returns the serialized representation of the value composed of the 
specified elements.
+     *
+     * @param elements the serialized values of the elements
+     * @return the serialized representation of the value composed of the 
specified elements.
+     */
+    public abstract ByteBuffer pack(List<ByteBuffer> elements);
+
+    /**
+     * Returns the serialized representation of the elements composing the 
specified value.
+     *
+     * @param value a serialized value of this type
+     * @return the serialized representation of the elements composing the 
specified value.
+     */
+    public abstract List<ByteBuffer> unpack(ByteBuffer value);
+
+    /**
+     * Checks if this type support bind makers for its elements when the type 
value is provided through a litteral.
+     * @return {@code true} if this type support bind makers for its elements, 
{@code false} otherwise.
+     */
+    public boolean supportElementBindMarkers()
+    {
+        return true;
+    }
+
+    /**
+     * Filter and sort the elements, if needed, before validating them.
+     * <p>
+     * This method take as input a list of elements, eliminate duplicates and 
reorder them if needed (e.g. {@code SetType} and {@code MapType}) and validate 
them.

Review Comment:
   ```suggestion
        * This method takes as input a list of elements, eliminates duplicates 
and reorders them if needed
        * (e.g. {@code SetType} and {@code MapType}) and validates them.
        *
   ```



##########
src/java/org/apache/cassandra/serializers/AbstractTextSerializer.java:
##########
@@ -53,7 +53,7 @@ public ByteBuffer serialize(String value)
     }
 
 
-        public String toString(String value)
+    public String toString(String value)

Review Comment:
   Nit: since we are here, we can add `@Override`



##########
src/java/org/apache/cassandra/serializers/CollectionSerializer.java:
##########
@@ -35,35 +37,106 @@
 public abstract class CollectionSerializer<T> extends TypeSerializer<T>
 {
     protected abstract List<ByteBuffer> serializeValues(T value);
-    protected abstract int getElementCount(T value);
 
     @Override
     public ByteBuffer serialize(T input)
     {
         List<ByteBuffer> values = serializeValues(input);
-        return pack(values, ByteBufferAccessor.instance, 
getElementCount(input));
+        return pack(values, ByteBufferAccessor.instance);
     }
 
-    public static ByteBuffer pack(Collection<ByteBuffer> values, int elements)
+    public ByteBuffer pack(Collection<ByteBuffer> values)

Review Comment:
   There are some unused methods that we can remove: `serializeList` and 
`serializedSizeList`. I think those were unused before this patch too.



##########
src/java/org/apache/cassandra/db/SystemKeyspace.java:
##########
@@ -1889,8 +1889,7 @@ public static void saveTopPartitions(TableMetadata 
metadata, String topType, Col
         List<ByteBuffer> tupleList = new ArrayList<>(topPartitions.size());
         topPartitions.forEach(tp -> {
             String key = metadata.partitionKeyType.getString(tp.key.getKey());
-            tupleList.add(TupleType.buildValue(new ByteBuffer[] { 
UTF8Type.instance.decompose(key),
-                                                                  
LongType.instance.decompose(tp.value)}));
+            
tupleList.add(TOP_TUPLE_TYPE.pack(Arrays.asList(UTF8Type.instance.decompose(key),
 LongType.instance.decompose(tp.value))));

Review Comment:
   We don't need to use `Arrays.asList`, this can use the `pack` overload that 
does the same.



##########
src/java/org/apache/cassandra/cql3/MultiElements.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.MultiElementType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.MarshalException;
+
+/**
+ * Static classes for MultiElementTypes.

Review Comment:
   Maybe `Terms representing individual values of multi-valued data types.`?



##########
test/unit/org/apache/cassandra/utils/bytecomparable/AbstractTypeByteSourceTest.java:
##########
@@ -909,11 +909,11 @@ public void testTupleType()
                 {
                     for (byte[] bytes : bytesValues)
                     {
-                        ByteBuffer tupleData = 
TupleType.buildValue(UTF8Type.instance.decompose(utf8),
-                                                                    decimal != 
null ? DecimalType.instance.decompose(decimal) : null,
-                                                                    varint != 
null ? IntegerType.instance.decompose(varint) : null,
-                                                                    // We 
could also use the wrapped bytes directly
-                                                                    
BytesType.instance.decompose(ByteBuffer.wrap(bytes)));
+                        ByteBuffer tupleData = 
tt.pack(Arrays.asList(UTF8Type.instance.decompose(utf8),

Review Comment:
   We don't need to use `Arrays.asList`, this can use the `pack` overload that 
does the same.



##########
src/java/org/apache/cassandra/db/marshal/MultiElementType.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Base type for the types being composed of multi-elements like Collections, 
Tuples, UDTs or Vectors.
+ * This class unifies the methods used by the CQL layer to work with those 
types.
+ */
+public abstract class MultiElementType<T> extends AbstractType<T>

Review Comment:
   Looking around for terminology, I think this new (and nice!) class might be 
better named `MultiValuedType`, referred to in docs as `multi-valued data 
type`, etc. wdyt?



##########
src/java/org/apache/cassandra/cql3/MultiElements.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.MultiElementType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.MarshalException;
+
+/**
+ * Static classes for MultiElementTypes.
+ */
+public final class MultiElements
+{
+    private MultiElements()
+    {
+    }
+
+    /**
+     * The serialized elements a multi-element type (collection, tuple, udt, 
...)
+     */
+    public static class Value extends Term.Terminal
+    {
+        /**
+         * The type represented by this {@code Value}
+         */
+        private final MultiElementType<?> type;
+
+        /**
+         * The serialized values of the elements composing this value.
+         */
+        private final List<ByteBuffer> elements;
+
+        /**
+         * Creates a {@code Value} from its serialized representation.
+         *
+         * @param value a serialized value from the specified type
+         * @param type the value type
+         * @return a {@code Value}
+         */
+        public static Value fromSerialized(ByteBuffer value, 
MultiElementType<?> type)
+        {
+            try
+            {
+                // We depend for SetType and MapType on the collections being 
without duplicated keys and sorted.
+                return new Value(type, 
type.filterSortAndValidateElements(type.unpack(value)));
+            }
+            catch (MarshalException e)
+            {
+                throw new InvalidRequestException(e.getMessage());
+            }
+        }
+
+        public Value(MultiElementType<?> type, List<ByteBuffer> elements)
+        {
+            this.type = type;
+            this.elements = elements;
+        }
+
+        @Override
+        public ByteBuffer get()
+        {
+            return type.pack(elements);
+        }
+
+        @Override
+        public List<ByteBuffer> getElements()
+        {
+            return elements;
+        }
+    }
+
+    /**
+     * The terms representing a multi-element value (collection, tuple, udt, 
...) where at least one of the terms
+     * represent a non-pure functions or a bind marker.

Review Comment:
   ```suggestion
        * represents a non-pure function or a bind marker.
   ```



##########
src/java/org/apache/cassandra/cql3/MultiElements.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.MultiElementType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.MarshalException;
+
+/**
+ * Static classes for MultiElementTypes.
+ */
+public final class MultiElements
+{
+    private MultiElements()
+    {
+    }
+
+    /**
+     * The serialized elements a multi-element type (collection, tuple, udt, 
...)
+     */
+    public static class Value extends Term.Terminal
+    {
+        /**
+         * The type represented by this {@code Value}
+         */
+        private final MultiElementType<?> type;
+
+        /**
+         * The serialized values of the elements composing this value.
+         */
+        private final List<ByteBuffer> elements;
+
+        /**
+         * Creates a {@code Value} from its serialized representation.
+         *
+         * @param value a serialized value from the specified type
+         * @param type the value type
+         * @return a {@code Value}
+         */
+        public static Value fromSerialized(ByteBuffer value, 
MultiElementType<?> type)
+        {
+            try
+            {
+                // We depend for SetType and MapType on the collections being 
without duplicated keys and sorted.
+                return new Value(type, 
type.filterSortAndValidateElements(type.unpack(value)));
+            }
+            catch (MarshalException e)
+            {
+                throw new InvalidRequestException(e.getMessage());
+            }
+        }
+
+        public Value(MultiElementType<?> type, List<ByteBuffer> elements)
+        {
+            this.type = type;
+            this.elements = elements;
+        }
+
+        @Override
+        public ByteBuffer get()
+        {
+            return type.pack(elements);
+        }
+
+        @Override
+        public List<ByteBuffer> getElements()
+        {
+            return elements;
+        }
+    }
+
+    /**
+     * The terms representing a multi-element value (collection, tuple, udt, 
...) where at least one of the terms
+     * represent a non-pure functions or a bind marker.
+     */
+    public static class DelayedValue extends Term.NonTerminal
+    {
+        /**
+         * The type represented by this {@code DelayedValue}

Review Comment:
   ```suggestion
            * The type of this value.
   ```



##########
src/java/org/apache/cassandra/cql3/Marker.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.MultiElementType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static 
org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+
+/**
+ * A placeholder, also called bind marker, for a single value represented in 
CQL as '?' for an unnamed marker or ':<name>' for a named marker.
+ * For example, 'SELECT ... WHERE pk = ?' or 'SELECT ... WHERE pk = :myKey '.
+ */
+public final class Marker extends Term.NonTerminal
+{
+    private final int bindIndex;
+
+    private final ColumnSpecification receiver;
+
+    private Marker(int bindIndex, ColumnSpecification receiver)
+    {
+        this.bindIndex = bindIndex;
+        this.receiver = receiver;
+    }
+
+    @Override
+    public void collectMarkerSpecification(VariableSpecifications boundNames)
+    {
+        boundNames.add(bindIndex, receiver);
+    }
+
+    @Override
+    public boolean containsBindMarker()
+    {
+        return true;
+    }
+
+    @Override
+    public void addFunctionsTo(List<Function> functions)
+    {
+    }
+
+    @Override
+    public Term.Terminal bind(QueryOptions options) throws 
InvalidRequestException
+    {
+        try
+        {
+            ByteBuffer bytes = options.getValues().get(bindIndex);
+            if (bytes == null)
+                return null;
+
+            if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+            {
+                checkFalse(receiver.type.isTuple(), "Invalid unset value for 
tuple %s", receiver.name);
+                return Constants.UNSET_VALUE;
+            }
+
+            if (receiver.type instanceof MultiElementType<?>)
+                return MultiElements.Value.fromSerialized(bytes, 
(MultiElementType<?>) receiver.type);
+
+            receiver.type.validate(bytes);
+            return new Constants.Value(bytes);
+        }
+            catch (MarshalException e)

Review Comment:
   Nit: extra tab



##########
src/java/org/apache/cassandra/cql3/MultiElements.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.MultiElementType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.MarshalException;
+
+/**
+ * Static classes for MultiElementTypes.
+ */
+public final class MultiElements
+{
+    private MultiElements()
+    {
+    }
+
+    /**
+     * The serialized elements a multi-element type (collection, tuple, udt, 
...)
+     */
+    public static class Value extends Term.Terminal
+    {
+        /**
+         * The type represented by this {@code Value}
+         */
+        private final MultiElementType<?> type;
+
+        /**
+         * The serialized values of the elements composing this value.
+         */
+        private final List<ByteBuffer> elements;
+
+        /**
+         * Creates a {@code Value} from its serialized representation.
+         *
+         * @param value a serialized value from the specified type
+         * @param type the value type
+         * @return a {@code Value}
+         */
+        public static Value fromSerialized(ByteBuffer value, 
MultiElementType<?> type)
+        {
+            try
+            {
+                // We depend for SetType and MapType on the collections being 
without duplicated keys and sorted.
+                return new Value(type, 
type.filterSortAndValidateElements(type.unpack(value)));
+            }
+            catch (MarshalException e)
+            {
+                throw new InvalidRequestException(e.getMessage());
+            }
+        }
+
+        public Value(MultiElementType<?> type, List<ByteBuffer> elements)
+        {
+            this.type = type;
+            this.elements = elements;
+        }
+
+        @Override
+        public ByteBuffer get()
+        {
+            return type.pack(elements);
+        }
+
+        @Override
+        public List<ByteBuffer> getElements()
+        {
+            return elements;
+        }
+    }
+
+    /**
+     * The terms representing a multi-element value (collection, tuple, udt, 
...) where at least one of the terms
+     * represent a non-pure functions or a bind marker.
+     */
+    public static class DelayedValue extends Term.NonTerminal
+    {
+        /**
+         * The type represented by this {@code DelayedValue}
+         */
+        private final MultiElementType<?> type;
+
+        /**
+         * The terms representing the elements composing this value.
+         */
+        private final List<Term> elements;
+
+        public DelayedValue(MultiElementType<?> type, List<Term> elements)
+        {
+            this.type = type;
+            this.elements = elements;
+        }
+
+        @Override
+        public void collectMarkerSpecification(VariableSpecifications 
boundNames)
+        {
+            if (type.supportElementBindMarkers())
+            {
+                for (int i = 0, m = elements.size(); i < m; i++)
+                    elements.get(i).collectMarkerSpecification(boundNames);
+            }
+        }
+
+        @Override
+        public Terminal bind(QueryOptions options)
+        {
+            try
+            {
+                List<ByteBuffer> buffers = new 
ArrayList<ByteBuffer>(elements.size());

Review Comment:
   ```suggestion
                   List<ByteBuffer> buffers = new ArrayList<>(elements.size());
   ```



##########
src/java/org/apache/cassandra/cql3/MultiElements.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.MultiElementType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.MarshalException;
+
+/**
+ * Static classes for MultiElementTypes.
+ */
+public final class MultiElements
+{
+    private MultiElements()
+    {
+    }
+
+    /**
+     * The serialized elements a multi-element type (collection, tuple, udt, 
...)

Review Comment:
   ```suggestion
        * The serialized elements of a multi-element type (collection, tuple, 
udt, ...)
   ```



##########
src/java/org/apache/cassandra/db/marshal/MultiElementType.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Base type for the types being composed of multi-elements like Collections, 
Tuples, UDTs or Vectors.
+ * This class unifies the methods used by the CQL layer to work with those 
types.
+ */
+public abstract class MultiElementType<T> extends AbstractType<T>
+{
+    protected MultiElementType(ComparisonType comparisonType)
+    {
+        super(comparisonType);
+    }
+
+    /**
+     * Returns the serialized representation of the value composed of the 
specified elements.
+     *
+     * @param elements the serialized values of the elements
+     * @return the serialized representation of the value composed of the 
specified elements.
+     */
+    public abstract ByteBuffer pack(List<ByteBuffer> elements);
+
+    /**
+     * Returns the serialized representation of the elements composing the 
specified value.
+     *
+     * @param value a serialized value of this type
+     * @return the serialized representation of the elements composing the 
specified value.
+     */
+    public abstract List<ByteBuffer> unpack(ByteBuffer value);
+
+    /**
+     * Checks if this type support bind makers for its elements when the type 
value is provided through a litteral.

Review Comment:
   ```suggestion
        * Checks if this type supports bind markers for its elements when the 
type value is provided through a literal.
        *
   ```



##########
src/java/org/apache/cassandra/db/marshal/MultiElementType.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Base type for the types being composed of multi-elements like Collections, 
Tuples, UDTs or Vectors.
+ * This class unifies the methods used by the CQL layer to work with those 
types.

Review Comment:
   Maybe we can clarify here that it covers both frozen and non-frozen data 
types. For example:
   ```suggestion
    * This class unifies the methods used by the CQL layer to work with those 
types,
    * and it includes both frozen and non-frozen multi-element types.
   ```



##########
src/java/org/apache/cassandra/db/marshal/MultiElementType.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Base type for the types being composed of multi-elements like Collections, 
Tuples, UDTs or Vectors.
+ * This class unifies the methods used by the CQL layer to work with those 
types.
+ */
+public abstract class MultiElementType<T> extends AbstractType<T>
+{
+    protected MultiElementType(ComparisonType comparisonType)
+    {
+        super(comparisonType);
+    }
+
+    /**
+     * Returns the serialized representation of the value composed of the 
specified elements.
+     *
+     * @param elements the serialized values of the elements
+     * @return the serialized representation of the value composed of the 
specified elements.
+     */
+    public abstract ByteBuffer pack(List<ByteBuffer> elements);
+
+    /**
+     * Returns the serialized representation of the elements composing the 
specified value.
+     *
+     * @param value a serialized value of this type
+     * @return the serialized representation of the elements composing the 
specified value.
+     */
+    public abstract List<ByteBuffer> unpack(ByteBuffer value);
+
+    /**
+     * Checks if this type support bind makers for its elements when the type 
value is provided through a litteral.
+     * @return {@code true} if this type support bind makers for its elements, 
{@code false} otherwise.
+     */
+    public boolean supportElementBindMarkers()

Review Comment:
   ```suggestion
       public boolean supportsElementBindMarkers()
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to