You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sn...@apache.org on 2020/02/12 08:58:51 UTC

[cassandra] branch trunk updated: Remove Java Driver dependency for UDFs and UDAs / Limit the dependencies used by UDFs/UDAs

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

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


The following commit(s) were added to refs/heads/trunk by this push:
     new f145071  Remove Java Driver dependency for UDFs and UDAs / Limit the dependencies used by UDFs/UDAs
f145071 is described below

commit f14507190b7eea7ad92c8c83dc04be9ef439b021
Author: Robert Stupp <sn...@snazy.de>
AuthorDate: Wed Aug 29 17:05:22 2018 +0200

    Remove Java Driver dependency for UDFs and UDAs / Limit the dependencies used by UDFs/UDAs
    
    patch by Robert Stupp; reviewed by Jason Brown, Jonathan Ellis, Ryan Svihla for CASSANDRA-14737
---
 NOTICE.txt                                         |    4 +
 .../cql3/functions/JavaBasedUDFunction.java        |    6 +-
 .../apache/cassandra/cql3/functions/JavaUDF.java   |    2 +-
 .../cql3/functions/ScriptBasedUDFunction.java      |    7 +-
 .../cassandra/cql3/functions/UDAggregate.java      |    2 +-
 .../cql3/functions/UDFByteCodeVerifier.java        |    2 +-
 .../cassandra/cql3/functions/UDFContext.java       |    4 +-
 .../cassandra/cql3/functions/UDFContextImpl.java   |    7 +-
 .../cassandra/cql3/functions/UDFunction.java       |   12 +-
 .../apache/cassandra/cql3/functions/UDHelper.java  |   58 +-
 .../types/AbstractAddressableByIndexData.java      |  330 +++
 .../cql3/functions/types/AbstractData.java         |  677 +++++
 .../types/AbstractGettableByIndexData.java         |  418 +++
 .../cql3/functions/types/AbstractGettableData.java |  325 ++
 .../cql3/functions/types/CodecRegistry.java        |  885 ++++++
 .../cassandra/cql3/functions/types/CodecUtils.java |  266 ++
 .../cassandra/cql3/functions/types/DataType.java   |  703 +++++
 .../functions/types/DataTypeClassNameParser.java   |  396 +++
 .../cassandra/cql3/functions/types/Duration.java   |  654 +++++
 .../cql3/functions/types/GettableByIndexData.java  |  592 ++++
 .../cql3/functions/types/GettableByNameData.java   |  593 ++++
 .../cql3/functions/types/GettableData.java         |   26 +
 .../cassandra/cql3/functions/types/LocalDate.java  |  212 ++
 .../cassandra/cql3/functions/types/Metadata.java   |  125 +
 .../cassandra/cql3/functions/types/ParseUtils.java |  625 ++++
 .../cql3/functions/types/SettableByIndexData.java  |  583 ++++
 .../cql3/functions/types/SettableByNameData.java   |  620 ++++
 .../cql3/functions/types/SettableData.java         |   26 +
 .../cassandra/cql3/functions/types/TupleType.java  |  201 ++
 .../cassandra/cql3/functions/types/TupleValue.java |   92 +
 .../cassandra/cql3/functions/types/TypeCodec.java  | 3103 ++++++++++++++++++++
 .../cassandra/cql3/functions/types/TypeTokens.java |  157 +
 .../cassandra/cql3/functions/types/UDTValue.java   |   96 +
 .../cassandra/cql3/functions/types/UserType.java   |  318 ++
 .../types/exceptions/CodecNotFoundException.java   |   42 +
 .../types/exceptions/DriverException.java          |   37 +
 .../types/exceptions/DriverInternalError.java      |   39 +
 .../types/exceptions/InvalidTypeException.java     |   39 +
 .../cql3/functions/types/package-info.java         |   37 +
 .../cql3/functions/types/utils/Bytes.java          |  219 ++
 .../cassandra/io/sstable/CQLSSTableWriter.java     |   28 +-
 .../cassandra/cql3/functions/JavaSourceUDF.txt     |    6 +-
 test/unit/org/apache/cassandra/cql3/CQLTester.java |    2 +-
 .../cql3/validation/entities/UFJavaTest.java       |   10 +-
 .../cql3/validation/entities/UFVerifierTest.java   |   28 +-
 .../validation/entities/udfverify/CallClone.java   |    2 +-
 .../entities/udfverify/CallComDatastax.java        |   50 -
 .../entities/udfverify/CallFinalize.java           |    2 +-
 .../entities/udfverify/CallOrgApache.java          |    2 +-
 .../entities/udfverify/ClassWithField.java         |    2 +-
 .../entities/udfverify/ClassWithInitializer.java   |    2 +-
 .../entities/udfverify/ClassWithInitializer2.java  |    2 +-
 .../entities/udfverify/ClassWithInitializer3.java  |    2 +-
 .../entities/udfverify/ClassWithInnerClass.java    |    2 +-
 .../entities/udfverify/ClassWithInnerClass2.java   |    2 +-
 .../udfverify/ClassWithStaticInitializer.java      |    2 +-
 .../udfverify/ClassWithStaticInnerClass.java       |    2 +-
 .../validation/entities/udfverify/GoodClass.java   |    2 +-
 .../entities/udfverify/UseOfSynchronized.java      |    2 +-
 .../udfverify/UseOfSynchronizedWithNotify.java     |    2 +-
 .../udfverify/UseOfSynchronizedWithNotifyAll.java  |    2 +-
 .../udfverify/UseOfSynchronizedWithWait.java       |    2 +-
 .../udfverify/UseOfSynchronizedWithWaitL.java      |    2 +-
 .../udfverify/UseOfSynchronizedWithWaitLI.java     |    2 +-
 .../entities/udfverify/UsingMapEntry.java          |    2 +-
 .../cassandra/io/sstable/CQLSSTableWriterTest.java |   15 +-
 .../io/sstable/StressCQLSSTableWriter.java         |   21 +-
 67 files changed, 12519 insertions(+), 219 deletions(-)

diff --git a/NOTICE.txt b/NOTICE.txt
index d9cda42..d135e1e 100644
--- a/NOTICE.txt
+++ b/NOTICE.txt
@@ -4,6 +4,10 @@ Copyright 2009-2020 The Apache Software Foundation
 This product includes software developed by The Apache Software
 Foundation (http://www.apache.org/).
 
+Parts of the DataStax Java Driver included in source form
+(https://github.com/datastax/java-driver)
+Copyright DataStax, Inc.
+
 Some alternate data structures provided by high-scale-lib from
 http://sourceforge.net/projects/high-scale-lib/.
 Written by Cliff Click and released as Public Domain.
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java b/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
index 71b9b89..d2bac5f 100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaBasedUDFunction.java
@@ -44,9 +44,9 @@ import com.google.common.reflect.TypeToken;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.datastax.driver.core.TypeCodec;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.functions.types.TypeCodec;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.transport.ProtocolVersion;
@@ -187,6 +187,8 @@ public final class JavaBasedUDFunction extends UDFunction
 
     private final JavaUDF javaUDF;
 
+    private static final Pattern patternJavaDriver = Pattern.compile("com\\.datastax\\.driver\\.core\\.");
+
     JavaBasedUDFunction(FunctionName name, List<ColumnIdentifier> argNames, List<AbstractType<?>> argTypes,
                         AbstractType<?> returnType, boolean calledOnNullInput, String body)
     {
@@ -223,7 +225,7 @@ public final class JavaBasedUDFunction extends UDFunction
                         break;
                     case "body":
                         lineOffset = countNewlines(javaSourceBuilder);
-                        s = body;
+                        s = patternJavaDriver.matcher(body).replaceAll("org.apache.cassandra.cql3.functions.types.");
                         break;
                     case "arguments":
                         s = generateArguments(javaParamTypes, argNames, false);
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaUDF.java b/src/java/org/apache/cassandra/cql3/functions/JavaUDF.java
index fab29f3..3134da9 100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaUDF.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaUDF.java
@@ -21,7 +21,7 @@ package org.apache.cassandra.cql3.functions;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import com.datastax.driver.core.TypeCodec;
+import org.apache.cassandra.cql3.functions.types.TypeCodec;
 import org.apache.cassandra.transport.ProtocolVersion;
 
 /**
diff --git a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java
index 08d832c..d7e5eb8 100644
--- a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDFunction.java
@@ -89,10 +89,9 @@ final class ScriptBasedUDFunction extends UDFunction
     "com.google.common.collect",
     "com.google.common.reflect",
     // following required by UDF
-    "com.datastax.driver.core",
-    "com.datastax.driver.core.utils",
-    //Driver Metadata class requires hashmap from this
-    "com.datastax.shaded.netty.util.collection"
+    "org.apache.cassandra.cql3.functions.types",
+    "org.apache.cassandra.cql3.functions.types.exceptions",
+    "org.apache.cassandra.cql3.functions.types.utils"
     };
 
     // use a JVM standard ExecutorService as DebuggableThreadPoolExecutor references internal
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
index 1a49b33..96cc556 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
@@ -25,7 +25,7 @@ import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.datastax.driver.core.TypeCodec;
+import org.apache.cassandra.cql3.functions.types.TypeCodec;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.ConfigurationException;
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java b/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java
index 8910298..e3b461c 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFByteCodeVerifier.java
@@ -47,7 +47,7 @@ public final class UDFByteCodeVerifier
 
     public static final String JAVA_UDF_NAME = JavaUDF.class.getName().replace('.', '/');
     public static final String OBJECT_NAME = Object.class.getName().replace('.', '/');
-    public static final String CTOR_SIG = "(Lcom/datastax/driver/core/TypeCodec;[Lcom/datastax/driver/core/TypeCodec;Lorg/apache/cassandra/cql3/functions/UDFContext;)V";
+    public static final String CTOR_SIG = "(Lorg/apache/cassandra/cql3/functions/types/TypeCodec;[Lorg/apache/cassandra/cql3/functions/types/TypeCodec;Lorg/apache/cassandra/cql3/functions/UDFContext;)V";
 
     private final Set<String> disallowedClasses = new HashSet<>();
     private final Multimap<String, String> disallowedMethodCalls = HashMultimap.create();
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFContext.java b/src/java/org/apache/cassandra/cql3/functions/UDFContext.java
index 4465aec..bb298ef 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFContext.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFContext.java
@@ -18,8 +18,8 @@
 
 package org.apache.cassandra.cql3.functions;
 
-import com.datastax.driver.core.TupleValue;
-import com.datastax.driver.core.UDTValue;
+import org.apache.cassandra.cql3.functions.types.TupleValue;
+import org.apache.cassandra.cql3.functions.types.UDTValue;
 
 /**
  * Provides context information for a particular user defined function.
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFContextImpl.java b/src/java/org/apache/cassandra/cql3/functions/UDFContextImpl.java
index 00625cd..d4bdf20 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFContextImpl.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFContextImpl.java
@@ -23,16 +23,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
-import com.datastax.driver.core.DataType;
-import com.datastax.driver.core.TupleType;
-import com.datastax.driver.core.TupleValue;
-import com.datastax.driver.core.TypeCodec;
-import com.datastax.driver.core.UDTValue;
-import com.datastax.driver.core.UserType;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.schema.CQLTypeParser;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.cql3.functions.types.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 8c5a07e..f98c75f 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -42,19 +42,19 @@ import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.datastax.driver.core.DataType;
-import com.datastax.driver.core.TypeCodec;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.marshal.UserType;
-import org.apache.cassandra.schema.Difference;
-import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.cql3.functions.types.TypeCodec;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.FunctionExecutionException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.Difference;
 import org.apache.cassandra.schema.Functions;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.ProtocolVersion;
@@ -98,7 +98,6 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
     //
     private static final String[] whitelistedPatterns =
     {
-    "com/datastax/driver/core/",
     "com/google/common/reflect/TypeToken",
     "java/io/IOException.class",
     "java/io/Serializable.class",
@@ -115,6 +114,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
     "java/text/",
     "java/time/",
     "java/util/",
+    "org/apache/cassandra/cql3/functions/types/",
     "org/apache/cassandra/cql3/functions/JavaUDF.class",
     "org/apache/cassandra/cql3/functions/UDFContext.class",
     "org/apache/cassandra/exceptions/",
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDHelper.java b/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
index 935b1b8..8c145d9 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
@@ -17,19 +17,14 @@
  */
 package org.apache.cassandra.cql3.functions;
 
-import java.lang.invoke.MethodHandle;
-import java.lang.invoke.MethodHandles;
-import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
 import java.util.List;
 
 import com.google.common.reflect.TypeToken;
 
-import com.datastax.driver.core.CodecRegistry;
-import com.datastax.driver.core.DataType;
-import com.datastax.driver.core.TypeCodec;
-import com.datastax.driver.core.exceptions.InvalidTypeException;
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.functions.types.*;
+import org.apache.cassandra.cql3.functions.types.exceptions.InvalidTypeException;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.transport.ProtocolVersion;
 
@@ -38,24 +33,7 @@ import org.apache.cassandra.transport.ProtocolVersion;
  */
 public final class UDHelper
 {
-    // TODO make these c'tors and methods public in Java-Driver - see https://datastax-oss.atlassian.net/browse/JAVA-502
-    private static final MethodHandle methodParseOne;
-    private static final CodecRegistry codecRegistry;
-    static
-    {
-        try
-        {
-            Class<?> cls = Class.forName("com.datastax.driver.core.DataTypeClassNameParser");
-            Method m = cls.getDeclaredMethod("parseOne", String.class, com.datastax.driver.core.ProtocolVersion.class, CodecRegistry.class);
-            m.setAccessible(true);
-            methodParseOne = MethodHandles.lookup().unreflect(m);
-            codecRegistry = new CodecRegistry();
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
+    private static final CodecRegistry codecRegistry = new CodecRegistry();
 
     static TypeCodec<Object>[] codecsFor(DataType[] dataType)
     {
@@ -71,7 +49,7 @@ public final class UDHelper
     }
 
     /**
-     * Construct an array containing the Java classes for the given Java Driver {@link com.datastax.driver.core.DataType}s.
+     * Construct an array containing the Java classes for the given {@link DataType}s.
      *
      * @param dataTypes  array with UDF argument types
      * @param calledOnNullInput whether to allow {@code null} as an argument value
@@ -108,11 +86,11 @@ public final class UDHelper
     }
 
     /**
-     * Construct an array containing the Java Driver {@link com.datastax.driver.core.DataType}s for the
+     * Construct an array containing the {@link DataType}s for the
      * C* internal types.
      *
      * @param abstractTypes list with UDF argument types
-     * @return array with argument types as {@link com.datastax.driver.core.DataType}
+     * @return array with argument types as {@link DataType}
      */
     public static DataType[] driverTypes(List<AbstractType<?>> abstractTypes)
     {
@@ -123,7 +101,7 @@ public final class UDHelper
     }
 
     /**
-     * Returns the Java Driver {@link com.datastax.driver.core.DataType} for the C* internal type.
+     * Returns the {@link DataType} for the C* internal type.
      */
     public static DataType driverType(AbstractType abstractType)
     {
@@ -134,26 +112,14 @@ public final class UDHelper
 
     public static DataType driverTypeFromAbstractType(String abstractTypeDef)
     {
-        try
-        {
-            return (DataType) methodParseOne.invoke(abstractTypeDef,
-                                                    com.datastax.driver.core.ProtocolVersion.fromInt(ProtocolVersion.CURRENT.asInt()),
-                                                    codecRegistry);
-        }
-        catch (RuntimeException | Error e)
-        {
-            // immediately rethrow these...
-            throw e;
-        }
-        catch (Throwable e)
-        {
-            throw new RuntimeException("cannot parse driver type " + abstractTypeDef, e);
-        }
+        return DataTypeClassNameParser.parseOne(abstractTypeDef,
+                                                ProtocolVersion.CURRENT,
+                                                codecRegistry);
     }
 
     public static Object deserialize(TypeCodec<?> codec, ProtocolVersion protocolVersion, ByteBuffer value)
     {
-        return codec.deserialize(value, com.datastax.driver.core.ProtocolVersion.fromInt(protocolVersion.asInt()));
+        return codec.deserialize(value, protocolVersion);
     }
 
     public static ByteBuffer serialize(TypeCodec<?> codec, ProtocolVersion protocolVersion, Object value)
@@ -161,7 +127,7 @@ public final class UDHelper
         if (!codec.getJavaType().getRawType().isAssignableFrom(value.getClass()))
             throw new InvalidTypeException("Invalid value for CQL type " + codec.getCqlType().getName());
 
-        return ((TypeCodec)codec).serialize(value, com.datastax.driver.core.ProtocolVersion.fromInt(protocolVersion.asInt()));
+        return ((TypeCodec)codec).serialize(value, protocolVersion);
     }
 
     public static Class<?> asJavaClass(TypeCodec<?> codec)
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/AbstractAddressableByIndexData.java b/src/java/org/apache/cassandra/cql3/functions/types/AbstractAddressableByIndexData.java
new file mode 100644
index 0000000..0e98c89
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/AbstractAddressableByIndexData.java
@@ -0,0 +1,330 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.reflect.TypeToken;
+
+import org.apache.cassandra.transport.ProtocolVersion;
+
+abstract class AbstractAddressableByIndexData<T extends SettableByIndexData<T>>
+extends AbstractGettableByIndexData implements SettableByIndexData<T>
+{
+
+    final ByteBuffer[] values;
+
+    AbstractAddressableByIndexData(ProtocolVersion protocolVersion, int size)
+    {
+        super(protocolVersion);
+        this.values = new ByteBuffer[size];
+    }
+
+    @SuppressWarnings("unchecked")
+    T setValue(int i, ByteBuffer value)
+    {
+        values[i] = value;
+        return (T) this;
+    }
+
+    @Override
+    protected ByteBuffer getValue(int i)
+    {
+        return values[i];
+    }
+
+    @Override
+    public T setBool(int i, boolean v)
+    {
+        TypeCodec<Boolean> codec = codecFor(i, Boolean.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveBooleanCodec)
+            bb = ((TypeCodec.PrimitiveBooleanCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setByte(int i, byte v)
+    {
+        TypeCodec<Byte> codec = codecFor(i, Byte.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveByteCodec)
+            bb = ((TypeCodec.PrimitiveByteCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setShort(int i, short v)
+    {
+        TypeCodec<Short> codec = codecFor(i, Short.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveShortCodec)
+            bb = ((TypeCodec.PrimitiveShortCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setInt(int i, int v)
+    {
+        TypeCodec<Integer> codec = codecFor(i, Integer.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveIntCodec)
+            bb = ((TypeCodec.PrimitiveIntCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setLong(int i, long v)
+    {
+        TypeCodec<Long> codec = codecFor(i, Long.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveLongCodec)
+            bb = ((TypeCodec.PrimitiveLongCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setTimestamp(int i, Date v)
+    {
+        return setValue(i, codecFor(i, Date.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setDate(int i, LocalDate v)
+    {
+        return setValue(i, codecFor(i, LocalDate.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setTime(int i, long v)
+    {
+        TypeCodec<Long> codec = codecFor(i, Long.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveLongCodec)
+            bb = ((TypeCodec.PrimitiveLongCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setFloat(int i, float v)
+    {
+        TypeCodec<Float> codec = codecFor(i, Float.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveFloatCodec)
+            bb = ((TypeCodec.PrimitiveFloatCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setDouble(int i, double v)
+    {
+        TypeCodec<Double> codec = codecFor(i, Double.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveDoubleCodec)
+            bb = ((TypeCodec.PrimitiveDoubleCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setString(int i, String v)
+    {
+        return setValue(i, codecFor(i, String.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setBytes(int i, ByteBuffer v)
+    {
+        return setValue(i, codecFor(i, ByteBuffer.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setBytesUnsafe(int i, ByteBuffer v)
+    {
+        return setValue(i, v == null ? null : v.duplicate());
+    }
+
+    @Override
+    public T setVarint(int i, BigInteger v)
+    {
+        return setValue(i, codecFor(i, BigInteger.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setDecimal(int i, BigDecimal v)
+    {
+        return setValue(i, codecFor(i, BigDecimal.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setUUID(int i, UUID v)
+    {
+        return setValue(i, codecFor(i, UUID.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setInet(int i, InetAddress v)
+    {
+        return setValue(i, codecFor(i, InetAddress.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <E> T setList(int i, List<E> v)
+    {
+        return setValue(i, codecFor(i).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <E> T setList(int i, List<E> v, Class<E> elementsClass)
+    {
+        return setValue(i, codecFor(i, TypeTokens.listOf(elementsClass)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <E> T setList(int i, List<E> v, TypeToken<E> elementsType)
+    {
+        return setValue(i, codecFor(i, TypeTokens.listOf(elementsType)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <K, V> T setMap(int i, Map<K, V> v)
+    {
+        return setValue(i, codecFor(i).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <K, V> T setMap(int i, Map<K, V> v, Class<K> keysClass, Class<V> valuesClass)
+    {
+        return setValue(
+        i, codecFor(i, TypeTokens.mapOf(keysClass, valuesClass)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <K, V> T setMap(int i, Map<K, V> v, TypeToken<K> keysType, TypeToken<V> valuesType)
+    {
+        return setValue(
+        i, codecFor(i, TypeTokens.mapOf(keysType, valuesType)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <E> T setSet(int i, Set<E> v)
+    {
+        return setValue(i, codecFor(i).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <E> T setSet(int i, Set<E> v, Class<E> elementsClass)
+    {
+        return setValue(i, codecFor(i, TypeTokens.setOf(elementsClass)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <E> T setSet(int i, Set<E> v, TypeToken<E> elementsType)
+    {
+        return setValue(i, codecFor(i, TypeTokens.setOf(elementsType)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setUDTValue(int i, UDTValue v)
+    {
+        return setValue(i, codecFor(i, UDTValue.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setTupleValue(int i, TupleValue v)
+    {
+        return setValue(i, codecFor(i, TupleValue.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <V> T set(int i, V v, Class<V> targetClass)
+    {
+        return set(i, v, codecFor(i, targetClass));
+    }
+
+    @Override
+    public <V> T set(int i, V v, TypeToken<V> targetType)
+    {
+        return set(i, v, codecFor(i, targetType));
+    }
+
+    @Override
+    public <V> T set(int i, V v, TypeCodec<V> codec)
+    {
+        checkType(i, codec.getCqlType().getName());
+        return setValue(i, codec.serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setToNull(int i)
+    {
+        return setValue(i, null);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof AbstractAddressableByIndexData)) return false;
+
+        AbstractAddressableByIndexData<?> that = (AbstractAddressableByIndexData<?>) o;
+        if (values.length != that.values.length) return false;
+
+        if (this.protocolVersion != that.protocolVersion) return false;
+
+        // Deserializing each value is slightly inefficient, but comparing
+        // the bytes could in theory be wrong (for varint for instance, 2 values
+        // can have different binary representation but be the same value due to
+        // leading zeros). So we don't take any risk.
+        for (int i = 0; i < values.length; i++)
+        {
+            DataType thisType = getType(i);
+            DataType thatType = that.getType(i);
+            if (!thisType.equals(thatType)) return false;
+
+            Object thisValue = this.codecFor(i).deserialize(this.values[i], this.protocolVersion);
+            Object thatValue = that.codecFor(i).deserialize(that.values[i], that.protocolVersion);
+            if (!Objects.equals(thisValue, thatValue)) return false;
+        }
+        return true;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        // Same as equals
+        int hash = 31;
+        for (int i = 0; i < values.length; i++)
+            hash +=
+            values[i] == null ? 1 : codecFor(i).deserialize(values[i], protocolVersion).hashCode();
+        return hash;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/AbstractData.java b/src/java/org/apache/cassandra/cql3/functions/types/AbstractData.java
new file mode 100644
index 0000000..adb7c0e
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/AbstractData.java
@@ -0,0 +1,677 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.reflect.TypeToken;
+
+import org.apache.cassandra.transport.ProtocolVersion;
+
+// We don't want to expose this one: it's less useful externally and it's a bit ugly to expose
+// anyway (but it's convenient).
+abstract class AbstractData<T extends SettableData<T>> extends AbstractGettableData
+implements SettableData<T>
+{
+
+    private final T wrapped;
+    final ByteBuffer[] values;
+
+    // Ugly, we could probably clean that: it is currently needed however because we sometimes
+    // want wrapped to be 'this' (UDTValue), and sometimes some other object (in BoundStatement).
+    @SuppressWarnings("unchecked")
+    protected AbstractData(ProtocolVersion protocolVersion, int size)
+    {
+        super(protocolVersion);
+        this.wrapped = (T) this;
+        this.values = new ByteBuffer[size];
+    }
+
+    protected AbstractData(ProtocolVersion protocolVersion, T wrapped, int size)
+    {
+        this(protocolVersion, wrapped, new ByteBuffer[size]);
+    }
+
+    protected AbstractData(ProtocolVersion protocolVersion, T wrapped, ByteBuffer[] values)
+    {
+        super(protocolVersion);
+        this.wrapped = wrapped;
+        this.values = values;
+    }
+
+    protected abstract int[] getAllIndexesOf(String name);
+
+    private T setValue(int i, ByteBuffer value)
+    {
+        values[i] = value;
+        return wrapped;
+    }
+
+    @Override
+    protected ByteBuffer getValue(int i)
+    {
+        return values[i];
+    }
+
+    @Override
+    protected int getIndexOf(String name)
+    {
+        return getAllIndexesOf(name)[0];
+    }
+
+    @Override
+    public T setBool(int i, boolean v)
+    {
+        TypeCodec<Boolean> codec = codecFor(i, Boolean.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveBooleanCodec)
+            bb = ((TypeCodec.PrimitiveBooleanCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setBool(String name, boolean v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setBool(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setByte(int i, byte v)
+    {
+        TypeCodec<Byte> codec = codecFor(i, Byte.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveByteCodec)
+            bb = ((TypeCodec.PrimitiveByteCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setByte(String name, byte v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setByte(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setShort(int i, short v)
+    {
+        TypeCodec<Short> codec = codecFor(i, Short.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveShortCodec)
+            bb = ((TypeCodec.PrimitiveShortCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setShort(String name, short v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setShort(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setInt(int i, int v)
+    {
+        TypeCodec<Integer> codec = codecFor(i, Integer.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveIntCodec)
+            bb = ((TypeCodec.PrimitiveIntCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setInt(String name, int v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setInt(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setLong(int i, long v)
+    {
+        TypeCodec<Long> codec = codecFor(i, Long.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveLongCodec)
+            bb = ((TypeCodec.PrimitiveLongCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setLong(String name, long v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setLong(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setTimestamp(int i, Date v)
+    {
+        return setValue(i, codecFor(i, Date.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setTimestamp(String name, Date v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setTimestamp(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setDate(int i, LocalDate v)
+    {
+        return setValue(i, codecFor(i, LocalDate.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setDate(String name, LocalDate v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setDate(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setTime(int i, long v)
+    {
+        TypeCodec<Long> codec = codecFor(i, Long.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveLongCodec)
+            bb = ((TypeCodec.PrimitiveLongCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setTime(String name, long v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setTime(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setFloat(int i, float v)
+    {
+        TypeCodec<Float> codec = codecFor(i, Float.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveFloatCodec)
+            bb = ((TypeCodec.PrimitiveFloatCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setFloat(String name, float v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setFloat(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setDouble(int i, double v)
+    {
+        TypeCodec<Double> codec = codecFor(i, Double.class);
+        ByteBuffer bb;
+        if (codec instanceof TypeCodec.PrimitiveDoubleCodec)
+            bb = ((TypeCodec.PrimitiveDoubleCodec) codec).serializeNoBoxing(v, protocolVersion);
+        else bb = codec.serialize(v, protocolVersion);
+        return setValue(i, bb);
+    }
+
+    @Override
+    public T setDouble(String name, double v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setDouble(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setString(int i, String v)
+    {
+        return setValue(i, codecFor(i, String.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setString(String name, String v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setString(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setBytes(int i, ByteBuffer v)
+    {
+        return setValue(i, codecFor(i, ByteBuffer.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setBytes(String name, ByteBuffer v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setBytes(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setBytesUnsafe(int i, ByteBuffer v)
+    {
+        return setValue(i, v == null ? null : v.duplicate());
+    }
+
+    @Override
+    public T setBytesUnsafe(String name, ByteBuffer v)
+    {
+        ByteBuffer value = v == null ? null : v.duplicate();
+        for (int i : getAllIndexesOf(name))
+        {
+            setValue(i, value);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setVarint(int i, BigInteger v)
+    {
+        return setValue(i, codecFor(i, BigInteger.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setVarint(String name, BigInteger v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setVarint(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setDecimal(int i, BigDecimal v)
+    {
+        return setValue(i, codecFor(i, BigDecimal.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setDecimal(String name, BigDecimal v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setDecimal(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setUUID(int i, UUID v)
+    {
+        return setValue(i, codecFor(i, UUID.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setUUID(String name, UUID v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setUUID(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setInet(int i, InetAddress v)
+    {
+        return setValue(i, codecFor(i, InetAddress.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setInet(String name, InetAddress v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setInet(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <E> T setList(int i, List<E> v)
+    {
+        return setValue(i, codecFor(i).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <E> T setList(int i, List<E> v, Class<E> elementsClass)
+    {
+        return setValue(i, codecFor(i, TypeTokens.listOf(elementsClass)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <E> T setList(int i, List<E> v, TypeToken<E> elementsType)
+    {
+        return setValue(i, codecFor(i, TypeTokens.listOf(elementsType)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <E> T setList(String name, List<E> v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setList(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public <E> T setList(String name, List<E> v, Class<E> elementsClass)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setList(i, v, elementsClass);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public <E> T setList(String name, List<E> v, TypeToken<E> elementsType)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setList(i, v, elementsType);
+        }
+        return wrapped;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <K, V> T setMap(int i, Map<K, V> v)
+    {
+        return setValue(i, codecFor(i).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <K, V> T setMap(int i, Map<K, V> v, Class<K> keysClass, Class<V> valuesClass)
+    {
+        return setValue(
+        i, codecFor(i, TypeTokens.mapOf(keysClass, valuesClass)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <K, V> T setMap(int i, Map<K, V> v, TypeToken<K> keysType, TypeToken<V> valuesType)
+    {
+        return setValue(
+        i, codecFor(i, TypeTokens.mapOf(keysType, valuesType)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <K, V> T setMap(String name, Map<K, V> v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setMap(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public <K, V> T setMap(String name, Map<K, V> v, Class<K> keysClass, Class<V> valuesClass)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setMap(i, v, keysClass, valuesClass);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public <K, V> T setMap(String name, Map<K, V> v, TypeToken<K> keysType, TypeToken<V> valuesType)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setMap(i, v, keysType, valuesType);
+        }
+        return wrapped;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <E> T setSet(int i, Set<E> v)
+    {
+        return setValue(i, codecFor(i).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <E> T setSet(int i, Set<E> v, Class<E> elementsClass)
+    {
+        return setValue(i, codecFor(i, TypeTokens.setOf(elementsClass)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <E> T setSet(int i, Set<E> v, TypeToken<E> elementsType)
+    {
+        return setValue(i, codecFor(i, TypeTokens.setOf(elementsType)).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <E> T setSet(String name, Set<E> v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setSet(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public <E> T setSet(String name, Set<E> v, Class<E> elementsClass)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setSet(i, v, elementsClass);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public <E> T setSet(String name, Set<E> v, TypeToken<E> elementsType)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setSet(i, v, elementsType);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setUDTValue(int i, UDTValue v)
+    {
+        return setValue(i, codecFor(i, UDTValue.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setUDTValue(String name, UDTValue v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setUDTValue(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setTupleValue(int i, TupleValue v)
+    {
+        return setValue(i, codecFor(i, TupleValue.class).serialize(v, protocolVersion));
+    }
+
+    @Override
+    public T setTupleValue(String name, TupleValue v)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setTupleValue(i, v);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public <V> T set(int i, V v, Class<V> targetClass)
+    {
+        return set(i, v, codecFor(i, targetClass));
+    }
+
+    @Override
+    public <V> T set(String name, V v, Class<V> targetClass)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            set(i, v, targetClass);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public <V> T set(int i, V v, TypeToken<V> targetType)
+    {
+        return set(i, v, codecFor(i, targetType));
+    }
+
+    @Override
+    public <V> T set(String name, V v, TypeToken<V> targetType)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            set(i, v, targetType);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public <V> T set(int i, V v, TypeCodec<V> codec)
+    {
+        checkType(i, codec.getCqlType().getName());
+        return setValue(i, codec.serialize(v, protocolVersion));
+    }
+
+    @Override
+    public <V> T set(String name, V v, TypeCodec<V> codec)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            set(i, v, codec);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public T setToNull(int i)
+    {
+        return setValue(i, null);
+    }
+
+    @Override
+    public T setToNull(String name)
+    {
+        for (int i : getAllIndexesOf(name))
+        {
+            setToNull(i);
+        }
+        return wrapped;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof AbstractData)) return false;
+
+        AbstractData<?> that = (AbstractData<?>) o;
+        if (values.length != that.values.length) return false;
+
+        if (this.protocolVersion != that.protocolVersion) return false;
+
+        // Deserializing each value is slightly inefficient, but comparing
+        // the bytes could in theory be wrong (for varint for instance, 2 values
+        // can have different binary representation but be the same value due to
+        // leading zeros). So we don't take any risk.
+        for (int i = 0; i < values.length; i++)
+        {
+            DataType thisType = getType(i);
+            DataType thatType = that.getType(i);
+            if (!thisType.equals(thatType)) return false;
+
+            Object thisValue = this.codecFor(i).deserialize(this.values[i], this.protocolVersion);
+            Object thatValue = that.codecFor(i).deserialize(that.values[i], that.protocolVersion);
+            if (!Objects.equals(thisValue, thatValue)) return false;
+        }
+        return true;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        // Same as equals
+        int hash = 31;
+        for (int i = 0; i < values.length; i++)
+            hash +=
+            values[i] == null ? 1 : codecFor(i).deserialize(values[i], protocolVersion).hashCode();
+        return hash;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/AbstractGettableByIndexData.java b/src/java/org/apache/cassandra/cql3/functions/types/AbstractGettableByIndexData.java
new file mode 100644
index 0000000..1552309
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/AbstractGettableByIndexData.java
@@ -0,0 +1,418 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.reflect.TypeToken;
+
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.cql3.functions.types.exceptions.InvalidTypeException;
+
+abstract class AbstractGettableByIndexData implements GettableByIndexData
+{
+
+    protected final ProtocolVersion protocolVersion;
+
+    AbstractGettableByIndexData(ProtocolVersion protocolVersion)
+    {
+        this.protocolVersion = protocolVersion;
+    }
+
+    /**
+     * Returns the type for the value at index {@code i}.
+     *
+     * @param i the index of the type to fetch.
+     * @return the type of the value at index {@code i}.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index.
+     */
+    protected abstract DataType getType(int i);
+
+    /**
+     * Returns the name corresponding to the value at index {@code i}.
+     *
+     * @param i the index of the name to fetch.
+     * @return the name corresponding to the value at index {@code i}.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index.
+     */
+    protected abstract String getName(int i);
+
+    /**
+     * Returns the value at index {@code i}.
+     *
+     * @param i the index to fetch.
+     * @return the value at index {@code i}.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index.
+     */
+    protected abstract ByteBuffer getValue(int i);
+
+    protected abstract CodecRegistry getCodecRegistry();
+
+    protected <T> TypeCodec<T> codecFor(int i)
+    {
+        return getCodecRegistry().codecFor(getType(i));
+    }
+
+    protected <T> TypeCodec<T> codecFor(int i, Class<T> javaClass)
+    {
+        return getCodecRegistry().codecFor(getType(i), javaClass);
+    }
+
+    protected <T> TypeCodec<T> codecFor(int i, TypeToken<T> javaType)
+    {
+        return getCodecRegistry().codecFor(getType(i), javaType);
+    }
+
+    protected <T> TypeCodec<T> codecFor(int i, T value)
+    {
+        return getCodecRegistry().codecFor(getType(i), value);
+    }
+
+    void checkType(int i, DataType.Name actual)
+    {
+        DataType.Name expected = getType(i).getName();
+        if (!actual.isCompatibleWith(expected))
+            throw new InvalidTypeException(
+            String.format("Value %s is of type %s, not %s", getName(i), expected, actual));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean isNull(int i)
+    {
+        return getValue(i) == null;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean getBool(int i)
+    {
+        ByteBuffer value = getValue(i);
+        TypeCodec<Boolean> codec = codecFor(i, Boolean.class);
+        if (codec instanceof TypeCodec.PrimitiveBooleanCodec)
+            return ((TypeCodec.PrimitiveBooleanCodec) codec).deserializeNoBoxing(value, protocolVersion);
+        else return codec.deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public byte getByte(int i)
+    {
+        ByteBuffer value = getValue(i);
+        TypeCodec<Byte> codec = codecFor(i, Byte.class);
+        if (codec instanceof TypeCodec.PrimitiveByteCodec)
+            return ((TypeCodec.PrimitiveByteCodec) codec).deserializeNoBoxing(value, protocolVersion);
+        else return codec.deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public short getShort(int i)
+    {
+        ByteBuffer value = getValue(i);
+        TypeCodec<Short> codec = codecFor(i, Short.class);
+        if (codec instanceof TypeCodec.PrimitiveShortCodec)
+            return ((TypeCodec.PrimitiveShortCodec) codec).deserializeNoBoxing(value, protocolVersion);
+        else return codec.deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int getInt(int i)
+    {
+        ByteBuffer value = getValue(i);
+        TypeCodec<Integer> codec = codecFor(i, Integer.class);
+        if (codec instanceof TypeCodec.PrimitiveIntCodec)
+            return ((TypeCodec.PrimitiveIntCodec) codec).deserializeNoBoxing(value, protocolVersion);
+        else return codec.deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public long getLong(int i)
+    {
+        ByteBuffer value = getValue(i);
+        TypeCodec<Long> codec = codecFor(i, Long.class);
+        if (codec instanceof TypeCodec.PrimitiveLongCodec)
+            return ((TypeCodec.PrimitiveLongCodec) codec).deserializeNoBoxing(value, protocolVersion);
+        else return codec.deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public Date getTimestamp(int i)
+    {
+        ByteBuffer value = getValue(i);
+        return codecFor(i, Date.class).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public LocalDate getDate(int i)
+    {
+        ByteBuffer value = getValue(i);
+        return codecFor(i, LocalDate.class).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public long getTime(int i)
+    {
+        ByteBuffer value = getValue(i);
+        TypeCodec<Long> codec = codecFor(i, Long.class);
+        if (codec instanceof TypeCodec.PrimitiveLongCodec)
+            return ((TypeCodec.PrimitiveLongCodec) codec).deserializeNoBoxing(value, protocolVersion);
+        else return codec.deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public float getFloat(int i)
+    {
+        ByteBuffer value = getValue(i);
+        TypeCodec<Float> codec = codecFor(i, Float.class);
+        if (codec instanceof TypeCodec.PrimitiveFloatCodec)
+            return ((TypeCodec.PrimitiveFloatCodec) codec).deserializeNoBoxing(value, protocolVersion);
+        else return codec.deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public double getDouble(int i)
+    {
+        ByteBuffer value = getValue(i);
+        TypeCodec<Double> codec = codecFor(i, Double.class);
+        if (codec instanceof TypeCodec.PrimitiveDoubleCodec)
+            return ((TypeCodec.PrimitiveDoubleCodec) codec).deserializeNoBoxing(value, protocolVersion);
+        else return codec.deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public ByteBuffer getBytesUnsafe(int i)
+    {
+        ByteBuffer value = getValue(i);
+        if (value == null) return null;
+        return value.duplicate();
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public ByteBuffer getBytes(int i)
+    {
+        ByteBuffer value = getValue(i);
+        return codecFor(i, ByteBuffer.class).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String getString(int i)
+    {
+        ByteBuffer value = getValue(i);
+        return codecFor(i, String.class).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public BigInteger getVarint(int i)
+    {
+        ByteBuffer value = getValue(i);
+        return codecFor(i, BigInteger.class).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public BigDecimal getDecimal(int i)
+    {
+        ByteBuffer value = getValue(i);
+        return codecFor(i, BigDecimal.class).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public UUID getUUID(int i)
+    {
+        ByteBuffer value = getValue(i);
+        return codecFor(i, UUID.class).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public InetAddress getInet(int i)
+    {
+        ByteBuffer value = getValue(i);
+        return codecFor(i, InetAddress.class).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> List<T> getList(int i, Class<T> elementsClass)
+    {
+        return getList(i, TypeToken.of(elementsClass));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> List<T> getList(int i, TypeToken<T> elementsType)
+    {
+        ByteBuffer value = getValue(i);
+        TypeToken<List<T>> javaType = TypeTokens.listOf(elementsType);
+        return codecFor(i, javaType).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> Set<T> getSet(int i, Class<T> elementsClass)
+    {
+        return getSet(i, TypeToken.of(elementsClass));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> Set<T> getSet(int i, TypeToken<T> elementsType)
+    {
+        ByteBuffer value = getValue(i);
+        TypeToken<Set<T>> javaType = TypeTokens.setOf(elementsType);
+        return codecFor(i, javaType).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    public <K, V> Map<K, V> getMap(int i, Class<K> keysClass, Class<V> valuesClass)
+    {
+        return getMap(i, TypeToken.of(keysClass), TypeToken.of(valuesClass));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    public <K, V> Map<K, V> getMap(int i, TypeToken<K> keysType, TypeToken<V> valuesType)
+    {
+        ByteBuffer value = getValue(i);
+        TypeToken<Map<K, V>> javaType = TypeTokens.mapOf(keysType, valuesType);
+        return codecFor(i, javaType).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    public UDTValue getUDTValue(int i)
+    {
+        ByteBuffer value = getValue(i);
+        return codecFor(i, UDTValue.class).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    @SuppressWarnings("unchecked")
+    public TupleValue getTupleValue(int i)
+    {
+        ByteBuffer value = getValue(i);
+        return codecFor(i, TupleValue.class).deserialize(value, protocolVersion);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public Object getObject(int i)
+    {
+        return get(i, codecFor(i));
+    }
+
+    @Override
+    public <T> T get(int i, Class<T> targetClass)
+    {
+        return get(i, codecFor(i, targetClass));
+    }
+
+    @Override
+    public <T> T get(int i, TypeToken<T> targetType)
+    {
+        return get(i, codecFor(i, targetType));
+    }
+
+    @Override
+    public <T> T get(int i, TypeCodec<T> codec)
+    {
+        checkType(i, codec.getCqlType().getName());
+        ByteBuffer value = getValue(i);
+        return codec.deserialize(value, protocolVersion);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/AbstractGettableData.java b/src/java/org/apache/cassandra/cql3/functions/types/AbstractGettableData.java
new file mode 100644
index 0000000..2ac7d98
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/AbstractGettableData.java
@@ -0,0 +1,325 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.reflect.TypeToken;
+
+import org.apache.cassandra.transport.ProtocolVersion;
+
+public abstract class AbstractGettableData extends AbstractGettableByIndexData
+implements GettableData
+{
+
+    /**
+     * Creates a new AbstractGettableData object.
+     *
+     * @param protocolVersion the protocol version in which values returned by {@link #getValue} will
+     *                        be returned. This must be a protocol version supported by this driver. In general, the
+     *                        correct value will be the value returned by {@code ProtocolOptions#getProtocolVersion}.
+     * @throws IllegalArgumentException if {@code protocolVersion} is not a valid protocol version.
+     */
+    AbstractGettableData(ProtocolVersion protocolVersion)
+    {
+        super(protocolVersion);
+    }
+
+    /**
+     * Returns the index corresponding to a given name.
+     *
+     * @param name the name for which to return the index of.
+     * @return the index for the value coressponding to {@code name}.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     */
+    protected abstract int getIndexOf(String name);
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean isNull(String name)
+    {
+        return isNull(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean getBool(String name)
+    {
+        return getBool(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public byte getByte(String name)
+    {
+        return getByte(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public short getShort(String name)
+    {
+        return getShort(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int getInt(String name)
+    {
+        return getInt(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public long getLong(String name)
+    {
+        return getLong(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public Date getTimestamp(String name)
+    {
+        return getTimestamp(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public LocalDate getDate(String name)
+    {
+        return getDate(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public long getTime(String name)
+    {
+        return getTime(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public float getFloat(String name)
+    {
+        return getFloat(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public double getDouble(String name)
+    {
+        return getDouble(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public ByteBuffer getBytesUnsafe(String name)
+    {
+        return getBytesUnsafe(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public ByteBuffer getBytes(String name)
+    {
+        return getBytes(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String getString(String name)
+    {
+        return getString(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public BigInteger getVarint(String name)
+    {
+        return getVarint(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public BigDecimal getDecimal(String name)
+    {
+        return getDecimal(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public UUID getUUID(String name)
+    {
+        return getUUID(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public InetAddress getInet(String name)
+    {
+        return getInet(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> List<T> getList(String name, Class<T> elementsClass)
+    {
+        return getList(getIndexOf(name), elementsClass);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> List<T> getList(String name, TypeToken<T> elementsType)
+    {
+        return getList(getIndexOf(name), elementsType);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> Set<T> getSet(String name, Class<T> elementsClass)
+    {
+        return getSet(getIndexOf(name), elementsClass);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> Set<T> getSet(String name, TypeToken<T> elementsType)
+    {
+        return getSet(getIndexOf(name), elementsType);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <K, V> Map<K, V> getMap(String name, Class<K> keysClass, Class<V> valuesClass)
+    {
+        return getMap(getIndexOf(name), keysClass, valuesClass);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <K, V> Map<K, V> getMap(String name, TypeToken<K> keysType, TypeToken<V> valuesType)
+    {
+        return getMap(getIndexOf(name), keysType, valuesType);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public UDTValue getUDTValue(String name)
+    {
+        return getUDTValue(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public TupleValue getTupleValue(String name)
+    {
+        return getTupleValue(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public Object getObject(String name)
+    {
+        return getObject(getIndexOf(name));
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> T get(String name, Class<T> targetClass)
+    {
+        return get(getIndexOf(name), targetClass);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> T get(String name, TypeToken<T> targetType)
+    {
+        return get(getIndexOf(name), targetType);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public <T> T get(String name, TypeCodec<T> codec)
+    {
+        return get(getIndexOf(name), codec);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/CodecRegistry.java b/src/java/org/apache/cassandra/cql3/functions/types/CodecRegistry.java
new file mode 100644
index 0000000..a979a57
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/CodecRegistry.java
@@ -0,0 +1,885 @@
+/*
+ * 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.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.cache.*;
+import com.google.common.reflect.TypeToken;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.cql3.functions.types.exceptions.CodecNotFoundException;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.cassandra.cql3.functions.types.DataType.Name.*;
+
+/**
+ * A registry for {@link TypeCodec}s. When the driver needs to serialize or deserialize a Java type
+ * to/from CQL, it will lookup in the registry for a suitable codec. The registry is initialized
+ * with default codecs that handle basic conversions (e.g. CQL {@code text} to {@code
+ * java.lang.String}), and users can add their own. Complex codecs can also be generated on-the-fly
+ * from simpler ones (more details below).
+ *
+ * <h3>Creating a registry </h3>
+ * <p>
+ * By default, the driver uses {@code CodecRegistry#DEFAULT_INSTANCE}, a shareable, JVM-wide
+ * instance initialized with built-in codecs for all the base CQL types. The only reason to create
+ * your own instances is if you have multiple {@code Cluster} objects that use different sets of
+ * codecs. In that case, use {@code
+ * Cluster.Builder#withCodecRegistry(CodecRegistry)} to associate the
+ * registry with the cluster:
+ *
+ * <pre>{@code
+ * CodecRegistry myCodecRegistry = new CodecRegistry();
+ * myCodecRegistry.register(myCodec1, myCodec2, myCodec3);
+ * Cluster cluster = Cluster.builder().withCodecRegistry(myCodecRegistry).build();
+ *
+ * // To retrieve the registry later:
+ * CodecRegistry registry = cluster.getConfiguration().getCodecRegistry();
+ * }</pre>
+ * <p>
+ * {@code CodecRegistry} instances are thread-safe.
+ *
+ * <p>It is possible to turn on log messages by setting the {@code
+ * CodecRegistry} logger level to {@code TRACE}. Beware that the registry
+ * can be very verbose at this log level.
+ *
+ * <h3>Registering and using custom codecs </h3>
+ * <p>
+ * To create a custom codec, write a class that extends {@link TypeCodec}, create an instance, and
+ * pass it to one of the {@link #register(TypeCodec) register} methods; for example, one could
+ * create a codec that maps CQL timestamps to JDK8's {@code java.time.LocalDate}:
+ *
+ * <pre>{@code
+ * class LocalDateCodec extends TypeCodec<java.time.LocalDate> {
+ *    ...
+ * }
+ * myCodecRegistry.register(new LocalDateCodec());
+ * }</pre>
+ * <p>
+ * The conversion will be available to:
+ *
+ * <ul>
+ * <li>all driver types that implement {@link GettableByIndexData}, {@link GettableByNameData},
+ * {@link SettableByIndexData} and/or {@link SettableByNameData}. Namely: {@code Row}, {@code
+ * BoundStatement}, {@link UDTValue} and {@link TupleValue};
+ * <li>{@code SimpleStatement#SimpleStatement(String, Object...) simple statements};
+ * <li>statements created with the {@code querybuilder.QueryBuilder Query
+ * builder}.
+ * </ul>
+ *
+ * <p>Example:
+ *
+ * <pre>{@code
+ * Row row = session.executeQuery("select date from some_table where pk = 1").one();
+ * java.time.LocalDate date = row.get(0, java.time.LocalDate.class); // uses LocalDateCodec registered above
+ * }</pre>
+ * <p>
+ * You can also bypass the codec registry by passing a standalone codec instance to methods such as
+ * {@link GettableByIndexData#get(int, TypeCodec)}.
+ *
+ * <h3>Codec generation </h3>
+ * <p>
+ * When a {@code CodecRegistry} cannot find a suitable codec among existing ones, it will attempt to
+ * create it on-the-fly. It can manage:
+ *
+ * <ul>
+ * <li>collections (lists, sets and maps) of known types. For example, if you registered a codec
+ * for JDK8's {@code java.time.LocalDate} like in the example above, you get {@code
+ * List<LocalDate>>} and {@code Set<LocalDate>>} handled for free, as well as all {@code Map}
+ * types whose keys and/or values are {@code java.time.LocalDate}. This works recursively for
+ * nested collections;
+ * <li>{@link UserType user types}, mapped to {@link UDTValue} objects. Custom codecs are
+ * available recursively to the UDT's fields, so if one of your fields is a {@code timestamp}
+ * you can use your {@code LocalDateCodec} to retrieve it as a {@code java.time.LocalDate};
+ * <li>{@link TupleType tuple types}, mapped to {@link TupleValue} (with the same rules for nested
+ * fields);
+ * <li>{@link DataType.CustomType custom types}, mapped to {@code
+ * ByteBuffer}.
+ * </ul>
+ * <p>
+ * If the codec registry encounters a mapping that it can't handle automatically, a {@link
+ * CodecNotFoundException} is thrown; you'll need to register a custom codec for it.
+ *
+ * <h3>Performance and caching </h3>
+ * <p>
+ * Whenever possible, the registry will cache the result of a codec lookup for a specific type
+ * mapping, including any generated codec. For example, if you registered {@code LocalDateCodec} and
+ * ask the registry for a codec to convert a CQL {@code list<timestamp>} to a Java {@code
+ * List<LocalDate>}:
+ *
+ * <ol>
+ * <li>the first lookup will generate a {@code TypeCodec<List<LocalDate>>} from {@code
+ * LocalDateCodec}, and put it in the cache;
+ * <li>the second lookup will hit the cache directly, and reuse the previously generated instance.
+ * </ol>
+ * <p>
+ * The javadoc for each {@link #codecFor(DataType) codecFor} variant specifies whether the result
+ * can be cached or not.
+ *
+ * <h3>Codec order </h3>
+ * <p>
+ * When the registry looks up a codec, the rules of precedence are:
+ *
+ * <ul>
+ * <li>if a result was previously cached for that mapping, it is returned;
+ * <li>otherwise, the registry checks the list of built-in codecs – the default ones – and the
+ * ones that were explicitly registered (in the order that they were registered). It calls
+ * each codec's {@code accepts} methods to determine if it can handle the mapping, and if so
+ * returns it;
+ * <li>otherwise, the registry tries to generate a codec, according to the rules outlined above.
+ * </ul>
+ * <p>
+ * It is currently impossible to override an existing codec. If you try to do so, {@link
+ * #register(TypeCodec)} will log a warning and ignore it.
+ */
+public final class CodecRegistry
+{
+
+    private static final Logger logger = LoggerFactory.getLogger(CodecRegistry.class);
+
+    private static final Map<DataType.Name, TypeCodec<?>> BUILT_IN_CODECS_MAP =
+    new EnumMap<>(DataType.Name.class);
+
+    static
+    {
+        BUILT_IN_CODECS_MAP.put(DataType.Name.ASCII, TypeCodec.ascii());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.BIGINT, TypeCodec.bigint());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.BLOB, TypeCodec.blob());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.BOOLEAN, TypeCodec.cboolean());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.COUNTER, TypeCodec.counter());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.DECIMAL, TypeCodec.decimal());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.DOUBLE, TypeCodec.cdouble());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.FLOAT, TypeCodec.cfloat());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.INET, TypeCodec.inet());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.INT, TypeCodec.cint());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.TEXT, TypeCodec.varchar());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.TIMESTAMP, TypeCodec.timestamp());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.UUID, TypeCodec.uuid());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.VARCHAR, TypeCodec.varchar());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.VARINT, TypeCodec.varint());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.TIMEUUID, TypeCodec.timeUUID());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.SMALLINT, TypeCodec.smallInt());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.TINYINT, TypeCodec.tinyInt());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.DATE, TypeCodec.date());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.TIME, TypeCodec.time());
+        BUILT_IN_CODECS_MAP.put(DataType.Name.DURATION, TypeCodec.duration());
+    }
+
+    // roughly sorted by popularity
+    private static final TypeCodec<?>[] BUILT_IN_CODECS =
+    new TypeCodec<?>[]{
+    TypeCodec
+    .varchar(), // must be declared before AsciiCodec so it gets chosen when CQL type not
+    // available
+    TypeCodec
+    .uuid(), // must be declared before TimeUUIDCodec so it gets chosen when CQL type not
+    // available
+    TypeCodec.timeUUID(),
+    TypeCodec.timestamp(),
+    TypeCodec.cint(),
+    TypeCodec.bigint(),
+    TypeCodec.blob(),
+    TypeCodec.cdouble(),
+    TypeCodec.cfloat(),
+    TypeCodec.decimal(),
+    TypeCodec.varint(),
+    TypeCodec.inet(),
+    TypeCodec.cboolean(),
+    TypeCodec.smallInt(),
+    TypeCodec.tinyInt(),
+    TypeCodec.date(),
+    TypeCodec.time(),
+    TypeCodec.duration(),
+    TypeCodec.counter(),
+    TypeCodec.ascii()
+    };
+
+    /**
+     * Cache key for the codecs cache.
+     */
+    private static final class CacheKey
+    {
+
+        private final DataType cqlType;
+
+        private final TypeToken<?> javaType;
+
+        CacheKey(DataType cqlType, TypeToken<?> javaType)
+        {
+            this.javaType = javaType;
+            this.cqlType = cqlType;
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            CacheKey cacheKey = (CacheKey) o;
+            return Objects.equals(cqlType, cacheKey.cqlType)
+                   && Objects.equals(javaType, cacheKey.javaType);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(cqlType, javaType);
+        }
+    }
+
+    /**
+     * Cache loader for the codecs cache.
+     */
+    private class TypeCodecCacheLoader extends CacheLoader<CacheKey, TypeCodec<?>>
+    {
+        @Override
+        public TypeCodec<?> load(CacheKey cacheKey)
+        {
+            checkNotNull(cacheKey.cqlType, "Parameter cqlType cannot be null");
+            if (logger.isTraceEnabled())
+                logger.trace(
+                "Loading codec into cache: [{} <-> {}]",
+                CodecRegistry.toString(cacheKey.cqlType),
+                CodecRegistry.toString(cacheKey.javaType));
+            for (TypeCodec<?> codec : codecs)
+            {
+                if (codec.accepts(cacheKey.cqlType)
+                    && (cacheKey.javaType == null || codec.accepts(cacheKey.javaType)))
+                {
+                    logger.trace("Already existing codec found: {}", codec);
+                    return codec;
+                }
+            }
+            return createCodec(cacheKey.cqlType, cacheKey.javaType);
+        }
+    }
+
+    /**
+     * A complexity-based weigher for the codecs cache. Weights are computed mainly according to the
+     * CQL type:
+     *
+     * <ol>
+     * <li>Manually-registered codecs always weigh 0;
+     * <li>Codecs for primitive types weigh 0;
+     * <li>Codecs for collections weigh the total weight of their inner types + the weight of their
+     * level of deepness;
+     * <li>Codecs for UDTs and tuples weigh the total weight of their inner types + the weight of
+     * their level of deepness, but cannot weigh less than 1;
+     * <li>Codecs for custom (non-CQL) types weigh 1.
+     * </ol>
+     * <p>
+     * A consequence of this algorithm is that codecs for primitive types and codecs for all "shallow"
+     * collections thereof are never evicted.
+     */
+    private class TypeCodecWeigher implements Weigher<CacheKey, TypeCodec<?>>
+    {
+
+        @Override
+        public int weigh(CacheKey key, TypeCodec<?> value)
+        {
+            return codecs.contains(value) ? 0 : weigh(value.cqlType, 0);
+        }
+
+        private int weigh(DataType cqlType, int level)
+        {
+            switch (cqlType.getName())
+            {
+                case LIST:
+                case SET:
+                case MAP:
+                {
+                    int weight = level;
+                    for (DataType eltType : cqlType.getTypeArguments())
+                    {
+                        weight += weigh(eltType, level + 1);
+                    }
+                    return weight;
+                }
+                case UDT:
+                {
+                    int weight = level;
+                    for (UserType.Field field : ((UserType) cqlType))
+                    {
+                        weight += weigh(field.getType(), level + 1);
+                    }
+                    return weight == 0 ? 1 : weight;
+                }
+                case TUPLE:
+                {
+                    int weight = level;
+                    for (DataType componentType : ((TupleType) cqlType).getComponentTypes())
+                    {
+                        weight += weigh(componentType, level + 1);
+                    }
+                    return weight == 0 ? 1 : weight;
+                }
+                case CUSTOM:
+                    return 1;
+                default:
+                    return 0;
+            }
+        }
+    }
+
+    /**
+     * Simple removal listener for the codec cache (can be used for debugging purposes by setting the
+     * {@code CodecRegistry} logger level to {@code TRACE}.
+     */
+    private static class TypeCodecRemovalListener implements RemovalListener<CacheKey, TypeCodec<?>>
+    {
+        @Override
+        public void onRemoval(RemovalNotification<CacheKey, TypeCodec<?>> notification)
+        {
+            logger.trace(
+            "Evicting codec from cache: {} (cause: {})",
+            notification.getValue(),
+            notification.getCause());
+        }
+    }
+
+    /**
+     * The list of user-registered codecs.
+     */
+    private final CopyOnWriteArrayList<TypeCodec<?>> codecs;
+
+    /**
+     * A LoadingCache to serve requests for codecs whenever possible. The cache can be used as long as
+     * at least the CQL type is known.
+     */
+    private final LoadingCache<CacheKey, TypeCodec<?>> cache;
+
+    /**
+     * Creates a new instance initialized with built-in codecs for all the base CQL types.
+     */
+    public CodecRegistry()
+    {
+        this.codecs = new CopyOnWriteArrayList<>();
+        this.cache = defaultCacheBuilder().build(new TypeCodecCacheLoader());
+    }
+
+    private CacheBuilder<CacheKey, TypeCodec<?>> defaultCacheBuilder()
+    {
+        CacheBuilder<CacheKey, TypeCodec<?>> builder =
+        CacheBuilder.newBuilder()
+                    // lists, sets and maps of 20 primitive types = 20 + 20 + 20*20 = 440 codecs,
+                    // so let's start with roughly 1/4 of that
+                    .initialCapacity(100)
+                    .maximumWeight(1000)
+                    .weigher(new TypeCodecWeigher());
+        if (logger.isTraceEnabled())
+            // do not bother adding a listener if it will be ineffective
+            builder = builder.removalListener(new TypeCodecRemovalListener());
+        return builder;
+    }
+
+    /**
+     * Register the given codec with this registry.
+     *
+     * <p>This method will log a warning and ignore the codec if it collides with a previously
+     * registered one. Note that this check is not done in a completely thread-safe manner; codecs
+     * should typically be registered at application startup, not in a highly concurrent context (if a
+     * race condition occurs, the worst possible outcome is that no warning gets logged, and the codec
+     * gets registered but will never actually be used).
+     *
+     * @param newCodec The codec to add to the registry.
+     * @return this CodecRegistry (for method chaining).
+     */
+    public CodecRegistry register(TypeCodec<?> newCodec)
+    {
+        for (TypeCodec<?> oldCodec : BUILT_IN_CODECS)
+        {
+            if (oldCodec.accepts(newCodec.getCqlType()) && oldCodec.accepts(newCodec.getJavaType()))
+            {
+                logger.warn(
+                "Ignoring codec {} because it collides with previously registered codec {}",
+                newCodec,
+                oldCodec);
+                return this;
+            }
+        }
+        for (TypeCodec<?> oldCodec : codecs)
+        {
+            if (oldCodec.accepts(newCodec.getCqlType()) && oldCodec.accepts(newCodec.getJavaType()))
+            {
+                logger.warn(
+                "Ignoring codec {} because it collides with previously registered codec {}",
+                newCodec,
+                oldCodec);
+                return this;
+            }
+        }
+        CacheKey key = new CacheKey(newCodec.getCqlType(), newCodec.getJavaType());
+        TypeCodec<?> existing = cache.getIfPresent(key);
+        if (existing != null)
+        {
+            logger.warn(
+            "Ignoring codec {} because it collides with previously generated codec {}",
+            newCodec,
+            existing);
+            return this;
+        }
+        this.codecs.add(newCodec);
+        return this;
+    }
+
+    /**
+     * Register the given codecs with this registry.
+     *
+     * @param codecs The codecs to add to the registry.
+     * @return this CodecRegistry (for method chaining).
+     * @see #register(TypeCodec)
+     */
+    public CodecRegistry register(TypeCodec<?>... codecs)
+    {
+        for (TypeCodec<?> codec : codecs) register(codec);
+        return this;
+    }
+
+    /**
+     * Register the given codecs with this registry.
+     *
+     * @param codecs The codecs to add to the registry.
+     * @return this CodecRegistry (for method chaining).
+     * @see #register(TypeCodec)
+     */
+    public CodecRegistry register(Iterable<? extends TypeCodec<?>> codecs)
+    {
+        for (TypeCodec<?> codec : codecs) register(codec);
+        return this;
+    }
+
+    /**
+     * Returns a {@link TypeCodec codec} that accepts the given value.
+     *
+     * <p>This method takes an arbitrary Java object and tries to locate a suitable codec for it.
+     * Codecs must perform a {@link TypeCodec#accepts(Object) runtime inspection} of the object to
+     * determine if they can accept it or not, which, depending on the implementations, can be
+     * expensive; besides, the resulting codec cannot be cached. Therefore there might be a
+     * performance penalty when using this method.
+     *
+     * <p>Furthermore, this method returns the first matching codec, regardless of its accepted CQL
+     * type. It should be reserved for situations where the target CQL type is not available or
+     * unknown. In the Java driver, this happens mainly when serializing a value in a {@code
+     * SimpleStatement#SimpleStatement(String, Object...) SimpleStatement} or in the {@code
+     * querybuilder.QueryBuilder}, where no CQL type information is
+     * available.
+     *
+     * <p>Codecs returned by this method are <em>NOT</em> cached (see the {@link CodecRegistry
+     * top-level documentation} of this class for more explanations about caching).
+     *
+     * @param value The value the codec should accept; must not be {@code null}.
+     * @return A suitable codec.
+     * @throws CodecNotFoundException if a suitable codec cannot be found.
+     */
+    public <T> TypeCodec<T> codecFor(T value)
+    {
+        return findCodec(null, value);
+    }
+
+    /**
+     * Returns a {@link TypeCodec codec} that accepts the given {@link DataType CQL type}.
+     *
+     * <p>This method returns the first matching codec, regardless of its accepted Java type. It
+     * should be reserved for situations where the Java type is not available or unknown. In the Java
+     * driver, this happens mainly when deserializing a value using the {@link
+     * GettableByIndexData#getObject(int) getObject} method.
+     *
+     * <p>Codecs returned by this method are cached (see the {@link CodecRegistry top-level
+     * documentation} of this class for more explanations about caching).
+     *
+     * @param cqlType The {@link DataType CQL type} the codec should accept; must not be {@code null}.
+     * @return A suitable codec.
+     * @throws CodecNotFoundException if a suitable codec cannot be found.
+     */
+    public <T> TypeCodec<T> codecFor(DataType cqlType) throws CodecNotFoundException
+    {
+        return lookupCodec(cqlType, null);
+    }
+
+    /**
+     * Returns a {@link TypeCodec codec} that accepts the given {@link DataType CQL type} and the
+     * given Java class.
+     *
+     * <p>This method can only handle raw (non-parameterized) Java types. For parameterized types, use
+     * {@link #codecFor(DataType, TypeToken)} instead.
+     *
+     * <p>Codecs returned by this method are cached (see the {@link CodecRegistry top-level
+     * documentation} of this class for more explanations about caching).
+     *
+     * @param cqlType  The {@link DataType CQL type} the codec should accept; must not be {@code null}.
+     * @param javaType The Java type the codec should accept; can be {@code null}.
+     * @return A suitable codec.
+     * @throws CodecNotFoundException if a suitable codec cannot be found.
+     */
+    public <T> TypeCodec<T> codecFor(DataType cqlType, Class<T> javaType)
+    throws CodecNotFoundException
+    {
+        return codecFor(cqlType, TypeToken.of(javaType));
+    }
+
+    /**
+     * Returns a {@link TypeCodec codec} that accepts the given {@link DataType CQL type} and the
+     * given Java type.
+     *
+     * <p>This method handles parameterized types thanks to Guava's {@link TypeToken} API.
+     *
+     * <p>Codecs returned by this method are cached (see the {@link CodecRegistry top-level
+     * documentation} of this class for more explanations about caching).
+     *
+     * @param cqlType  The {@link DataType CQL type} the codec should accept; must not be {@code null}.
+     * @param javaType The {@link TypeToken Java type} the codec should accept; can be {@code null}.
+     * @return A suitable codec.
+     * @throws CodecNotFoundException if a suitable codec cannot be found.
+     */
+    public <T> TypeCodec<T> codecFor(DataType cqlType, TypeToken<T> javaType)
+    throws CodecNotFoundException
+    {
+        return lookupCodec(cqlType, javaType);
+    }
+
+    /**
+     * Returns a {@link TypeCodec codec} that accepts the given {@link DataType CQL type} and the
+     * given value.
+     *
+     * <p>This method takes an arbitrary Java object and tries to locate a suitable codec for it.
+     * Codecs must perform a {@link TypeCodec#accepts(Object) runtime inspection} of the object to
+     * determine if they can accept it or not, which, depending on the implementations, can be
+     * expensive; besides, the resulting codec cannot be cached. Therefore there might be a
+     * performance penalty when using this method.
+     *
+     * <p>Codecs returned by this method are <em>NOT</em> cached (see the {@link CodecRegistry
+     * top-level documentation} of this class for more explanations about caching).
+     *
+     * @param cqlType The {@link DataType CQL type} the codec should accept; can be {@code null}.
+     * @param value   The value the codec should accept; must not be {@code null}.
+     * @return A suitable codec.
+     * @throws CodecNotFoundException if a suitable codec cannot be found.
+     */
+    public <T> TypeCodec<T> codecFor(DataType cqlType, T value)
+    {
+        return findCodec(cqlType, value);
+    }
+
+    @SuppressWarnings("unchecked")
+    private <T> TypeCodec<T> lookupCodec(DataType cqlType, TypeToken<T> javaType)
+    {
+        checkNotNull(cqlType, "Parameter cqlType cannot be null");
+        TypeCodec<?> codec = BUILT_IN_CODECS_MAP.get(cqlType.getName());
+        if (codec != null && (javaType == null || codec.accepts(javaType)))
+        {
+            logger.trace("Returning built-in codec {}", codec);
+            return (TypeCodec<T>) codec;
+        }
+        if (logger.isTraceEnabled())
+            logger.trace("Querying cache for codec [{} <-> {}]", toString(cqlType), toString(javaType));
+        try
+        {
+            CacheKey cacheKey = new CacheKey(cqlType, javaType);
+            codec = cache.get(cacheKey);
+        }
+        catch (UncheckedExecutionException e)
+        {
+            if (e.getCause() instanceof CodecNotFoundException)
+            {
+                throw (CodecNotFoundException) e.getCause();
+            }
+            throw new CodecNotFoundException(e.getCause());
+        }
+        catch (RuntimeException | ExecutionException e)
+        {
+            throw new CodecNotFoundException(e.getCause());
+        }
+        logger.trace("Returning cached codec {}", codec);
+        return (TypeCodec<T>) codec;
+    }
+
+    @SuppressWarnings("unchecked")
+    private <T> TypeCodec<T> findCodec(DataType cqlType, TypeToken<T> javaType)
+    {
+        checkNotNull(cqlType, "Parameter cqlType cannot be null");
+        if (logger.isTraceEnabled())
+            logger.trace("Looking for codec [{} <-> {}]", toString(cqlType), toString(javaType));
+
+        // Look at the built-in codecs first
+        for (TypeCodec<?> codec : BUILT_IN_CODECS)
+        {
+            if (codec.accepts(cqlType) && (javaType == null || codec.accepts(javaType)))
+            {
+                logger.trace("Built-in codec found: {}", codec);
+                return (TypeCodec<T>) codec;
+            }
+        }
+
+        // Look at the user-registered codecs next
+        for (TypeCodec<?> codec : codecs)
+        {
+            if (codec.accepts(cqlType) && (javaType == null || codec.accepts(javaType)))
+            {
+                logger.trace("Already registered codec found: {}", codec);
+                return (TypeCodec<T>) codec;
+            }
+        }
+        return createCodec(cqlType, javaType);
+    }
+
+    @SuppressWarnings("unchecked")
+    private <T> TypeCodec<T> findCodec(DataType cqlType, T value)
+    {
+        checkNotNull(value, "Parameter value cannot be null");
+        if (logger.isTraceEnabled())
+            logger.trace("Looking for codec [{} <-> {}]", toString(cqlType), value.getClass());
+
+        // Look at the built-in codecs first
+        for (TypeCodec<?> codec : BUILT_IN_CODECS)
+        {
+            if ((cqlType == null || codec.accepts(cqlType)) && codec.accepts(value))
+            {
+                logger.trace("Built-in codec found: {}", codec);
+                return (TypeCodec<T>) codec;
+            }
+        }
+
+        // Look at the user-registered codecs next
+        for (TypeCodec<?> codec : codecs)
+        {
+            if ((cqlType == null || codec.accepts(cqlType)) && codec.accepts(value))
+            {
+                logger.trace("Already registered codec found: {}", codec);
+                return (TypeCodec<T>) codec;
+            }
+        }
+        return createCodec(cqlType, value);
+    }
+
+    private <T> TypeCodec<T> createCodec(DataType cqlType, TypeToken<T> javaType)
+    {
+        TypeCodec<T> codec = maybeCreateCodec(cqlType, javaType);
+        if (codec == null) throw notFound(cqlType, javaType);
+        // double-check that the created codec satisfies the initial request
+        // this check can fail specially when creating codecs for collections
+        // e.g. if B extends A and there is a codec registered for A and
+        // we request a codec for List<B>, the registry would generate a codec for List<A>
+        if (!codec.accepts(cqlType) || (javaType != null && !codec.accepts(javaType)))
+            throw notFound(cqlType, javaType);
+        logger.trace("Codec created: {}", codec);
+        return codec;
+    }
+
+    private <T> TypeCodec<T> createCodec(DataType cqlType, T value)
+    {
+        TypeCodec<T> codec = maybeCreateCodec(cqlType, value);
+        if (codec == null) throw notFound(cqlType, TypeToken.of(value.getClass()));
+        // double-check that the created codec satisfies the initial request
+        if ((cqlType != null && !codec.accepts(cqlType)) || !codec.accepts(value))
+            throw notFound(cqlType, TypeToken.of(value.getClass()));
+        logger.trace("Codec created: {}", codec);
+        return codec;
+    }
+
+    @SuppressWarnings("unchecked")
+    private <T> TypeCodec<T> maybeCreateCodec(DataType cqlType, TypeToken<T> javaType)
+    {
+        checkNotNull(cqlType);
+
+        if (cqlType.getName() == LIST
+            && (javaType == null || List.class.isAssignableFrom(javaType.getRawType())))
+        {
+            TypeToken<?> elementType = null;
+            if (javaType != null && javaType.getType() instanceof ParameterizedType)
+            {
+                Type[] typeArguments = ((ParameterizedType) javaType.getType()).getActualTypeArguments();
+                elementType = TypeToken.of(typeArguments[0]);
+            }
+            TypeCodec<?> eltCodec = findCodec(cqlType.getTypeArguments().get(0), elementType);
+            return (TypeCodec<T>) TypeCodec.list(eltCodec);
+        }
+
+        if (cqlType.getName() == SET
+            && (javaType == null || Set.class.isAssignableFrom(javaType.getRawType())))
+        {
+            TypeToken<?> elementType = null;
+            if (javaType != null && javaType.getType() instanceof ParameterizedType)
+            {
+                Type[] typeArguments = ((ParameterizedType) javaType.getType()).getActualTypeArguments();
+                elementType = TypeToken.of(typeArguments[0]);
+            }
+            TypeCodec<?> eltCodec = findCodec(cqlType.getTypeArguments().get(0), elementType);
+            return (TypeCodec<T>) TypeCodec.set(eltCodec);
+        }
+
+        if (cqlType.getName() == MAP
+            && (javaType == null || Map.class.isAssignableFrom(javaType.getRawType())))
+        {
+            TypeToken<?> keyType = null;
+            TypeToken<?> valueType = null;
+            if (javaType != null && javaType.getType() instanceof ParameterizedType)
+            {
+                Type[] typeArguments = ((ParameterizedType) javaType.getType()).getActualTypeArguments();
+                keyType = TypeToken.of(typeArguments[0]);
+                valueType = TypeToken.of(typeArguments[1]);
+            }
+            TypeCodec<?> keyCodec = findCodec(cqlType.getTypeArguments().get(0), keyType);
+            TypeCodec<?> valueCodec = findCodec(cqlType.getTypeArguments().get(1), valueType);
+            return (TypeCodec<T>) TypeCodec.map(keyCodec, valueCodec);
+        }
+
+        if (cqlType instanceof TupleType
+            && (javaType == null || TupleValue.class.isAssignableFrom(javaType.getRawType())))
+        {
+            return (TypeCodec<T>) TypeCodec.tuple((TupleType) cqlType);
+        }
+
+        if (cqlType instanceof UserType
+            && (javaType == null || UDTValue.class.isAssignableFrom(javaType.getRawType())))
+        {
+            return (TypeCodec<T>) TypeCodec.userType((UserType) cqlType);
+        }
+
+        if (cqlType instanceof DataType.CustomType
+            && (javaType == null || ByteBuffer.class.isAssignableFrom(javaType.getRawType())))
+        {
+            return (TypeCodec<T>) TypeCodec.custom((DataType.CustomType) cqlType);
+        }
+
+        return null;
+    }
+
+    @SuppressWarnings({ "unchecked", "rawtypes" })
+    private <T> TypeCodec<T> maybeCreateCodec(DataType cqlType, T value)
+    {
+        checkNotNull(value);
+
+        if ((cqlType == null || cqlType.getName() == LIST) && value instanceof List)
+        {
+            List list = (List) value;
+            if (list.isEmpty())
+            {
+                DataType elementType =
+                (cqlType == null || cqlType.getTypeArguments().isEmpty())
+                ? DataType.blob()
+                : cqlType.getTypeArguments().get(0);
+                return (TypeCodec<T>) TypeCodec.list(findCodec(elementType, (TypeToken) null));
+            }
+            else
+            {
+                DataType elementType =
+                (cqlType == null || cqlType.getTypeArguments().isEmpty())
+                ? null
+                : cqlType.getTypeArguments().get(0);
+                return (TypeCodec<T>) TypeCodec.list(findCodec(elementType, list.iterator().next()));
+            }
+        }
+
+        if ((cqlType == null || cqlType.getName() == SET) && value instanceof Set)
+        {
+            Set set = (Set) value;
+            if (set.isEmpty())
+            {
+                DataType elementType =
+                (cqlType == null || cqlType.getTypeArguments().isEmpty())
+                ? DataType.blob()
+                : cqlType.getTypeArguments().get(0);
+                return (TypeCodec<T>) TypeCodec.set(findCodec(elementType, (TypeToken) null));
+            }
+            else
+            {
+                DataType elementType =
+                (cqlType == null || cqlType.getTypeArguments().isEmpty())
+                ? null
+                : cqlType.getTypeArguments().get(0);
+                return (TypeCodec<T>) TypeCodec.set(findCodec(elementType, set.iterator().next()));
+            }
+        }
+
+        if ((cqlType == null || cqlType.getName() == MAP) && value instanceof Map)
+        {
+            Map map = (Map) value;
+            if (map.isEmpty())
+            {
+                DataType keyType =
+                (cqlType == null || cqlType.getTypeArguments().size() < 1)
+                ? DataType.blob()
+                : cqlType.getTypeArguments().get(0);
+                DataType valueType =
+                (cqlType == null || cqlType.getTypeArguments().size() < 2)
+                ? DataType.blob()
+                : cqlType.getTypeArguments().get(1);
+                return (TypeCodec<T>) TypeCodec.map(
+                findCodec(keyType, (TypeToken) null), findCodec(valueType, (TypeToken) null));
+            }
+            else
+            {
+                DataType keyType =
+                (cqlType == null || cqlType.getTypeArguments().size() < 1)
+                ? null
+                : cqlType.getTypeArguments().get(0);
+                DataType valueType =
+                (cqlType == null || cqlType.getTypeArguments().size() < 2)
+                ? null
+                : cqlType.getTypeArguments().get(1);
+                Map.Entry entry = (Map.Entry) map.entrySet().iterator().next();
+                return (TypeCodec<T>)
+                       TypeCodec.map(
+                       findCodec(keyType, entry.getKey()), findCodec(valueType, entry.getValue()));
+            }
+        }
+
+        if ((cqlType == null || cqlType.getName() == DataType.Name.TUPLE)
+            && value instanceof TupleValue)
+        {
+            return (TypeCodec<T>)
+                   TypeCodec.tuple(cqlType == null ? ((TupleValue) value).getType() : (TupleType) cqlType);
+        }
+
+        if ((cqlType == null || cqlType.getName() == DataType.Name.UDT) && value instanceof UDTValue)
+        {
+            return (TypeCodec<T>)
+                   TypeCodec.userType(cqlType == null ? ((UDTValue) value).getType() : (UserType) cqlType);
+        }
+
+        if ((cqlType instanceof DataType.CustomType)
+            && value instanceof ByteBuffer)
+        {
+            return (TypeCodec<T>) TypeCodec.custom((DataType.CustomType) cqlType);
+        }
+
+        return null;
+    }
+
+    private static CodecNotFoundException notFound(DataType cqlType, TypeToken<?> javaType)
+    {
+        String msg =
+        String.format(
+        "Codec not found for requested operation: [%s <-> %s]",
+        toString(cqlType), toString(javaType));
+        return new CodecNotFoundException(msg);
+    }
+
+    private static String toString(Object value)
+    {
+        return value == null ? "ANY" : value.toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/CodecUtils.java b/src/java/org/apache/cassandra/cql3/functions/types/CodecUtils.java
new file mode 100644
index 0000000..4e97e5c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/CodecUtils.java
@@ -0,0 +1,266 @@
+/*
+ * 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 org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * A set of utility methods to deal with type conversion and serialization.
+ */
+public final class CodecUtils
+{
+
+    private static final long MAX_CQL_LONG_VALUE = ((1L << 32) - 1);
+
+    private static final long EPOCH_AS_CQL_LONG = (1L << 31);
+
+    private CodecUtils()
+    {
+    }
+
+    /**
+     * Utility method that "packs" together a list of {@link ByteBuffer}s containing serialized
+     * collection elements. Mainly intended for use with collection codecs when serializing
+     * collections.
+     *
+     * @param buffers  the collection elements
+     * @param elements the total number of elements
+     * @param version  the protocol version to use
+     * @return The serialized collection
+     */
+    public static ByteBuffer pack(ByteBuffer[] buffers, int elements, ProtocolVersion version)
+    {
+        int size = 0;
+        for (ByteBuffer bb : buffers)
+        {
+            int elemSize = sizeOfValue(bb, version);
+            size += elemSize;
+        }
+        ByteBuffer result = ByteBuffer.allocate(sizeOfCollectionSize(version) + size);
+        writeSize(result, elements, version);
+        for (ByteBuffer bb : buffers) writeValue(result, bb, version);
+        return (ByteBuffer) result.flip();
+    }
+
+    /**
+     * Utility method that reads a size value. Mainly intended for collection codecs when
+     * deserializing CQL collections.
+     *
+     * @param input   The ByteBuffer to read from.
+     * @param version The protocol version to use.
+     * @return The size value.
+     */
+    static int readSize(ByteBuffer input, ProtocolVersion version)
+    {
+        switch (version)
+        {
+            case V1:
+            case V2:
+                return getUnsignedShort(input);
+            case V3:
+            case V4:
+            case V5:
+                return input.getInt();
+            default:
+                throw new IllegalArgumentException(String.valueOf(version));
+        }
+    }
+
+    /**
+     * Utility method that writes a size value. Mainly intended for collection codecs when serializing
+     * CQL collections.
+     *
+     * @param output  The ByteBuffer to write to.
+     * @param size    The collection size.
+     * @param version The protocol version to use.
+     */
+    private static void writeSize(ByteBuffer output, int size, ProtocolVersion version)
+    {
+        switch (version)
+        {
+            case V1:
+            case V2:
+                if (size > 65535)
+                    throw new IllegalArgumentException(
+                    String.format(
+                    "Native protocol version %d supports up to 65535 elements in any collection - but collection contains %d elements",
+                    version.asInt(), size));
+                output.putShort((short) size);
+                break;
+            case V3:
+            case V4:
+            case V5:
+                output.putInt(size);
+                break;
+            default:
+                throw new IllegalArgumentException(String.valueOf(version));
+        }
+    }
+
+    /**
+     * Utility method that reads a value. Mainly intended for collection codecs when deserializing CQL
+     * collections.
+     *
+     * @param input   The ByteBuffer to read from.
+     * @param version The protocol version to use.
+     * @return The collection element.
+     */
+    public static ByteBuffer readValue(ByteBuffer input, ProtocolVersion version)
+    {
+        int size = readSize(input, version);
+        return size < 0 ? null : readBytes(input, size);
+    }
+
+    /**
+     * Utility method that writes a value. Mainly intended for collection codecs when deserializing
+     * CQL collections.
+     *
+     * @param output  The ByteBuffer to write to.
+     * @param value   The value to write.
+     * @param version The protocol version to use.
+     */
+    public static void writeValue(ByteBuffer output, ByteBuffer value, ProtocolVersion version)
+    {
+        switch (version)
+        {
+            case V1:
+            case V2:
+                assert value != null;
+                output.putShort((short) value.remaining());
+                output.put(value.duplicate());
+                break;
+            case V3:
+            case V4:
+            case V5:
+                if (value == null)
+                {
+                    output.putInt(-1);
+                }
+                else
+                {
+                    output.putInt(value.remaining());
+                    output.put(value.duplicate());
+                }
+                break;
+            default:
+                throw new IllegalArgumentException(String.valueOf(version));
+        }
+    }
+
+    /**
+     * Read {@code length} bytes from {@code bb} into a new ByteBuffer.
+     *
+     * @param bb     The ByteBuffer to read.
+     * @param length The number of bytes to read.
+     * @return The read bytes.
+     */
+    public static ByteBuffer readBytes(ByteBuffer bb, int length)
+    {
+        ByteBuffer copy = bb.duplicate();
+        copy.limit(copy.position() + length);
+        bb.position(bb.position() + length);
+        return copy;
+    }
+
+    /**
+     * Converts an "unsigned" int read from a DATE value into a signed int.
+     *
+     * <p>The protocol encodes DATE values as <em>unsigned</em> ints with the Epoch in the middle of
+     * the range (2^31). This method handles the conversion from an "unsigned" to a signed int.
+     */
+    static int fromUnsignedToSignedInt(int unsigned)
+    {
+        return unsigned + Integer.MIN_VALUE; // this relies on overflow for "negative" values
+    }
+
+    /**
+     * Converts an int into an "unsigned" int suitable to be written as a DATE value.
+     *
+     * <p>The protocol encodes DATE values as <em>unsigned</em> ints with the Epoch in the middle of
+     * the range (2^31). This method handles the conversion from a signed to an "unsigned" int.
+     */
+    static int fromSignedToUnsignedInt(int signed)
+    {
+        return signed - Integer.MIN_VALUE;
+    }
+
+    /**
+     * Convert from a raw CQL long representing a numeric DATE literal to the number of days since the
+     * Epoch. In CQL, numeric DATE literals are longs (unsigned integers actually) between 0 and 2^32
+     * - 1, with the epoch in the middle; this method re-centers the epoch at 0.
+     *
+     * @param raw The CQL date value to convert.
+     * @return The number of days since the Epoch corresponding to the given raw value.
+     * @throws IllegalArgumentException if the value is out of range.
+     */
+    static int fromCqlDateToDaysSinceEpoch(long raw)
+    {
+        if (raw < 0 || raw > MAX_CQL_LONG_VALUE)
+            throw new IllegalArgumentException(
+            String.format(
+            "Numeric literals for DATE must be between 0 and %d (got %d)",
+            MAX_CQL_LONG_VALUE, raw));
+        return (int) (raw - EPOCH_AS_CQL_LONG);
+    }
+
+    private static int sizeOfCollectionSize(ProtocolVersion version)
+    {
+        switch (version)
+        {
+            case V1:
+            case V2:
+                return 2;
+            case V3:
+            case V4:
+            case V5:
+                return 4;
+            default:
+                throw new IllegalArgumentException(String.valueOf(version));
+        }
+    }
+
+    private static int sizeOfValue(ByteBuffer value, ProtocolVersion version)
+    {
+        switch (version)
+        {
+            case V1:
+            case V2:
+                int elemSize = value.remaining();
+                if (elemSize > 65535)
+                    throw new IllegalArgumentException(
+                    String.format(
+                    "Native protocol version %d supports only elements with size up to 65535 bytes - but element size is %d bytes",
+                    version.asInt(), elemSize));
+                return 2 + elemSize;
+            case V3:
+            case V4:
+            case V5:
+                return value == null ? 4 : 4 + value.remaining();
+            default:
+                throw new IllegalArgumentException(String.valueOf(version));
+        }
+    }
+
+    private static int getUnsignedShort(ByteBuffer bb)
+    {
+        int length = (bb.get() & 0xFF) << 8;
+        return length | (bb.get() & 0xFF);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/DataType.java b/src/java/org/apache/cassandra/cql3/functions/types/DataType.java
new file mode 100644
index 0000000..5412720
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/DataType.java
@@ -0,0 +1,703 @@
+/*
+ * 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.*;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * Data types supported by cassandra.
+ */
+public abstract class DataType
+{
+
+    /**
+     * The CQL type name.
+     */
+    public enum Name
+    {
+        CUSTOM(0),
+        ASCII(1),
+        BIGINT(2),
+        BLOB(3),
+        BOOLEAN(4),
+        COUNTER(5),
+        DECIMAL(6),
+        DOUBLE(7),
+        FLOAT(8),
+        INT(9),
+        TEXT(10)
+        {
+            @Override
+            public boolean isCompatibleWith(Name that)
+            {
+                return this == that || that == VARCHAR;
+            }
+        },
+        TIMESTAMP(11),
+        UUID(12),
+        VARCHAR(13)
+        {
+            @Override
+            public boolean isCompatibleWith(Name that)
+            {
+                return this == that || that == TEXT;
+            }
+        },
+        VARINT(14),
+        TIMEUUID(15),
+        INET(16),
+        DATE(17, ProtocolVersion.V4),
+        TIME(18, ProtocolVersion.V4),
+        SMALLINT(19, ProtocolVersion.V4),
+        TINYINT(20, ProtocolVersion.V4),
+        DURATION(21, ProtocolVersion.V5),
+        LIST(32),
+        MAP(33),
+        SET(34),
+        UDT(48, ProtocolVersion.V3),
+        TUPLE(49, ProtocolVersion.V3);
+
+        final int protocolId;
+
+        final ProtocolVersion minProtocolVersion;
+
+        private static final Name[] nameToIds;
+
+        static
+        {
+            int maxCode = -1;
+            for (Name name : Name.values()) maxCode = Math.max(maxCode, name.protocolId);
+            nameToIds = new Name[maxCode + 1];
+            for (Name name : Name.values())
+            {
+                if (nameToIds[name.protocolId] != null) throw new IllegalStateException("Duplicate Id");
+                nameToIds[name.protocolId] = name;
+            }
+        }
+
+        Name(int protocolId)
+        {
+            this(protocolId, ProtocolVersion.V1);
+        }
+
+        Name(int protocolId, ProtocolVersion minProtocolVersion)
+        {
+            this.protocolId = protocolId;
+            this.minProtocolVersion = minProtocolVersion;
+        }
+
+        /**
+         * Return {@code true} if the provided Name is equal to this one, or if they are aliases for
+         * each other, and {@code false} otherwise.
+         *
+         * @param that the Name to compare with the current one.
+         * @return {@code true} if the provided Name is equal to this one, or if they are aliases for
+         * each other, and {@code false} otherwise.
+         */
+        public boolean isCompatibleWith(Name that)
+        {
+            return this == that;
+        }
+
+        @Override
+        public String toString()
+        {
+            return super.toString().toLowerCase();
+        }
+    }
+
+    private static final Map<Name, DataType> primitiveTypeMap =
+    new EnumMap<>(Name.class);
+
+    static
+    {
+        primitiveTypeMap.put(Name.ASCII, new DataType.NativeType(Name.ASCII));
+        primitiveTypeMap.put(Name.BIGINT, new DataType.NativeType(Name.BIGINT));
+        primitiveTypeMap.put(Name.BLOB, new DataType.NativeType(Name.BLOB));
+        primitiveTypeMap.put(Name.BOOLEAN, new DataType.NativeType(Name.BOOLEAN));
+        primitiveTypeMap.put(Name.COUNTER, new DataType.NativeType(Name.COUNTER));
+        primitiveTypeMap.put(Name.DECIMAL, new DataType.NativeType(Name.DECIMAL));
+        primitiveTypeMap.put(Name.DOUBLE, new DataType.NativeType(Name.DOUBLE));
+        primitiveTypeMap.put(Name.FLOAT, new DataType.NativeType(Name.FLOAT));
+        primitiveTypeMap.put(Name.INET, new DataType.NativeType(Name.INET));
+        primitiveTypeMap.put(Name.INT, new DataType.NativeType(Name.INT));
+        primitiveTypeMap.put(Name.TEXT, new DataType.NativeType(Name.TEXT));
+        primitiveTypeMap.put(Name.TIMESTAMP, new DataType.NativeType(Name.TIMESTAMP));
+        primitiveTypeMap.put(Name.UUID, new DataType.NativeType(Name.UUID));
+        primitiveTypeMap.put(Name.VARCHAR, new DataType.NativeType(Name.VARCHAR));
+        primitiveTypeMap.put(Name.VARINT, new DataType.NativeType(Name.VARINT));
+        primitiveTypeMap.put(Name.TIMEUUID, new DataType.NativeType(Name.TIMEUUID));
+        primitiveTypeMap.put(Name.SMALLINT, new DataType.NativeType(Name.SMALLINT));
+        primitiveTypeMap.put(Name.TINYINT, new DataType.NativeType(Name.TINYINT));
+        primitiveTypeMap.put(Name.DATE, new DataType.NativeType(Name.DATE));
+        primitiveTypeMap.put(Name.TIME, new DataType.NativeType(Name.TIME));
+        primitiveTypeMap.put(Name.DURATION, new DataType.NativeType(Name.DURATION));
+    }
+
+    protected final DataType.Name name;
+
+    protected DataType(DataType.Name name)
+    {
+        this.name = name;
+    }
+
+    /**
+     * Returns the ASCII type.
+     *
+     * @return The ASCII type.
+     */
+    public static DataType ascii()
+    {
+        return primitiveTypeMap.get(Name.ASCII);
+    }
+
+    /**
+     * Returns the BIGINT type.
+     *
+     * @return The BIGINT type.
+     */
+    public static DataType bigint()
+    {
+        return primitiveTypeMap.get(Name.BIGINT);
+    }
+
+    /**
+     * Returns the BLOB type.
+     *
+     * @return The BLOB type.
+     */
+    public static DataType blob()
+    {
+        return primitiveTypeMap.get(Name.BLOB);
+    }
+
+    /**
+     * Returns the BOOLEAN type.
+     *
+     * @return The BOOLEAN type.
+     */
+    public static DataType cboolean()
+    {
+        return primitiveTypeMap.get(Name.BOOLEAN);
+    }
+
+    /**
+     * Returns the COUNTER type.
+     *
+     * @return The COUNTER type.
+     */
+    public static DataType counter()
+    {
+        return primitiveTypeMap.get(Name.COUNTER);
+    }
+
+    /**
+     * Returns the DECIMAL type.
+     *
+     * @return The DECIMAL type.
+     */
+    public static DataType decimal()
+    {
+        return primitiveTypeMap.get(Name.DECIMAL);
+    }
+
+    /**
+     * Returns the DOUBLE type.
+     *
+     * @return The DOUBLE type.
+     */
+    public static DataType cdouble()
+    {
+        return primitiveTypeMap.get(Name.DOUBLE);
+    }
+
+    /**
+     * Returns the FLOAT type.
+     *
+     * @return The FLOAT type.
+     */
+    public static DataType cfloat()
+    {
+        return primitiveTypeMap.get(Name.FLOAT);
+    }
+
+    /**
+     * Returns the INET type.
+     *
+     * @return The INET type.
+     */
+    public static DataType inet()
+    {
+        return primitiveTypeMap.get(Name.INET);
+    }
+
+    /**
+     * Returns the TINYINT type.
+     *
+     * @return The TINYINT type.
+     */
+    public static DataType tinyint()
+    {
+        return primitiveTypeMap.get(Name.TINYINT);
+    }
+
+    /**
+     * Returns the SMALLINT type.
+     *
+     * @return The SMALLINT type.
+     */
+    public static DataType smallint()
+    {
+        return primitiveTypeMap.get(Name.SMALLINT);
+    }
+
+    /**
+     * Returns the INT type.
+     *
+     * @return The INT type.
+     */
+    public static DataType cint()
+    {
+        return primitiveTypeMap.get(Name.INT);
+    }
+
+    /**
+     * Returns the TEXT type.
+     *
+     * @return The TEXT type.
+     */
+    public static DataType text()
+    {
+        return primitiveTypeMap.get(Name.TEXT);
+    }
+
+    /**
+     * Returns the TIMESTAMP type.
+     *
+     * @return The TIMESTAMP type.
+     */
+    public static DataType timestamp()
+    {
+        return primitiveTypeMap.get(Name.TIMESTAMP);
+    }
+
+    /**
+     * Returns the DATE type.
+     *
+     * @return The DATE type.
+     */
+    public static DataType date()
+    {
+        return primitiveTypeMap.get(Name.DATE);
+    }
+
+    /**
+     * Returns the TIME type.
+     *
+     * @return The TIME type.
+     */
+    public static DataType time()
+    {
+        return primitiveTypeMap.get(Name.TIME);
+    }
+
+    /**
+     * Returns the UUID type.
+     *
+     * @return The UUID type.
+     */
+    public static DataType uuid()
+    {
+        return primitiveTypeMap.get(Name.UUID);
+    }
+
+    /**
+     * Returns the VARCHAR type.
+     *
+     * @return The VARCHAR type.
+     */
+    public static DataType varchar()
+    {
+        return primitiveTypeMap.get(Name.VARCHAR);
+    }
+
+    /**
+     * Returns the VARINT type.
+     *
+     * @return The VARINT type.
+     */
+    public static DataType varint()
+    {
+        return primitiveTypeMap.get(Name.VARINT);
+    }
+
+    /**
+     * Returns the TIMEUUID type.
+     *
+     * @return The TIMEUUID type.
+     */
+    public static DataType timeuuid()
+    {
+        return primitiveTypeMap.get(Name.TIMEUUID);
+    }
+
+    /**
+     * Returns the type of lists of {@code elementType} elements.
+     *
+     * @param elementType the type of the list elements.
+     * @param frozen      whether the list is frozen.
+     * @return the type of lists of {@code elementType} elements.
+     */
+    public static CollectionType list(DataType elementType, boolean frozen)
+    {
+        return new DataType.CollectionType(Name.LIST, ImmutableList.of(elementType), frozen);
+    }
+
+    /**
+     * Returns the type of "not frozen" lists of {@code elementType} elements.
+     *
+     * <p>This is a shorthand for {@code list(elementType, false);}.
+     *
+     * @param elementType the type of the list elements.
+     * @return the type of "not frozen" lists of {@code elementType} elements.
+     */
+    public static CollectionType list(DataType elementType)
+    {
+        return list(elementType, false);
+    }
+
+    /**
+     * Returns the type of sets of {@code elementType} elements.
+     *
+     * @param elementType the type of the set elements.
+     * @param frozen      whether the set is frozen.
+     * @return the type of sets of {@code elementType} elements.
+     */
+    public static CollectionType set(DataType elementType, boolean frozen)
+    {
+        return new DataType.CollectionType(Name.SET, ImmutableList.of(elementType), frozen);
+    }
+
+    /**
+     * Returns the type of "not frozen" sets of {@code elementType} elements.
+     *
+     * <p>This is a shorthand for {@code set(elementType, false);}.
+     *
+     * @param elementType the type of the set elements.
+     * @return the type of "not frozen" sets of {@code elementType} elements.
+     */
+    public static CollectionType set(DataType elementType)
+    {
+        return set(elementType, false);
+    }
+
+    /**
+     * Returns the type of maps of {@code keyType} to {@code valueType} elements.
+     *
+     * @param keyType   the type of the map keys.
+     * @param valueType the type of the map values.
+     * @param frozen    whether the map is frozen.
+     * @return the type of maps of {@code keyType} to {@code valueType} elements.
+     */
+    public static CollectionType map(DataType keyType, DataType valueType, boolean frozen)
+    {
+        return new DataType.CollectionType(Name.MAP, ImmutableList.of(keyType, valueType), frozen);
+    }
+
+    /**
+     * Returns the type of "not frozen" maps of {@code keyType} to {@code valueType} elements.
+     *
+     * <p>This is a shorthand for {@code map(keyType, valueType, false);}.
+     *
+     * @param keyType   the type of the map keys.
+     * @param valueType the type of the map values.
+     * @return the type of "not frozen" maps of {@code keyType} to {@code valueType} elements.
+     */
+    public static CollectionType map(DataType keyType, DataType valueType)
+    {
+        return map(keyType, valueType, false);
+    }
+
+    /**
+     * Returns a Custom type.
+     *
+     * <p>A custom type is defined by the name of the class used on the Cassandra side to implement
+     * it. Note that the support for custom types by the driver is limited.
+     *
+     * <p>The use of custom types is rarely useful and is thus not encouraged.
+     *
+     * @param typeClassName the server-side fully qualified class name for the type.
+     * @return the custom type for {@code typeClassName}.
+     */
+    public static DataType.CustomType custom(String typeClassName)
+    {
+        if (typeClassName == null) throw new NullPointerException();
+        return new DataType.CustomType(Name.CUSTOM, typeClassName);
+    }
+
+    /**
+     * Returns the Duration type, introduced in Cassandra 3.10.
+     *
+     * <p>Note that a Duration type does not have a native representation in CQL, and technically, is
+     * merely a special {@link DataType#custom(String) custom type} from the driver's point of view.
+     *
+     * @return the Duration type. The returned instance is a singleton.
+     */
+    public static DataType duration()
+    {
+        return primitiveTypeMap.get(Name.DURATION);
+    }
+
+    /**
+     * Returns the name of that type.
+     *
+     * @return the name of that type.
+     */
+    public Name getName()
+    {
+        return name;
+    }
+
+    /**
+     * Returns whether this data type is frozen.
+     *
+     * <p>This applies to User Defined Types, tuples and nested collections. Frozen types are
+     * serialized as a single value in Cassandra's storage engine, whereas non-frozen types are stored
+     * in a form that allows updates to individual subfields.
+     *
+     * @return whether this data type is frozen.
+     */
+    public abstract boolean isFrozen();
+
+    /**
+     * Returns whether this data type represent a CQL {@link
+     * DataType.CollectionType collection type}, that is, a list, set or map.
+     *
+     * @return whether this data type name represent the name of a collection type.
+     */
+    public boolean isCollection()
+    {
+        return this instanceof CollectionType;
+    }
+
+    /**
+     * Returns the type arguments of this type.
+     *
+     * <p>Note that only the collection types (LIST, MAP, SET) have type arguments. For the other
+     * types, this will return an empty list.
+     *
+     * <p>For the collection types:
+     *
+     * <ul>
+     * <li>For lists and sets, this method returns one argument, the type of the elements.
+     * <li>For maps, this method returns two arguments, the first one is the type of the map keys,
+     * the second one is the type of the map values.
+     * </ul>
+     *
+     * @return an immutable list containing the type arguments of this type.
+     */
+    public List<DataType> getTypeArguments()
+    {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Returns a String representation of this data type suitable for inclusion as a parameter type in
+     * a function or aggregate signature.
+     *
+     * <p>In such places, the String representation might vary from the canonical one as returned by
+     * {@link #toString()}; e.g. the {@code frozen} keyword is not accepted.
+     *
+     * @return a String representation of this data type suitable for inclusion as a parameter type in
+     * a function or aggregate signature.
+     */
+    public String asFunctionParameterString()
+    {
+        return toString();
+    }
+
+    /**
+     * Instances of this class represent CQL native types, also known as CQL primitive types.
+     */
+    public static class NativeType extends DataType
+    {
+
+        private NativeType(DataType.Name name)
+        {
+            super(name);
+        }
+
+        @Override
+        public boolean isFrozen()
+        {
+            return false;
+        }
+
+        @Override
+        public final int hashCode()
+        {
+            return (name == Name.TEXT) ? Name.VARCHAR.hashCode() : name.hashCode();
+        }
+
+        @Override
+        public final boolean equals(Object o)
+        {
+            if (!(o instanceof DataType.NativeType)) return false;
+
+            NativeType that = (DataType.NativeType) o;
+            return this.name.isCompatibleWith(that.name);
+        }
+
+        @Override
+        public String toString()
+        {
+            return name.toString();
+        }
+    }
+
+    /**
+     * Instances of this class represent collection types, that is, lists, sets or maps.
+     */
+    public static class CollectionType extends DataType
+    {
+
+        private final List<DataType> typeArguments;
+        private final boolean frozen;
+
+        private CollectionType(DataType.Name name, List<DataType> typeArguments, boolean frozen)
+        {
+            super(name);
+            this.typeArguments = typeArguments;
+            this.frozen = frozen;
+        }
+
+        @Override
+        public boolean isFrozen()
+        {
+            return frozen;
+        }
+
+        @Override
+        public List<DataType> getTypeArguments()
+        {
+            return typeArguments;
+        }
+
+        @Override
+        public final int hashCode()
+        {
+            return Objects.hash(name, typeArguments);
+        }
+
+        @Override
+        public final boolean equals(Object o)
+        {
+            if (!(o instanceof DataType.CollectionType)) return false;
+
+            DataType.CollectionType d = (DataType.CollectionType) o;
+            return name == d.name && typeArguments.equals(d.typeArguments);
+        }
+
+        @Override
+        public String toString()
+        {
+            if (name == Name.MAP)
+            {
+                String template = frozen ? "frozen<%s<%s, %s>>" : "%s<%s, %s>";
+                return String.format(template, name, typeArguments.get(0), typeArguments.get(1));
+            }
+            else
+            {
+                String template = frozen ? "frozen<%s<%s>>" : "%s<%s>";
+                return String.format(template, name, typeArguments.get(0));
+            }
+        }
+
+        @Override
+        public String asFunctionParameterString()
+        {
+            if (name == Name.MAP)
+            {
+                String template = "%s<%s, %s>";
+                return String.format(
+                template,
+                name,
+                typeArguments.get(0).asFunctionParameterString(),
+                typeArguments.get(1).asFunctionParameterString());
+            }
+            else
+            {
+                String template = "%s<%s>";
+                return String.format(template, name, typeArguments.get(0).asFunctionParameterString());
+            }
+        }
+    }
+
+    /**
+     * A "custom" type is a type that cannot be expressed as a CQL type.
+     *
+     * <p>Each custom type is merely identified by the fully qualified {@code
+     * #getCustomTypeClassName() class name} that represents this type server-side.
+     *
+     * <p>The driver provides a minimal support for such types through instances of this class.
+     *
+     * <p>A codec for custom types can be obtained via {@link TypeCodec#custom(DataType.CustomType)}.
+     */
+    public static class CustomType extends DataType
+    {
+
+        private final String customClassName;
+
+        private CustomType(DataType.Name name, String className)
+        {
+            super(name);
+            this.customClassName = className;
+        }
+
+        @Override
+        public boolean isFrozen()
+        {
+            return false;
+        }
+
+        @Override
+        public final int hashCode()
+        {
+            return Objects.hash(name, customClassName);
+        }
+
+        @Override
+        public final boolean equals(Object o)
+        {
+            if (!(o instanceof DataType.CustomType)) return false;
+
+            DataType.CustomType d = (DataType.CustomType) o;
+            return name == d.name && Objects.equals(customClassName, d.customClassName);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("'%s'", customClassName);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/DataTypeClassNameParser.java b/src/java/org/apache/cassandra/cql3/functions/types/DataTypeClassNameParser.java
new file mode 100644
index 0000000..7064ba2
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/DataTypeClassNameParser.java
@@ -0,0 +1,396 @@
+/*
+ * 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.*;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.cql3.functions.types.exceptions.DriverInternalError;
+import org.apache.cassandra.cql3.functions.types.utils.Bytes;
+
+/*
+ * Parse data types from schema tables, for Cassandra 3.0 and above.
+ * In these versions, data types appear as class names, like "org.apache.cassandra.db.marshal.AsciiType"
+ * or "org.apache.cassandra.db.marshal.TupleType(org.apache.cassandra.db.marshal.Int32Type,org.apache.cassandra.db.marshal.Int32Type)".
+ *
+ * This is modified (and simplified) from Cassandra's TypeParser class to suit
+ * our needs. In particular it's not very efficient, but it doesn't really matter
+ * since it's rarely used and never in a critical path.
+ *
+ * Note that those methods all throw DriverInternalError when there is a parsing
+ * problem because in theory we'll only parse class names coming from Cassandra and
+ * so there shouldn't be anything wrong with them.
+ */
+public class DataTypeClassNameParser
+{
+    private static final Logger logger = LoggerFactory.getLogger(DataTypeClassNameParser.class);
+
+    private static final String REVERSED_TYPE = "org.apache.cassandra.db.marshal.ReversedType";
+    private static final String FROZEN_TYPE = "org.apache.cassandra.db.marshal.FrozenType";
+    private static final String LIST_TYPE = "org.apache.cassandra.db.marshal.ListType";
+    private static final String SET_TYPE = "org.apache.cassandra.db.marshal.SetType";
+    private static final String MAP_TYPE = "org.apache.cassandra.db.marshal.MapType";
+    private static final String UDT_TYPE = "org.apache.cassandra.db.marshal.UserType";
+    private static final String TUPLE_TYPE = "org.apache.cassandra.db.marshal.TupleType";
+    private static final String DURATION_TYPE = "org.apache.cassandra.db.marshal.DurationType";
+
+    private static final ImmutableMap<String, DataType> cassTypeToDataType =
+    new ImmutableMap.Builder<String, DataType>()
+    .put("org.apache.cassandra.db.marshal.AsciiType", DataType.ascii())
+    .put("org.apache.cassandra.db.marshal.LongType", DataType.bigint())
+    .put("org.apache.cassandra.db.marshal.BytesType", DataType.blob())
+    .put("org.apache.cassandra.db.marshal.BooleanType", DataType.cboolean())
+    .put("org.apache.cassandra.db.marshal.CounterColumnType", DataType.counter())
+    .put("org.apache.cassandra.db.marshal.DecimalType", DataType.decimal())
+    .put("org.apache.cassandra.db.marshal.DoubleType", DataType.cdouble())
+    .put("org.apache.cassandra.db.marshal.FloatType", DataType.cfloat())
+    .put("org.apache.cassandra.db.marshal.InetAddressType", DataType.inet())
+    .put("org.apache.cassandra.db.marshal.Int32Type", DataType.cint())
+    .put("org.apache.cassandra.db.marshal.UTF8Type", DataType.text())
+    .put("org.apache.cassandra.db.marshal.TimestampType", DataType.timestamp())
+    .put("org.apache.cassandra.db.marshal.SimpleDateType", DataType.date())
+    .put("org.apache.cassandra.db.marshal.TimeType", DataType.time())
+    .put("org.apache.cassandra.db.marshal.UUIDType", DataType.uuid())
+    .put("org.apache.cassandra.db.marshal.IntegerType", DataType.varint())
+    .put("org.apache.cassandra.db.marshal.TimeUUIDType", DataType.timeuuid())
+    .put("org.apache.cassandra.db.marshal.ByteType", DataType.tinyint())
+    .put("org.apache.cassandra.db.marshal.ShortType", DataType.smallint())
+    .put(DURATION_TYPE, DataType.duration())
+    .build();
+
+    public static DataType parseOne(
+    String className, ProtocolVersion protocolVersion, CodecRegistry codecRegistry)
+    {
+        boolean frozen = false;
+        if (isReversed(className))
+        {
+            // Just skip the ReversedType part, we don't care
+            className = getNestedClassName(className);
+        }
+        else if (isFrozen(className))
+        {
+            frozen = true;
+            className = getNestedClassName(className);
+        }
+
+        Parser parser = new Parser(className, 0);
+        String next = parser.parseNextName();
+
+        if (next.startsWith(LIST_TYPE))
+            return DataType.list(
+            parseOne(parser.getTypeParameters().get(0), protocolVersion, codecRegistry), frozen);
+
+        if (next.startsWith(SET_TYPE))
+            return DataType.set(
+            parseOne(parser.getTypeParameters().get(0), protocolVersion, codecRegistry), frozen);
+
+        if (next.startsWith(MAP_TYPE))
+        {
+            List<String> params = parser.getTypeParameters();
+            return DataType.map(
+            parseOne(params.get(0), protocolVersion, codecRegistry),
+            parseOne(params.get(1), protocolVersion, codecRegistry),
+            frozen);
+        }
+
+        if (frozen)
+            logger.warn(
+            "Got o.a.c.db.marshal.FrozenType for something else than a collection, "
+            + "this driver version might be too old for your version of Cassandra");
+
+        if (isUserType(next))
+        {
+            ++parser.idx; // skipping '('
+
+            String keyspace = parser.readOne();
+            parser.skipBlankAndComma();
+            String typeName =
+            TypeCodec.varchar()
+                     .deserialize(Bytes.fromHexString("0x" + parser.readOne()), protocolVersion);
+            parser.skipBlankAndComma();
+            Map<String, String> rawFields = parser.getNameAndTypeParameters();
+            List<UserType.Field> fields = new ArrayList<>(rawFields.size());
+            for (Map.Entry<String, String> entry : rawFields.entrySet())
+                fields.add(
+                new UserType.Field(
+                entry.getKey(), parseOne(entry.getValue(), protocolVersion, codecRegistry)));
+            // create a frozen UserType since C* 2.x UDTs are always frozen.
+            return new UserType(keyspace, typeName, true, fields, protocolVersion, codecRegistry);
+        }
+
+        if (isTupleType(next))
+        {
+            List<String> rawTypes = parser.getTypeParameters();
+            List<DataType> types = new ArrayList<>(rawTypes.size());
+            for (String rawType : rawTypes)
+            {
+                types.add(parseOne(rawType, protocolVersion, codecRegistry));
+            }
+            return new TupleType(types, protocolVersion, codecRegistry);
+        }
+
+        DataType type = cassTypeToDataType.get(next);
+        return type == null ? DataType.custom(className) : type;
+    }
+
+    public static boolean isReversed(String className)
+    {
+        return className.startsWith(REVERSED_TYPE);
+    }
+
+    public static boolean isFrozen(String className)
+    {
+        return className.startsWith(FROZEN_TYPE);
+    }
+
+    private static String getNestedClassName(String className)
+    {
+        Parser p = new Parser(className, 0);
+        p.parseNextName();
+        List<String> l = p.getTypeParameters();
+        if (l.size() != 1) throw new IllegalStateException();
+        className = l.get(0);
+        return className;
+    }
+
+    private static boolean isUserType(String className)
+    {
+        return className.startsWith(UDT_TYPE);
+    }
+
+    private static boolean isTupleType(String className)
+    {
+        return className.startsWith(TUPLE_TYPE);
+    }
+
+    private static class Parser
+    {
+
+        private final String str;
+        private int idx;
+
+        private Parser(String str, int idx)
+        {
+            this.str = str;
+            this.idx = idx;
+        }
+
+        String parseNextName()
+        {
+            skipBlank();
+            return readNextIdentifier();
+        }
+
+        String readOne()
+        {
+            String name = parseNextName();
+            String args = readRawArguments();
+            return name + args;
+        }
+
+        // Assumes we have just read a class name and read it's potential arguments
+        // blindly. I.e. it assume that either parsing is done or that we're on a '('
+        // and this reads everything up until the corresponding closing ')'. It
+        // returns everything read, including the enclosing parenthesis.
+        private String readRawArguments()
+        {
+            skipBlank();
+
+            if (isEOS() || str.charAt(idx) == ')' || str.charAt(idx) == ',') return "";
+
+            if (str.charAt(idx) != '(')
+                throw new IllegalStateException(
+                String.format(
+                "Expecting char %d of %s to be '(' but '%c' found", idx, str, str.charAt(idx)));
+
+            int i = idx;
+            int open = 1;
+            while (open > 0)
+            {
+                ++idx;
+
+                if (isEOS()) throw new IllegalStateException("Non closed parenthesis");
+
+                if (str.charAt(idx) == '(')
+                {
+                    open++;
+                }
+                else if (str.charAt(idx) == ')')
+                {
+                    open--;
+                }
+            }
+            // we've stopped at the last closing ')' so move past that
+            ++idx;
+            return str.substring(i, idx);
+        }
+
+        List<String> getTypeParameters()
+        {
+            List<String> list = new ArrayList<>();
+
+            if (isEOS()) return list;
+
+            if (str.charAt(idx) != '(') throw new IllegalStateException();
+
+            ++idx; // skipping '('
+
+            while (skipBlankAndComma())
+            {
+                if (str.charAt(idx) == ')')
+                {
+                    ++idx;
+                    return list;
+                }
+
+                try
+                {
+                    list.add(readOne());
+                }
+                catch (DriverInternalError e)
+                {
+                    throw new DriverInternalError(
+                    String.format("Exception while parsing '%s' around char %d", str, idx), e);
+                }
+            }
+            throw new DriverInternalError(
+            String.format(
+            "Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
+        }
+
+        // Must be at the start of the first parameter to read
+        Map<String, String> getNameAndTypeParameters()
+        {
+            // The order of the hashmap matters for UDT
+            Map<String, String> map = new LinkedHashMap<>();
+
+            while (skipBlankAndComma())
+            {
+                if (str.charAt(idx) == ')')
+                {
+                    ++idx;
+                    return map;
+                }
+
+                String bbHex = readNextIdentifier();
+                String name = null;
+                try
+                {
+                    name =
+                    TypeCodec.varchar()
+                             .deserialize(Bytes.fromHexString("0x" + bbHex), ProtocolVersion.CURRENT);
+                }
+                catch (NumberFormatException e)
+                {
+                    throwSyntaxError(e.getMessage());
+                }
+
+                skipBlank();
+                if (str.charAt(idx) != ':') throwSyntaxError("expecting ':' token");
+
+                ++idx;
+                skipBlank();
+                try
+                {
+                    map.put(name, readOne());
+                }
+                catch (DriverInternalError e)
+                {
+                    throw new DriverInternalError(
+                    String.format("Exception while parsing '%s' around char %d", str, idx), e);
+                }
+            }
+            throw new DriverInternalError(
+            String.format(
+            "Syntax error parsing '%s' at char %d: unexpected end of string", str, idx));
+        }
+
+        private void throwSyntaxError(String msg)
+        {
+            throw new DriverInternalError(
+            String.format("Syntax error parsing '%s' at char %d: %s", str, idx, msg));
+        }
+
+        private boolean isEOS()
+        {
+            return isEOS(str, idx);
+        }
+
+        private static boolean isEOS(String str, int i)
+        {
+            return i >= str.length();
+        }
+
+        private void skipBlank()
+        {
+            idx = skipBlank(str, idx);
+        }
+
+        private static int skipBlank(String str, int i)
+        {
+            while (!isEOS(str, i) && ParseUtils.isBlank(str.charAt(i))) ++i;
+
+            return i;
+        }
+
+        // skip all blank and at best one comma, return true if there not EOS
+        private boolean skipBlankAndComma()
+        {
+            boolean commaFound = false;
+            while (!isEOS())
+            {
+                int c = str.charAt(idx);
+                if (c == ',')
+                {
+                    if (commaFound) return true;
+                    else commaFound = true;
+                }
+                else if (!ParseUtils.isBlank(c))
+                {
+                    return true;
+                }
+                ++idx;
+            }
+            return false;
+        }
+
+        // left idx positioned on the character stopping the read
+        String readNextIdentifier()
+        {
+            int i = idx;
+            while (!isEOS() && ParseUtils.isIdentifierChar(str.charAt(idx))) ++idx;
+
+            return str.substring(i, idx);
+        }
+
+        @Override
+        public String toString()
+        {
+            return str.substring(0, idx)
+                   + '['
+                   + (idx == str.length() ? "" : str.charAt(idx))
+                   + ']'
+                   + str.substring(idx + 1);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/Duration.java b/src/java/org/apache/cassandra/cql3/functions/types/Duration.java
new file mode 100644
index 0000000..71d3e93
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/Duration.java
@@ -0,0 +1,654 @@
+/*
+ * 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.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.base.Objects;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * Represents a duration. A duration stores separately months, days, and seconds due to the fact
+ * that the number of days in a month varies, and a day can have 23 or 25 hours if a daylight saving
+ * is involved.
+ */
+public final class Duration
+{
+
+    private static final long NANOS_PER_MICRO = 1000L;
+    private static final long NANOS_PER_MILLI = 1000 * NANOS_PER_MICRO;
+    private static final long NANOS_PER_SECOND = 1000 * NANOS_PER_MILLI;
+    private static final long NANOS_PER_MINUTE = 60 * NANOS_PER_SECOND;
+    private static final long NANOS_PER_HOUR = 60 * NANOS_PER_MINUTE;
+    private static final int DAYS_PER_WEEK = 7;
+    private static final int MONTHS_PER_YEAR = 12;
+
+    /**
+     * The Regexp used to parse the duration provided as String.
+     */
+    private static final Pattern STANDARD_PATTERN =
+    Pattern.compile(
+    "\\G(\\d+)(y|Y|mo|MO|mO|Mo|w|W|d|D|h|H|s|S|ms|MS|mS|Ms|us|US|uS|Us|µs|µS|ns|NS|nS|Ns|m|M)");
+
+    /**
+     * The Regexp used to parse the duration when provided in the ISO 8601 format with designators.
+     */
+    private static final Pattern ISO8601_PATTERN =
+    Pattern.compile("P((\\d+)Y)?((\\d+)M)?((\\d+)D)?(T((\\d+)H)?((\\d+)M)?((\\d+)S)?)?");
+
+    /**
+     * The Regexp used to parse the duration when provided in the ISO 8601 format with designators.
+     */
+    private static final Pattern ISO8601_WEEK_PATTERN = Pattern.compile("P(\\d+)W");
+
+    /**
+     * The Regexp used to parse the duration when provided in the ISO 8601 alternative format.
+     */
+    private static final Pattern ISO8601_ALTERNATIVE_PATTERN =
+    Pattern.compile("P(\\d{4})-(\\d{2})-(\\d{2})T(\\d{2}):(\\d{2}):(\\d{2})");
+
+    /**
+     * The number of months.
+     */
+    private final int months;
+
+    /**
+     * The number of days.
+     */
+    private final int days;
+
+    /**
+     * The number of nanoseconds.
+     */
+    private final long nanoseconds;
+
+    private Duration(int months, int days, long nanoseconds)
+    {
+        // Makes sure that all the values are negative if one of them is
+        if ((months < 0 || days < 0 || nanoseconds < 0)
+            && ((months > 0 || days > 0 || nanoseconds > 0)))
+        {
+            throw new IllegalArgumentException(
+            String.format(
+            "All values must be either negative or positive, got %d months, %d days, %d nanoseconds",
+            months, days, nanoseconds));
+        }
+        this.months = months;
+        this.days = days;
+        this.nanoseconds = nanoseconds;
+    }
+
+    /**
+     * Creates a duration with the given number of months, days and nanoseconds.
+     *
+     * <p>A duration can be negative. In this case, all the non zero values must be negative.
+     *
+     * @param months      the number of months
+     * @param days        the number of days
+     * @param nanoseconds the number of nanoseconds
+     * @throws IllegalArgumentException if the values are not all negative or all positive
+     */
+    public static Duration newInstance(int months, int days, long nanoseconds)
+    {
+        return new Duration(months, days, nanoseconds);
+    }
+
+    /**
+     * Converts a <code>String</code> into a duration.
+     *
+     * <p>The accepted formats are:
+     *
+     * <ul>
+     * <li>multiple digits followed by a time unit like: 12h30m where the time unit can be:
+     * <ul>
+     * <li>{@code y}: years
+     * <li>{@code m}: months
+     * <li>{@code w}: weeks
+     * <li>{@code d}: days
+     * <li>{@code h}: hours
+     * <li>{@code m}: minutes
+     * <li>{@code s}: seconds
+     * <li>{@code ms}: milliseconds
+     * <li>{@code us} or {@code µs}: microseconds
+     * <li>{@code ns}: nanoseconds
+     * </ul>
+     * <li>ISO 8601 format: P[n]Y[n]M[n]DT[n]H[n]M[n]S or P[n]W
+     * <li>ISO 8601 alternative format: P[YYYY]-[MM]-[DD]T[hh]:[mm]:[ss]
+     * </ul>
+     *
+     * @param input the <code>String</code> to convert
+     * @return a {@link Duration}
+     */
+    public static Duration from(String input)
+    {
+        boolean isNegative = input.startsWith("-");
+        String source = isNegative ? input.substring(1) : input;
+
+        if (source.startsWith("P"))
+        {
+            if (source.endsWith("W")) return parseIso8601WeekFormat(isNegative, source);
+
+            if (source.contains("-")) return parseIso8601AlternativeFormat(isNegative, source);
+
+            return parseIso8601Format(isNegative, source);
+        }
+        return parseStandardFormat(isNegative, source);
+    }
+
+    private static Duration parseIso8601Format(boolean isNegative, String source)
+    {
+        Matcher matcher = ISO8601_PATTERN.matcher(source);
+        if (!matcher.matches())
+            throw new IllegalArgumentException(
+            String.format("Unable to convert '%s' to a duration", source));
+
+        Builder builder = new Builder(isNegative);
+        if (matcher.group(1) != null) builder.addYears(groupAsLong(matcher, 2));
+
+        if (matcher.group(3) != null) builder.addMonths(groupAsLong(matcher, 4));
+
+        if (matcher.group(5) != null) builder.addDays(groupAsLong(matcher, 6));
+
+        // Checks if the String contains time information
+        if (matcher.group(7) != null)
+        {
+            if (matcher.group(8) != null) builder.addHours(groupAsLong(matcher, 9));
+
+            if (matcher.group(10) != null) builder.addMinutes(groupAsLong(matcher, 11));
+
+            if (matcher.group(12) != null) builder.addSeconds(groupAsLong(matcher, 13));
+        }
+        return builder.build();
+    }
+
+    private static Duration parseIso8601AlternativeFormat(boolean isNegative, String source)
+    {
+        Matcher matcher = ISO8601_ALTERNATIVE_PATTERN.matcher(source);
+        if (!matcher.matches())
+            throw new IllegalArgumentException(
+            String.format("Unable to convert '%s' to a duration", source));
+
+        return new Builder(isNegative)
+               .addYears(groupAsLong(matcher, 1))
+               .addMonths(groupAsLong(matcher, 2))
+               .addDays(groupAsLong(matcher, 3))
+               .addHours(groupAsLong(matcher, 4))
+               .addMinutes(groupAsLong(matcher, 5))
+               .addSeconds(groupAsLong(matcher, 6))
+               .build();
+    }
+
+    private static Duration parseIso8601WeekFormat(boolean isNegative, String source)
+    {
+        Matcher matcher = ISO8601_WEEK_PATTERN.matcher(source);
+        if (!matcher.matches())
+            throw new IllegalArgumentException(
+            String.format("Unable to convert '%s' to a duration", source));
+
+        return new Builder(isNegative).addWeeks(groupAsLong(matcher, 1)).build();
+    }
+
+    private static Duration parseStandardFormat(boolean isNegative, String source)
+    {
+        Matcher matcher = STANDARD_PATTERN.matcher(source);
+        if (!matcher.find())
+            throw new IllegalArgumentException(
+            String.format("Unable to convert '%s' to a duration", source));
+
+        Builder builder = new Builder(isNegative);
+        boolean done;
+
+        do
+        {
+            long number = groupAsLong(matcher, 1);
+            String symbol = matcher.group(2);
+            add(builder, number, symbol);
+            done = matcher.end() == source.length();
+        } while (matcher.find());
+
+        if (!done)
+            throw new IllegalArgumentException(
+            String.format("Unable to convert '%s' to a duration", source));
+
+        return builder.build();
+    }
+
+    private static long groupAsLong(Matcher matcher, int group)
+    {
+        return Long.parseLong(matcher.group(group));
+    }
+
+    private static Builder add(Builder builder, long number, String symbol)
+    {
+        String s = symbol.toLowerCase();
+        if (s.equals("y"))
+        {
+            return builder.addYears(number);
+        }
+        else if (s.equals("mo"))
+        {
+            return builder.addMonths(number);
+        }
+        else if (s.equals("w"))
+        {
+            return builder.addWeeks(number);
+        }
+        else if (s.equals("d"))
+        {
+            return builder.addDays(number);
+        }
+        else if (s.equals("h"))
+        {
+            return builder.addHours(number);
+        }
+        else if (s.equals("m"))
+        {
+            return builder.addMinutes(number);
+        }
+        else if (s.equals("s"))
+        {
+            return builder.addSeconds(number);
+        }
+        else if (s.equals("ms"))
+        {
+            return builder.addMillis(number);
+        }
+        else if (s.equals("us") || s.equals("µs"))
+        {
+            return builder.addMicros(number);
+        }
+        else if (s.equals("ns"))
+        {
+            return builder.addNanos(number);
+        }
+        throw new IllegalArgumentException(String.format("Unknown duration symbol '%s'", symbol));
+    }
+
+    /**
+     * Appends the result of the division to the specified builder if the dividend is not zero.
+     *
+     * @param builder  the builder to append to
+     * @param dividend the dividend
+     * @param divisor  the divisor
+     * @param unit     the time unit to append after the result of the division
+     * @return the remainder of the division
+     */
+    private static long append(StringBuilder builder, long dividend, long divisor, String unit)
+    {
+        if (dividend == 0 || dividend < divisor) return dividend;
+
+        builder.append(dividend / divisor).append(unit);
+        return dividend % divisor;
+    }
+
+    /**
+     * Returns the number of months in this duration.
+     *
+     * @return the number of months in this duration.
+     */
+    public int getMonths()
+    {
+        return months;
+    }
+
+    /**
+     * Returns the number of days in this duration.
+     *
+     * @return the number of days in this duration.
+     */
+    public int getDays()
+    {
+        return days;
+    }
+
+    /**
+     * Returns the number of nanoseconds in this duration.
+     *
+     * @return the number of months in this duration.
+     */
+    public long getNanoseconds()
+    {
+        return nanoseconds;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(days, months, nanoseconds);
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        if (!(obj instanceof Duration)) return false;
+
+        Duration other = (Duration) obj;
+        return days == other.days && months == other.months && nanoseconds == other.nanoseconds;
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder builder = new StringBuilder();
+
+        if (months < 0 || days < 0 || nanoseconds < 0) builder.append('-');
+
+        long remainder = append(builder, Math.abs(months), MONTHS_PER_YEAR, "y");
+        append(builder, remainder, 1, "mo");
+
+        append(builder, Math.abs(days), 1, "d");
+
+        if (nanoseconds != 0)
+        {
+            remainder = append(builder, Math.abs(nanoseconds), NANOS_PER_HOUR, "h");
+            remainder = append(builder, remainder, NANOS_PER_MINUTE, "m");
+            remainder = append(builder, remainder, NANOS_PER_SECOND, "s");
+            remainder = append(builder, remainder, NANOS_PER_MILLI, "ms");
+            remainder = append(builder, remainder, NANOS_PER_MICRO, "us");
+            append(builder, remainder, 1, "ns");
+        }
+        return builder.toString();
+    }
+
+    private static class Builder
+    {
+        /**
+         * {@code true} if the duration is a negative one, {@code false} otherwise.
+         */
+        private final boolean isNegative;
+
+        /**
+         * The number of months.
+         */
+        private int months;
+
+        /**
+         * The number of days.
+         */
+        private int days;
+
+        /**
+         * The number of nanoseconds.
+         */
+        private long nanoseconds;
+
+        /**
+         * We need to make sure that the values for each units are provided in order.
+         */
+        private int currentUnitIndex;
+
+        public Builder(boolean isNegative)
+        {
+            this.isNegative = isNegative;
+        }
+
+        /**
+         * Adds the specified amount of years.
+         *
+         * @param numberOfYears the number of years to add.
+         * @return this {@code Builder}
+         */
+        public Builder addYears(long numberOfYears)
+        {
+            validateOrder(1);
+            validateMonths(numberOfYears, MONTHS_PER_YEAR);
+            months += numberOfYears * MONTHS_PER_YEAR;
+            return this;
+        }
+
+        /**
+         * Adds the specified amount of months.
+         *
+         * @param numberOfMonths the number of months to add.
+         * @return this {@code Builder}
+         */
+        public Builder addMonths(long numberOfMonths)
+        {
+            validateOrder(2);
+            validateMonths(numberOfMonths, 1);
+            months += numberOfMonths;
+            return this;
+        }
+
+        /**
+         * Adds the specified amount of weeks.
+         *
+         * @param numberOfWeeks the number of weeks to add.
+         * @return this {@code Builder}
+         */
+        public Builder addWeeks(long numberOfWeeks)
+        {
+            validateOrder(3);
+            validateDays(numberOfWeeks, DAYS_PER_WEEK);
+            days += numberOfWeeks * DAYS_PER_WEEK;
+            return this;
+        }
+
+        /**
+         * Adds the specified amount of days.
+         *
+         * @param numberOfDays the number of days to add.
+         * @return this {@code Builder}
+         */
+        public Builder addDays(long numberOfDays)
+        {
+            validateOrder(4);
+            validateDays(numberOfDays, 1);
+            days += numberOfDays;
+            return this;
+        }
+
+        /**
+         * Adds the specified amount of hours.
+         *
+         * @param numberOfHours the number of hours to add.
+         * @return this {@code Builder}
+         */
+        public Builder addHours(long numberOfHours)
+        {
+            validateOrder(5);
+            validateNanos(numberOfHours, NANOS_PER_HOUR);
+            nanoseconds += numberOfHours * NANOS_PER_HOUR;
+            return this;
+        }
+
+        /**
+         * Adds the specified amount of minutes.
+         *
+         * @param numberOfMinutes the number of minutes to add.
+         * @return this {@code Builder}
+         */
+        public Builder addMinutes(long numberOfMinutes)
+        {
+            validateOrder(6);
+            validateNanos(numberOfMinutes, NANOS_PER_MINUTE);
+            nanoseconds += numberOfMinutes * NANOS_PER_MINUTE;
+            return this;
+        }
+
+        /**
+         * Adds the specified amount of seconds.
+         *
+         * @param numberOfSeconds the number of seconds to add.
+         * @return this {@code Builder}
+         */
+        public Builder addSeconds(long numberOfSeconds)
+        {
+            validateOrder(7);
+            validateNanos(numberOfSeconds, NANOS_PER_SECOND);
+            nanoseconds += numberOfSeconds * NANOS_PER_SECOND;
+            return this;
+        }
+
+        /**
+         * Adds the specified amount of milliseconds.
+         *
+         * @param numberOfMillis the number of milliseconds to add.
+         * @return this {@code Builder}
+         */
+        public Builder addMillis(long numberOfMillis)
+        {
+            validateOrder(8);
+            validateNanos(numberOfMillis, NANOS_PER_MILLI);
+            nanoseconds += numberOfMillis * NANOS_PER_MILLI;
+            return this;
+        }
+
+        /**
+         * Adds the specified amount of microseconds.
+         *
+         * @param numberOfMicros the number of microseconds to add.
+         * @return this {@code Builder}
+         */
+        public Builder addMicros(long numberOfMicros)
+        {
+            validateOrder(9);
+            validateNanos(numberOfMicros, NANOS_PER_MICRO);
+            nanoseconds += numberOfMicros * NANOS_PER_MICRO;
+            return this;
+        }
+
+        /**
+         * Adds the specified amount of nanoseconds.
+         *
+         * @param numberOfNanos the number of nanoseconds to add.
+         * @return this {@code Builder}
+         */
+        public Builder addNanos(long numberOfNanos)
+        {
+            validateOrder(10);
+            validateNanos(numberOfNanos, 1);
+            nanoseconds += numberOfNanos;
+            return this;
+        }
+
+        /**
+         * Validates that the total number of months can be stored.
+         *
+         * @param units         the number of units that need to be added
+         * @param monthsPerUnit the number of days per unit
+         */
+        private void validateMonths(long units, int monthsPerUnit)
+        {
+            validate(units, (Integer.MAX_VALUE - months) / monthsPerUnit, "months");
+        }
+
+        /**
+         * Validates that the total number of days can be stored.
+         *
+         * @param units       the number of units that need to be added
+         * @param daysPerUnit the number of days per unit
+         */
+        private void validateDays(long units, int daysPerUnit)
+        {
+            validate(units, (Integer.MAX_VALUE - days) / daysPerUnit, "days");
+        }
+
+        /**
+         * Validates that the total number of nanoseconds can be stored.
+         *
+         * @param units        the number of units that need to be added
+         * @param nanosPerUnit the number of nanoseconds per unit
+         */
+        private void validateNanos(long units, long nanosPerUnit)
+        {
+            validate(units, (Long.MAX_VALUE - nanoseconds) / nanosPerUnit, "nanoseconds");
+        }
+
+        /**
+         * Validates that the specified amount is less than the limit.
+         *
+         * @param units    the number of units to check
+         * @param limit    the limit on the number of units
+         * @param unitName the unit name
+         */
+        private void validate(long units, long limit, String unitName)
+        {
+            checkArgument(
+            units <= limit,
+            "Invalid duration. The total number of %s must be less or equal to %s",
+            unitName,
+            Integer.MAX_VALUE);
+        }
+
+        /**
+         * Validates that the duration values are added in the proper order.
+         *
+         * @param unitIndex the unit index (e.g. years=1, months=2, ...)
+         */
+        private void validateOrder(int unitIndex)
+        {
+            if (unitIndex == currentUnitIndex)
+                throw new IllegalArgumentException(
+                String.format(
+                "Invalid duration. The %s are specified multiple times", getUnitName(unitIndex)));
+
+            if (unitIndex <= currentUnitIndex)
+                throw new IllegalArgumentException(
+                String.format(
+                "Invalid duration. The %s should be after %s",
+                getUnitName(currentUnitIndex), getUnitName(unitIndex)));
+
+            currentUnitIndex = unitIndex;
+        }
+
+        /**
+         * Returns the name of the unit corresponding to the specified index.
+         *
+         * @param unitIndex the unit index
+         * @return the name of the unit corresponding to the specified index.
+         */
+        private String getUnitName(int unitIndex)
+        {
+            switch (unitIndex)
+            {
+                case 1:
+                    return "years";
+                case 2:
+                    return "months";
+                case 3:
+                    return "weeks";
+                case 4:
+                    return "days";
+                case 5:
+                    return "hours";
+                case 6:
+                    return "minutes";
+                case 7:
+                    return "seconds";
+                case 8:
+                    return "milliseconds";
+                case 9:
+                    return "microseconds";
+                case 10:
+                    return "nanoseconds";
+                default:
+                    throw new AssertionError("unknown unit index: " + unitIndex);
+            }
+        }
+
+        public Duration build()
+        {
+            return isNegative
+                   ? new Duration(-months, -days, -nanoseconds)
+                   : new Duration(months, days, nanoseconds);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/GettableByIndexData.java b/src/java/org/apache/cassandra/cql3/functions/types/GettableByIndexData.java
new file mode 100644
index 0000000..bcc5585
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/GettableByIndexData.java
@@ -0,0 +1,592 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.reflect.TypeToken;
+
+import org.apache.cassandra.cql3.functions.types.exceptions.CodecNotFoundException;
+import org.apache.cassandra.cql3.functions.types.exceptions.InvalidTypeException;
+
+/**
+ * Collection of (typed) CQL values that can be retrieved by index (starting at zero).
+ */
+public interface GettableByIndexData
+{
+
+    /**
+     * Returns whether the {@code i}th value is NULL.
+     *
+     * @param i the index ({@code 0 <= i < size()}) of the value to check.
+     * @return whether the {@code i}th value is NULL.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     */
+    public boolean isNull(int i);
+
+    /**
+     * Returns the {@code i}th value as a boolean.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code boolean} (for CQL type {@code boolean}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the boolean value of the {@code i}th element. If the value is NULL, {@code false} is
+     * returned. If you need to distinguish NULL and false values, check first with {@link
+     * #isNull(int)} or use {@code get(i, Boolean.class)}.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a boolean.
+     */
+    public boolean getBool(int i);
+
+    /**
+     * Returns the {@code i}th value as a byte.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code byte} (for CQL type {@code tinyint}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a byte. If the value is NULL, {@code 0} is
+     * returned. If you need to distinguish NULL and 0, check first with {@link #isNull(int)} or
+     * use {@code get(i, Byte.class)}.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a byte.
+     */
+    public byte getByte(int i);
+
+    /**
+     * Returns the {@code i}th value as a short.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code short} (for CQL type {@code smallint}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a short. If the value is NULL, {@code 0} is
+     * returned. If you need to distinguish NULL and 0, check first with {@link #isNull(int)} or
+     * use {@code get(i, Short.class)}.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a short.
+     */
+    public short getShort(int i);
+
+    /**
+     * Returns the {@code i}th value as an integer.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code int} (for CQL type {@code int}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as an integer. If the value is NULL, {@code 0} is
+     * returned. If you need to distinguish NULL and 0, check first with {@link #isNull(int)} or
+     * use {@code get(i, Integer.class)}.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to an int.
+     */
+    public int getInt(int i);
+
+    /**
+     * Returns the {@code i}th value as a long.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code byte} (for CQL types {@code bigint} and {@code counter}, this will be the
+     * built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a long. If the value is NULL, {@code 0L} is
+     * returned. If you need to distinguish NULL and 0L, check first with {@link #isNull(int)} or
+     * use {@code get(i, Long.class)}.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a long.
+     */
+    public long getLong(int i);
+
+    /**
+     * Returns the {@code i}th value as a date.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code Date} (for CQL type {@code timestamp}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a data. If the value is NULL, {@code null} is
+     * returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a {@code Date}.
+     */
+    public Date getTimestamp(int i);
+
+    /**
+     * Returns the {@code i}th value as a date (without time).
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@link LocalDate} (for CQL type {@code date}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as an date. If the value is NULL, {@code null} is
+     * returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a {@code LocalDate}.
+     */
+    public LocalDate getDate(int i);
+
+    /**
+     * Returns the {@code i}th value as a long in nanoseconds since midnight.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code long} (for CQL type {@code time}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a long. If the value is NULL, {@code 0L} is
+     * returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a long.
+     */
+    public long getTime(int i);
+
+    /**
+     * Returns the {@code i}th value as a float.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code float} (for CQL type {@code float}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a float. If the value is NULL, {@code 0.0f} is
+     * returned. If you need to distinguish NULL and 0.0f, check first with {@link #isNull(int)}
+     * or use {@code get(i, Float.class)}.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a float.
+     */
+    public float getFloat(int i);
+
+    /**
+     * Returns the {@code i}th value as a double.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code double} (for CQL type {@code double}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a double. If the value is NULL, {@code 0.0} is
+     * returned. If you need to distinguish NULL and 0.0, check first with {@link #isNull(int)} or
+     * use {@code get(i, Double.class)}.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a double.
+     */
+    public double getDouble(int i);
+
+    /**
+     * Returns the {@code i}th value as a {@code ByteBuffer}.
+     *
+     * <p>This method does not use any codec; it returns a copy of the binary representation of the
+     * value. It is up to the caller to convert the returned value appropriately.
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a ByteBuffer. If the value is NULL, {@code
+     * null} is returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     */
+    public ByteBuffer getBytesUnsafe(int i);
+
+    /**
+     * Returns the {@code i}th value as a byte array.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code ByteBuffer} (for CQL type {@code blob}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a byte array. If the value is NULL, {@code
+     * null} is returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a {@code ByteBuffer}.
+     */
+    public ByteBuffer getBytes(int i);
+
+    /**
+     * Returns the {@code i}th value as a string.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java string (for CQL types {@code text}, {@code varchar} and {@code ascii}, this will
+     * be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a string. If the value is NULL, {@code null} is
+     * returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a string.
+     */
+    public String getString(int i);
+
+    /**
+     * Returns the {@code i}th value as a variable length integer.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code BigInteger} (for CQL type {@code varint}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a variable length integer. If the value is
+     * NULL, {@code null} is returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a {@code BigInteger}.
+     */
+    public BigInteger getVarint(int i);
+
+    /**
+     * Returns the {@code i}th value as a variable length decimal.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code BigDecimal} (for CQL type {@code decimal}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a variable length decimal. If the value is
+     * NULL, {@code null} is returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a {@code BigDecimal}.
+     */
+    public BigDecimal getDecimal(int i);
+
+    /**
+     * Returns the {@code i}th value as a UUID.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code UUID} (for CQL types {@code uuid} and {@code timeuuid}, this will be the
+     * built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a UUID. If the value is NULL, {@code null} is
+     * returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a {@code UUID}.
+     */
+    public UUID getUUID(int i);
+
+    /**
+     * Returns the {@code i}th value as an InetAddress.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to an {@code InetAddress} (for CQL type {@code inet}, this will be the built-in codec).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as an InetAddress. If the value is NULL, {@code
+     * null} is returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a {@code InetAddress}.
+     */
+    public InetAddress getInet(int i);
+
+    /**
+     * Returns the {@code i}th value as a list.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a list of the specified type.
+     *
+     * <p>If the type of the elements is generic, use {@link #getList(int, TypeToken)}.
+     *
+     * <p>Implementation note: the actual {@link List} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will be mapped to an empty collection (note that Cassandra
+     * makes no distinction between {@code NULL} and an empty collection).
+     *
+     * @param i             the index ({@code 0 <= i < size()}) to retrieve.
+     * @param elementsClass the class for the elements of the list to retrieve.
+     * @return the value of the {@code i}th element as a list of {@code T} objects.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a list.
+     */
+    public <T> List<T> getList(int i, Class<T> elementsClass);
+
+    /**
+     * Returns the {@code i}th value as a list.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a list of the specified type.
+     *
+     * <p>Use this variant with nested collections, which produce a generic element type:
+     *
+     * <pre>
+     * {@code List<List<String>> l = row.getList(1, new TypeToken<List<String>>() {});}
+     * </pre>
+     *
+     * <p>Implementation note: the actual {@link List} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param i            the index ({@code 0 <= i < size()}) to retrieve.
+     * @param elementsType the type of the elements of the list to retrieve.
+     * @return the value of the {@code i}th element as a list of {@code T} objects.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a list.
+     */
+    public <T> List<T> getList(int i, TypeToken<T> elementsType);
+
+    /**
+     * Returns the {@code i}th value as a set.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a set of the specified type.
+     *
+     * <p>If the type of the elements is generic, use {@link #getSet(int, TypeToken)}.
+     *
+     * <p>Implementation note: the actual {@link Set} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param i             the index ({@code 0 <= i < size()}) to retrieve.
+     * @param elementsClass the class for the elements of the set to retrieve.
+     * @return the value of the {@code i}th element as a set of {@code T} objects.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a set.
+     */
+    public <T> Set<T> getSet(int i, Class<T> elementsClass);
+
+    /**
+     * Returns the {@code i}th value as a set.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a set of the specified type.
+     *
+     * <p>Use this variant with nested collections, which produce a generic element type:
+     *
+     * <pre>
+     * {@code Set<List<String>> l = row.getSet(1, new TypeToken<List<String>>() {});}
+     * </pre>
+     *
+     * <p>Implementation note: the actual {@link Set} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param i            the index ({@code 0 <= i < size()}) to retrieve.
+     * @param elementsType the type for the elements of the set to retrieve.
+     * @return the value of the {@code i}th element as a set of {@code T} objects.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a set.
+     */
+    public <T> Set<T> getSet(int i, TypeToken<T> elementsType);
+
+    /**
+     * Returns the {@code i}th value as a map.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a map of the specified types.
+     *
+     * <p>If the type of the keys and/or values is generic, use {@link #getMap(int, TypeToken,
+     * TypeToken)}.
+     *
+     * <p>Implementation note: the actual {@link Map} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param i           the index ({@code 0 <= i < size()}) to retrieve.
+     * @param keysClass   the class for the keys of the map to retrieve.
+     * @param valuesClass the class for the values of the map to retrieve.
+     * @return the value of the {@code i}th element as a map of {@code K} to {@code V} objects.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a map.
+     */
+    public <K, V> Map<K, V> getMap(int i, Class<K> keysClass, Class<V> valuesClass);
+
+    /**
+     * Returns the {@code i}th value as a map.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a map of the specified types.
+     *
+     * <p>Use this variant with nested collections, which produce a generic element type:
+     *
+     * <pre>
+     * {@code Map<Int, List<String>> l = row.getMap(1, TypeToken.of(Integer.class), new TypeToken<List<String>>() {});}
+     * </pre>
+     *
+     * <p>Implementation note: the actual {@link Map} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param i          the index ({@code 0 <= i < size()}) to retrieve.
+     * @param keysType   the type for the keys of the map to retrieve.
+     * @param valuesType the type for the values of the map to retrieve.
+     * @return the value of the {@code i}th element as a map of {@code K} to {@code V} objects.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a map.
+     */
+    public <K, V> Map<K, V> getMap(int i, TypeToken<K> keysType, TypeToken<V> valuesType);
+
+    /**
+     * Return the {@code i}th value as a UDT value.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code UDTValue} (if the CQL type is a UDT, the registry will generate a codec
+     * automatically).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a UDT value. If the value is NULL, then {@code
+     * null} will be returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a {@code UDTValue}.
+     */
+    public UDTValue getUDTValue(int i);
+
+    /**
+     * Return the {@code i}th value as a tuple value.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code TupleValue} (if the CQL type is a tuple, the registry will generate a codec
+     * automatically).
+     *
+     * @param i the index ({@code 0 <= i < size()}) to retrieve.
+     * @return the value of the {@code i}th element as a tuple value. If the value is NULL, then
+     * {@code null} will be returned.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to a {@code TupleValue}.
+     */
+    public TupleValue getTupleValue(int i);
+
+    /**
+     * Returns the {@code i}th value as the Java type matching its CQL type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find the first codec that handles the
+     * underlying CQL type. The Java type of the returned object will be determined by the codec that
+     * was selected.
+     *
+     * <p>Use this method to dynamically inspect elements when types aren't known in advance, for
+     * instance if you're writing a generic row logger. If you know the target Java type, it is
+     * generally preferable to use typed getters, such as the ones for built-in types ({@link
+     * #getBool(int)}, {@link #getInt(int)}, etc.), or {@link #get(int, Class)} and {@link #get(int,
+     * TypeToken)} for custom types.
+     *
+     * @param i the index to retrieve.
+     * @return the value of the {@code i}th value as the Java type matching its CQL type.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @see CodecRegistry#codecFor(DataType)
+     */
+    public Object getObject(int i);
+
+    /**
+     * Returns the {@code i}th value converted to the given Java type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to the given Java type.
+     *
+     * <p>If the target type is generic, use {@link #get(int, TypeToken)}.
+     *
+     * <p>Implementation note: the actual object returned by this method will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent; by default, a CQL {@code NULL} value translates to
+     * {@code null} for simple CQL types, UDTs and tuples, and to empty collections for all CQL
+     * collection types.
+     *
+     * @param i           the index to retrieve.
+     * @param targetClass The Java type the value should be converted to.
+     * @return the value of the {@code i}th value converted to the given Java type.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to {@code targetClass}.
+     */
+    <T> T get(int i, Class<T> targetClass);
+
+    /**
+     * Returns the {@code i}th value converted to the given Java type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to the given Java type.
+     *
+     * <p>Implementation note: the actual object returned by this method will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent; by default, a CQL {@code NULL} value translates to
+     * {@code null} for simple CQL types, UDTs and tuples, and to empty collections for all CQL
+     * collection types.
+     *
+     * @param i          the index to retrieve.
+     * @param targetType The Java type the value should be converted to.
+     * @return the value of the {@code i}th value converted to the given Java type.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the element's CQL
+     *                                   type to {@code targetType}.
+     */
+    <T> T get(int i, TypeToken<T> targetType);
+
+    /**
+     * Returns the {@code i}th value converted using the given {@link TypeCodec}.
+     *
+     * <p>This method entirely bypasses the {@link CodecRegistry} and forces the driver to use the
+     * given codec instead. This can be useful if the codec would collide with a previously registered
+     * one, or if you want to use the codec just once without registering it.
+     *
+     * <p>It is the caller's responsibility to ensure that the given codec {@link
+     * TypeCodec#accepts(DataType) accepts} the underlying CQL type; failing to do so may result in
+     * {@link InvalidTypeException}s being thrown.
+     *
+     * <p>Implementation note: the actual object returned by this method will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent; by default, a CQL {@code NULL} value translates to
+     * {@code null} for simple CQL types, UDTs and tuples, and to empty collections for all CQL
+     * collection types.
+     *
+     * @param i     the index to retrieve.
+     * @param codec The {@link TypeCodec} to use to deserialize the value; may not be {@code null}.
+     * @return the value of the {@code i}th value converted using the given {@link TypeCodec}.
+     * @throws InvalidTypeException      if the given codec does not {@link TypeCodec#accepts(DataType)
+     *                                   accept} the underlying CQL type.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     */
+    <T> T get(int i, TypeCodec<T> codec);
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/GettableByNameData.java b/src/java/org/apache/cassandra/cql3/functions/types/GettableByNameData.java
new file mode 100644
index 0000000..c578ff4
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/GettableByNameData.java
@@ -0,0 +1,593 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.reflect.TypeToken;
+
+import org.apache.cassandra.cql3.functions.types.exceptions.CodecNotFoundException;
+import org.apache.cassandra.cql3.functions.types.exceptions.InvalidTypeException;
+
+/**
+ * Collection of (typed) CQL values that can be retrieved by name.
+ */
+public interface GettableByNameData
+{
+
+    /**
+     * Returns whether the value for {@code name} is NULL.
+     *
+     * @param name the name to check.
+     * @return whether the value for {@code name} is NULL.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     */
+    public boolean isNull(String name);
+
+    /**
+     * Returns the value for {@code name} as a boolean.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code boolean} (for CQL type {@code boolean}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the boolean value for {@code name}. If the value is NULL, {@code false} is returned. If
+     * you need to distinguish NULL and false values, check first with {@link #isNull(String)} or
+     * use {@code get(name, Boolean.class)}.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a boolean.
+     */
+    public boolean getBool(String name);
+
+    /**
+     * Returns the value for {@code name} as a byte.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code byte} (for CQL type {@code tinyint}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a byte. If the value is NULL, {@code 0} is returned. If
+     * you need to distinguish NULL and 0, check first with {@link #isNull(String)} or use {@code
+     * get(name, Byte.class)}. {@code 0} is returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a byte.
+     */
+    public byte getByte(String name);
+
+    /**
+     * Returns the value for {@code name} as a short.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code short} (for CQL type {@code smallint}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a short. If the value is NULL, {@code 0} is returned. If
+     * you need to distinguish NULL and 0, check first with {@link #isNull(String)} or use {@code
+     * get(name, Short.class)}. {@code 0} is returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a short.
+     */
+    public short getShort(String name);
+
+    /**
+     * Returns the value for {@code name} as an integer.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code int} (for CQL type {@code int}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as an integer. If the value is NULL, {@code 0} is returned.
+     * If you need to distinguish NULL and 0, check first with {@link #isNull(String)} or use
+     * {@code get(name, Integer.class)}. {@code 0} is returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to an int.
+     */
+    public int getInt(String name);
+
+    /**
+     * Returns the value for {@code name} as a long.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code byte} (for CQL types {@code bigint} and {@code counter}, this will be the
+     * built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a long. If the value is NULL, {@code 0L} is returned. If
+     * you need to distinguish NULL and 0L, check first with {@link #isNull(String)} or use {@code
+     * get(name, Long.class)}.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a long.
+     */
+    public long getLong(String name);
+
+    /**
+     * Returns the value for {@code name} as a date.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code Date} (for CQL type {@code timestamp}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a date. If the value is NULL, {@code null} is returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a {@code Date}.
+     */
+    public Date getTimestamp(String name);
+
+    /**
+     * Returns the value for {@code name} as a date (without time).
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@link LocalDate} (for CQL type {@code date}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a date. If the value is NULL, {@code null} is returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a {@code LocalDate}.
+     */
+    public LocalDate getDate(String name);
+
+    /**
+     * Returns the value for {@code name} as a long in nanoseconds since midnight.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code long} (for CQL type {@code time}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a long. If the value is NULL, {@code 0L} is returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a long.
+     */
+    public long getTime(String name);
+
+    /**
+     * Returns the value for {@code name} as a float.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code float} (for CQL type {@code float}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a float. If the value is NULL, {@code 0.0f} is returned.
+     * If you need to distinguish NULL and 0.0f, check first with {@link #isNull(String)} or use
+     * {@code get(name, Float.class)}.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a float.
+     */
+    public float getFloat(String name);
+
+    /**
+     * Returns the value for {@code name} as a double.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code double} (for CQL type {@code double}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a double. If the value is NULL, {@code 0.0} is returned.
+     * If you need to distinguish NULL and 0.0, check first with {@link #isNull(String)} or use
+     * {@code get(name, Double.class)}.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a double.
+     */
+    public double getDouble(String name);
+
+    /**
+     * Returns the value for {@code name} as a ByteBuffer.
+     *
+     * <p>This method does not use any codec; it returns a copy of the binary representation of the
+     * value. It is up to the caller to convert the returned value appropriately.
+     *
+     * <p>Note: this method always return the bytes composing the value, even if the column is not of
+     * type BLOB. That is, this method never throw an InvalidTypeException. However, if the type is
+     * not BLOB, it is up to the caller to handle the returned value correctly.
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a ByteBuffer. If the value is NULL, {@code null} is
+     * returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     */
+    public ByteBuffer getBytesUnsafe(String name);
+
+    /**
+     * Returns the value for {@code name} as a byte array.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java {@code ByteBuffer} (for CQL type {@code blob}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a byte array. If the value is NULL, {@code null} is
+     * returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a {@code ByteBuffer}.
+     */
+    public ByteBuffer getBytes(String name);
+
+    /**
+     * Returns the value for {@code name} as a string.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a Java string (for CQL types {@code text}, {@code varchar} and {@code ascii}, this will
+     * be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a string. If the value is NULL, {@code null} is returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a string.
+     */
+    public String getString(String name);
+
+    /**
+     * Returns the value for {@code name} as a variable length integer.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code BigInteger} (for CQL type {@code varint}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a variable length integer. If the value is NULL, {@code
+     * null} is returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a {@code BigInteger}.
+     */
+    public BigInteger getVarint(String name);
+
+    /**
+     * Returns the value for {@code name} as a variable length decimal.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code BigDecimal} (for CQL type {@code decimal}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a variable length decimal. If the value is NULL, {@code
+     * null} is returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a {@code BigDecimal}.
+     */
+    public BigDecimal getDecimal(String name);
+
+    /**
+     * Returns the value for {@code name} as a UUID.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code UUID} (for CQL types {@code uuid} and {@code timeuuid}, this will be the
+     * built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as a UUID. If the value is NULL, {@code null} is returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a {@code UUID}.
+     */
+    public UUID getUUID(String name);
+
+    /**
+     * Returns the value for {@code name} as an InetAddress.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to an {@code InetAddress} (for CQL type {@code inet}, this will be the built-in codec).
+     *
+     * @param name the name to retrieve.
+     * @return the value for {@code name} as an InetAddress. If the value is NULL, {@code null} is
+     * returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a {@code InetAddress}.
+     */
+    public InetAddress getInet(String name);
+
+    /**
+     * Returns the value for {@code name} as a list.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a list of the specified type.
+     *
+     * <p>If the type of the elements is generic, use {@link #getList(String, TypeToken)}.
+     *
+     * <p>Implementation note: the actual {@link List} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param name          the name to retrieve.
+     * @param elementsClass the class for the elements of the list to retrieve.
+     * @return the value of the {@code i}th element as a list of {@code T} objects.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a list.
+     */
+    public <T> List<T> getList(String name, Class<T> elementsClass);
+
+    /**
+     * Returns the value for {@code name} as a list.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a list of the specified type.
+     *
+     * <p>Use this variant with nested collections, which produce a generic element type:
+     *
+     * <pre>
+     * {@code List<List<String>> l = row.getList("theColumn", new TypeToken<List<String>>() {});}
+     * </pre>
+     *
+     * <p>Implementation note: the actual {@link List} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param name         the name to retrieve.
+     * @param elementsType the type for the elements of the list to retrieve.
+     * @return the value of the {@code i}th element as a list of {@code T} objects.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a list.
+     */
+    public <T> List<T> getList(String name, TypeToken<T> elementsType);
+
+    /**
+     * Returns the value for {@code name} as a set.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a set of the specified type.
+     *
+     * <p>If the type of the elements is generic, use {@link #getSet(String, TypeToken)}.
+     *
+     * <p>Implementation note: the actual {@link Set} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param name          the name to retrieve.
+     * @param elementsClass the class for the elements of the set to retrieve.
+     * @return the value of the {@code i}th element as a set of {@code T} objects.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a set.
+     */
+    public <T> Set<T> getSet(String name, Class<T> elementsClass);
+
+    /**
+     * Returns the value for {@code name} as a set.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a set of the specified type.
+     *
+     * <p>Use this variant with nested collections, which produce a generic element type:
+     *
+     * <pre>
+     * {@code Set<List<String>> l = row.getSet("theColumn", new TypeToken<List<String>>() {});}
+     * </pre>
+     *
+     * <p>Implementation note: the actual {@link Set} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param name         the name to retrieve.
+     * @param elementsType the type for the elements of the set to retrieve.
+     * @return the value of the {@code i}th element as a set of {@code T} objects.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a set.
+     */
+    public <T> Set<T> getSet(String name, TypeToken<T> elementsType);
+
+    /**
+     * Returns the value for {@code name} as a map.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a map of the specified types.
+     *
+     * <p>If the type of the keys and/or values is generic, use {@link #getMap(String, TypeToken,
+     * TypeToken)}.
+     *
+     * <p>Implementation note: the actual {@link Map} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param name        the name to retrieve.
+     * @param keysClass   the class for the keys of the map to retrieve.
+     * @param valuesClass the class for the values of the map to retrieve.
+     * @return the value of {@code name} as a map of {@code K} to {@code V} objects.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a map.
+     */
+    public <K, V> Map<K, V> getMap(String name, Class<K> keysClass, Class<V> valuesClass);
+
+    /**
+     * Returns the value for {@code name} as a map.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a map of the specified types.
+     *
+     * <p>Use this variant with nested collections, which produce a generic element type:
+     *
+     * <pre>
+     * {@code Map<Int, List<String>> l = row.getMap("theColumn", TypeToken.of(Integer.class), new TypeToken<List<String>>() {});}
+     * </pre>
+     *
+     * <p>Implementation note: the actual {@link Map} implementation will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent. By default, the driver will return mutable
+     * instances, and a CQL {@code NULL} will mapped to an empty collection (note that Cassandra makes
+     * no distinction between {@code NULL} and an empty collection).
+     *
+     * @param name       the name to retrieve.
+     * @param keysType   the class for the keys of the map to retrieve.
+     * @param valuesType the class for the values of the map to retrieve.
+     * @return the value of {@code name} as a map of {@code K} to {@code V} objects.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a map.
+     */
+    public <K, V> Map<K, V> getMap(String name, TypeToken<K> keysType, TypeToken<V> valuesType);
+
+    /**
+     * Return the value for {@code name} as a UDT value.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code UDTValue} (if the CQL type is a UDT, the registry will generate a codec
+     * automatically).
+     *
+     * @param name the name to retrieve.
+     * @return the value of {@code name} as a UDT value. If the value is NULL, then {@code null} will
+     * be returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a {@code UDTValue}.
+     */
+    public UDTValue getUDTValue(String name);
+
+    /**
+     * Return the value for {@code name} as a tuple value.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to a {@code TupleValue} (if the CQL type is a tuple, the registry will generate a codec
+     * automatically).
+     *
+     * @param name the name to retrieve.
+     * @return the value of {@code name} as a tuple value. If the value is NULL, then {@code null}
+     * will be returned.
+     * @throws IllegalArgumentException if {@code name} is not valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to a {@code TupleValue}.
+     */
+    public TupleValue getTupleValue(String name);
+
+    /**
+     * Returns the value for {@code name} as the Java type matching its CQL type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find the first codec that handles the
+     * underlying CQL type. The Java type of the returned object will be determined by the codec that
+     * was selected.
+     *
+     * <p>Use this method to dynamically inspect elements when types aren't known in advance, for
+     * instance if you're writing a generic row logger. If you know the target Java type, it is
+     * generally preferable to use typed getters, such as the ones for built-in types ({@link
+     * #getBool(String)}, {@link #getInt(String)}, etc.), or {@link #get(String, Class)} and {@link
+     * #get(String, TypeToken)} for custom types.
+     *
+     * @param name the name to retrieve.
+     * @return the value of {@code name} as the Java type matching its CQL type. If the value is NULL
+     * and is a simple type, UDT or tuple, {@code null} is returned. If it is NULL and is a
+     * collection type, an empty (immutable) collection is returned.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @see CodecRegistry#codecFor(DataType)
+     */
+    Object getObject(String name);
+
+    /**
+     * Returns the value for {@code name} converted to the given Java type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to the given Java type.
+     *
+     * <p>If the target type is generic, use {@link #get(String, TypeToken)}.
+     *
+     * <p>Implementation note: the actual object returned by this method will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent; by default, a CQL {@code NULL} value translates to
+     * {@code null} for simple CQL types, UDTs and tuples, and to empty collections for all CQL
+     * collection types.
+     *
+     * @param name        the name to retrieve.
+     * @param targetClass The Java type the value should be converted to.
+     * @return the value for {@code name} value converted to the given Java type.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to {@code targetClass}.
+     */
+    <T> T get(String name, Class<T> targetClass);
+
+    /**
+     * Returns the value for {@code name} converted to the given Java type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to convert the underlying CQL
+     * type to the given Java type.
+     *
+     * <p>Implementation note: the actual object returned by this method will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent; by default, a CQL {@code NULL} value translates to
+     * {@code null} for simple CQL types, UDTs and tuples, and to empty collections for all CQL
+     * collection types.
+     *
+     * @param name       the name to retrieve.
+     * @param targetType The Java type the value should be converted to.
+     * @return the value for {@code name} value converted to the given Java type.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the underlying CQL
+     *                                  type to {@code targetType}.
+     */
+    <T> T get(String name, TypeToken<T> targetType);
+
+    /**
+     * Returns the value for {@code name} converted using the given {@link TypeCodec}.
+     *
+     * <p>This method entirely bypasses the {@link CodecRegistry} and forces the driver to use the
+     * given codec instead. This can be useful if the codec would collide with a previously registered
+     * one, or if you want to use the codec just once without registering it.
+     *
+     * <p>It is the caller's responsibility to ensure that the given codec {@link
+     * TypeCodec#accepts(DataType) accepts} the underlying CQL type; failing to do so may result in
+     * {@link InvalidTypeException}s being thrown.
+     *
+     * <p>Implementation note: the actual object returned by this method will depend on the {@link
+     * TypeCodec codec} being used; therefore, callers should make no assumptions concerning its
+     * mutability nor its thread-safety. Furthermore, the behavior of this method in respect to CQL
+     * {@code NULL} values is also codec-dependent; by default, a CQL {@code NULL} value translates to
+     * {@code null} for simple CQL types, UDTs and tuples, and to empty collections for all CQL
+     * collection types.
+     *
+     * @param name  the name to retrieve.
+     * @param codec The {@link TypeCodec} to use to deserialize the value; may not be {@code null}.
+     * @return the value of the {@code i}th value converted using the given {@link TypeCodec}.
+     * @throws InvalidTypeException      if the given codec does not {@link TypeCodec#accepts(DataType)
+     *                                   accept} the underlying CQL type.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     */
+    <T> T get(String name, TypeCodec<T> codec);
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/GettableData.java b/src/java/org/apache/cassandra/cql3/functions/types/GettableData.java
new file mode 100644
index 0000000..e8f2b72
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/GettableData.java
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+/**
+ * Collection of (typed) CQL values that can be retrieved either by index (starting at zero) or by
+ * name.
+ */
+public interface GettableData extends GettableByIndexData, GettableByNameData
+{
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/LocalDate.java b/src/java/org/apache/cassandra/cql3/functions/types/LocalDate.java
new file mode 100644
index 0000000..dead6ec
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/LocalDate.java
@@ -0,0 +1,212 @@
+/*
+ * 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.*;
+import java.util.concurrent.TimeUnit;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * A date with no time components, no time zone, in the ISO 8601 calendar.
+ *
+ * <p>Note that ISO 8601 has a number of differences with the default gregorian calendar used in
+ * Java:
+ *
+ * <ul>
+ * <li>it uses a proleptic gregorian calendar, meaning that it's gregorian indefinitely back in
+ * the past (there is no gregorian change);
+ * <li>there is a year 0.
+ * </ul>
+ *
+ * <p>This class implements these differences, so that year/month/day fields match exactly the ones
+ * in CQL string literals.
+ *
+ * @since 2.2
+ */
+public final class LocalDate
+{
+
+    private static final TimeZone UTC = TimeZone.getTimeZone("UTC");
+
+    private final long millisSinceEpoch;
+    private final int daysSinceEpoch;
+
+    // This gets initialized lazily if we ever need it. Once set, it is effectively immutable.
+    private volatile GregorianCalendar calendar;
+
+    private LocalDate(int daysSinceEpoch)
+    {
+        this.daysSinceEpoch = daysSinceEpoch;
+        this.millisSinceEpoch = TimeUnit.DAYS.toMillis(daysSinceEpoch);
+    }
+
+    /**
+     * Builds a new instance from a number of days since January 1st, 1970 GMT.
+     *
+     * @param daysSinceEpoch the number of days.
+     * @return the new instance.
+     */
+    static LocalDate fromDaysSinceEpoch(int daysSinceEpoch)
+    {
+        return new LocalDate(daysSinceEpoch);
+    }
+
+    /**
+     * Builds a new instance from a number of milliseconds since January 1st, 1970 GMT. Note that if
+     * the given number does not correspond to a whole number of days, it will be rounded towards 0.
+     *
+     * @param millisSinceEpoch the number of milliseconds since January 1st, 1970 GMT.
+     * @return the new instance.
+     * @throws IllegalArgumentException if the date is not in the range [-5877641-06-23;
+     *                                  5881580-07-11].
+     */
+    static LocalDate fromMillisSinceEpoch(long millisSinceEpoch)
+    throws IllegalArgumentException
+    {
+        long daysSinceEpoch = TimeUnit.MILLISECONDS.toDays(millisSinceEpoch);
+        checkArgument(
+        daysSinceEpoch >= Integer.MIN_VALUE && daysSinceEpoch <= Integer.MAX_VALUE,
+        "Date should be in the range [-5877641-06-23; 5881580-07-11]");
+
+        return new LocalDate((int) daysSinceEpoch);
+    }
+
+    /**
+     * Returns the number of days since January 1st, 1970 GMT.
+     *
+     * @return the number of days.
+     */
+    int getDaysSinceEpoch()
+    {
+        return daysSinceEpoch;
+    }
+
+    /**
+     * Returns the year.
+     *
+     * @return the year.
+     */
+    public int getYear()
+    {
+        GregorianCalendar c = getCalendar();
+        int year = c.get(Calendar.YEAR);
+        if (c.get(Calendar.ERA) == GregorianCalendar.BC) year = -year + 1;
+        return year;
+    }
+
+    /**
+     * Returns the month.
+     *
+     * @return the month. It is 1-based, e.g. 1 for January.
+     */
+    public int getMonth()
+    {
+        return getCalendar().get(Calendar.MONTH) + 1;
+    }
+
+    /**
+     * Returns the day in the month.
+     *
+     * @return the day in the month.
+     */
+    public int getDay()
+    {
+        return getCalendar().get(Calendar.DAY_OF_MONTH);
+    }
+
+    /**
+     * Return a new {@link LocalDate} with the specified (signed) amount of time added to (or
+     * subtracted from) the given {@link Calendar} field, based on the calendar's rules.
+     *
+     * <p>Note that adding any amount to a field smaller than {@link Calendar#DAY_OF_MONTH} will
+     * remain without effect, as this class does not keep time components.
+     *
+     * <p>See {@link Calendar} javadocs for more information.
+     *
+     * @param field  a {@link Calendar} field to modify.
+     * @param amount the amount of date or time to be added to the field.
+     * @return a new {@link LocalDate} with the specified (signed) amount of time added to (or
+     * subtracted from) the given {@link Calendar} field.
+     * @throws IllegalArgumentException if the new date is not in the range [-5877641-06-23;
+     *                                  5881580-07-11].
+     */
+    public LocalDate add(int field, int amount)
+    {
+        GregorianCalendar newCalendar = isoCalendar();
+        newCalendar.setTimeInMillis(millisSinceEpoch);
+        newCalendar.add(field, amount);
+        LocalDate newDate = fromMillisSinceEpoch(newCalendar.getTimeInMillis());
+        newDate.calendar = newCalendar;
+        return newDate;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+
+        if (o instanceof LocalDate)
+        {
+            LocalDate that = (LocalDate) o;
+            return this.daysSinceEpoch == that.daysSinceEpoch;
+        }
+        return false;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return daysSinceEpoch;
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("%d-%s-%s", getYear(), pad2(getMonth()), pad2(getDay()));
+    }
+
+    private static String pad2(int i)
+    {
+        String s = Integer.toString(i);
+        return s.length() == 2 ? s : '0' + s;
+    }
+
+    private GregorianCalendar getCalendar()
+    {
+        // Two threads can race and both create a calendar. This is not a problem.
+        if (calendar == null)
+        {
+
+            // Use a local variable to only expose after we're done mutating it.
+            GregorianCalendar tmp = isoCalendar();
+            tmp.setTimeInMillis(millisSinceEpoch);
+
+            calendar = tmp;
+        }
+        return calendar;
+    }
+
+    // This matches what Cassandra uses server side (from Joda Time's LocalDate)
+    private static GregorianCalendar isoCalendar()
+    {
+        GregorianCalendar calendar = new GregorianCalendar(UTC);
+        calendar.setGregorianChange(new Date(Long.MIN_VALUE));
+        return calendar;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/Metadata.java b/src/java/org/apache/cassandra/cql3/functions/types/Metadata.java
new file mode 100644
index 0000000..24ab05b
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/Metadata.java
@@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.cassandra.cql3.ColumnIdentifier;
+
+/**
+ * Keeps metadata on the connected cluster, including known nodes and schema definitions.
+ */
+public class Metadata
+{
+    /*
+     * Deal with case sensitivity for a given element id (keyspace, table, column, etc.)
+     *
+     * This method is used to convert identifiers provided by the client (through methods such as getKeyspace(String)),
+     * to the format used internally by the driver.
+     *
+     * We expect client-facing APIs to behave like cqlsh, that is:
+     * - identifiers that are mixed-case or contain special characters should be quoted.
+     * - unquoted identifiers will be lowercased: getKeyspace("Foo") will look for a keyspace named "foo"
+     */
+    static String handleId(String id)
+    {
+        // Shouldn't really happen for this method, but no reason to fail here
+        if (id == null) return null;
+
+        boolean isAlphanumericLowCase = true;
+        boolean isAlphanumeric = true;
+        for (int i = 0; i < id.length(); i++)
+        {
+            char c = id.charAt(i);
+            if (c >= 65 && c <= 90)
+            { // A-Z
+                isAlphanumericLowCase = false;
+            }
+            else if (!((c >= 48 && c <= 57) // 0-9
+                       || (c == 95) // _ (underscore)
+                       || (c >= 97 && c <= 122) // a-z
+            ))
+            {
+                isAlphanumeric = false;
+                isAlphanumericLowCase = false;
+                break;
+            }
+        }
+
+        if (isAlphanumericLowCase)
+        {
+            return id;
+        }
+        if (isAlphanumeric)
+        {
+            return id.toLowerCase();
+        }
+
+        // Check if it's enclosed in quotes. If it is, remove them and unescape internal double quotes
+        return ParseUtils.unDoubleQuote(id);
+    }
+
+    /**
+     * Quotes a CQL identifier if necessary.
+     *
+     * <p>This is similar to {@link #quote(String)}, except that it won't quote the input string if it
+     * can safely be used as-is. For example:
+     *
+     * <ul>
+     * <li>{@code quoteIfNecessary("foo").equals("foo")} (no need to quote).
+     * <li>{@code quoteIfNecessary("Foo").equals("\"Foo\"")} (identifier is mixed case so case
+     * sensitivity is required)
+     * <li>{@code quoteIfNecessary("foo bar").equals("\"foo bar\"")} (identifier contains special
+     * characters)
+     * <li>{@code quoteIfNecessary("table").equals("\"table\"")} (identifier is a reserved CQL
+     * keyword)
+     * </ul>
+     *
+     * @param id the "internal" form of the identifier. That is, the identifier as it would appear in
+     *           Cassandra system tables (such as {@code system_schema.tables}, {@code
+     *           system_schema.columns}, etc.)
+     * @return the identifier as it would appear in a CQL query string. This is also how you need to
+     * pass it to public driver methods, such as {@code #getKeyspace(String)}.
+     */
+    static String quoteIfNecessary(String id)
+    {
+        return ColumnIdentifier.maybeQuote(id);
+    }
+
+    /**
+     * Quote a keyspace, table or column identifier to make it case sensitive.
+     *
+     * <p>CQL identifiers, including keyspace, table and column ones, are case insensitive by default.
+     * Case sensitive identifiers can however be provided by enclosing the identifier in double quotes
+     * (see the <a href="http://cassandra.apache.org/doc/cql3/CQL.html#identifiers">CQL
+     * documentation</a> for details). If you are using case sensitive identifiers, this method can be
+     * used to enclose such identifiers in double quotes, making them case sensitive.
+     *
+     * <p>Note that <a
+     * href="https://docs.datastax.com/en/cql/3.0/cql/cql_reference/keywords_r.html">reserved CQL
+     * keywords</a> should also be quoted. You can check if a given identifier is a reserved keyword
+     * by calling {@code #isReservedCqlKeyword(String)}.
+     *
+     * @param id the keyspace or table identifier.
+     * @return {@code id} enclosed in double-quotes, for use in methods like {@code #getReplicas},
+     * {@code #getKeyspace}, {@code KeyspaceMetadata#getTable} or even {@code
+     * Cluster#connect(String)}.
+     */
+    public static String quote(String id)
+    {
+        return ParseUtils.doubleQuote(id);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/ParseUtils.java b/src/java/org/apache/cassandra/cql3/functions/types/ParseUtils.java
new file mode 100644
index 0000000..8972bee
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/ParseUtils.java
@@ -0,0 +1,625 @@
+/*
+ * 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.text.*;
+import java.util.Date;
+import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Simple utility class used to help parsing CQL values (mainly UDT and collection ones).
+ */
+public abstract class ParseUtils
+{
+
+    /**
+     * Valid ISO-8601 patterns for CQL timestamp literals.
+     */
+    private static final String[] iso8601Patterns =
+    new String[]{
+    "yyyy-MM-dd HH:mm",
+    "yyyy-MM-dd HH:mm:ss",
+    "yyyy-MM-dd HH:mmZ",
+    "yyyy-MM-dd HH:mm:ssZ",
+    "yyyy-MM-dd HH:mm:ss.SSS",
+    "yyyy-MM-dd HH:mm:ss.SSSZ",
+    "yyyy-MM-dd'T'HH:mm",
+    "yyyy-MM-dd'T'HH:mmZ",
+    "yyyy-MM-dd'T'HH:mm:ss",
+    "yyyy-MM-dd'T'HH:mm:ssZ",
+    "yyyy-MM-dd'T'HH:mm:ss.SSS",
+    "yyyy-MM-dd'T'HH:mm:ss.SSSZ",
+    "yyyy-MM-dd",
+    "yyyy-MM-ddZ"
+    };
+
+    /**
+     * Returns the index of the first character in toParse from idx that is not a "space".
+     *
+     * @param toParse the string to skip space on.
+     * @param idx     the index to start skipping space from.
+     * @return the index of the first character in toParse from idx that is not a "space.
+     */
+    static int skipSpaces(String toParse, int idx)
+    {
+        while (isBlank(toParse.charAt(idx))) ++idx;
+        return idx;
+    }
+
+    /**
+     * Assuming that idx points to the beginning of a CQL value in toParse, returns the index of the
+     * first character after this value.
+     *
+     * @param toParse the string to skip a value form.
+     * @param idx     the index to start parsing a value from.
+     * @return the index ending the CQL value starting at {@code idx}.
+     * @throws IllegalArgumentException if idx doesn't point to the start of a valid CQL value.
+     */
+    static int skipCQLValue(String toParse, int idx)
+    {
+        if (idx >= toParse.length()) throw new IllegalArgumentException();
+
+        if (isBlank(toParse.charAt(idx))) throw new IllegalArgumentException();
+
+        int cbrackets = 0;
+        int sbrackets = 0;
+        int parens = 0;
+        boolean inString = false;
+
+        do
+        {
+            char c = toParse.charAt(idx);
+            if (inString)
+            {
+                if (c == '\'')
+                {
+                    if (idx + 1 < toParse.length() && toParse.charAt(idx + 1) == '\'')
+                    {
+                        ++idx; // this is an escaped quote, skip it
+                    }
+                    else
+                    {
+                        inString = false;
+                        if (cbrackets == 0 && sbrackets == 0 && parens == 0) return idx + 1;
+                    }
+                }
+                // Skip any other character
+            }
+            else if (c == '\'')
+            {
+                inString = true;
+            }
+            else if (c == '{')
+            {
+                ++cbrackets;
+            }
+            else if (c == '[')
+            {
+                ++sbrackets;
+            }
+            else if (c == '(')
+            {
+                ++parens;
+            }
+            else if (c == '}')
+            {
+                if (cbrackets == 0) return idx;
+
+                --cbrackets;
+                if (cbrackets == 0 && sbrackets == 0 && parens == 0) return idx + 1;
+            }
+            else if (c == ']')
+            {
+                if (sbrackets == 0) return idx;
+
+                --sbrackets;
+                if (cbrackets == 0 && sbrackets == 0 && parens == 0) return idx + 1;
+            }
+            else if (c == ')')
+            {
+                if (parens == 0) return idx;
+
+                --parens;
+                if (cbrackets == 0 && sbrackets == 0 && parens == 0) return idx + 1;
+            }
+            else if (isBlank(c) || !isIdentifierChar(c))
+            {
+                if (cbrackets == 0 && sbrackets == 0 && parens == 0) return idx;
+            }
+        } while (++idx < toParse.length());
+
+        if (inString || cbrackets != 0 || sbrackets != 0 || parens != 0)
+            throw new IllegalArgumentException();
+        return idx;
+    }
+
+    /**
+     * Assuming that idx points to the beginning of a CQL identifier in toParse, returns the index of
+     * the first character after this identifier.
+     *
+     * @param toParse the string to skip an identifier from.
+     * @param idx     the index to start parsing an identifier from.
+     * @return the index ending the CQL identifier starting at {@code idx}.
+     * @throws IllegalArgumentException if idx doesn't point to the start of a valid CQL identifier.
+     */
+    static int skipCQLId(String toParse, int idx)
+    {
+        if (idx >= toParse.length()) throw new IllegalArgumentException();
+
+        char c = toParse.charAt(idx);
+        if (isIdentifierChar(c))
+        {
+            while (idx < toParse.length() && isIdentifierChar(toParse.charAt(idx))) idx++;
+            return idx;
+        }
+
+        if (c != '"') throw new IllegalArgumentException();
+
+        while (++idx < toParse.length())
+        {
+            c = toParse.charAt(idx);
+            if (c != '"') continue;
+
+            if (idx + 1 < toParse.length() && toParse.charAt(idx + 1) == '\"')
+                ++idx; // this is an escaped double quote, skip it
+            else return idx + 1;
+        }
+        throw new IllegalArgumentException();
+    }
+
+    /**
+     * Return {@code true} if the given character is allowed in a CQL identifier, that is, if it is in
+     * the range: {@code [0..9a..zA..Z-+._&]}.
+     *
+     * @param c The character to inspect.
+     * @return {@code true} if the given character is allowed in a CQL identifier, {@code false}
+     * otherwise.
+     */
+    static boolean isIdentifierChar(int c)
+    {
+        return (c >= '0' && c <= '9')
+               || (c >= 'a' && c <= 'z')
+               || (c >= 'A' && c <= 'Z')
+               || c == '-'
+               || c == '+'
+               || c == '.'
+               || c == '_'
+               || c == '&';
+    }
+
+    /**
+     * Return {@code true} if the given character is a valid whitespace character in CQL, that is, if
+     * it is a regular space, a tabulation sign, or a new line sign.
+     *
+     * @param c The character to inspect.
+     * @return {@code true} if the given character is a valid whitespace character, {@code false}
+     * otherwise.
+     */
+    static boolean isBlank(int c)
+    {
+        return c == ' ' || c == '\t' || c == '\n';
+    }
+
+    /**
+     * Check whether the given string corresponds to a valid CQL long literal. Long literals are
+     * composed solely by digits, but can have an optional leading minus sign.
+     *
+     * @param str The string to inspect.
+     * @return {@code true} if the given string corresponds to a valid CQL integer literal, {@code
+     * false} otherwise.
+     */
+    static boolean isLongLiteral(String str)
+    {
+        if (str == null || str.isEmpty()) return false;
+        char[] chars = str.toCharArray();
+        for (int i = 0; i < chars.length; i++)
+        {
+            char c = chars[i];
+            if ((c < '0' && (i != 0 || c != '-')) || c > '9') return false;
+        }
+        return true;
+    }
+
+    /**
+     * Return {@code true} if the given string is surrounded by single quotes, and {@code false}
+     * otherwise.
+     *
+     * @param value The string to inspect.
+     * @return {@code true} if the given string is surrounded by single quotes, and {@code false}
+     * otherwise.
+     */
+    static boolean isQuoted(String value)
+    {
+        return isQuoted(value, '\'');
+    }
+
+    /**
+     * Quote the given string; single quotes are escaped. If the given string is null, this method
+     * returns a quoted empty string ({@code ''}).
+     *
+     * @param value The value to quote.
+     * @return The quoted string.
+     */
+    public static String quote(String value)
+    {
+        return quote(value, '\'');
+    }
+
+    /**
+     * Unquote the given string if it is quoted; single quotes are unescaped. If the given string is
+     * not quoted, it is returned without any modification.
+     *
+     * @param value The string to unquote.
+     * @return The unquoted string.
+     */
+    static String unquote(String value)
+    {
+        return unquote(value, '\'');
+    }
+
+    /**
+     * Double quote the given string; double quotes are escaped. If the given string is null, this
+     * method returns a quoted empty string ({@code ""}).
+     *
+     * @param value The value to double quote.
+     * @return The double quoted string.
+     */
+    static String doubleQuote(String value)
+    {
+        return quote(value, '"');
+    }
+
+    /**
+     * Unquote the given string if it is double quoted; double quotes are unescaped. If the given
+     * string is not double quoted, it is returned without any modification.
+     *
+     * @param value The string to un-double quote.
+     * @return The un-double quoted string.
+     */
+    static String unDoubleQuote(String value)
+    {
+        return unquote(value, '"');
+    }
+
+    /**
+     * Parse the given string as a date, using one of the accepted ISO-8601 date patterns.
+     *
+     * <p>This method is adapted from Apache Commons {@code DateUtils.parseStrictly()} method (that is
+     * used Cassandra side to parse date strings)..
+     *
+     * @throws ParseException If the given string is not a valid ISO-8601 date.
+     * @see <a href="https://cassandra.apache.org/doc/cql3/CQL-2.2.html#usingtimestamps">'Working with
+     * timestamps' section of CQL specification</a>
+     */
+    static Date parseDate(String str) throws ParseException
+    {
+        SimpleDateFormat parser = new SimpleDateFormat();
+        parser.setLenient(false);
+        // set a default timezone for patterns that do not provide one
+        parser.setTimeZone(TimeZone.getTimeZone("UTC"));
+        // Java 6 has very limited support for ISO-8601 time zone formats,
+        // so we need to transform the string first
+        // so that accepted patterns are correctly handled,
+        // such as Z for UTC, or "+00:00" instead of "+0000".
+        // Note: we cannot use the X letter in the pattern
+        // because it has been introduced in Java 7.
+        str = str.replaceAll("(\\+|\\-)(\\d\\d):(\\d\\d)$", "$1$2$3");
+        str = str.replaceAll("Z$", "+0000");
+        ParsePosition pos = new ParsePosition(0);
+        for (String parsePattern : iso8601Patterns)
+        {
+            parser.applyPattern(parsePattern);
+            pos.setIndex(0);
+            Date date = parser.parse(str, pos);
+            if (date != null && pos.getIndex() == str.length())
+            {
+                return date;
+            }
+        }
+        throw new ParseException("Unable to parse the date: " + str, -1);
+    }
+
+    /**
+     * Parse the given string as a date, using the supplied date pattern.
+     *
+     * <p>This method is adapted from Apache Commons {@code DateUtils.parseStrictly()} method (that is
+     * used Cassandra side to parse date strings)..
+     *
+     * @throws ParseException If the given string cannot be parsed with the given pattern.
+     * @see <a href="https://cassandra.apache.org/doc/cql3/CQL-2.2.html#usingtimestamps">'Working with
+     * timestamps' section of CQL specification</a>
+     */
+    static Date parseDate(String str, String pattern) throws ParseException
+    {
+        SimpleDateFormat parser = new SimpleDateFormat();
+        parser.setLenient(false);
+        // set a default timezone for patterns that do not provide one
+        parser.setTimeZone(TimeZone.getTimeZone("UTC"));
+        // Java 6 has very limited support for ISO-8601 time zone formats,
+        // so we need to transform the string first
+        // so that accepted patterns are correctly handled,
+        // such as Z for UTC, or "+00:00" instead of "+0000".
+        // Note: we cannot use the X letter in the pattern
+        // because it has been introduced in Java 7.
+        str = str.replaceAll("(\\+|\\-)(\\d\\d):(\\d\\d)$", "$1$2$3");
+        str = str.replaceAll("Z$", "+0000");
+        ParsePosition pos = new ParsePosition(0);
+        parser.applyPattern(pattern);
+        pos.setIndex(0);
+        Date date = parser.parse(str, pos);
+        if (date != null && pos.getIndex() == str.length())
+        {
+            return date;
+        }
+        throw new ParseException("Unable to parse the date: " + str, -1);
+    }
+
+    /**
+     * Parse the given string as a time, using the following time pattern: {@code
+     * hh:mm:ss[.fffffffff]}.
+     *
+     * <p>This method is loosely based on {@code java.sql.Timestamp}.
+     *
+     * @param str The string to parse.
+     * @return A long value representing the number of nanoseconds since midnight.
+     * @throws ParseException if the string cannot be parsed.
+     * @see <a href="https://cassandra.apache.org/doc/cql3/CQL-2.2.html#usingtime">'Working with time'
+     * section of CQL specification</a>
+     */
+    static long parseTime(String str) throws ParseException
+    {
+        String nanos_s;
+
+        long hour;
+        long minute;
+        long second;
+        long a_nanos = 0;
+
+        String formatError = "Timestamp format must be hh:mm:ss[.fffffffff]";
+        String zeros = "000000000";
+
+        if (str == null) throw new IllegalArgumentException(formatError);
+        str = str.trim();
+
+        // Parse the time
+        int firstColon = str.indexOf(':');
+        int secondColon = str.indexOf(':', firstColon + 1);
+
+        // Convert the time; default missing nanos
+        if (firstColon > 0 && secondColon > 0 && secondColon < str.length() - 1)
+        {
+            int period = str.indexOf('.', secondColon + 1);
+            hour = Integer.parseInt(str.substring(0, firstColon));
+            if (hour < 0 || hour >= 24) throw new IllegalArgumentException("Hour out of bounds.");
+
+            minute = Integer.parseInt(str.substring(firstColon + 1, secondColon));
+            if (minute < 0 || minute >= 60) throw new IllegalArgumentException("Minute out of bounds.");
+
+            if (period > 0 && period < str.length() - 1)
+            {
+                second = Integer.parseInt(str.substring(secondColon + 1, period));
+                if (second < 0 || second >= 60) throw new IllegalArgumentException("Second out of bounds.");
+
+                nanos_s = str.substring(period + 1);
+                if (nanos_s.length() > 9) throw new IllegalArgumentException(formatError);
+                if (!Character.isDigit(nanos_s.charAt(0))) throw new IllegalArgumentException(formatError);
+                nanos_s = nanos_s + zeros.substring(0, 9 - nanos_s.length());
+                a_nanos = Integer.parseInt(nanos_s);
+            }
+            else if (period > 0) throw new ParseException(formatError, -1);
+            else
+            {
+                second = Integer.parseInt(str.substring(secondColon + 1));
+                if (second < 0 || second >= 60) throw new ParseException("Second out of bounds.", -1);
+            }
+        }
+        else throw new ParseException(formatError, -1);
+
+        long rawTime = 0;
+        rawTime += TimeUnit.HOURS.toNanos(hour);
+        rawTime += TimeUnit.MINUTES.toNanos(minute);
+        rawTime += TimeUnit.SECONDS.toNanos(second);
+        rawTime += a_nanos;
+        return rawTime;
+    }
+
+    /**
+     * Format the given long value as a CQL time literal, using the following time pattern: {@code
+     * hh:mm:ss[.fffffffff]}.
+     *
+     * @param value A long value representing the number of nanoseconds since midnight.
+     * @return The formatted value.
+     * @see <a href="https://cassandra.apache.org/doc/cql3/CQL-2.2.html#usingtime">'Working with time'
+     * section of CQL specification</a>
+     */
+    static String formatTime(long value)
+    {
+        int nano = (int) (value % 1000000000);
+        value -= nano;
+        value /= 1000000000;
+        int seconds = (int) (value % 60);
+        value -= seconds;
+        value /= 60;
+        int minutes = (int) (value % 60);
+        value -= minutes;
+        value /= 60;
+        int hours = (int) (value % 24);
+        value -= hours;
+        value /= 24;
+        assert (value == 0);
+        StringBuilder sb = new StringBuilder();
+        leftPadZeros(hours, 2, sb);
+        sb.append(':');
+        leftPadZeros(minutes, 2, sb);
+        sb.append(':');
+        leftPadZeros(seconds, 2, sb);
+        sb.append('.');
+        leftPadZeros(nano, 9, sb);
+        return sb.toString();
+    }
+
+    /**
+     * Return {@code true} if the given string is surrounded by the quote character given, and {@code
+     * false} otherwise.
+     *
+     * @param value The string to inspect.
+     * @return {@code true} if the given string is surrounded by the quote character, and {@code
+     * false} otherwise.
+     */
+    private static boolean isQuoted(String value, char quoteChar)
+    {
+        return value != null
+               && value.length() > 1
+               && value.charAt(0) == quoteChar
+               && value.charAt(value.length() - 1) == quoteChar;
+    }
+
+    /**
+     * @param quoteChar " or '
+     * @return A quoted empty string.
+     */
+    private static String emptyQuoted(char quoteChar)
+    {
+        // don't handle non quote characters, this is done so that these are interned and don't create
+        // repeated empty quoted strings.
+        assert quoteChar == '"' || quoteChar == '\'';
+        if (quoteChar == '"') return "\"\"";
+        else return "''";
+    }
+
+    /**
+     * Quotes text and escapes any existing quotes in the text. {@code String.replace()} is a bit too
+     * inefficient (see JAVA-67, JAVA-1262).
+     *
+     * @param text      The text.
+     * @param quoteChar The character to use as a quote.
+     * @return The text with surrounded in quotes with all existing quotes escaped with (i.e. '
+     * becomes '')
+     */
+    private static String quote(String text, char quoteChar)
+    {
+        if (text == null || text.isEmpty()) return emptyQuoted(quoteChar);
+
+        int nbMatch = 0;
+        int start = -1;
+        do
+        {
+            start = text.indexOf(quoteChar, start + 1);
+            if (start != -1) ++nbMatch;
+        } while (start != -1);
+
+        // no quotes found that need to be escaped, simply surround in quotes and return.
+        if (nbMatch == 0) return quoteChar + text + quoteChar;
+
+        // 2 for beginning and end quotes.
+        // length for original text
+        // nbMatch for escape characters to add to quotes to be escaped.
+        int newLength = 2 + text.length() + nbMatch;
+        char[] result = new char[newLength];
+        result[0] = quoteChar;
+        result[newLength - 1] = quoteChar;
+        int newIdx = 1;
+        for (int i = 0; i < text.length(); i++)
+        {
+            char c = text.charAt(i);
+            if (c == quoteChar)
+            {
+                // escape quote with another occurrence.
+                result[newIdx++] = c;
+                result[newIdx++] = c;
+            }
+            else
+            {
+                result[newIdx++] = c;
+            }
+        }
+        return new String(result);
+    }
+
+    /**
+     * Unquotes text and unescapes non surrounding quotes. {@code String.replace()} is a bit too
+     * inefficient (see JAVA-67, JAVA-1262).
+     *
+     * @param text      The text
+     * @param quoteChar The character to use as a quote.
+     * @return The text with surrounding quotes removed and non surrounding quotes unescaped (i.e. ''
+     * becomes ')
+     */
+    private static String unquote(String text, char quoteChar)
+    {
+        if (!isQuoted(text, quoteChar)) return text;
+
+        if (text.length() == 2) return "";
+
+        String search = emptyQuoted(quoteChar);
+        int nbMatch = 0;
+        int start = -1;
+        do
+        {
+            start = text.indexOf(search, start + 2);
+            // ignore the second to last character occurrence, as the last character is a quote.
+            if (start != -1 && start != text.length() - 2) ++nbMatch;
+        } while (start != -1);
+
+        // no escaped quotes found, simply remove surrounding quotes and return.
+        if (nbMatch == 0) return text.substring(1, text.length() - 1);
+
+        // length of the new string will be its current length - the number of occurrences.
+        int newLength = text.length() - nbMatch - 2;
+        char[] result = new char[newLength];
+        int newIdx = 0;
+        // track whenever a quoteChar is encountered and the previous character is not a quoteChar.
+        boolean firstFound = false;
+        for (int i = 1; i < text.length() - 1; i++)
+        {
+            char c = text.charAt(i);
+            if (c == quoteChar)
+            {
+                if (firstFound)
+                {
+                    // The previous character was a quoteChar, don't add this to result, this action in
+                    // effect removes consecutive quotes.
+                    firstFound = false;
+                }
+                else
+                {
+                    // found a quoteChar and the previous character was not a quoteChar, include in result.
+                    firstFound = true;
+                    result[newIdx++] = c;
+                }
+            }
+            else
+            {
+                // non quoteChar encountered, include in result.
+                result[newIdx++] = c;
+                firstFound = false;
+            }
+        }
+        return new String(result);
+    }
+
+    private static void leftPadZeros(int value, int digits, StringBuilder sb)
+    {
+        sb.append(String.format("%0" + digits + 'd', value));
+    }
+
+    private ParseUtils()
+    {
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/SettableByIndexData.java b/src/java/org/apache/cassandra/cql3/functions/types/SettableByIndexData.java
new file mode 100644
index 0000000..a9d0898
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/SettableByIndexData.java
@@ -0,0 +1,583 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.reflect.TypeToken;
+
+import org.apache.cassandra.cql3.functions.types.exceptions.CodecNotFoundException;
+import org.apache.cassandra.cql3.functions.types.exceptions.InvalidTypeException;
+
+/**
+ * Collection of (typed) CQL values that can be set by index (starting at zero).
+ */
+public interface SettableByIndexData<T extends SettableByIndexData<T>>
+{
+
+    /**
+     * Sets the {@code i}th value to the provided boolean.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code boolean}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set. To set the value to NULL, use {@link #setToNull(int)} or {@code
+     *          set(i, v, Boolean.class)}
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setBool(int i, boolean v);
+
+    /**
+     * Set the {@code i}th value to the provided byte.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code tinyint}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set. To set the value to NULL, use {@link #setToNull(int)} or {@code
+     *          set(i, v, Byte.class)}
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setByte(int i, byte v);
+
+    /**
+     * Set the {@code i}th value to the provided short.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code smallint}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set. To set the value to NULL, use {@link #setToNull(int)} or {@code
+     *          set(i, v, Short.class)}
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setShort(int i, short v);
+
+    /**
+     * Set the {@code i}th value to the provided integer.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code int}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set. To set the value to NULL, use {@link #setToNull(int)} or {@code
+     *          set(i, v, Integer.class)}
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setInt(int i, int v);
+
+    /**
+     * Sets the {@code i}th value to the provided long.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code bigint}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set. To set the value to NULL, use {@link #setToNull(int)} or {@code
+     *          set(i, v, Long.class)}
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setLong(int i, long v);
+
+    /**
+     * Set the {@code i}th value to the provided date.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code timestamp}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setTimestamp(int i, Date v);
+
+    /**
+     * Set the {@code i}th value to the provided date (without time).
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code date}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setDate(int i, LocalDate v);
+
+    /**
+     * Set the {@code i}th value to the provided time as a long in nanoseconds since midnight.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code time}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setTime(int i, long v);
+
+    /**
+     * Sets the {@code i}th value to the provided float.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code float}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set. To set the value to NULL, use {@link #setToNull(int)} or {@code
+     *          set(i, v, Float.class)}
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setFloat(int i, float v);
+
+    /**
+     * Sets the {@code i}th value to the provided double.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code double}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set. To set the value to NULL, use {@link #setToNull(int)} or {@code
+     *          set(i, v, Double.class)}.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setDouble(int i, double v);
+
+    /**
+     * Sets the {@code i}th value to the provided string.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL types {@code text}, {@code varchar} and {@code ascii}, this will
+     * be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setString(int i, String v);
+
+    /**
+     * Sets the {@code i}th value to the provided byte buffer.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code blob}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setBytes(int i, ByteBuffer v);
+
+    /**
+     * Sets the {@code i}th value to the provided byte buffer.
+     *
+     * <p>This method does not use any codec; it sets the value in its binary form directly. If you
+     * insert data that is not compatible with the underlying CQL type, you will get an {@code
+     * InvalidQueryException} at execute time.
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     */
+    public T setBytesUnsafe(int i, ByteBuffer v);
+
+    /**
+     * Sets the {@code i}th value to the provided big integer.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code varint}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setVarint(int i, BigInteger v);
+
+    /**
+     * Sets the {@code i}th value to the provided big decimal.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code decimal}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setDecimal(int i, BigDecimal v);
+
+    /**
+     * Sets the {@code i}th value to the provided UUID.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL types {@code uuid} and {@code timeuuid}, this will be the built-in
+     * codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setUUID(int i, UUID v);
+
+    /**
+     * Sets the {@code i}th value to the provided inet address.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code inet}, this will be the built-in codec).
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setInet(int i, InetAddress v);
+
+    /**
+     * Sets the {@code i}th value to the provided list.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (the type of the elements in the Java list is not considered). If two or
+     * more codecs target that CQL type, the one that was first registered will be used. For this
+     * reason, it is generally preferable to use the more deterministic methods {@link #setList(int,
+     * List, Class)} or {@link #setList(int, List, TypeToken)}.
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set. Note that {@code null} values inside collections are not supported
+     *          by CQL.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws NullPointerException      if {@code v} contains null values. Nulls are not supported in
+     *                                   collections by CQL.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public <E> T setList(int i, List<E> v);
+
+    /**
+     * Sets the {@code i}th value to the provided list, which elements are of the provided Java class.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of lists
+     * of the given Java type to the underlying CQL type.
+     *
+     * <p>If the type of the elements is generic, use {@link #setList(int, List, TypeToken)}.
+     *
+     * @param i             the index of the value to set.
+     * @param v             the value to set. Note that {@code null} values inside collections are not supported
+     *                      by CQL.
+     * @param elementsClass the class for the elements of the list.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws NullPointerException      if {@code v} contains null values. Nulls are not supported in
+     *                                   collections by CQL.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public <E> T setList(int i, List<E> v, Class<E> elementsClass);
+
+    /**
+     * Sets the {@code i}th value to the provided list, which elements are of the provided Java type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of lists
+     * of the given Java type to the underlying CQL type.
+     *
+     * @param i            the index of the value to set.
+     * @param v            the value to set. Note that {@code null} values inside collections are not supported
+     *                     by CQL.
+     * @param elementsType the type for the elements of the list.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws NullPointerException      if {@code v} contains null values. Nulls are not supported in
+     *                                   collections by CQL.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public <E> T setList(int i, List<E> v, TypeToken<E> elementsType);
+
+    /**
+     * Sets the {@code i}th value to the provided map.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (the type of the elements in the Java map is not considered). If two or
+     * more codecs target that CQL type, the one that was first registered will be used. For this
+     * reason, it is generally preferable to use the more deterministic methods {@link #setMap(int,
+     * Map, Class, Class)} or {@link #setMap(int, Map, TypeToken, TypeToken)}.
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set. Note that {@code null} values inside collections are not supported
+     *          by CQL.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws NullPointerException      if {@code v} contains null values. Nulls are not supported in
+     *                                   collections by CQL.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public <K, V> T setMap(int i, Map<K, V> v);
+
+    /**
+     * Sets the {@code i}th value to the provided map, which keys and values are of the provided Java
+     * classes.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of lists
+     * of the given Java types to the underlying CQL type.
+     *
+     * <p>If the type of the keys or values is generic, use {@link #setMap(int, Map, TypeToken,
+     * TypeToken)}.
+     *
+     * @param i           the index of the value to set.
+     * @param v           the value to set. Note that {@code null} values inside collections are not supported
+     *                    by CQL.
+     * @param keysClass   the class for the keys of the map.
+     * @param valuesClass the class for the values of the map.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws NullPointerException      if {@code v} contains null values. Nulls are not supported in
+     *                                   collections by CQL.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public <K, V> T setMap(int i, Map<K, V> v, Class<K> keysClass, Class<V> valuesClass);
+
+    /**
+     * Sets the {@code i}th value to the provided map, which keys and values are of the provided Java
+     * types.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of lists
+     * of the given Java types to the underlying CQL type.
+     *
+     * @param i          the index of the value to set.
+     * @param v          the value to set. Note that {@code null} values inside collections are not supported
+     *                   by CQL.
+     * @param keysType   the type for the keys of the map.
+     * @param valuesType the type for the values of the map.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws NullPointerException      if {@code v} contains null values. Nulls are not supported in
+     *                                   collections by CQL.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public <K, V> T setMap(int i, Map<K, V> v, TypeToken<K> keysType, TypeToken<V> valuesType);
+
+    /**
+     * Sets the {@code i}th value to the provided set.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (the type of the elements in the Java set is not considered). If two or
+     * more codecs target that CQL type, the one that was first registered will be used. For this
+     * reason, it is generally preferable to use the more deterministic methods {@link #setSet(int,
+     * Set, Class)} or {@link #setSet(int, Set, TypeToken)}.
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set. Note that {@code null} values inside collections are not supported
+     *          by CQL.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws NullPointerException      if {@code v} contains null values. Nulls are not supported in
+     *                                   collections by CQL.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public <E> T setSet(int i, Set<E> v);
+
+    /**
+     * Sets the {@code i}th value to the provided set, which elements are of the provided Java class.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of sets
+     * of the given Java type to the underlying CQL type.
+     *
+     * <p>If the type of the elements is generic, use {@link #setSet(int, Set, TypeToken)}.
+     *
+     * @param i             the index of the value to set.
+     * @param v             the value to set. Note that {@code null} values inside collections are not supported
+     *                      by CQL.
+     * @param elementsClass the class for the elements of the set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws NullPointerException      if {@code v} contains null values. Nulls are not supported in
+     *                                   collections by CQL.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public <E> T setSet(int i, Set<E> v, Class<E> elementsClass);
+
+    /**
+     * Sets the {@code i}th value to the provided set, which elements are of the provided Java type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of sets
+     * of the given Java type to the underlying CQL type.
+     *
+     * @param i            the index of the value to set.
+     * @param v            the value to set. Note that {@code null} values inside collections are not supported
+     *                     by CQL.
+     * @param elementsType the type for the elements of the set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws NullPointerException      if {@code v} contains null values. Nulls are not supported in
+     *                                   collections by CQL.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public <E> T setSet(int i, Set<E> v, TypeToken<E> elementsType);
+
+    /**
+     * Sets the {@code i}th value to the provided UDT value.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of
+     * {@code UDTValue} to the underlying CQL type.
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setUDTValue(int i, UDTValue v);
+
+    /**
+     * Sets the {@code i}th value to the provided tuple value.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of
+     * {@code TupleValue} to the underlying CQL type.
+     *
+     * @param i the index of the value to set.
+     * @param v the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    public T setTupleValue(int i, TupleValue v);
+
+    /**
+     * Sets the {@code i}th value to {@code null}.
+     *
+     * <p>This is mainly intended for CQL types which map to native Java types.
+     *
+     * @param i the index of the value to set.
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     */
+    public T setToNull(int i);
+
+    /**
+     * Sets the {@code i}th value to the provided value of the provided Java class.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of the
+     * provided Java class to the underlying CQL type.
+     *
+     * <p>If the Java type is generic, use {@link #set(int, Object, TypeToken)} instead.
+     *
+     * @param i           the index of the value to set.
+     * @param v           the value to set; may be {@code null}.
+     * @param targetClass The Java class to convert to; must not be {@code null};
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    <V> T set(int i, V v, Class<V> targetClass);
+
+    /**
+     * Sets the {@code i}th value to the provided value of the provided Java type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of the
+     * provided Java type to the underlying CQL type.
+     *
+     * @param i          the index of the value to set.
+     * @param v          the value to set; may be {@code null}.
+     * @param targetType The Java type to convert to; must not be {@code null};
+     * @return this object.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     * @throws CodecNotFoundException    if there is no registered codec to convert the value to the
+     *                                   underlying CQL type.
+     */
+    <V> T set(int i, V v, TypeToken<V> targetType);
+
+    /**
+     * Sets the {@code i}th value to the provided value, converted using the given {@link TypeCodec}.
+     *
+     * <p>This method entirely bypasses the {@link CodecRegistry} and forces the driver to use the
+     * given codec instead. This can be useful if the codec would collide with a previously registered
+     * one, or if you want to use the codec just once without registering it.
+     *
+     * <p>It is the caller's responsibility to ensure that the given codec {@link
+     * TypeCodec#accepts(DataType) accepts} the underlying CQL type; failing to do so may result in
+     * {@link InvalidTypeException}s being thrown.
+     *
+     * @param i     the index of the value to set.
+     * @param v     the value to set; may be {@code null}.
+     * @param codec The {@link TypeCodec} to use to serialize the value; may not be {@code null}.
+     * @return this object.
+     * @throws InvalidTypeException      if the given codec does not {@link TypeCodec#accepts(DataType)
+     *                                   accept} the underlying CQL type.
+     * @throws IndexOutOfBoundsException if {@code i} is not a valid index for this object.
+     */
+    <V> T set(int i, V v, TypeCodec<V> codec);
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/SettableByNameData.java b/src/java/org/apache/cassandra/cql3/functions/types/SettableByNameData.java
new file mode 100644
index 0000000..514ba60
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/SettableByNameData.java
@@ -0,0 +1,620 @@
+/*
+ * 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.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.reflect.TypeToken;
+
+import org.apache.cassandra.cql3.functions.types.exceptions.CodecNotFoundException;
+import org.apache.cassandra.cql3.functions.types.exceptions.InvalidTypeException;
+
+/**
+ * Collection of (typed) CQL values that can set by name.
+ */
+public interface SettableByNameData<T extends SettableData<T>>
+{
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided boolean.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code boolean}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set. To set the value to NULL, use {@link #setToNull(String)} or {@code
+     *             set(name, v, Boolean.class)}.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setBool(String name, boolean v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided byte.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code tinyint}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set. To set the value to NULL, use {@link #setToNull(String)} or {@code
+     *             set(name, v, Byte.class)}.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setByte(String name, byte v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided short.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code smallint}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set. To set the value to NULL, use {@link #setToNull(String)} or {@code
+     *             set(name, v, Short.class)}.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setShort(String name, short v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided integer.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code int}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set. To set the value to NULL, use {@link #setToNull(String)} or {@code
+     *             set(name, v, Integer.class)}.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setInt(String name, int v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided long.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code bigint}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set. To set the value to NULL, use {@link #setToNull(String)} or {@code
+     *             set(name, v, Long.class)}.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setLong(String name, long v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided date.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code timestamp}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setTimestamp(String name, Date v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided date (without
+     * time).
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code date}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setDate(String name, LocalDate v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided time as a long in
+     * nanoseconds since midnight.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code time}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setTime(String name, long v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided float.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code float}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set. To set the value to NULL, use {@link #setToNull(String)} or {@code
+     *             set(name, v, Float.class)}.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setFloat(String name, float v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided double.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code double}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set. To set the value to NULL, use {@link #setToNull(String)} or {@code
+     *             set(name, v, Double.class)}.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setDouble(String name, double v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided string.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL types {@code text}, {@code varchar} and {@code ascii}, this will
+     * be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setString(String name, String v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided byte buffer.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code blob}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setBytes(String name, ByteBuffer v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided byte buffer.
+     *
+     * <p>This method does not use any codec; it sets the value in its binary form directly. If you
+     * insert data that is not compatible with the underlying CQL type, you will get an {@code
+     * InvalidQueryException} at execute time.
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     */
+    public T setBytesUnsafe(String name, ByteBuffer v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided big integer.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code varint}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setVarint(String name, BigInteger v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided big decimal.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code decimal}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setDecimal(String name, BigDecimal v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided UUID.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL types {@code uuid} and {@code timeuuid}, this will be the built-in
+     * codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setUUID(String name, UUID v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided inet address.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (for CQL type {@code inet}, this will be the built-in codec).
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setInet(String name, InetAddress v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided list.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (the type of the elements in the Java list is not considered). If two or
+     * more codecs target that CQL type, the one that was first registered will be used. For this
+     * reason, it is generally preferable to use the more deterministic methods {@link
+     * #setList(String, List, Class)} or {@link #setList(String, List, TypeToken)}.
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set. Note that {@code null} values inside collections are not supported
+     *             by CQL.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws NullPointerException     if {@code v} contains null values. Nulls are not supported in
+     *                                  collections by CQL.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public <E> T setList(String name, List<E> v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided list, which
+     * elements are of the provided Java class.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of lists
+     * of the given Java type to the underlying CQL type.
+     *
+     * <p>If the type of the elements is generic, use {@link #setList(String, List, TypeToken)}.
+     *
+     * @param name          the name of the value to set; if {@code name} is present multiple
+     * @param v             the value to set. Note that {@code null} values inside collections are not supported
+     *                      by CQL.
+     * @param elementsClass the class for the elements of the list.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws NullPointerException     if {@code v} contains null values. Nulls are not supported in
+     *                                  collections by CQL.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public <E> T setList(String name, List<E> v, Class<E> elementsClass);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided list, which
+     * elements are of the provided Java type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of lists
+     * of the given Java type to the underlying CQL type.
+     *
+     * @param name         the name of the value to set; if {@code name} is present multiple
+     * @param v            the value to set. Note that {@code null} values inside collections are not supported
+     *                     by CQL.
+     * @param elementsType the type for the elements of the list.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws NullPointerException     if {@code v} contains null values. Nulls are not supported in
+     *                                  collections by CQL.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public <E> T setList(String name, List<E> v, TypeToken<E> elementsType);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided map.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (the type of the elements in the Java map is not considered). If two or
+     * more codecs target that CQL type, the one that was first registered will be used. For this
+     * reason, it is generally preferable to use the more deterministic methods {@link #setMap(String,
+     * Map, Class, Class)} or {@link #setMap(String, Map, TypeToken, TypeToken)}.
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set. Note that {@code null} values inside collections are not supported
+     *             by CQL.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws NullPointerException     if {@code v} contains null values. Nulls are not supported in
+     *                                  collections by CQL.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public <K, V> T setMap(String name, Map<K, V> v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided map, which keys
+     * and values are of the provided Java classes.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of lists
+     * of the given Java types to the underlying CQL type.
+     *
+     * <p>If the type of the keys or values is generic, use {@link #setMap(String, Map, TypeToken,
+     * TypeToken)}.
+     *
+     * @param name        the name of the value to set; if {@code name} is present multiple times, all its
+     *                    values are set.
+     * @param v           the value to set. Note that {@code null} values inside collections are not supported
+     *                    by CQL.
+     * @param keysClass   the class for the keys of the map.
+     * @param valuesClass the class for the values of the map.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws NullPointerException     if {@code v} contains null values. Nulls are not supported in
+     *                                  collections by CQL.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public <K, V> T setMap(String name, Map<K, V> v, Class<K> keysClass, Class<V> valuesClass);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided map, which keys
+     * and values are of the provided Java types.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of lists
+     * of the given Java types to the underlying CQL type.
+     *
+     * @param name       the name of the value to set; if {@code name} is present multiple times, all its
+     *                   values are set.
+     * @param v          the value to set. Note that {@code null} values inside collections are not supported
+     *                   by CQL.
+     * @param keysType   the type for the keys of the map.
+     * @param valuesType the type for the values of the map.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws NullPointerException     if {@code v} contains null values. Nulls are not supported in
+     *                                  collections by CQL.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public <K, V> T setMap(String name, Map<K, V> v, TypeToken<K> keysType, TypeToken<V> valuesType);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided set.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion to the
+     * underlying CQL type (the type of the elements in the Java set is not considered). If two or
+     * more codecs target that CQL type, the one that was first registered will be used. For this
+     * reason, it is generally preferable to use the more deterministic methods {@link #setSet(String,
+     * Set, Class)} or {@link #setSet(String, Set, TypeToken)}.
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set. Note that {@code null} values inside collections are not supported
+     *             by CQL.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws NullPointerException     if {@code v} contains null values. Nulls are not supported in
+     *                                  collections by CQL.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public <E> T setSet(String name, Set<E> v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided set, which
+     * elements are of the provided Java class.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of sets
+     * of the given Java type to the underlying CQL type.
+     *
+     * <p>If the type of the elements is generic, use {@link #setSet(String, Set, TypeToken)}.
+     *
+     * @param name          the name of the value to set; if {@code name} is present multiple
+     * @param v             the value to set. Note that {@code null} values inside collections are not supported
+     *                      by CQL.
+     * @param elementsClass the class for the elements of the set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws NullPointerException     if {@code v} contains null values. Nulls are not supported in
+     *                                  collections by CQL.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public <E> T setSet(String name, Set<E> v, Class<E> elementsClass);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided set, which
+     * elements are of the provided Java type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of sets
+     * of the given Java type to the underlying CQL type.
+     *
+     * @param name         the name of the value to set; if {@code name} is present multiple
+     * @param v            the value to set. Note that {@code null} values inside collections are not supported
+     *                     by CQL.
+     * @param elementsType the type for the elements of the set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws NullPointerException     if {@code v} contains null values. Nulls are not supported in
+     *                                  collections by CQL.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public <E> T setSet(String name, Set<E> v, TypeToken<E> elementsType);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided UDT value.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of
+     * {@code UDTValue} to the underlying CQL type.
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setUDTValue(String name, UDTValue v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided tuple value.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of
+     * {@code TupleValue} to the underlying CQL type.
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @param v    the value to set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    public T setTupleValue(String name, TupleValue v);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to {@code null}.
+     *
+     * <p>This is mainly intended for CQL types which map to native Java types.
+     *
+     * @param name the name of the value to set; if {@code name} is present multiple times, all its
+     *             values are set.
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     */
+    public T setToNull(String name);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided value of the
+     * provided Java class.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of the
+     * provided Java class to the underlying CQL type.
+     *
+     * <p>If the Java type is generic, use {@link #set(String, Object, TypeToken)} instead.
+     *
+     * @param name        the name of the value to set; if {@code name} is present multiple times, all its
+     *                    values are set.
+     * @param v           the value to set; may be {@code null}.
+     * @param targetClass The Java class to convert to; must not be {@code null};
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    <V> T set(String name, V v, Class<V> targetClass);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided value of the
+     * provided Java type.
+     *
+     * <p>This method uses the {@link CodecRegistry} to find a codec to handle the conversion of the
+     * provided Java type to the underlying CQL type.
+     *
+     * @param name       the name of the value to set; if {@code name} is present multiple times, all its
+     *                   values are set.
+     * @param v          the value to set; may be {@code null}.
+     * @param targetType The Java type to convert to; must not be {@code null};
+     * @return this object.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     * @throws CodecNotFoundException   if there is no registered codec to convert the value to the
+     *                                  underlying CQL type.
+     */
+    <V> T set(String name, V v, TypeToken<V> targetType);
+
+    /**
+     * Sets the value for (all occurrences of) variable {@code name} to the provided value, converted
+     * using the given {@link TypeCodec}.
+     *
+     * <p>This method entirely bypasses the {@link CodecRegistry} and forces the driver to use the
+     * given codec instead. This can be useful if the codec would collide with a previously registered
+     * one, or if you want to use the codec just once without registering it.
+     *
+     * <p>It is the caller's responsibility to ensure that the given codec {@link
+     * TypeCodec#accepts(DataType) accepts} the underlying CQL type; failing to do so may result in
+     * {@link InvalidTypeException}s being thrown.
+     *
+     * @param name  the name of the value to set; if {@code name} is present multiple times, all its
+     *              values are set.
+     * @param v     the value to set; may be {@code null}.
+     * @param codec The {@link TypeCodec} to use to serialize the value; may not be {@code null}.
+     * @return this object.
+     * @throws InvalidTypeException     if the given codec does not {@link TypeCodec#accepts(DataType)
+     *                                  accept} the underlying CQL type.
+     * @throws IllegalArgumentException if {@code name} is not a valid name for this object.
+     */
+    <V> T set(String name, V v, TypeCodec<V> codec);
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/SettableData.java b/src/java/org/apache/cassandra/cql3/functions/types/SettableData.java
new file mode 100644
index 0000000..a60e738
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/SettableData.java
@@ -0,0 +1,26 @@
+/*
+ * 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;
+
+/**
+ * Collection of (typed) CQL values that can be set either by index (starting at zero) or by name.
+ */
+public interface SettableData<T extends SettableData<T>>
+extends SettableByIndexData<T>, SettableByNameData<T>
+{
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/TupleType.java b/src/java/org/apache/cassandra/cql3/functions/types/TupleType.java
new file mode 100644
index 0000000..8b8f452
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/TupleType.java
@@ -0,0 +1,201 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.cql3.functions.types.exceptions.InvalidTypeException;
+
+/**
+ * A tuple type.
+ *
+ * <p>A tuple type is a essentially a list of types.
+ */
+public class TupleType extends DataType
+{
+
+    private final List<DataType> types;
+    private final ProtocolVersion protocolVersion;
+    private final CodecRegistry codecRegistry;
+
+    TupleType(List<DataType> types, ProtocolVersion protocolVersion, CodecRegistry codecRegistry)
+    {
+        super(DataType.Name.TUPLE);
+        this.types = ImmutableList.copyOf(types);
+        this.protocolVersion = protocolVersion;
+        this.codecRegistry = codecRegistry;
+    }
+
+    /**
+     * Creates a "disconnected" tuple type (<b>you should prefer {@code
+     * Metadata#newTupleType(DataType...) cluster.getMetadata().newTupleType(...)} whenever
+     * possible</b>).
+     *
+     * <p>This method is only exposed for situations where you don't have a {@code Cluster} instance
+     * available. If you create a type with this method and use it with a {@code Cluster} later, you
+     * won't be able to set tuple fields with custom codecs registered against the cluster, or you
+     * might get errors if the protocol versions don't match.
+     *
+     * @param protocolVersion the protocol version to use.
+     * @param codecRegistry   the codec registry to use.
+     * @param types           the types for the tuple type.
+     * @return the newly created tuple type.
+     */
+    public static TupleType of(
+    ProtocolVersion protocolVersion, CodecRegistry codecRegistry, DataType... types)
+    {
+        return new TupleType(Arrays.asList(types), protocolVersion, codecRegistry);
+    }
+
+    /**
+     * The (immutable) list of types composing this tuple type.
+     *
+     * @return the (immutable) list of types composing this tuple type.
+     */
+    List<DataType> getComponentTypes()
+    {
+        return types;
+    }
+
+    /**
+     * Returns a new empty value for this tuple type.
+     *
+     * @return an empty (with all component to {@code null}) value for this user type definition.
+     */
+    public TupleValue newValue()
+    {
+        return new TupleValue(this);
+    }
+
+    /**
+     * Returns a new value for this tuple type that uses the provided values for the components.
+     *
+     * <p>The numbers of values passed to this method must correspond to the number of components in
+     * this tuple type. The {@code i}th parameter value will then be assigned to the {@code i}th
+     * component of the resulting tuple value.
+     *
+     * @param values the values to use for the component of the resulting tuple.
+     * @return a new tuple values based on the provided values.
+     * @throws IllegalArgumentException if the number of {@code values} provided does not correspond
+     *                                  to the number of components in this tuple type.
+     * @throws InvalidTypeException     if any of the provided value is not of the correct type for the
+     *                                  component.
+     */
+    public TupleValue newValue(Object... values)
+    {
+        if (values.length != types.size())
+            throw new IllegalArgumentException(
+            String.format(
+            "Invalid number of values. Expecting %d but got %d", types.size(), values.length));
+
+        TupleValue t = newValue();
+        for (int i = 0; i < values.length; i++)
+        {
+            DataType dataType = types.get(i);
+            if (values[i] == null) t.setValue(i, null);
+            else
+                t.setValue(
+                i, codecRegistry.codecFor(dataType, values[i]).serialize(values[i], protocolVersion));
+        }
+        return t;
+    }
+
+    @Override
+    public boolean isFrozen()
+    {
+        return true;
+    }
+
+    /**
+     * Return the protocol version that has been used to deserialize this tuple type, or that will be
+     * used to serialize it. In most cases this should be the version currently in use by the cluster
+     * instance that this tuple type belongs to, as reported by {@code
+     * ProtocolOptions#getProtocolVersion()}.
+     *
+     * @return the protocol version that has been used to deserialize this tuple type, or that will be
+     * used to serialize it.
+     */
+    ProtocolVersion getProtocolVersion()
+    {
+        return protocolVersion;
+    }
+
+    CodecRegistry getCodecRegistry()
+    {
+        return codecRegistry;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Arrays.hashCode(new Object[]{ name, types });
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof TupleType)) return false;
+
+        TupleType d = (TupleType) o;
+        return name == d.name && types.equals(d.types);
+    }
+
+    /**
+     * Return {@code true} if this tuple type contains the given tuple type, and {@code false}
+     * otherwise.
+     *
+     * <p>A tuple type is said to contain another one if the latter has fewer components than the
+     * former, but all of them are of the same type. E.g. the type {@code tuple<int, text>} is
+     * contained by the type {@code tuple<int, text, double>}.
+     *
+     * <p>A contained type can be seen as a "partial" view of a containing type, where the missing
+     * components are supposed to be {@code null}.
+     *
+     * @param other the tuple type to compare against the current one
+     * @return {@code true} if this tuple type contains the given tuple type, and {@code false}
+     * otherwise.
+     */
+    public boolean contains(TupleType other)
+    {
+        if (this.equals(other)) return true;
+        if (other.types.size() > this.types.size()) return false;
+        return types.subList(0, other.types.size()).equals(other.types);
+    }
+
+    @Override
+    public String toString()
+    {
+        return "frozen<" + asFunctionParameterString() + '>';
+    }
+
+    @Override
+    public String asFunctionParameterString()
+    {
+        StringBuilder sb = new StringBuilder();
+        for (DataType type : types)
+        {
+            sb.append(sb.length() == 0 ? "tuple<" : ", ");
+            sb.append(type.asFunctionParameterString());
+        }
+        return sb.append('>').toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/TupleValue.java b/src/java/org/apache/cassandra/cql3/functions/types/TupleValue.java
new file mode 100644
index 0000000..68348e3
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/TupleValue.java
@@ -0,0 +1,92 @@
+/*
+ * 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;
+
+/**
+ * A value for a Tuple.
+ */
+public class TupleValue extends AbstractAddressableByIndexData<TupleValue>
+{
+
+    private final TupleType type;
+
+    /**
+     * Builds a new value for a tuple.
+     *
+     * @param type the {@link TupleType} instance defining this tuple's components.
+     */
+    TupleValue(TupleType type)
+    {
+        super(type.getProtocolVersion(), type.getComponentTypes().size());
+        this.type = type;
+    }
+
+    protected DataType getType(int i)
+    {
+        return type.getComponentTypes().get(i);
+    }
+
+    @Override
+    protected String getName(int i)
+    {
+        // This is used for error messages
+        return "component " + i;
+    }
+
+    @Override
+    protected CodecRegistry getCodecRegistry()
+    {
+        return type.getCodecRegistry();
+    }
+
+    /**
+     * The tuple type this is a value of.
+     *
+     * @return The tuple type this is a value of.
+     */
+    public TupleType getType()
+    {
+        return type;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof TupleValue)) return false;
+
+        TupleValue that = (TupleValue) o;
+        if (!type.equals(that.type)) return false;
+
+        return super.equals(o);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return super.hashCode();
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder();
+        TypeCodec<Object> codec = getCodecRegistry().codecFor(type);
+        sb.append(codec.format(this));
+        return sb.toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/types/TypeCodec.java b/src/java/org/apache/cassandra/cql3/functions/types/TypeCodec.java
new file mode 100644
index 0000000..a728a1c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/types/TypeCodec.java
@@ -0,0 +1,3103 @@
+/*
+ * 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.io.DataInput;
+import java.io.IOException;
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.text.ParseException;
+import java.util.*;
+import java.util.regex.Pattern;
+
+import com.google.common.io.ByteArrayDataOutput;
+import com.google.common.io.ByteStreams;
+import com.google.common.reflect.TypeToken;
+
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.cql3.functions.types.DataType.CollectionType;
+import org.apache.cassandra.cql3.functions.types.DataType.Name;
+import org.apache.cassandra.cql3.functions.types.exceptions.InvalidTypeException;
+import org.apache.cassandra.cql3.functions.types.utils.Bytes;
+import org.apache.cassandra.utils.vint.VIntCoding;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.cassandra.cql3.functions.types.DataType.*;
+
+/**
+ * A Codec that can serialize and deserialize to and from a given {@link #getCqlType() CQL type} and
+ * a given {@link #getJavaType() Java Type}.
+ *
+ * <p>
+ *
+ * <h3>Serializing and deserializing</h3>
+ *
+ * <p>Two methods handle the serialization and deserialization of Java types into CQL types
+ * according to the native protocol specifications:
+ *
+ * <ol>
+ * <li>{@link #serialize(Object, ProtocolVersion)}: used to serialize from the codec's Java type
+ * to a {@link ByteBuffer} instance corresponding to the codec's CQL type;
+ * <li>{@link #deserialize(ByteBuffer, ProtocolVersion)}: used to deserialize a {@link ByteBuffer}
+ * instance corresponding to the codec's CQL type to the codec's Java type.
+ * </ol>
+ *
+ * <p>
+ *
+ * <h3>Formatting and parsing</h3>
+ *
+ * <p>Two methods handle the formatting and parsing of Java types into CQL strings:
+ *
+ * <ol>
+ * <li>{@link #format(Object)}: formats the Java type handled by the codec as a CQL string;
+ * <li>{@link #parse(String)}; parses a CQL string into the Java type handled by the codec.
+ * </ol>
+ *
+ * <p>
+ *
+ * <h3>Inspection</h3>
+ *
+ * <p>Codecs also have the following inspection methods:
+ *
+ * <p>
+ *
+ * <ol>
+ * <li>{@link #accepts(DataType)}: returns true if the codec can deserialize the given CQL type;
+ * <li>{@link #accepts(TypeToken)}: returns true if the codec can serialize the given Java type;
+ * <li>{@link #accepts(Object)}; returns true if the codec can serialize the given object.
+ * </ol>
+ *
+ * <p>
+ *
+ * <h3>Implementation notes</h3>
+ *
+ * <p>
+ *
+ * <ol>
+ * <li>TypeCodec implementations <em>must</em> be thread-safe.
+ * <li>TypeCodec implementations <em>must</em> perform fast and never block.
+ * <li>TypeCodec implementations <em>must</em> support all native protocol versions; it is not
+ * possible to use different codecs for the same types but under different protocol versions.
+ * <li>TypeCodec implementations must comply with the native protocol specifications; failing to
+ * do so will result in unexpected results and could cause the driver to crash.
+ * <li>TypeCodec implementations <em>should</em> be stateless and immutable.
+ * <li>TypeCodec implementations <em>should</em> interpret {@code null} values and empty
+ * ByteBuffers (i.e. <code>{@link ByteBuffer#remaining()} == 0</code>) in a
+ * <em>reasonable</em> way; usually, {@code NULL} CQL values should map to {@code null}
+ * references, but exceptions exist; e.g. for varchar types, a {@code NULL} CQL value maps to
+ * a {@code null} reference, whereas an empty buffer maps to an empty String. For collection
+ * types, it is also admitted that {@code NULL} CQL values map to empty Java collections
+ * instead of {@code null} references. In any case, the codec's behavior in respect to {@code
+ * null} values and empty ByteBuffers should be clearly documented.
+ * <li>TypeCodec implementations that wish to handle Java primitive types <em>must</em> be
+ * instantiated with the wrapper Java class instead, and implement the appropriate interface
+ * (e.g. {@link TypeCodec.PrimitiveBooleanCodec} for primitive {@code
+ * boolean} types; there is one such interface for each Java primitive type).
+ * <li>When deserializing, TypeCodec implementations should not consume {@link ByteBuffer}
+ * instances by performing relative read operations that modify their current position; codecs
+ * should instead prefer absolute read methods, or, if necessary, they should {@link
+ * ByteBuffer#duplicate() duplicate} their byte buffers prior to reading them.
+ * </ol>
+ *
+ * @param <T> The codec's Java type
+ */
+public abstract class TypeCodec<T>
+{
+
+    /**
+     * Return the default codec for the CQL type {@code boolean}. The returned codec maps the CQL type
+     * {@code boolean} into the Java type {@link Boolean}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code boolean}.
+     */
+    public static PrimitiveBooleanCodec cboolean()
+    {
+        return BooleanCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code tinyint}. The returned codec maps the CQL type
+     * {@code tinyint} into the Java type {@link Byte}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code tinyint}.
+     */
+    public static PrimitiveByteCodec tinyInt()
+    {
+        return TinyIntCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code smallint}. The returned codec maps the CQL
+     * type {@code smallint} into the Java type {@link Short}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code smallint}.
+     */
+    public static PrimitiveShortCodec smallInt()
+    {
+        return SmallIntCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code int}. The returned codec maps the CQL type
+     * {@code int} into the Java type {@link Integer}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code int}.
+     */
+    public static PrimitiveIntCodec cint()
+    {
+        return IntCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code bigint}. The returned codec maps the CQL type
+     * {@code bigint} into the Java type {@link Long}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code bigint}.
+     */
+    public static PrimitiveLongCodec bigint()
+    {
+        return BigintCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code counter}. The returned codec maps the CQL type
+     * {@code counter} into the Java type {@link Long}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code counter}.
+     */
+    public static PrimitiveLongCodec counter()
+    {
+        return CounterCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code float}. The returned codec maps the CQL type
+     * {@code float} into the Java type {@link Float}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code float}.
+     */
+    public static PrimitiveFloatCodec cfloat()
+    {
+        return FloatCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code double}. The returned codec maps the CQL type
+     * {@code double} into the Java type {@link Double}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code double}.
+     */
+    public static PrimitiveDoubleCodec cdouble()
+    {
+        return DoubleCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code varint}. The returned codec maps the CQL type
+     * {@code varint} into the Java type {@link BigInteger}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code varint}.
+     */
+    public static TypeCodec<BigInteger> varint()
+    {
+        return VarintCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code decimal}. The returned codec maps the CQL type
+     * {@code decimal} into the Java type {@link BigDecimal}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code decimal}.
+     */
+    public static TypeCodec<BigDecimal> decimal()
+    {
+        return DecimalCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code ascii}. The returned codec maps the CQL type
+     * {@code ascii} into the Java type {@link String}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code ascii}.
+     */
+    public static TypeCodec<String> ascii()
+    {
+        return AsciiCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code varchar}. The returned codec maps the CQL type
+     * {@code varchar} into the Java type {@link String}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code varchar}.
+     */
+    public static TypeCodec<String> varchar()
+    {
+        return VarcharCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code blob}. The returned codec maps the CQL type
+     * {@code blob} into the Java type {@link ByteBuffer}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code blob}.
+     */
+    public static TypeCodec<ByteBuffer> blob()
+    {
+        return BlobCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code date}. The returned codec maps the CQL type
+     * {@code date} into the Java type {@link LocalDate}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code date}.
+     */
+    public static TypeCodec<LocalDate> date()
+    {
+        return DateCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code time}. The returned codec maps the CQL type
+     * {@code time} into the Java type {@link Long}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code time}.
+     */
+    public static PrimitiveLongCodec time()
+    {
+        return TimeCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code timestamp}. The returned codec maps the CQL
+     * type {@code timestamp} into the Java type {@link Date}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code timestamp}.
+     */
+    public static TypeCodec<Date> timestamp()
+    {
+        return TimestampCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code uuid}. The returned codec maps the CQL type
+     * {@code uuid} into the Java type {@link UUID}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code uuid}.
+     */
+    public static TypeCodec<UUID> uuid()
+    {
+        return UUIDCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code timeuuid}. The returned codec maps the CQL
+     * type {@code timeuuid} into the Java type {@link UUID}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code timeuuid}.
+     */
+    public static TypeCodec<UUID> timeUUID()
+    {
+        return TimeUUIDCodec.instance;
+    }
+
+    /**
+     * Return the default codec for the CQL type {@code inet}. The returned codec maps the CQL type
+     * {@code inet} into the Java type {@link InetAddress}. The returned instance is a singleton.
+     *
+     * @return the default codec for CQL type {@code inet}.
+     */
+    public static TypeCodec<InetAddress> inet()
+    {
+        return InetCodec.instance;
+    }
+
+    /**
+     * Return a newly-created codec for the CQL type {@code list} whose element type is determined by
+     * the given element codec. The returned codec maps the CQL type {@code list} into the Java type
+     * {@link List}. This method does not cache returned instances and returns a newly-allocated
+     * object at each invocation.
+     *
+     * @param elementCodec the codec that will handle elements of this list.
+     * @return A newly-created codec for CQL type {@code list}.
+     */
+    public static <T> TypeCodec<List<T>> list(TypeCodec<T> elementCodec)
+    {
+        return new ListCodec<>(elementCodec);
+    }
+
+    /**
+     * Return a newly-created codec for the CQL type {@code set} whose element type is determined by
+     * the given element codec. The returned codec maps the CQL type {@code set} into the Java type
+     * {@link Set}. This method does not cache returned instances and returns a newly-allocated object
+     * at each invocation.
+     *
+     * @param elementCodec the codec that will handle elements of this set.
+     * @return A newly-created codec for CQL type {@code set}.
+     */
+    public static <T> TypeCodec<Set<T>> set(TypeCodec<T> elementCodec)
+    {
+        return new SetCodec<>(elementCodec);
+    }
+
+    /**
+     * Return a newly-created codec for the CQL type {@code map} whose key type and value type are
+     * determined by the given codecs. The returned codec maps the CQL type {@code map} into the Java
+     * type {@link Map}. This method does not cache returned instances and returns a newly-allocated
+     * object at each invocation.
+     *
+     * @param keyCodec   the codec that will handle keys of this map.
+     * @param valueCodec the codec that will handle values of this map.
+     * @return A newly-created codec for CQL type {@code map}.
+     */
+    public static <K, V> TypeCodec<Map<K, V>> map(TypeCodec<K> keyCodec, TypeCodec<V> valueCodec)
+    {
+        return new MapCodec<>(keyCodec, valueCodec);
+    }
+
+    /**
+     * Return a newly-created codec for the given user-defined CQL type. The returned codec maps the
+     * user-defined type into the Java type {@link UDTValue}. This method does not cache returned
+     * instances and returns a newly-allocated object at each invocation.
+     *
+     * @param type the user-defined type this codec should handle.
+     * @return A newly-created codec for the given user-defined CQL type.
+     */
+    public static TypeCodec<UDTValue> userType(UserType type)
+    {
+        return new UDTCodec(type);
+    }
+
+    /**
+     * Return a newly-created codec for the given CQL tuple type. The returned codec maps the tuple
+     * type into the Java type {@link TupleValue}. This method does not cache returned instances and
+     * returns a newly-allocated object at each invocation.
+     *
+     * @param type the tuple type this codec should handle.
+     * @return A newly-created codec for the given CQL tuple type.
+     */
+    public static TypeCodec<TupleValue> tuple(TupleType type)
+    {
+        return new TupleCodec(type);
+    }
+
+    /**
+     * Return a newly-created codec for the given CQL custom type.
+     *
+     * <p>The returned codec maps the custom type into the Java type {@link ByteBuffer}, thus
+     * providing a (very lightweight) support for Cassandra types that do not have a CQL equivalent.
+     *
+     * <p>Note that the returned codec assumes that CQL literals for the given custom type are
+     * expressed in binary form as well, e.g. {@code 0xcafebabe}. If this is not the case, <em>the
+     * returned codec might be unable to {@link #parse(String) parse} and {@link #format(Object)
+     * format} literals for this type</em>. This is notoriously true for types inheriting from {@code
+     * org.apache.cassandra.db.marshal.AbstractCompositeType}, whose CQL literals are actually
+     * expressed as quoted strings.
+     *
+     * <p>This method does not cache returned instances and returns a newly-allocated object at each
+     * invocation.
+     *
+     * @param type the custom type this codec should handle.
+     * @return A newly-created codec for the given CQL custom type.
+     */
+    public static TypeCodec<ByteBuffer> custom(DataType.CustomType type)
+    {
+        return new CustomCodec(type);
+    }
+
+    /**
+     * Returns the default codec for the {@link DataType#duration() Duration type}.
+     *
+     * <p>This codec maps duration types to the driver's built-in {@link Duration} class, thus
+     * providing a more user-friendly mapping than the low-level mapping provided by regular {@link
+     * #custom(DataType.CustomType) custom type codecs}.
+     *
+     * <p>The returned instance is a singleton.
+     *
+     * @return the default codec for the Duration type.
+     */
+    public static TypeCodec<Duration> duration()
+    {
+        return DurationCodec.instance;
+    }
+
+    private final TypeToken<T> javaType;
+
+    final DataType cqlType;
+
+    /**
+     * This constructor can only be used for non parameterized types. For parameterized ones, please
+     * use {@link #TypeCodec(DataType, TypeToken)} instead.
+     *
+     * @param javaClass The Java class this codec serializes from and deserializes to.
+     */
+    protected TypeCodec(DataType cqlType, Class<T> javaClass)
+    {
+        this(cqlType, TypeToken.of(javaClass));
+    }
+
+    protected TypeCodec(DataType cqlType, TypeToken<T> javaType)
+    {
+        checkNotNull(cqlType, "cqlType cannot be null");
+        checkNotNull(javaType, "javaType cannot be null");
+        checkArgument(
+        !javaType.isPrimitive(),
+        "Cannot create a codec for a primitive Java type (%s), please use the wrapper type instead",
+        javaType);
+        this.cqlType = cqlType;
+        this.javaType = javaType;
+    }
+
+    /**
+     * Return the Java type that this codec deserializes to and serializes from.
+     *
+     * @return The Java type this codec deserializes to and serializes from.
+     */
+    public TypeToken<T> getJavaType()
+    {
+        return javaType;
+    }
+
+    /**
+     * Return the CQL type that this codec deserializes from and serializes to.
+     *
+     * @return The Java type this codec deserializes from and serializes to.
+     */
+    public DataType getCqlType()
+    {
+        return cqlType;
+    }
+
+    /**
+     * Serialize the given value according to the CQL type handled by this codec.
+     *
+     * <p>Implementation notes:
+     *
+     * <ol>
+     * <li>Null values should be gracefully handled and no exception should be raised; these should
+     * be considered as the equivalent of a NULL CQL value;
+     * <li>Codecs for CQL collection types should not permit null elements;
+     * <li>Codecs for CQL collection types should treat a {@code null} input as the equivalent of an
+     * empty collection.
+     * </ol>
+     *
+     * @param value           An instance of T; may be {@code null}.
+     * @param protocolVersion the protocol version to use when serializing {@code bytes}. In most
+     *                        cases, the proper value to provide for this argument is the value returned by {@code
+     *                        ProtocolOptions#getProtocolVersion} (which is the protocol version in use by the driver).
+     * @return A {@link ByteBuffer} instance containing the serialized form of T
+     * @throws InvalidTypeException if the given value does not have the expected type
+     */
+    public abstract ByteBuffer serialize(T value, ProtocolVersion protocolVersion)
+    throws InvalidTypeException;
+
+    /**
+     * Deserialize the given {@link ByteBuffer} instance according to the CQL type handled by this
+     * codec.
+     *
+     * <p>Implementation notes:
+     *
+     * <ol>
+     * <li>Null or empty buffers should be gracefully handled and no exception should be raised;
+     * these should be considered as the equivalent of a NULL CQL value and, in most cases,
+     * should map to {@code null} or a default value for the corresponding Java type, if
+     * applicable;
+     * <li>Codecs for CQL collection types should clearly document whether they return immutable
+     * collections or not (note that the driver's default collection codecs return
+     * <em>mutable</em> collections);
+     * <li>Codecs for CQL collection types should avoid returning {@code null}; they should return
+     * empty collections instead (the driver's default collection codecs all comply with this
+     * rule).
+     * <li>The provided {@link ByteBuffer} should never be consumed by read operations that modify
+     * its current position; if necessary, {@link ByteBuffer#duplicate()} duplicate} it before
+     * consuming.
+     * </ol>
+     *
+     * @param bytes           A {@link ByteBuffer} instance containing the serialized form of T; may be {@code
+     *                        null} or empty.
+     * @param protocolVersion the protocol version to use when serializing {@code bytes}. In most
+     *                        cases, the proper value to provide for this argument is the value returned by {@code
+     *                        ProtocolOptions#getProtocolVersion} (which is the protocol version in use by the driver).
+     * @return An instance of T
+     * @throws InvalidTypeException if the given {@link ByteBuffer} instance cannot be deserialized
+     */
+    public abstract T deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion)
+    throws InvalidTypeException;
+
+    /**
+     * Parse the given CQL literal into an instance of the Java type handled by this codec.
+     *
+     * <p>Implementors should take care of unquoting and unescaping the given CQL string where
+     * applicable. Null values and empty Strings should be accepted, as well as the string {@code
+     * "NULL"}; in most cases, implementations should interpret these inputs has equivalent to a
+     * {@code null} reference.
+     *
+     * <p>Implementing this method is not strictly mandatory: internally, the driver only uses it to
+     * parse the INITCOND when building the metadata of an aggregate function (and in most cases it
+     * will use a built-in codec, unless the INITCOND has a custom type).
+     *
+     * @param value The CQL string to parse, may be {@code null} or empty.
+     * @return An instance of T; may be {@code null} on a {@code null input}.
+     * @throws InvalidTypeException if the given value cannot be parsed into the expected type
+     */
+    public abstract T parse(String value) throws InvalidTypeException;
+
+    /**
+     * Format the given value as a valid CQL literal according to the CQL type handled by this codec.
+     *
+     * <p>Implementors should take care of quoting and escaping the resulting CQL literal where
+     * applicable. Null values should be accepted; in most cases, implementations should return the
+     * CQL keyword {@code "NULL"} for {@code null} inputs.
+     *
+     * <p>Implementing this method is not strictly mandatory. It is used:
+     *
+     * <ol>
+     * <li>in the query builder, when values are inlined in the query string (see {@code
+     * querybuilder.BuiltStatement} for a detailed explanation of when
+     * this happens);
+     * <li>in the {@code QueryLogger}, if parameter logging is enabled;
+     * <li>to format the INITCOND in {@code AggregateMetadata#asCQLQuery(boolean)};
+     * <li>in the {@code toString()} implementation of some objects ({@link UDTValue}, {@link
+     * TupleValue}, and the internal representation of a {@code ROWS} response), which may
+     * appear in driver logs.
+     * </ol>
+     * <p>
+     * If you choose not to implement this method, you should not throw an exception but instead
+     * return a constant string (for example "XxxCodec.format not implemented").
+     *
+     * @param value An instance of T; may be {@code null}.
+     * @return CQL string
+     * @throws InvalidTypeException if the given value does not have the expected type
+     */
+    public abstract String format(T value) throws InvalidTypeException;
+
+    /**
+     * Return {@code true} if this codec is capable of serializing the given {@code javaType}.
+     *
+     * <p>The implementation is <em>invariant</em> with respect to the passed argument (through the
+     * usage of {@link TypeToken#equals(Object)} and <em>it's strongly recommended not to modify this
+     * behavior</em>. This means that a codec will only ever return {@code true} for the
+     * <em>exact</em> Java type that it has been created for.
+     *
+     * <p>If the argument represents a Java primitive type, its wrapper type is considered instead.
+     *
+     * @param javaType The Java type this codec should serialize from and deserialize to; cannot be
+     *                 {@code null}.
+     * @return {@code true} if the codec is capable of serializing the given {@code javaType}, and
+     * {@code false} otherwise.
+     * @throws NullPointerException if {@code javaType} is {@code null}.
+     */
+    public boolean accepts(TypeToken<?> javaType)
+    {
+        checkNotNull(javaType, "Parameter javaType cannot be null");
+        return this.javaType.equals(javaType.wrap());
+    }
+
+    /**
+     * Return {@code true} if this codec is capable of serializing the given {@code javaType}.
+     *
+     * <p>This implementation simply calls {@link #accepts(TypeToken)}.
+     *
+     * @param javaType The Java type this codec should serialize from and deserialize to; cannot be
+     *                 {@code null}.
+     * @return {@code true} if the codec is capable of serializing the given {@code javaType}, and
+     * {@code false} otherwise.
+     * @throws NullPointerException if {@code javaType} is {@code null}.
+     */
+    public boolean accepts(Class<?> javaType)
+    {
+        checkNotNull(javaType, "Parameter javaType cannot be null");
+        return accepts(TypeToken.of(javaType));
+    }
+
+    /**
+     * Return {@code true} if this codec is capable of deserializing the given {@code cqlType}.
+     *
+     * @param cqlType The CQL type this codec should deserialize from and serialize to; cannot be
+     *                {@code null}.
+     * @return {@code true} if the codec is capable of deserializing the given {@code cqlType}, and
+     * {@code false} otherwise.
+     * @throws NullPointerException if {@code cqlType} is {@code null}.
+     */
+    public boolean accepts(DataType cqlType)
+    {
+        checkNotNull(cqlType, "Parameter cqlType cannot be null");
+        return this.cqlType.equals(cqlType);
+    }
+
+    /**
+     * Return {@code true} if this codec is capable of serializing the given object. Note that the
+     * object's Java type is inferred from the object' runtime (raw) type, contrary to {@link
+     * #accepts(TypeToken)} which is capable of handling generic types.
+     *
+     * <p>This method is intended mostly to be used by the QueryBuilder when no type information is
+     * available when the codec is used.
+     *
+     * <p>Implementation notes:
+     *
+     * <ol>
+     * <li>The default implementation is <em>covariant</em> with respect to the passed argument
+     * (through the usage of {@code TypeToken#isAssignableFrom(TypeToken)} or {@link
+     * TypeToken#isSupertypeOf(Type)}) and <em>it's strongly recommended not to modify this
+     * behavior</em>. This means that, by default, a codec will accept <em>any subtype</em> of
+     * the Java type that it has been created for.
+     * <li>The base implementation provided here can only handle non-parameterized types; codecs
+     * handling parameterized types, such as collection types, must override this method and
+     * perform some sort of "manual" inspection of the actual type parameters.
+     * <li>Similarly, codecs that only accept a partial subset of all possible values must override
+     * this method and manually inspect the object to check if it complies or not with the
+     * codec's limitations.
+     * </ol>
+     *
+     * @param value The Java type this codec should serialize from and deserialize to; cannot be
+     *              {@code null}.
+     * @return {@code true} if the codec is capable of serializing the given {@code javaType}, and
+     * {@code false} otherwise.
+     * @throws NullPointerException if {@code value} is {@code null}.
+     */
+    public boolean accepts(Object value)
+    {
+        checkNotNull(value, "Parameter value cannot be null");
+        return this.javaType.isSupertypeOf(TypeToken.of(value.getClass()));
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("%s [%s <-> %s]", this.getClass().getSimpleName(), cqlType, javaType);
+    }
+
+    /**
+     * A codec that is capable of handling primitive booleans, thus avoiding the overhead of boxing
+     * and unboxing such primitives.
+     */
+    public abstract static class PrimitiveBooleanCodec extends TypeCodec<Boolean>
+    {
+
+        PrimitiveBooleanCodec(DataType cqlType)
+        {
+            super(cqlType, Boolean.class);
+        }
+
+        public abstract ByteBuffer serializeNoBoxing(boolean v, ProtocolVersion protocolVersion);
+
+        public abstract boolean deserializeNoBoxing(ByteBuffer v, ProtocolVersion protocolVersion);
+
+        @Override
+        public ByteBuffer serialize(Boolean value, ProtocolVersion protocolVersion)
+        {
+            return value == null ? null : serializeNoBoxing(value, protocolVersion);
+        }
+
+        @Override
+        public Boolean deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion)
+        {
+            return bytes == null || bytes.remaining() == 0
+                   ? null
+                   : deserializeNoBoxing(bytes, protocolVersion);
+        }
+    }
+
+    /**
+     * A codec that is capable of handling primitive bytes, thus avoiding the overhead of boxing and
+     * unboxing such primitives.
+     */
+    public abstract static class PrimitiveByteCodec extends TypeCodec<Byte>
+    {
+
+        PrimitiveByteCodec(DataType cqlType)
+        {
+            super(cqlType, Byte.class);
+        }
+
+        public abstract ByteBuffer serializeNoBoxing(byte v, ProtocolVersion protocolVersion);
+
+        public abstract byte deserializeNoBoxing(ByteBuffer v, ProtocolVersion protocolVersion);
+
+        @Override
+        public ByteBuffer serialize(Byte value, ProtocolVersion protocolVersion)
+        {
+            return value == null ? null : serializeNoBoxing(value, protocolVersion);
+        }
+
+        @Override
+        public Byte deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion)
+        {
+            return bytes == null || bytes.remaining() == 0
+                   ? null
+                   : deserializeNoBoxing(bytes, protocolVersion);
+        }
+    }
+
+    /**
+     * A codec that is capable of handling primitive shorts, thus avoiding the overhead of boxing and
+     * unboxing such primitives.
+     */
+    public abstract static class PrimitiveShortCodec extends TypeCodec<Short>
+    {
+
+        PrimitiveShortCodec(DataType cqlType)
+        {
+            super(cqlType, Short.class);
+        }
+
+        public abstract ByteBuffer serializeNoBoxing(short v, ProtocolVersion protocolVersion);
+
+        public abstract short deserializeNoBoxing(ByteBuffer v, ProtocolVersion protocolVersion);
+
+        @Override
+        public ByteBuffer serialize(Short value, ProtocolVersion protocolVersion)
+        {
+            return value == null ? null : serializeNoBoxing(value, protocolVersion);
+        }
+
+        @Override
+        public Short deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion)
+        {
+            return bytes == null || bytes.remaining() == 0
+                   ? null
+                   : deserializeNoBoxing(bytes, protocolVersion);
+        }
+    }
+
+    /**
+     * A codec that is capable of handling primitive ints, thus avoiding the overhead of boxing and
+     * unboxing such primitives.
+     */
+    public abstract static class PrimitiveIntCodec extends TypeCodec<Integer>
+    {
+
+        PrimitiveIntCodec(DataType cqlType)
+        {
+            super(cqlType, Integer.class);
+        }
+
+        public abstract ByteBuffer serializeNoBoxing(int v, ProtocolVersion protocolVersion);
+
+        public abstract int deserializeNoBoxing(ByteBuffer v, ProtocolVersion protocolVersion);
+
+        @Override
+        public ByteBuffer serialize(Integer value, ProtocolVersion protocolVersion)
+        {
+            return value == null ? null : serializeNoBoxing(value, protocolVersion);
+        }
+
+        @Override
+        public Integer deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion)
+        {
+            return bytes == null || bytes.remaining() == 0
+                   ? null
+                   : deserializeNoBoxing(bytes, protocolVersion);
+        }
+    }
+
+    /**
+     * A codec that is capable of handling primitive longs, thus avoiding the overhead of boxing and
+     * unboxing such primitives.
+     */
+    public abstract static class PrimitiveLongCodec extends TypeCodec<Long>
+    {
+
+        PrimitiveLongCodec(DataType cqlType)
+        {
+            super(cqlType, Long.class);
+        }
+
+        public abstract ByteBuffer serializeNoBoxing(long v, ProtocolVersion protocolVersion);
+
+        public abstract long deserializeNoBoxing(ByteBuffer v, ProtocolVersion protocolVersion);
+
+        @Override
+        public ByteBuffer serialize(Long value, ProtocolVersion protocolVersion)
+        {
+            return value == null ? null : serializeNoBoxing(value, protocolVersion);
+        }
+
+        @Override
+        public Long deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion)
+        {
+            return bytes == null || bytes.remaining() == 0
+                   ? null
+                   : deserializeNoBoxing(bytes, protocolVersion);
+        }
+    }
+
+    /**
+     * A codec that is capable of handling primitive floats, thus avoiding the overhead of boxing and
+     * unboxing such primitives.
+     */
+    public abstract static class PrimitiveFloatCodec extends TypeCodec<Float>
+    {
+
+        PrimitiveFloatCodec(DataType cqlType)
+        {
+            super(cqlType, Float.class);
+        }
+
+        public abstract ByteBuffer serializeNoBoxing(float v, ProtocolVersion protocolVersion);
+
+        public abstract float deserializeNoBoxing(ByteBuffer v, ProtocolVersion protocolVersion);
+
+        @Override
+        public ByteBuffer serialize(Float value, ProtocolVersion protocolVersion)
+        {
+            return value == null ? null : serializeNoBoxing(value, protocolVersion);
+        }
+
+        @Override
+        public Float deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion)
+        {
+            return bytes == null || bytes.remaining() == 0
+                   ? null
+                   : deserializeNoBoxing(bytes, protocolVersion);
+        }
+    }
+
+    /**
+     * A codec that is capable of handling primitive doubles, thus avoiding the overhead of boxing and
+     * unboxing such primitives.
+     */
+    public abstract static class PrimitiveDoubleCodec extends TypeCodec<Double>
+    {
+
+        PrimitiveDoubleCodec(DataType cqlType)
+        {
+            super(cqlType, Double.class);
+        }
+
+        public abstract ByteBuffer serializeNoBoxing(double v, ProtocolVersion protocolVersion);
+
+        public abstract double deserializeNoBoxing(ByteBuffer v, ProtocolVersion protocolVersion);
+
+        @Override
+        public ByteBuffer serialize(Double value, ProtocolVersion protocolVersion)
+        {
+            return value == null ? null : serializeNoBoxing(value, protocolVersion);
+        }
+
+        @Override
+        public Double deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion)
+        {
+            return bytes == null || bytes.remaining() == 0
+                   ? null
+                   : deserializeNoBoxing(bytes, protocolVersion);
+        }
+    }
+
+    /**
+     * Base class for codecs handling CQL string types such as {@link DataType#varchar()}, {@link
+     * DataType#text()} or {@link DataType#ascii()}.
+     */
+    private abstract static class StringCodec extends TypeCodec<String>
+    {
+
+        private final Charset charset;
+
+        private StringCodec(DataType cqlType, Charset charset)
+        {
+            super(cqlType, String.class);
+            this.charset = charset;
+        }
+
+        @Override
+        public String parse(String value)
+        {
+            if (value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")) return null;
+            if (!ParseUtils.isQuoted(value))
+                throw new InvalidTypeException("text or varchar values must be enclosed by single quotes");
+
+            return ParseUtils.unquote(value);
+        }
+
+        @Override
+        public String format(String value)
+        {
+            if (value == null) return "NULL";
+            return ParseUtils.quote(value);
+        }
+
+        @Override
+        public ByteBuffer serialize(String value, ProtocolVersion protocolVersion)
+        {
+            return value == null ? null : ByteBuffer.wrap(value.getBytes(charset));
+        }
+
+        /**
+         * {@inheritDoc}
+         *
+         * <p>Implementation note: this method treats {@code null}s and empty buffers differently: the
+         * formers are mapped to {@code null}s while the latters are mapped to empty strings.
+         */
+        @Override
+        public String deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion)
+        {
+            if (bytes == null) return null;
+            if (bytes.remaining() == 0) return "";
+            return new String(Bytes.getArray(bytes), charset);
+        }
+    }
+
+    /**
+     * This codec maps a CQL {@link DataType#varchar()} to a Java {@link String}. Note that this codec
+     * also handles {@link DataType#text()}, which is merely an alias for {@link DataType#varchar()}.
+     */
+    private static class VarcharCodec extends StringCodec
+    {
+
+        private static final VarcharCodec instance = new VarcharCodec();
+
+        private VarcharCodec()
+        {
+            super(DataType.varchar(), Charset.forName("UTF-8"));
+        }
+    }
+
+    /**
+     * This codec maps a CQL {@link DataType#ascii()} to a Java {@link String}.
+     */
+    private static class AsciiCodec extends StringCodec
+    {
+
+        private static final AsciiCodec instance = new AsciiCodec();
+
+        private static final Pattern ASCII_PATTERN = Pattern.compile("^\\p{ASCII}*$");
+
+        private AsciiCodec()
+        {
+            super(DataType.ascii(), Charset.forName("US-ASCII"));
+        }
+
+        @Override
+        public ByteBuffer serialize(String value, ProtocolVersion protocolVersion)
+        {
+            if (value != null && !ASCII_PATTERN.matcher(value).matches())
+            {
+                throw new InvalidTypeException(String.format("%s is not a valid ASCII String", value));
+            }
+            return super.serialize(value, protocolVersion);
+        }
+
+        @Override
+        public String format(String value)
+        {
+            if (value != null && !ASCII_PATTERN.matcher(value).matches())
+            {
+                throw new InvalidTypeException(String.format("%s is not a valid ASCII String", value));
+            }
+            return super.format(value);
+        }
+    }
+
+    /**
+     * Base class for codecs handling CQL 8-byte integer types such as {@link DataType#bigint()},
+     * {@link DataType#counter()} or {@link DataType#time()}.
+     */
+    private abstract static class LongCodec extends PrimitiveLongCodec
+    {
+
+        private LongCodec(DataType cqlType)
+        {
+            super(cqlType);
+        }
+
+        @Override
+        public Long parse(String value)
+        {
+            try
+            {
+                return value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")
+                       ? null
+                       : Long.parseLong(value);
+            }
+            catch (NumberFormatException e)
+            {
+                throw new InvalidTypeException(
+                String.format("Cannot parse 64-bits long value from \"%s\"", value));
+            }
+        }
+
+        @Override
+        public String format(Long value)
+        {
+            if (value == null) return "NULL";
+            return Long.toString(value);
+        }
+
+        @Override
+        public ByteBuffer serializeNoBoxing(long value, ProtocolVersion protocolVersion)
+        {
+            ByteBuffer bb = ByteBuffer.allocate(8);
+            bb.putLong(0, value);
+            return bb;
+        }
+
+        @Override
+        public long deserializeNoBoxing(ByteBuffer bytes, ProtocolVersion protocolVersion)
+        {
+            if (bytes == null || bytes.remaining() == 0) return 0;
+            if (bytes.remaining() != 8)
+                throw new InvalidTypeException(
+                "Invalid 64-bits long value, expecting 8 bytes but got " + bytes.remaining());
+
+            return bytes.getLong(bytes.position());
+        }
+    }
+
+    /**
+     * This codec maps a CQL {@link DataType#bigint()} to a Java {@link Long}.
+     */
+    private static class BigintCodec extends LongCodec
+    {
+
+        private static final BigintCodec instance = new BigintCodec();
+
+        private BigintCodec()
+        {
+            super(DataType.bigint());
+        }
+    }
+
+    /**
+     * This codec maps a CQL {@link DataType#counter()} to a Java {@link Long}.
+     */
+    private static class CounterCodec extends LongCodec
+    {
+
+        private static final CounterCodec instance = new CounterCodec();
+
+        private CounterCodec()
+        {
+            super(DataType.counter());
+        }
+    }
+
+    /**
+     * This codec maps a CQL {@link DataType#blob()} to a Java {@link ByteBuffer}.
+     */
+    private static class BlobCodec extends TypeCodec<ByteBuffer>
+    {
+
+        private static final BlobCodec instance = new BlobCodec();
+
+        private BlobCodec()
+        {
+            super(DataType.blob(), ByteBuffer.class);
+        }
+
+        @Override
+        public ByteBuffer parse(String value)
+        {
+            return value == null || value.isEmpty() || value.equalsIgnoreCase("NULL")
+                   ? null
+                   : Bytes.fromHexString(value);
+        }
+
+        @Override
+        public String format(ByteBuffer value)
+        {
+            if (value == null) return "NULL";
+            return Bytes.toHexString(value);
+        }
+
+        @Override
+        public ByteBuffer serialize(ByteBuffer value, ProtocolVersion protocolVersion)
+        {
+            return value == null ? null : value.duplicate();
+        }
+
+        @Override
+        public ByteBuffer deserialize(ByteBuffer bytes, ProtocolVersion protocolVersion)
+        {
... 3628 lines suppressed ...


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