You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2016/11/21 17:55:49 UTC

[3/5] cassandra git commit: Add support for arithmetic operators

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java b/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java
new file mode 100644
index 0000000..1f115a9
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java
@@ -0,0 +1,380 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.OperationExecutionException;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * Operation functions (Mathematics).
+ *
+ */
+public final class OperationFcts
+{
+    private static enum OPERATION
+    {
+        ADDITION('+', "_add")
+        {
+            protected ByteBuffer execute(NumberType<?> resultType,
+                                         NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right)
+            {
+                return resultType.add(leftType, left, rightType, right);
+            }
+        },
+        SUBSTRACTION('-', "_substract")
+        {
+            protected ByteBuffer execute(NumberType<?> resultType,
+                                         NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right)
+            {
+                return resultType.substract(leftType, left, rightType, right);
+            }
+        },
+        MULTIPLICATION('*', "_multiply")
+        {
+            protected ByteBuffer execute(NumberType<?> resultType,
+                                         NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right)
+            {
+                return resultType.multiply(leftType, left, rightType, right);
+            }
+        },
+        DIVISION('/', "_divide")
+        {
+            protected ByteBuffer execute(NumberType<?> resultType,
+                                         NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right)
+            {
+                return resultType.divide(leftType, left, rightType, right);
+            }
+        },
+        MODULO('%', "_modulo")
+        {
+            protected ByteBuffer execute(NumberType<?> resultType,
+                                         NumberType<?> leftType,
+                                         ByteBuffer left,
+                                         NumberType<?> rightType,
+                                         ByteBuffer right)
+            {
+                return resultType.mod(leftType, left, rightType, right);
+            }
+        };
+
+        /**
+         * The operator symbol.
+         */
+        private final char symbol;
+
+        /**
+         * The name of the function associated to this operation
+         */
+        private final String functionName;
+
+        private OPERATION(char symbol, String functionName)
+        {
+            this.symbol = symbol;
+            this.functionName = functionName;
+        }
+
+        /**
+         * Executes the operation between the specified operand.
+         *
+         * @param resultType the result ype of the operation
+         * @param leftType the type of the left operand
+         * @param left the left operand
+         * @param rightType the type of the right operand
+         * @param right the right operand
+         * @return the operation result
+         */
+        protected abstract ByteBuffer execute(NumberType<?> resultType,
+                                              NumberType<?> leftType,
+                                              ByteBuffer left,
+                                              NumberType<?> rightType,
+                                              ByteBuffer right);
+
+        /**
+         * Returns the {@code OPERATOR} associated to the specified function.
+         * @param functionName the function name
+         * @return the {@code OPERATOR} associated to the specified function
+         */
+        public static OPERATION fromFunctionName(String functionName)
+        {
+            for (OPERATION operator : values())
+            {
+                if (operator.functionName.equals(functionName))
+                    return operator;
+            }
+            return null;
+        }
+
+        /**
+         * Returns the {@code OPERATOR} with the specified symbol.
+         * @param functionName the function name
+         * @return the {@code OPERATOR} with the specified symbol
+         */
+        public static OPERATION fromSymbol(char symbol)
+        {
+            for (OPERATION operator : values())
+            {
+                if (operator.symbol == symbol)
+                    return operator;
+            }
+            return null;
+        }
+    }
+
+    /**
+     * The name of the function used to perform negations
+     */
+    public static final String NEGATION_FUNCTION_NAME = "_negate";
+
+    public static Collection<Function> all()
+    {
+        List<Function> functions = new ArrayList<>();
+
+        final NumberType<?>[] numericTypes = new NumberType[] { ByteType.instance,
+                                                                ShortType.instance,
+                                                                Int32Type.instance,
+                                                                LongType.instance,
+                                                                FloatType.instance,
+                                                                DoubleType.instance,
+                                                                DecimalType.instance,
+                                                                IntegerType.instance,
+                                                                CounterColumnType.instance};
+
+        for (NumberType<?> left : numericTypes)
+        {
+            for (NumberType<?> right : numericTypes)
+            {
+                NumberType<?> returnType = returnType(left, right);
+                functions.add(new OperationFunction(returnType, left, OPERATION.ADDITION, right));
+                functions.add(new OperationFunction(returnType, left, OPERATION.SUBSTRACTION, right));
+                functions.add(new OperationFunction(returnType, left, OPERATION.MULTIPLICATION, right));
+                functions.add(new OperationFunction(returnType, left, OPERATION.DIVISION, right));
+                functions.add(new OperationFunction(returnType, left, OPERATION.MODULO, right));
+            }
+            functions.add(new NegationFunction(left));
+        }
+        return functions;
+    }
+
+    /**
+     * Checks if the function with the specified name is an operation.
+     *
+     * @param function the function name
+     * @return {@code true} if the function is an operation, {@code false} otherwise.
+     */
+    public static boolean isOperation(FunctionName function)
+    {
+        return SchemaConstants.SYSTEM_KEYSPACE_NAME.equals(function.keyspace)
+                && OPERATION.fromFunctionName(function.name) != null;
+    }
+
+    /**
+     * Checks if the function with the specified name is a negation.
+     *
+     * @param function the function name
+     * @return {@code true} if the function is an negation, {@code false} otherwise.
+     */
+    public static boolean isNegation(FunctionName function)
+    {
+        return SchemaConstants.SYSTEM_KEYSPACE_NAME.equals(function.keyspace)&& NEGATION_FUNCTION_NAME.equals(function.name);
+    }
+
+    /**
+     * Returns the operator associated to the specified function.
+     *
+     * @return the operator associated to the specified function.
+     */
+    public static char getOperator(FunctionName function)
+    {
+        assert SchemaConstants.SYSTEM_KEYSPACE_NAME.equals(function.keyspace);
+        return OPERATION.fromFunctionName(function.name).symbol;
+    }
+
+    /**
+     * Returns the name of the function associated to the specified operator.
+     *
+     * @param operator the operator
+     * @return the name of the function associated to the specified operator
+     */
+    public static FunctionName getFunctionNameFromOperator(char operator)
+    {
+        return FunctionName.nativeFunction(OPERATION.fromSymbol(operator).functionName);
+    }
+
+    /**
+     * Determine the return type for an operation between the specified types.
+     *
+     * @param left the type of the left operand
+     * @param right the type of the right operand
+     * @return the return type for an operation between the specified types
+     */
+    private static NumberType<?> returnType(NumberType<?> left, NumberType<?> right)
+    {
+        boolean isFloatingPoint = left.isFloatingPoint() || right.isFloatingPoint();
+        int size = Math.max(size(left), size(right));
+        return isFloatingPoint
+             ? floatPointType(size)
+             : integerType(size);
+    }
+
+    /**
+     * Returns the number of bytes used to represent a value of this type.
+     * @return the number of bytes used to represent a value of this type or {@code Integer.MAX} if the number of bytes
+     * is not limited.
+     */
+    private static int size(NumberType<?> type)
+    {
+        int size = type.valueLengthIfFixed();
+
+        if (size > 0)
+            return size;
+
+        if (type.isCounter())
+            return LongType.instance.valueLengthIfFixed();
+
+        return Integer.MAX_VALUE;
+    }
+
+    private static NumberType<?> floatPointType(int size)
+    {
+        switch (size)
+        {
+            case 4: return FloatType.instance;
+            case 8: return DoubleType.instance;
+            default: return DecimalType.instance;
+        }
+    }
+
+    private static NumberType<?> integerType(int size)
+    {
+        switch (size)
+        {
+            case 1: return ByteType.instance;
+            case 2: return ShortType.instance;
+            case 4: return Int32Type.instance;
+            case 8: return LongType.instance;
+            default: return IntegerType.instance;
+        }
+    }
+
+    /**
+     * The class must not be instantiated.
+     */
+    private OperationFcts()
+    {
+    }
+
+    /**
+     * Function that execute operations.
+     */
+    private static class OperationFunction extends NativeScalarFunction
+    {
+        private final OPERATION operation;
+
+        public OperationFunction(NumberType<?> returnType,
+                                 NumberType<?> left,
+                                 OPERATION operation,
+                                 NumberType<?> right)
+        {
+            super(operation.functionName, returnType, left, right);
+            this.operation = operation;
+        }
+
+        @Override
+        public final String columnName(List<String> columnNames)
+        {
+            return String.format("%s %s %s", columnNames.get(0), getOperator(), columnNames.get(1));
+        }
+
+        public final ByteBuffer execute(ProtocolVersion protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer left = parameters.get(0);
+            ByteBuffer right = parameters.get(1);
+            if (left == null || !left.hasRemaining() || right == null || !right.hasRemaining())
+                return null;
+
+            NumberType<?> leftType = (NumberType<?>) argTypes().get(0);
+            NumberType<?> rightType = (NumberType<?>) argTypes().get(1);
+            NumberType<?> resultType = (NumberType<?>) returnType();
+
+            try
+            {
+                return operation.execute(resultType, leftType, left, rightType, right);
+            }
+            catch (Exception e)
+            {
+                throw OperationExecutionException.create(getOperator(), argTypes, e);
+            }
+        }
+
+        /**
+         * Returns the operator symbol.
+         * @return the operator symbol
+         */
+        private char getOperator()
+        {
+            return operation.symbol;
+        }
+    }
+
+    /**
+     * Function that negate a number.
+     */
+    private static class NegationFunction extends NativeScalarFunction
+    {
+        public NegationFunction(NumberType<?> inputType)
+        {
+            super(NEGATION_FUNCTION_NAME, inputType, inputType);
+        }
+
+        @Override
+        public final String columnName(List<String> columnNames)
+        {
+            return String.format("-%s", columnNames.get(0));
+        }
+
+        public final ByteBuffer execute(ProtocolVersion protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer input = parameters.get(0);
+            if (input == null)
+                return null;
+
+            NumberType<?> inputType = (NumberType<?>) argTypes().get(0);
+
+            return inputType.negate(input);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java b/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
new file mode 100644
index 0000000..de5fd93
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
@@ -0,0 +1,91 @@
+/*
+ * 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.selection;
+
+import java.util.List;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selector.Factory;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+/**
+ * A base <code>Selector.Factory</code> for collections or tuples.
+ */
+abstract class CollectionFactory extends Factory
+{
+    /**
+     * The collection or tuple type.
+     */
+    private final AbstractType<?> type;
+
+    /**
+     * The collection or tuple element factories.
+     */
+    private final SelectorFactories factories;
+
+    public CollectionFactory(AbstractType<?> type, SelectorFactories factories)
+    {
+        this.type = type;
+        this.factories = factories;
+    }
+
+    protected final AbstractType<?> getReturnType()
+    {
+        return type;
+    }
+
+    @Override
+    public final void addFunctionsTo(List<Function> functions)
+    {
+        factories.addFunctionsTo(functions);
+    }
+
+    @Override
+    public final boolean isAggregateSelectorFactory()
+    {
+        return factories.doesAggregation();
+    }
+
+    @Override
+    public final boolean isWritetimeSelectorFactory()
+    {
+        return factories.containsWritetimeSelectorFactory();
+    }
+
+    @Override
+    public final boolean isTTLSelectorFactory()
+    {
+        return factories.containsTTLSelectorFactory();
+    }
+
+    protected final void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+    {
+        SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
+        for (Factory factory : factories)
+           factory.addColumnMapping(tmpMapping, resultsColumn);
+
+        if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
+            // add a null mapping for cases where the collection is empty
+            mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+        else
+            // collate the mapped columns from the child factories & add those
+            mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java b/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java
new file mode 100644
index 0000000..50b3ded
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java
@@ -0,0 +1,90 @@
+/*
+ * 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.selection;
+
+import java.util.List;
+
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selector.Factory;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A <code>Selector.Factory</code> which forwards all its method calls to another factory.
+ * Subclasses should override one or more methods to modify the behavior of the backing factory as desired per
+ * the decorator pattern.
+ */
+abstract class ForwardingFactory extends Factory
+{
+    /**
+     * Returns the backing delegate instance that methods are forwarded to.
+     */
+    protected abstract Factory delegate();
+
+    public Selector newInstance(QueryOptions options) throws InvalidRequestException
+    {
+        return delegate().newInstance(options);
+    }
+
+    protected String getColumnName()
+    {
+        return delegate().getColumnName();
+    }
+
+    protected AbstractType<?> getReturnType()
+    {
+        return delegate().getReturnType();
+    }
+
+    protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+    {
+        delegate().addColumnMapping(mapping, resultsColumn);
+    }
+
+    @Override
+    public void addFunctionsTo(List<Function> functions)
+    {
+        delegate().addFunctionsTo(functions);
+    }
+
+    @Override
+    public boolean isAggregateSelectorFactory()
+    {
+        return delegate().isAggregateSelectorFactory();
+    }
+
+    @Override
+    public boolean isWritetimeSelectorFactory()
+    {
+        return delegate().isWritetimeSelectorFactory();
+    }
+
+    @Override
+    public boolean isTTLSelectorFactory()
+    {
+        return delegate().isTTLSelectorFactory();
+    }
+
+    @Override
+    public boolean isSimpleSelectorFactory(int index)
+    {
+        return delegate().isSimpleSelectorFactory(index);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ListSelector.java b/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
new file mode 100644
index 0000000..940bd9c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
@@ -0,0 +1,104 @@
+/*
+ * 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.selection;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.CollectionSerializer;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * <code>Selector</code> for literal list (e.g. [min(value), max(value), count(value)]).
+ *
+ */
+final class ListSelector extends Selector
+{
+    /**
+     * The list type.
+     */
+    private final AbstractType<?> type;
+
+    /**
+     * The list elements
+     */
+    private final List<Selector> elements;
+
+    public static Factory newFactory(final AbstractType<?> type, final SelectorFactories factories)
+    {
+        return new CollectionFactory(type, factories)
+        {
+            protected String getColumnName()
+            {
+                return Lists.listToString(factories, Factory::getColumnName);
+            }
+
+            public Selector newInstance(final QueryOptions options)
+            {
+                return new ListSelector(type, factories.newInstances(options));
+            }
+        };
+    }
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).addInput(protocolVersion, rs);
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        List<ByteBuffer> buffers = new ArrayList<>(elements.size());
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            buffers.add(elements.get(i).getOutput(protocolVersion));
+        }
+        return CollectionSerializer.pack(buffers, buffers.size(), protocolVersion);
+    }
+
+    public void reset()
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).reset();
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return Lists.listToString(elements);
+    }
+
+    private ListSelector(AbstractType<?> type, List<Selector> elements)
+    {
+        this.type = type;
+        this.elements = elements;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/MapSelector.java b/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
new file mode 100644
index 0000000..8bbae8c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
@@ -0,0 +1,195 @@
+/*
+ * 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.selection;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.CollectionSerializer;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * <code>Selector</code> for literal map (e.g. {'min' : min(value), 'max' : max(value), 'count' : count(value)}).
+ *
+ */
+final class MapSelector extends Selector
+{
+    /**
+     * The map type.
+     */
+    private final MapType<?, ?> type;
+
+    /**
+     * The map elements
+     */
+    private final List<Pair<Selector, Selector>> elements;
+
+    public static Factory newFactory(final AbstractType<?> type, final List<Pair<Factory, Factory>> factories)
+    {
+        return new Factory()
+        {
+            protected String getColumnName()
+            {
+                return Maps.mapToString(factories, Factory::getColumnName);
+            }
+
+            protected AbstractType<?> getReturnType()
+            {
+                return type;
+            }
+
+            protected final void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+            {
+                SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    entry.left.addColumnMapping(tmpMapping, resultsColumn);
+                    entry.right.addColumnMapping(tmpMapping, resultsColumn);
+                }
+
+                if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
+                    // add a null mapping for cases where the collection is empty
+                    mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+                else
+                    // collate the mapped columns from the child factories & add those
+                    mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());
+            }
+
+            public Selector newInstance(final QueryOptions options)
+            {
+                return new MapSelector(type,
+                                        factories.stream()
+                                                 .map(p -> Pair.create(p.left.newInstance(options),
+                                                                       p.right.newInstance(options)))
+                                                 .collect(Collectors.toList()));
+            }
+
+            @Override
+            public boolean isAggregateSelectorFactory()
+            {
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    if (entry.left.isAggregateSelectorFactory() || entry.right.isAggregateSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+
+            @Override
+            public void addFunctionsTo(List<Function> functions)
+            {
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    entry.left.addFunctionsTo(functions);
+                    entry.right.addFunctionsTo(functions);
+                }
+            }
+
+            @Override
+            public boolean isWritetimeSelectorFactory()
+            {
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    if (entry.left.isWritetimeSelectorFactory() || entry.right.isWritetimeSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+
+            @Override
+            public boolean isTTLSelectorFactory()
+            {
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    if (entry.left.isTTLSelectorFactory() || entry.right.isTTLSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+        };
+    }
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            Pair<Selector, Selector> pair = elements.get(i);
+            pair.left.addInput(protocolVersion, rs);
+            pair.right.addInput(protocolVersion, rs);
+        }
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        Map<ByteBuffer, ByteBuffer> map = new TreeMap<>(type.getKeysType());
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            Pair<Selector, Selector> pair = elements.get(i);
+            map.put(pair.left.getOutput(protocolVersion), pair.right.getOutput(protocolVersion));
+        }
+
+        List<ByteBuffer> buffers = new ArrayList<>(elements.size() * 2);
+        for (Map.Entry<ByteBuffer, ByteBuffer> entry : map.entrySet())
+        {
+            buffers.add(entry.getKey());
+            buffers.add(entry.getValue());
+        }
+        return CollectionSerializer.pack(buffers, elements.size(), protocolVersion);
+    }
+
+    public void reset()
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            Pair<Selector, Selector> pair = elements.get(i);
+            pair.left.reset();
+            pair.right.reset();
+        }
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return Maps.mapToString(elements);
+    }
+
+    private MapSelector(AbstractType<?> type, List<Pair<Selector, Selector>> elements)
+    {
+        this.type = (MapType<?, ?>) type;
+        this.elements = elements;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
index c05cdaa..15e0210 100644
--- a/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
@@ -28,15 +28,6 @@ import org.apache.cassandra.transport.ProtocolVersion;
 
 final class ScalarFunctionSelector extends AbstractFunctionSelector<ScalarFunction>
 {
-    public boolean isAggregate()
-    {
-        // We cannot just return true as it is possible to have a scalar function wrapping an aggregation function
-        if (argSelectors.isEmpty())
-            return false;
-
-        return argSelectors.get(0).isAggregate();
-    }
-
     public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
     {
         for (int i = 0, m = argSelectors.size(); i < m; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
index 80e2ae8..20719ea 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@ -18,17 +18,22 @@
  */
 package org.apache.cassandra.cql3.selection;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
+import java.util.*;
+import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.text.StrBuilder;
+
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.cql3.selection.Selector.Factory;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.Pair;
+
+import static org.apache.cassandra.cql3.selection.SelectorFactories.createFactoriesAndCollectColumnDefinitions;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 
 public interface Selectable extends AssignmentTestable
 {
@@ -154,7 +159,7 @@ public interface Selectable extends AssignmentTestable
         @Override
         public String toString()
         {
-            return rawTerm.toString();
+            return rawTerm.getText();
         }
 
         public static class Raw extends Selectable.Raw
@@ -244,11 +249,7 @@ public interface Selectable extends AssignmentTestable
         @Override
         public String toString()
         {
-            return new StrBuilder().append(function.name())
-                                   .append("(")
-                                   .appendWithSeparators(args, ", ")
-                                   .append(")")
-                                   .toString();
+            return function.columnName(args.stream().map(Object::toString).collect(Collectors.toList()));
         }
 
         public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
@@ -279,6 +280,18 @@ public interface Selectable extends AssignmentTestable
                                Collections.emptyList());
             }
 
+            public static Raw newOperation(char operator, Selectable.Raw left, Selectable.Raw right)
+            {
+                return new Raw(OperationFcts.getFunctionNameFromOperator(operator),
+                               Arrays.asList(left, right));
+            }
+
+            public static Raw newNegation(Selectable.Raw arg)
+            {
+                return new Raw(FunctionName.nativeFunction(OperationFcts.NEGATION_FUNCTION_NAME),
+                               Collections.singletonList(arg));
+            }
+
             public Selectable prepare(CFMetaData cfm)
             {
                 List<Selectable> preparedArgs = new ArrayList<>(args.size());
@@ -433,7 +446,16 @@ public interface Selectable extends AssignmentTestable
 
         public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
         {
-            Selector.Factory factory = selected.newSelectorFactory(cfm, null, defs, boundNames);
+            AbstractType<?> expectedUdtType = null;
+
+            // If the UDT is between parentheses, we know that it is not a tuple with a single element.
+            if (selected instanceof BetweenParenthesesOrWithTuple)
+            {
+                BetweenParenthesesOrWithTuple betweenParentheses = (BetweenParenthesesOrWithTuple) selected;
+                expectedUdtType = betweenParentheses.selectables.get(0).getExactTypeIfKnown(cfm.ksName);
+            }
+
+            Selector.Factory factory = selected.newSelectorFactory(cfm, expectedUdtType, defs, boundNames);
             AbstractType<?> type = factory.getColumnSpecification(cfm).type;
             if (!type.isUDT())
             {
@@ -485,4 +507,609 @@ public interface Selectable extends AssignmentTestable
             }
         }
     }
+
+    /**
+     * {@code Selectable} for {@code Selectable} between parentheses or tuples.
+     * <p>The parser cannot differentiate between a single element between parentheses or a single element tuple.
+     * By consequence, we are forced to wait until the type is known to be able to differentiate them.</p>
+     */
+    public static class BetweenParenthesesOrWithTuple implements Selectable
+    {
+        /**
+         * The tuple elements or the element between the parentheses
+         */
+        private final List<Selectable> selectables;
+
+        public BetweenParenthesesOrWithTuple(List<Selectable> selectables)
+        {
+            this.selectables = selectables;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            if (selectables.size() == 1 && !receiver.type.isTuple())
+                return selectables.get(0).testAssignment(keyspace, receiver);
+
+            return Tuples.testTupleAssignment(receiver, selectables);
+        }
+
+        @Override
+        public Factory newSelectorFactory(CFMetaData cfm,
+                                          AbstractType<?> expectedType,
+                                          List<ColumnDefinition> defs,
+                                          VariableSpecifications boundNames)
+        {
+            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            if (type == null)
+            {
+                type = expectedType;
+                if (type == null)
+                    throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)",
+                                         this);
+            }
+
+            if (selectables.size() == 1 && !type.isTuple())
+                return newBetweenParenthesesSelectorFactory(cfm, expectedType, defs, boundNames);
+
+            return newTupleSelectorFactory(cfm, (TupleType) type, defs, boundNames);
+        }
+
+        private Factory newBetweenParenthesesSelectorFactory(CFMetaData cfm,
+                                                             AbstractType<?> expectedType,
+                                                             List<ColumnDefinition> defs,
+                                                             VariableSpecifications boundNames)
+        {
+            Selectable selectable = selectables.get(0);
+            final Factory factory = selectable.newSelectorFactory(cfm, expectedType, defs, boundNames);
+
+            return new ForwardingFactory()
+            {
+                protected Factory delegate()
+                {
+                    return factory;
+                }
+
+                protected String getColumnName()
+                {
+                    return String.format("(%s)", factory.getColumnName());
+                }
+            };
+        }
+
+        private Factory newTupleSelectorFactory(CFMetaData cfm,
+                                                TupleType tupleType,
+                                                List<ColumnDefinition> defs,
+                                                VariableSpecifications boundNames)
+        {
+            SelectorFactories factories = createFactoriesAndCollectColumnDefinitions(selectables,
+                                                                                     tupleType.allTypes(),
+                                                                                     cfm,
+                                                                                     defs,
+                                                                                     boundNames);
+
+            return TupleSelector.newFactory(tupleType, factories);
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            // If there is only one element we cannot know if it is an element between parentheses or a tuple
+            // with only one element. By consequence, we need to force the user to specify the type.
+            if (selectables.size() == 1)
+                return null;
+
+            return Tuples.getExactTupleTypeIfKnown(selectables, p -> p.getExactTypeIfKnown(keyspace));
+        }
+
+        @Override
+        public String toString()
+        {
+            return Tuples.tupleToString(selectables);
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final List<Selectable.Raw> raws;
+
+            public Raw(List<Selectable.Raw> raws)
+            {
+                this.raws = raws;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                return new BetweenParenthesesOrWithTuple(raws.stream().map(p -> p.prepare(cfm)).collect(Collectors.toList()));
+            }
+        }
+    }
+
+    /**
+     * <code>Selectable</code> for literal Lists.
+     */
+    public static class WithList implements Selectable
+    {
+        /**
+         * The list elements
+         */
+        private final List<Selectable> selectables;
+
+        public WithList(List<Selectable> selectables)
+        {
+            this.selectables = selectables;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            return Lists.testListAssignment(receiver, selectables);
+        }
+
+        @Override
+        public Factory newSelectorFactory(CFMetaData cfm,
+                                          AbstractType<?> expectedType,
+                                          List<ColumnDefinition> defs,
+                                          VariableSpecifications boundNames)
+        {
+            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            if (type == null)
+            {
+                type = expectedType;
+                if (type == null)
+                    throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)",
+                                         this);
+            }
+
+            ListType<?> listType = (ListType<?>) type;
+
+            List<AbstractType<?>> expectedTypes = new ArrayList<>(selectables.size());
+            for (int i = 0, m = selectables.size(); i < m; i++)
+                expectedTypes.add(listType.getElementsType());
+
+            SelectorFactories factories = createFactoriesAndCollectColumnDefinitions(selectables,
+                                                                                     expectedTypes,
+                                                                                     cfm,
+                                                                                     defs,
+                                                                                     boundNames);
+            return ListSelector.newFactory(type, factories);
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return Lists.getExactListTypeIfKnown(selectables, p -> p.getExactTypeIfKnown(keyspace));
+        }
+
+        @Override
+        public String toString()
+        {
+            return Lists.listToString(selectables);
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final List<Selectable.Raw> raws;
+
+            public Raw(List<Selectable.Raw> raws)
+            {
+                this.raws = raws;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                return new WithList(raws.stream().map(p -> p.prepare(cfm)).collect(Collectors.toList()));
+            }
+        }
+    }
+
+    /**
+     * <code>Selectable</code> for literal Sets.
+     */
+    public static class WithSet implements Selectable
+    {
+        /**
+         * The set elements
+         */
+        private final List<Selectable> selectables;
+
+        public WithSet(List<Selectable> selectables)
+        {
+            this.selectables = selectables;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            return Sets.testSetAssignment(receiver, selectables);
+        }
+
+        @Override
+        public Factory newSelectorFactory(CFMetaData cfm,
+                                          AbstractType<?> expectedType,
+                                          List<ColumnDefinition> defs,
+                                          VariableSpecifications boundNames)
+        {
+            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            if (type == null)
+            {
+                type = expectedType;
+                if (type == null)
+                    throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)",
+                                         this);
+            }
+
+            // The parser treats empty Maps as Sets so if the type is a MapType we know that the Map is empty
+            if (type instanceof MapType)
+                return MapSelector.newFactory(type, Collections.emptyList());
+
+            SetType<?> setType = (SetType<?>) type;
+
+            if (setType.getElementsType() == DurationType.instance)
+                throw invalidRequest("Durations are not allowed inside sets: %s", setType.asCQL3Type());
+
+            List<AbstractType<?>> expectedTypes = new ArrayList<>(selectables.size());
+            for (int i = 0, m = selectables.size(); i < m; i++)
+                expectedTypes.add(setType.getElementsType());
+
+            SelectorFactories factories = createFactoriesAndCollectColumnDefinitions(selectables,
+                                                                                     expectedTypes,
+                                                                                     cfm,
+                                                                                     defs,
+                                                                                     boundNames);
+
+            return SetSelector.newFactory(type, factories);
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return Sets.getExactSetTypeIfKnown(selectables, p -> p.getExactTypeIfKnown(keyspace));
+        }
+
+        @Override
+        public String toString()
+        {
+            return Sets.setToString(selectables);
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final List<Selectable.Raw> raws;
+
+            public Raw(List<Selectable.Raw> raws)
+            {
+                this.raws = raws;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                return new WithSet(raws.stream().map(p -> p.prepare(cfm)).collect(Collectors.toList()));
+            }
+        }
+    }
+
+    /**
+     * {@code Selectable} for literal Maps or UDTs.
+     * <p>The parser cannot differentiate between a Map or a UDT in the selection cause because a
+     * {@code ColumnDefinition} is equivalent to a {@code FieldIdentifier} from a syntax point of view.
+     * By consequence, we are forced to wait until the type is known to be able to differentiate them.</p>
+     */
+    public static class WithMapOrUdt implements Selectable
+    {
+        /**
+         * The column family metadata. We need to store them to be able to build the proper data once the type has been
+         * identified.
+         */
+        private final CFMetaData cfm;
+
+        /**
+         * The Map or UDT raw elements.
+         */
+        private final List<Pair<Selectable.Raw, Selectable.Raw>> raws;
+
+        public WithMapOrUdt(CFMetaData cfm, List<Pair<Selectable.Raw, Selectable.Raw>> raws)
+        {
+            this.cfm = cfm;
+            this.raws = raws;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            return receiver.type.isUDT() ? UserTypes.testUserTypeAssignment(receiver, getUdtFields((UserType) receiver.type))
+                                         : Maps.testMapAssignment(receiver, getMapEntries(cfm));
+        }
+
+        @Override
+        public Factory newSelectorFactory(CFMetaData cfm,
+                                          AbstractType<?> expectedType,
+                                          List<ColumnDefinition> defs,
+                                          VariableSpecifications boundNames)
+        {
+            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            if (type == null)
+            {
+                type = expectedType;
+                if (type == null)
+                    throw invalidRequest("Cannot infer type for term %s in selection clause (try using a cast to force a type)",
+                                         this);
+            }
+
+            if (type.isUDT())
+                return newUdtSelectorFactory(cfm, expectedType, defs, boundNames);
+
+            return newMapSelectorFactory(cfm, defs, boundNames, type);
+        }
+
+        private Factory newMapSelectorFactory(CFMetaData cfm,
+                                              List<ColumnDefinition> defs,
+                                              VariableSpecifications boundNames,
+                                              AbstractType<?> type)
+        {
+            MapType<?, ?> mapType = (MapType<?, ?>) type;
+
+            if (mapType.getKeysType() == DurationType.instance)
+                throw invalidRequest("Durations are not allowed as map keys: %s", mapType.asCQL3Type());
+
+            return MapSelector.newFactory(type, getMapEntries(cfm).stream()
+                                                                  .map(p -> Pair.create(p.left.newSelectorFactory(cfm, mapType.getKeysType(), defs, boundNames),
+                                                                                        p.right.newSelectorFactory(cfm, mapType.getValuesType(), defs, boundNames)))
+                                                                  .collect(Collectors.toList()));
+        }
+
+        private Factory newUdtSelectorFactory(CFMetaData cfm,
+                                              AbstractType<?> expectedType,
+                                              List<ColumnDefinition> defs,
+                                              VariableSpecifications boundNames)
+        {
+            UserType ut = (UserType) expectedType;
+            Map<FieldIdentifier, Factory> factories = new LinkedHashMap<>(ut.size());
+
+            for (Pair<Selectable.Raw, Selectable.Raw> raw : raws)
+            {
+                if (!(raw.left instanceof RawIdentifier))
+                    throw invalidRequest("%s is not a valid field identifier of type %s ",
+                                         raw.left,
+                                         ut.getNameAsString());
+
+                FieldIdentifier fieldName = ((RawIdentifier) raw.left).toFieldIdentifier();
+                int fieldPosition = ut.fieldPosition(fieldName);
+
+                if (fieldPosition == -1)
+                    throw invalidRequest("Unknown field '%s' in value of user defined type %s",
+                                         fieldName,
+                                         ut.getNameAsString());
+
+                AbstractType<?> fieldType = ut.fieldType(fieldPosition);
+                factories.put(fieldName,
+                              raw.right.prepare(cfm).newSelectorFactory(cfm, fieldType, defs, boundNames));
+            }
+
+            return UserTypeSelector.newFactory(expectedType, factories);
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            // Lets force the user to specify the type.
+            return null;
+        }
+
+        @Override
+        public String toString()
+        {
+            return raws.stream()
+                       .map(p -> String.format("%s: %s",
+                                               p.left instanceof RawIdentifier ? p.left : p.left.prepare(cfm),
+                                               p.right.prepare(cfm)))
+                       .collect(Collectors.joining(", ", "{", "}"));
+        }
+
+        private List<Pair<Selectable, Selectable>> getMapEntries(CFMetaData cfm)
+        {
+            return raws.stream()
+                       .map(p -> Pair.create(p.left.prepare(cfm), p.right.prepare(cfm)))
+                       .collect(Collectors.toList());
+        }
+
+        private Map<FieldIdentifier, Selectable> getUdtFields(UserType ut)
+        {
+            Map<FieldIdentifier, Selectable> fields = new LinkedHashMap<>(ut.size());
+
+            for (Pair<Selectable.Raw, Selectable.Raw> raw : raws)
+            {
+                if (!(raw.left instanceof RawIdentifier))
+                    throw invalidRequest("%s is not a valid field identifier of type %s ",
+                                         raw.left,
+                                         ut.getNameAsString());
+
+                FieldIdentifier fieldName = ((RawIdentifier) raw.left).toFieldIdentifier();
+                int fieldPosition = ut.fieldPosition(fieldName);
+
+                if (fieldPosition == -1)
+                    throw invalidRequest("Unknown field '%s' in value of user defined type %s",
+                                         fieldName,
+                                         ut.getNameAsString());
+
+                fields.put(fieldName, raw.right.prepare(cfm));
+            }
+
+            return fields;
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final List<Pair<Selectable.Raw, Selectable.Raw>> raws;
+
+            public Raw(List<Pair<Selectable.Raw, Selectable.Raw>> raws)
+            {
+                this.raws = raws;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                return new WithMapOrUdt(cfm, raws);
+            }
+        }
+    }
+
+    /**
+     * <code>Selectable</code> for type hints (e.g. (int) ?).
+     */
+    public static class WithTypeHint implements Selectable
+    {
+
+        /**
+         * The name of the type as specified in the query.
+         */
+        private final String typeName;
+
+        /**
+         * The type specified by the hint.
+         */
+        private final AbstractType<?> type;
+
+       /**
+         * The selectable to which the hint should be applied.
+         */
+        private final Selectable selectable;
+
+        public WithTypeHint(String typeName, AbstractType<?> type, Selectable selectable)
+        {
+            this.typeName = typeName;
+            this.type = type;
+            this.selectable = selectable;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            if (receiver.type.equals(type))
+                return AssignmentTestable.TestResult.EXACT_MATCH;
+            else if (receiver.type.isValueCompatibleWith(type))
+                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+            else
+                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+        }
+
+        @Override
+        public Factory newSelectorFactory(CFMetaData cfm,
+                                          AbstractType<?> expectedType,
+                                          List<ColumnDefinition> defs,
+                                          VariableSpecifications boundNames)
+        {
+            final ColumnSpecification receiver = new ColumnSpecification(cfm.ksName, cfm.cfName, new ColumnIdentifier(toString(), true), type);
+
+            if (!selectable.testAssignment(cfm.ksName, receiver).isAssignable())
+                throw new InvalidRequestException(String.format("Cannot assign value %s to %s of type %s", this, receiver.name, receiver.type.asCQL3Type()));
+
+            final Factory factory = selectable.newSelectorFactory(cfm, type, defs, boundNames);
+
+            return new ForwardingFactory()
+            {
+                protected Factory delegate()
+                {
+                    return factory;
+                }
+
+                protected AbstractType<?> getReturnType()
+                {
+                    return type;
+                }
+
+                protected String getColumnName()
+                {
+                    return String.format("(%s)%s", typeName, factory.getColumnName());
+                }
+            };
+        }
+
+        @Override
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            return type;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("(%s)%s", typeName, selectable);
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final CQL3Type.Raw typeRaw;
+
+            private final Selectable.Raw raw;
+
+            public Raw( CQL3Type.Raw typeRaw, Selectable.Raw raw)
+            {
+                this.typeRaw = typeRaw;
+                this.raw = raw;
+            }
+
+            public Selectable prepare(CFMetaData cfm)
+            {
+                Selectable selectable = raw.prepare(cfm);
+                AbstractType<?> type = this.typeRaw.prepare(cfm.ksName).getType();
+                if (type.isFreezable())
+                    type = type.freeze();
+                return new WithTypeHint(typeRaw.toString(), type, selectable);
+            }
+        }
+    }
+
+    /**
+     * In the selection clause, the parser cannot differentiate between Maps and UDTs as a column identifier and field
+     * identifier have the same syntax. By consequence, we need to wait until the type is known to create the proper
+     * Object: {@code ColumnDefinition} or {@code FieldIdentifier}.
+     */
+    public static final class RawIdentifier extends Selectable.Raw
+    {
+        private final String text;
+
+        private final boolean quoted;
+
+        /**
+         * Creates a {@code RawIdentifier} from an unquoted identifier string.
+         */
+        public static Raw forUnquoted(String text)
+        {
+            return new RawIdentifier(text, false);
+        }
+
+        /**
+         * Creates a {@code RawIdentifier} from a quoted identifier string.
+         */
+        public static Raw forQuoted(String text)
+        {
+            return new RawIdentifier(text, true);
+        }
+
+        private RawIdentifier(String text, boolean quoted)
+        {
+            this.text = text;
+            this.quoted = quoted;
+        }
+
+        @Override
+        public Selectable prepare(CFMetaData cfm)
+        {
+            ColumnDefinition.Raw raw = quoted ? ColumnDefinition.Raw.forQuoted(text)
+                                              : ColumnDefinition.Raw.forUnquoted(text);
+            return raw.prepare(cfm);
+        }
+
+        public FieldIdentifier toFieldIdentifier()
+        {
+            return quoted ? FieldIdentifier.forQuoted(text)
+                          : FieldIdentifier.forUnquoted(text);
+        }
+
+        @Override
+        public String toString()
+        {
+            return text;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/Selector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selector.java b/src/java/org/apache/cassandra/cql3/selection/Selector.java
index 922b57f..6f83dfc 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selector.java
@@ -172,17 +172,6 @@ public abstract class Selector
     public abstract AbstractType<?> getType();
 
     /**
-     * Checks if this <code>Selector</code> is creating aggregates.
-     *
-     * @return <code>true</code> if this <code>Selector</code> is creating aggregates <code>false</code>
-     * otherwise.
-     */
-    public boolean isAggregate()
-    {
-        return false;
-    }
-
-    /**
      * Reset the internal state of this <code>Selector</code>.
      */
     public abstract void reset();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/SetSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SetSelector.java b/src/java/org/apache/cassandra/cql3/selection/SetSelector.java
new file mode 100644
index 0000000..34de078
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/SetSelector.java
@@ -0,0 +1,106 @@
+/*
+ * 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.selection;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.CollectionSerializer;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * <code>Selector</code> for literal set (e.g. {min(value), max(value), count(value)}).
+ *
+ */
+final class SetSelector extends Selector
+{
+    /**
+     * The set type.
+     */
+    private final SetType<?> type;
+
+    /**
+     * The set elements
+     */
+    private final List<Selector> elements;
+
+    public static Factory newFactory(final AbstractType<?> type, final SelectorFactories factories)
+    {
+        return new CollectionFactory(type, factories)
+        {
+            protected String getColumnName()
+            {
+                return Sets.setToString(factories, Factory::getColumnName);
+            }
+
+            public Selector newInstance(final QueryOptions options)
+            {
+                return new SetSelector(type, factories.newInstances(options));
+            }
+        };
+    }
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).addInput(protocolVersion, rs);
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        Set<ByteBuffer> buffers = new TreeSet<>(type.getElementsType());
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            buffers.add(elements.get(i).getOutput(protocolVersion));
+        }
+        return CollectionSerializer.pack(buffers, buffers.size(), protocolVersion);
+    }
+
+    public void reset()
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).reset();
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return Sets.setToString(elements);
+    }
+
+    private SetSelector(AbstractType<?> type, List<Selector> elements)
+    {
+        this.type = (SetType<?>) type;
+        this.elements = elements;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/TupleSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/TupleSelector.java b/src/java/org/apache/cassandra/cql3/selection/TupleSelector.java
new file mode 100644
index 0000000..9f4c381
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/TupleSelector.java
@@ -0,0 +1,101 @@
+/*
+ * 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.selection;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Tuples;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * <code>Selector</code> for literal tuples (e.g. (min(value), max(value), count(value))).
+ *
+ */
+final class TupleSelector extends Selector
+{
+    /**
+     * The tuple type.
+     */
+    private final AbstractType<?> type;
+
+    /**
+     * The tuple elements
+     */
+    private final List<Selector> elements;
+
+    public static Factory newFactory(final AbstractType<?> type, final SelectorFactories factories)
+    {
+        return new CollectionFactory(type, factories)
+        {
+            protected String getColumnName()
+            {
+                return Tuples.tupleToString(factories, Factory::getColumnName);
+            }
+
+            public Selector newInstance(final QueryOptions options)
+            {
+                return new TupleSelector(type, factories.newInstances(options));
+            }
+        };
+    }
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).addInput(protocolVersion, rs);
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        ByteBuffer[] buffers = new ByteBuffer[elements.size()];
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            buffers[i] = elements.get(i).getOutput(protocolVersion);
+        }
+        return TupleType.buildValue(buffers);
+    }
+
+    public void reset()
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).reset();
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return Tuples.tupleToString(elements);
+    }
+
+    private TupleSelector(AbstractType<?> type, List<Selector> elements)
+    {
+        this.type = type;
+        this.elements = elements;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java b/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java
new file mode 100644
index 0000000..3c298b5
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java
@@ -0,0 +1,177 @@
+/*
+ * 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.selection;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.FieldIdentifier;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.UserTypes;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+/**
+ * <code>Selector</code> for literal map (e.g. {'min' : min(value), 'max' : max(value), 'count' : count(value)}).
+ *
+ */
+final class UserTypeSelector extends Selector
+{
+    /**
+     * The map type.
+     */
+    private final AbstractType<?> type;
+
+    /**
+     * The user type fields
+     */
+    private final Map<FieldIdentifier, Selector> fields;
+
+    public static Factory newFactory(final AbstractType<?> type, final Map<FieldIdentifier, Factory> factories)
+    {
+        return new Factory()
+        {
+            protected String getColumnName()
+            {
+                return UserTypes.userTypeToString(factories, Factory::getColumnName);
+            }
+
+            protected AbstractType<?> getReturnType()
+            {
+                return type;
+            }
+
+            protected final void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+            {
+                SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
+                for (Factory factory : factories.values())
+                {
+                    factory.addColumnMapping(tmpMapping, resultsColumn);
+                }
+
+                if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
+                    // add a null mapping for cases where the collection is empty
+                    mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+                else
+                    // collate the mapped columns from the child factories & add those
+                    mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());
+            }
+
+            public Selector newInstance(final QueryOptions options)
+            {
+                Map<FieldIdentifier, Selector> fields = new HashMap<>(factories.size());
+                for (Entry<FieldIdentifier, Factory> factory : factories.entrySet())
+                    fields.put(factory.getKey(), factory.getValue().newInstance(options));
+
+                return new UserTypeSelector(type, fields);
+            }
+
+            @Override
+            public boolean isAggregateSelectorFactory()
+            {
+                for (Factory factory : factories.values())
+                {
+                    if (factory.isAggregateSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+
+            @Override
+            public void addFunctionsTo(List<Function> functions)
+            {
+                for (Factory factory : factories.values())
+                    factory.addFunctionsTo(functions);
+            }
+
+            @Override
+            public boolean isWritetimeSelectorFactory()
+            {
+                for (Factory factory : factories.values())
+                {
+                    if (factory.isWritetimeSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+
+            @Override
+            public boolean isTTLSelectorFactory()
+            {
+                for (Factory factory : factories.values())
+                {
+                    if (factory.isTTLSelectorFactory())
+                        return true;
+                }
+                return false;
+            }
+        };
+    }
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (Selector field : fields.values())
+            field.addInput(protocolVersion, rs);
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        UserType userType = (UserType) type;
+        ByteBuffer[] buffers = new ByteBuffer[userType.size()];
+        for (int i = 0, m = userType.size(); i < m; i++)
+        {
+            Selector selector = fields.get(userType.fieldName(i));
+            if (selector != null)
+                buffers[i] = selector.getOutput(protocolVersion);
+        }
+        return TupleType.buildValue(buffers);
+    }
+
+    public void reset()
+    {
+        for (Selector field : fields.values())
+            field.reset();
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return UserTypes.userTypeToString(fields);
+    }
+
+    private UserTypeSelector(AbstractType<?> type, Map<FieldIdentifier, Selector> fields)
+    {
+        this.type = type;
+        this.fields = fields;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 7f6b6ab..31a461b 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -477,6 +477,7 @@ public final class SystemKeyspace
                         .add(BytesConversionFcts.all())
                         .add(AggregateFcts.all())
                         .add(CastFcts.all())
+                        .add(OperationFcts.all())
                         .build();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/AbstractType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index 37a1959..99df8a2 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -319,6 +319,11 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
         return false;
     }
 
+    public boolean isTuple()
+    {
+        return false;
+    }
+
     public boolean isMultiCell()
     {
         return false;
@@ -384,7 +389,7 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
     /**
      * The length of values for this type if all values are of fixed length, -1 otherwise.
      */
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return -1;
     }
@@ -487,9 +492,13 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>, Assignm
 
     public final AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
     {
+        return testAssignment(receiver.type);
+    }
+
+    public final AssignmentTestable.TestResult testAssignment(AbstractType<?> receiverType)
+    {
         // We should ignore the fact that the output type is frozen in our comparison as functions do not support
         // frozen types for arguments
-        AbstractType<?> receiverType = receiver.type;
         if (isFreezable() && !isMultiCell())
             receiverType = receiverType.freeze();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/BooleanType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/BooleanType.java b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
index 1dbd1af..475cae6 100644
--- a/src/java/org/apache/cassandra/db/marshal/BooleanType.java
+++ b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
@@ -97,7 +97,7 @@ public class BooleanType extends AbstractType<Boolean>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 1;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/ByteType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteType.java b/src/java/org/apache/cassandra/db/marshal/ByteType.java
index 55aea8f..517e734 100644
--- a/src/java/org/apache/cassandra/db/marshal/ByteType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ByteType.java
@@ -28,7 +28,7 @@ import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class ByteType extends AbstractType<Byte>
+public class ByteType extends NumberType<Byte>
 {
     public static final ByteType instance = new ByteType();
 
@@ -84,8 +84,62 @@ public class ByteType extends AbstractType<Byte>
     }
 
     @Override
+    public int valueLengthIfFixed()
+    {
+        return 1;
+    }
+
+    @Override
     public TypeSerializer<Byte> getSerializer()
     {
         return ByteSerializer.instance;
     }
+
+    @Override
+    public byte toByte(ByteBuffer value)
+    {
+        return ByteBufferUtil.toByte(value);
+    }
+
+    @Override
+    public short toShort(ByteBuffer value)
+    {
+        return toByte(value);
+    }
+
+    @Override
+    protected int toInt(ByteBuffer value)
+    {
+        return toByte(value);
+    }
+
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((byte) (leftType.toByte(left) + rightType.toByte(right)));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((byte) (leftType.toByte(left) - rightType.toByte(right)));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((byte) (leftType.toByte(left) * rightType.toByte(right)));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((byte) (leftType.toByte(left) / rightType.toByte(right)));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes((byte) (leftType.toByte(left) % rightType.toByte(right)));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return ByteBufferUtil.bytes((byte) -toByte(input));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
index 8bb1a25..8777e0e 100644
--- a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
@@ -22,13 +22,13 @@ import java.nio.ByteBuffer;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.db.context.CounterContext;
-import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.CounterSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class CounterColumnType extends AbstractType<Long>
+public class CounterColumnType extends NumberType<Long>
 {
     public static final CounterColumnType instance = new CounterColumnType();
 
@@ -93,4 +93,40 @@ public class CounterColumnType extends AbstractType<Long>
     {
         return CounterSerializer.instance;
     }
+
+    @Override
+    protected long toLong(ByteBuffer value)
+    {
+        return ByteBufferUtil.toLong(value);
+    }
+
+    public ByteBuffer add(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) + rightType.toLong(right));
+    }
+
+    public ByteBuffer substract(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) - rightType.toLong(right));
+    }
+
+    public ByteBuffer multiply(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) * rightType.toLong(right));
+    }
+
+    public ByteBuffer divide(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) / rightType.toLong(right));
+    }
+
+    public ByteBuffer mod(NumberType<?> leftType, ByteBuffer left, NumberType<?> rightType, ByteBuffer right)
+    {
+        return ByteBufferUtil.bytes(leftType.toLong(left) % rightType.toLong(right));
+    }
+
+    public ByteBuffer negate(ByteBuffer input)
+    {
+        return ByteBufferUtil.bytes(-toLong(input));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8b3de2f4/src/java/org/apache/cassandra/db/marshal/DateType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/DateType.java b/src/java/org/apache/cassandra/db/marshal/DateType.java
index 87b2cad..473cedf 100644
--- a/src/java/org/apache/cassandra/db/marshal/DateType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DateType.java
@@ -119,7 +119,7 @@ public class DateType extends AbstractType<Date>
     }
 
     @Override
-    protected int valueLengthIfFixed()
+    public int valueLengthIfFixed()
     {
         return 8;
     }