cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From slebre...@apache.org
Subject [4/5] CQL3 refactor to allow conversion function
Date Fri, 08 Feb 2013 17:45:59 GMT
http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/Maps.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Maps.java b/src/java/org/apache/cassandra/cql3/Maps.java
new file mode 100644
index 0000000..e62c101
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/Maps.java
@@ -0,0 +1,271 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.MarshalException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Static helper methods and classes for maps.
+ */
+public abstract class Maps
+{
+    private Maps() {}
+
+    public static ColumnSpecification keySpecOf(ColumnSpecification column)
+    {
+        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("key(" + column.name + ")", true), ((MapType)column.type).keys);
+    }
+
+    public static ColumnSpecification valueSpecOf(ColumnSpecification column)
+    {
+        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((MapType)column.type).values);
+    }
+
+    public static class Literal implements Term.Raw
+    {
+        public final List<Pair<Term.Raw, Term.Raw>> entries;
+
+        public Literal(List<Pair<Term.Raw, Term.Raw>> entries)
+        {
+            this.entries = entries;
+        }
+
+        public Value prepare(ColumnSpecification receiver) throws InvalidRequestException
+        {
+            validateAssignableTo(receiver);
+
+            ColumnSpecification keySpec = Maps.keySpecOf(receiver);
+            ColumnSpecification valueSpec = Maps.valueSpecOf(receiver);
+            Map<ByteBuffer, ByteBuffer> values = new TreeMap<ByteBuffer, ByteBuffer>(((MapType)receiver.type).keys);
+            for (Pair<Term.Raw, Term.Raw> entry : entries)
+            {
+                Term k = entry.left.prepare(keySpec);
+                Term v = entry.right.prepare(valueSpec);
+
+                if (!(k instanceof Constants.Value && v instanceof Constants.Value))
+                {
+                    if (k instanceof Term.NonTerminal || v instanceof Term.NonTerminal)
+                        throw new InvalidRequestException(String.format("Invalid map literal for %s: bind variables are not supported inside collection literals", receiver));
+                    else
+                        throw new InvalidRequestException(String.format("Invalid map literal for %s: nested collections are not supported", receiver));
+                }
+
+                if (values.put(((Constants.Value)k).bytes, ((Constants.Value)v).bytes) != null)
+                    throw new InvalidRequestException(String.format("Invalid map literal: duplicate entry for key %s", entry.left));
+            }
+            return new Value(values);
+        }
+
+        private void validateAssignableTo(ColumnSpecification receiver) throws InvalidRequestException
+        {
+            if (!(receiver.type instanceof MapType))
+                throw new InvalidRequestException(String.format("Invalid map literal for %s of type %s", receiver, receiver.type.asCQL3Type()));
+
+            ColumnSpecification keySpec = Maps.keySpecOf(receiver);
+            ColumnSpecification valueSpec = Maps.valueSpecOf(receiver);
+            for (Pair<Term.Raw, Term.Raw> entry : entries)
+            {
+                if (!entry.left.isAssignableTo(keySpec))
+                    throw new InvalidRequestException(String.format("Invalid map literal for %s: key %s is not of type %s", receiver, entry.left, keySpec.type.asCQL3Type()));
+                if (!entry.right.isAssignableTo(valueSpec))
+                    throw new InvalidRequestException(String.format("Invalid map literal for %s: value %s is not of type %s", receiver, entry.right, valueSpec.type.asCQL3Type()));
+            }
+        }
+
+        public boolean isAssignableTo(ColumnSpecification receiver)
+        {
+            try
+            {
+                validateAssignableTo(receiver);
+                return true;
+            }
+            catch (InvalidRequestException e)
+            {
+                return false;
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            StringBuilder sb = new StringBuilder();
+            sb.append("{");
+            for (int i = 0; i < entries.size(); i++)
+            {
+                if (i > 0) sb.append(", ");
+                sb.append(entries.get(i).left).append(":").append(entries.get(i).right);
+            }
+            sb.append("}");
+            return sb.toString();
+        }
+    }
+
+    public static class Value extends Term.Terminal
+    {
+        public final Map<ByteBuffer, ByteBuffer> map;
+
+        public Value(Map<ByteBuffer, ByteBuffer> map)
+        {
+            this.map = map;
+        }
+
+        public static Value fromSerialized(ByteBuffer value, MapType type) throws InvalidRequestException
+        {
+            try
+            {
+                // Collections have this small hack that validate cannot be called on a serialized object,
+                // but compose does the validation (so we're fine).
+                Map<?, ?> m = type.compose(value);
+                Map<ByteBuffer, ByteBuffer> map = new LinkedHashMap<ByteBuffer, ByteBuffer>(m.size());
+                for (Map.Entry<?, ?> entry : m.entrySet())
+                    map.put(type.keys.decompose(entry.getKey()), type.values.decompose(entry.getValue()));
+                return new Value(map);
+            }
+            catch (MarshalException e)
+            {
+                throw new InvalidRequestException(e.getMessage());
+            }
+        }
+
+        public ByteBuffer get()
+        {
+            List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(2 * map.size());
+            for (Map.Entry<ByteBuffer, ByteBuffer> entry : map.entrySet())
+            {
+                buffers.add(entry.getKey());
+                buffers.add(entry.getValue());
+            }
+            return CollectionType.pack(buffers, map.size());
+        }
+    }
+
+    public static class Marker extends AbstractMarker
+    {
+        protected Marker(int bindIndex, ColumnSpecification receiver)
+        {
+            super(bindIndex, receiver);
+            assert receiver.type instanceof MapType;
+        }
+
+        public Value bind(List<ByteBuffer> values) throws InvalidRequestException
+        {
+            ByteBuffer value = values.get(bindIndex);
+            return Value.fromSerialized(value, (MapType)receiver.type);
+        }
+    }
+
+    public static class Setter extends Operation
+    {
+        public Setter(ColumnIdentifier column, Term t)
+        {
+            super(column, t);
+        }
+
+        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
+        {
+            // delete + put
+            ColumnNameBuilder column = prefix.add(columnName.key);
+            cf.addAtom(params.makeTombstoneForOverwrite(column.build(), column.buildAsEndOfRange()));
+            Putter.doPut(t, cf, column, params);
+        }
+    }
+
+    public static class SetterByKey extends Operation
+    {
+        private final Term k;
+
+        public SetterByKey(ColumnIdentifier column, Term k, Term t)
+        {
+            super(column, t);
+            this.k = k;
+        }
+
+        @Override
+        public void collectMarkerSpecification(ColumnSpecification[] boundNames)
+        {
+            super.collectMarkerSpecification(boundNames);
+            k.collectMarkerSpecification(boundNames);
+        }
+
+        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
+        {
+            Term.Terminal key = k.bind(params.variables);
+            Term.Terminal value = t.bind(params.variables);
+            assert key instanceof Constants.Value && value instanceof Constants.Value;
+
+            ByteBuffer cellName = prefix.add(columnName.key).add(((Constants.Value)key).bytes).build();
+            cf.addColumn(params.makeColumn(cellName, ((Constants.Value)value).bytes));
+        }
+    }
+
+    public static class Putter extends Operation
+    {
+        public Putter(ColumnIdentifier column, Term t)
+        {
+            super(column, t);
+        }
+
+        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
+        {
+            doPut(t, cf, prefix.add(columnName.key), params);
+        }
+
+        static void doPut(Term t, ColumnFamily cf, ColumnNameBuilder columnName, UpdateParameters params) throws InvalidRequestException
+        {
+            Term.Terminal value = t.bind(params.variables);
+            assert value instanceof Maps.Value;
+
+            Map<ByteBuffer, ByteBuffer> toAdd = ((Maps.Value)value).map;
+            for (Map.Entry<ByteBuffer, ByteBuffer> entry : toAdd.entrySet())
+            {
+                ByteBuffer cellName = columnName.copy().add(entry.getKey()).build();
+                cf.addColumn(params.makeColumn(cellName, entry.getValue()));
+            }
+        }
+    }
+
+    public static class DiscarderByKey extends Operation
+    {
+        public DiscarderByKey(ColumnIdentifier column, Term k)
+        {
+            super(column, k);
+        }
+
+        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
+        {
+            Term.Terminal key = t.bind(params.variables);
+            assert key instanceof Constants.Value;
+
+            ByteBuffer cellName = prefix.add(columnName.key).add(((Constants.Value)key).bytes).build();
+            cf.addColumn(params.makeTombstone(cellName));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/Operation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Operation.java b/src/java/org/apache/cassandra/cql3/Operation.java
new file mode 100644
index 0000000..9754f50
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/Operation.java
@@ -0,0 +1,408 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * An UPDATE or DELETE operation.
+ *
+ * For UPDATE this includes:
+ *   - setting a constant
+ *   - counter operations
+ *   - collections operations
+ * and for DELETE:
+ *   - deleting a column
+ *   - deleting an element of collection column
+ *
+ * Fine grained operation are obtained from their raw counterpart (Operation.Raw, which
+ * correspond to a parsed, non-checked operation) by provided the receiver for the operation.
+ */
+public abstract class Operation
+{
+    // Name of the column the operation applies to
+    public final ColumnIdentifier columnName;
+
+    // Term involved in the operation. In theory this should not be here since some operation
+    // may require none of more than one term, but most need 1 so it simplify things a bit.
+    protected final Term t;
+
+    protected Operation(ColumnIdentifier columnName, Term t)
+    {
+        this.columnName = columnName;
+        this.t = t;
+    }
+
+    /**
+     * @return whether the operation requires a read of the previous value to be executed
+     * (only lists setterByIdx, discard and discardByIdx requires that).
+     */
+    public boolean requiresRead()
+    {
+        return false;
+    }
+
+    /**
+     * Collects the column specification for the bind variables of this operation.
+     *
+     * @param boundNames the list of column specification where to collect the
+     * bind variables of this term in.
+     */
+    public void collectMarkerSpecification(ColumnSpecification[] boundNames)
+    {
+        if (t != null)
+            t.collectMarkerSpecification(boundNames);
+    }
+
+    /**
+     * Execute the operation.
+     *
+     * @param rowKey row key for the update.
+     * @param cf the column family to which to add the updates generated by this operation.
+     * @param namePrefix the prefix that identify the CQL3 row this operation applies to (callers should not reuse
+     * the ColumnNameBuilder they pass here).
+     * @param params parameters of the update.
+     */
+    public abstract void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder namePrefix, UpdateParameters params) throws InvalidRequestException;
+
+    /**
+     * A parsed raw UPDATE operation.
+     *
+     * This can be one of:
+     *   - Setting a value: c = v
+     *   - Setting an element of a collection: c[x] = v
+     *   - An addition/substraction to a variable: c = c +/- v (where v can be a collection literal)
+     *   - An prepend operation: c = v + c
+     */
+    public interface RawUpdate
+    {
+        /**
+         * This method validates the operation (i.e. validate it is well typed)
+         * based on the specification of the receiver of the operation.
+         *
+         * It returns an Operation which can be though as post-preparation well-typed
+         * Operation.
+         *
+         * @param receiver the "column" this operation applies to. Note that
+         * contrarly to the method of same name in Term.Raw, the receiver should always
+         * be a true column.
+         * @return the prepared update operation.
+         */
+        public Operation prepare(CFDefinition.Name receiver) throws InvalidRequestException;
+
+        /**
+         * @return whether this operation can be applied alongside the {@code
+         * other} update (in the same UPDATE statement for the same column).
+         */
+        public boolean isCompatibleWith(RawUpdate other);
+    }
+
+    /**
+     * A parsed raw DELETE operation.
+     *
+     * This can be one of:
+     *   - Deleting a column
+     *   - Deleting an element of a collection
+     */
+    public interface RawDeletion
+    {
+        /**
+         * The name of the column affected by this delete operation.
+         */
+        public ColumnIdentifier affectedColumn();
+
+        /**
+         * This method validates the operation (i.e. validate it is well typed)
+         * based on the specification of the column affected by the operation (i.e the
+         * one returned by affectedColumn()).
+         *
+         * It returns an Operation which can be though as post-preparation well-typed
+         * Operation.
+         *
+         * @param receiver the "column" this operation applies to.
+         * @return the prepared delete operation.
+         */
+        public Operation prepare(ColumnSpecification receiver) throws InvalidRequestException;
+    }
+
+    public static class SetValue implements RawUpdate
+    {
+        private final Term.Raw value;
+
+        public SetValue(Term.Raw value)
+        {
+            this.value = value;
+        }
+
+        public Operation prepare(CFDefinition.Name receiver) throws InvalidRequestException
+        {
+            Term v = value.prepare(receiver);
+
+            if (!(receiver.type instanceof CollectionType))
+                return new Constants.Setter(receiver.kind == CFDefinition.Name.Kind.VALUE_ALIAS ? null : receiver.name, v);
+
+            switch (((CollectionType)receiver.type).kind)
+            {
+                case LIST:
+                    return new Lists.Setter(receiver.name, v);
+                case SET:
+                    return new Sets.Setter(receiver.name, v);
+                case MAP:
+                    return new Maps.Setter(receiver.name, v);
+            }
+            throw new AssertionError();
+        }
+
+        protected String toString(ColumnSpecification column)
+        {
+            return String.format("%s = %s", column, value);
+        }
+
+        public boolean isCompatibleWith(RawUpdate other)
+        {
+            // We don't allow setting multiple time the same column, because 1)
+            // it's stupid and 2) the result would seem random to the user.
+            return false;
+        }
+    }
+
+    public static class SetElement implements RawUpdate
+    {
+        private final Term.Raw selector;
+        private final Term.Raw value;
+
+        public SetElement(Term.Raw selector, Term.Raw value)
+        {
+            this.selector = selector;
+            this.value = value;
+        }
+
+        public Operation prepare(CFDefinition.Name receiver) throws InvalidRequestException
+        {
+            if (!(receiver.type instanceof CollectionType))
+                throw new InvalidRequestException(String.format("Invalid operation (%s) for non collection column %s", toString(receiver), receiver));
+
+            switch (((CollectionType)receiver.type).kind)
+            {
+                case LIST:
+                    Term idx = selector.prepare(Lists.indexSpecOf(receiver));
+                    Term lval = value.prepare(Lists.valueSpecOf(receiver));
+                    return new Lists.SetterByIndex(receiver.name, idx, lval);
+                case SET:
+                    throw new InvalidRequestException(String.format("Invalid operation (%s) for set column %s", toString(receiver), receiver));
+                case MAP:
+                    Term key = selector.prepare(Maps.keySpecOf(receiver));
+                    Term mval = value.prepare(Maps.valueSpecOf(receiver));
+                    return new Maps.SetterByKey(receiver.name, key, mval);
+            }
+            throw new AssertionError();
+        }
+
+        protected String toString(ColumnSpecification column)
+        {
+            return String.format("%s[%s] = %s", column, selector, value);
+        }
+
+        public boolean isCompatibleWith(RawUpdate other)
+        {
+            // TODO: we could check that the other operation is not setting the same element
+            // too (but since the index/key set may be a bind variables we can't always do it at this point)
+            return !(other instanceof SetValue);
+        }
+    }
+
+    public static class Addition implements RawUpdate
+    {
+        private final Term.Raw value;
+
+        public Addition(Term.Raw value)
+        {
+            this.value = value;
+        }
+
+        public Operation prepare(CFDefinition.Name receiver) throws InvalidRequestException
+        {
+            Term v = value.prepare(receiver);
+
+            if (!(receiver.type instanceof CollectionType))
+            {
+                if (!(receiver.type instanceof CounterColumnType))
+                    throw new InvalidRequestException(String.format("Invalid operation for non counter column %s", toString(receiver), receiver));
+                return new Constants.Adder(receiver.kind == CFDefinition.Name.Kind.VALUE_ALIAS ? null : receiver.name, v);
+            }
+
+            switch (((CollectionType)receiver.type).kind)
+            {
+                case LIST:
+                    return new Lists.Appender(receiver.name, v);
+                case SET:
+                    return new Sets.Adder(receiver.name, v);
+                case MAP:
+                    return new Maps.Putter(receiver.name, v);
+            }
+            throw new AssertionError();
+        }
+
+        protected String toString(ColumnSpecification column)
+        {
+            return String.format("%s = %s + %s", column, column, value);
+        }
+
+        public boolean isCompatibleWith(RawUpdate other)
+        {
+            return !(other instanceof SetValue);
+        }
+    }
+
+    public static class Substraction implements RawUpdate
+    {
+        private final Term.Raw value;
+
+        public Substraction(Term.Raw value)
+        {
+            this.value = value;
+        }
+
+        public Operation prepare(CFDefinition.Name receiver) throws InvalidRequestException
+        {
+            Term v = value.prepare(receiver);
+
+            if (!(receiver.type instanceof CollectionType))
+            {
+                if (!(receiver.type instanceof CounterColumnType))
+                    throw new InvalidRequestException(String.format("Invalid operation (%s) for non counter column %s", toString(receiver), receiver));
+                return new Constants.Substracter(receiver.kind == CFDefinition.Name.Kind.VALUE_ALIAS ? null : receiver.name, v);
+            }
+
+            switch (((CollectionType)receiver.type).kind)
+            {
+                case LIST:
+                    return new Lists.Discarder(receiver.name, v);
+                case SET:
+                    return new Sets.Discarder(receiver.name, v);
+                case MAP:
+                    throw new InvalidRequestException(String.format("Invalid operation (%s) for map column %s", toString(receiver), receiver));
+            }
+            throw new AssertionError();
+        }
+
+        protected String toString(ColumnSpecification column)
+        {
+            return String.format("%s = %s - %s", column, column, value);
+        }
+
+        public boolean isCompatibleWith(RawUpdate other)
+        {
+            return !(other instanceof SetValue);
+        }
+    }
+
+    public static class Prepend implements RawUpdate
+    {
+        private final Term.Raw value;
+
+        public Prepend(Term.Raw value)
+        {
+            this.value = value;
+        }
+
+        public Operation prepare(CFDefinition.Name receiver) throws InvalidRequestException
+        {
+            Term v = value.prepare(receiver);
+
+            if (!(receiver.type instanceof ListType))
+                throw new InvalidRequestException(String.format("Invalid operation (%s) for non list column %s", toString(receiver), receiver));
+
+            return new Lists.Prepender(receiver.name, v);
+        }
+
+        protected String toString(ColumnSpecification column)
+        {
+            return String.format("%s = %s - %s", column, value, column);
+        }
+
+        public boolean isCompatibleWith(RawUpdate other)
+        {
+            return !(other instanceof SetValue);
+        }
+    }
+
+    public static class ColumnDeletion implements RawDeletion
+    {
+        private final ColumnIdentifier id;
+
+        public ColumnDeletion(ColumnIdentifier id)
+        {
+            this.id = id;
+        }
+
+        public ColumnIdentifier affectedColumn()
+        {
+            return id;
+        }
+
+        public Operation prepare(ColumnSpecification receiver) throws InvalidRequestException
+        {
+            // No validation, deleting a column is always "well typed"
+            return new Constants.Deleter(id, receiver.type instanceof CollectionType);
+        }
+    }
+
+    public static class ElementDeletion implements RawDeletion
+    {
+        private final ColumnIdentifier id;
+        private final Term.Raw element;
+
+        public ElementDeletion(ColumnIdentifier id, Term.Raw element)
+        {
+            this.id = id;
+            this.element = element;
+        }
+
+        public ColumnIdentifier affectedColumn()
+        {
+            return id;
+        }
+
+        public Operation prepare(ColumnSpecification receiver) throws InvalidRequestException
+        {
+            if (!(receiver.type instanceof CollectionType))
+                throw new InvalidRequestException(String.format("Invalid deletion operation for non collection column %s", receiver));
+
+            switch (((CollectionType)receiver.type).kind)
+            {
+                case LIST:
+                    Term idx = element.prepare(Lists.indexSpecOf(receiver));
+                    return new Lists.DiscarderByIndex(id, idx);
+                case SET:
+                    Term elt = element.prepare(Sets.valueSpecOf(receiver));
+                    return new Lists.Discarder(id, elt);
+                case MAP:
+                    Term key = element.prepare(Maps.keySpecOf(receiver));
+                    return new Maps.DiscarderByKey(id, key);
+            }
+            throw new AssertionError();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index f18cdcc..cbaf5d3 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -137,7 +137,10 @@ public class QueryProcessor
     throws RequestExecutionException, RequestValidationException
     {
         logger.trace("CQL QUERY: {}", queryString);
-        return processStatement(getStatement(queryString, queryState.getClientState()).statement, cl, queryState, Collections.<ByteBuffer>emptyList());
+        CQLStatement prepared = getStatement(queryString, queryState.getClientState()).statement;
+        if (prepared.getBoundsTerms() > 0)
+            throw new InvalidRequestException("Cannot execute query with bind variables");
+        return processStatement(prepared, cl, queryState, Collections.<ByteBuffer>emptyList());
     }
 
     public static UntypedResultSet process(String query) throws RequestExecutionException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/Relation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Relation.java b/src/java/org/apache/cassandra/cql3/Relation.java
index 7fcf2e4..0217565 100644
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@ -29,32 +29,16 @@ public class Relation
 {
     private final ColumnIdentifier entity;
     private final Type relationType;
-    private final Term value;
-    private final List<Term> inValues;
+    private final Term.Raw value;
+    private final List<Term.Raw> inValues;
     public final boolean onToken;
 
     public static enum Type
     {
         EQ, LT, LTE, GTE, GT, IN;
-
-        public static Type forString(String s)
-        {
-            if (s.equals("="))
-                return EQ;
-            else if (s.equals("<"))
-                return LT;
-            else if (s.equals("<="))
-                return LTE;
-            else if (s.equals(">="))
-                return GTE;
-            else if (s.equals(">"))
-                return GT;
-
-            return null;
-        }
     }
 
-    private Relation(ColumnIdentifier entity, Type type, Term value, List<Term> inValues, boolean onToken)
+    private Relation(ColumnIdentifier entity, Type type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
     {
         this.entity = entity;
         this.relationType = type;
@@ -70,19 +54,19 @@ public class Relation
      * @param type the type that describes how this entity relates to the value.
      * @param value the value being compared.
      */
-    public Relation(ColumnIdentifier entity, String type, Term value)
+    public Relation(ColumnIdentifier entity, Type type, Term.Raw value)
     {
-        this(entity, Type.forString(type), value, null, false);
+        this(entity, type, value, null, false);
     }
 
-    public Relation(ColumnIdentifier entity, String type, Term value, boolean onToken)
+    public Relation(ColumnIdentifier entity, Type type, Term.Raw value, boolean onToken)
     {
-        this(entity, Type.forString(type), value, null, onToken);
+        this(entity, type, value, null, onToken);
     }
 
     public static Relation createInRelation(ColumnIdentifier entity)
     {
-        return new Relation(entity, Type.IN, null, new ArrayList<Term>(), false);
+        return new Relation(entity, Type.IN, null, new ArrayList<Term.Raw>(), false);
     }
 
     public Type operator()
@@ -95,19 +79,19 @@ public class Relation
         return entity;
     }
 
-    public Term getValue()
+    public Term.Raw getValue()
     {
         assert relationType != Type.IN;
         return value;
     }
 
-    public List<Term> getInValues()
+    public List<Term.Raw> getInValues()
     {
         assert relationType == Type.IN;
         return inValues;
     }
 
-    public void addInValue(Term t)
+    public void addInValue(Term.Raw t)
     {
         inValues.add(t);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/ResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index 9bc19ca..290653f 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -58,6 +58,12 @@ public class ResultSet
         return rows.size();
     }
 
+    public void addRow(List<ByteBuffer> row)
+    {
+        assert row.size() == metadata.names.size();
+        rows.add(row);
+    }
+
     public void addColumnValue(ByteBuffer value)
     {
         if (rows.isEmpty() || lastRow().size() == metadata.names.size())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/Sets.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Sets.java b/src/java/org/apache/cassandra/cql3/Sets.java
new file mode 100644
index 0000000..1537507
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/Sets.java
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import com.google.common.base.Joiner;
+
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.MarshalException;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * Static helper methods and classes for sets.
+ */
+public abstract class Sets
+{
+    private Sets() {}
+
+    public static ColumnSpecification valueSpecOf(ColumnSpecification column)
+    {
+        return new ColumnSpecification(column.ksName, column.cfName, new ColumnIdentifier("value(" + column.name + ")", true), ((SetType)column.type).elements);
+    }
+
+    public static class Literal implements Term.Raw
+    {
+        private final List<Term.Raw> elements;
+
+        public Literal(List<Term.Raw> elements)
+        {
+            this.elements = elements;
+        }
+
+        public Term.Terminal prepare(ColumnSpecification receiver) throws InvalidRequestException
+        {
+            validateAssignableTo(receiver);
+
+            // We've parsed empty maps as a set literal to break the ambiguity so
+            // handle that case now
+            if (receiver.type instanceof MapType && elements.isEmpty())
+                return new Maps.Value(Collections.<ByteBuffer, ByteBuffer>emptyMap());
+
+            ColumnSpecification valueSpec = Sets.valueSpecOf(receiver);
+            Set<ByteBuffer> values = new TreeSet<ByteBuffer>(((SetType)receiver.type).elements);
+            for (Term.Raw rt : elements)
+            {
+                Term t = rt.prepare(valueSpec);
+
+                if (!(t instanceof Constants.Value))
+                {
+                    if (t instanceof Term.NonTerminal)
+                        throw new InvalidRequestException(String.format("Invalid set literal for %s: bind variables are not supported inside collection literals", receiver));
+                    else
+                        throw new InvalidRequestException(String.format("Invalid set literal for %s: nested collections are not supported", receiver));
+                }
+
+                if (!values.add(((Constants.Value)t).bytes))
+                    throw new InvalidRequestException(String.format("Invalid set literal: duplicate value %s", rt));
+            }
+            return new Value(values);
+        }
+
+        private void validateAssignableTo(ColumnSpecification receiver) throws InvalidRequestException
+        {
+            if (!(receiver.type instanceof SetType))
+            {
+                // We've parsed empty maps as a set literal to break the ambiguity so
+                // handle that case now
+                if (receiver.type instanceof MapType && elements.isEmpty())
+                    return;
+
+                throw new InvalidRequestException(String.format("Invalid set literal for %s of type %s", receiver, receiver.type.asCQL3Type()));
+            }
+
+            ColumnSpecification valueSpec = Sets.valueSpecOf(receiver);
+            for (Term.Raw rt : elements)
+            {
+                if (!rt.isAssignableTo(valueSpec))
+                    throw new InvalidRequestException(String.format("Invalid set literal for %s: value %s is not of type %s", receiver, rt, valueSpec.type.asCQL3Type()));
+            }
+        }
+
+        public boolean isAssignableTo(ColumnSpecification receiver)
+        {
+            try
+            {
+                validateAssignableTo(receiver);
+                return true;
+            }
+            catch (InvalidRequestException e)
+            {
+                return false;
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "{" + Joiner.on(", ").join(elements) + "}";
+        }
+    }
+
+    public static class Value extends Term.Terminal
+    {
+        public final Set<ByteBuffer> elements;
+
+        public Value(Set<ByteBuffer> elements)
+        {
+            this.elements = elements;
+        }
+
+        public static Value fromSerialized(ByteBuffer value, SetType type) throws InvalidRequestException
+        {
+            try
+            {
+                // Collections have this small hack that validate cannot be called on a serialized object,
+                // but compose does the validation (so we're fine).
+                Set<?> s = type.compose(value);
+                Set<ByteBuffer> elements = new LinkedHashSet<ByteBuffer>(s.size());
+                for (Object element : s)
+                    elements.add(type.elements.decompose(element));
+                return new Value(elements);
+            }
+            catch (MarshalException e)
+            {
+                throw new InvalidRequestException(e.getMessage());
+            }
+        }
+
+        public ByteBuffer get()
+        {
+            return CollectionType.pack(new ArrayList<ByteBuffer>(elements), elements.size());
+        }
+    }
+
+    public static class Marker extends AbstractMarker
+    {
+        protected Marker(int bindIndex, ColumnSpecification receiver)
+        {
+            super(bindIndex, receiver);
+            assert receiver.type instanceof SetType;
+        }
+
+        public Value bind(List<ByteBuffer> values) throws InvalidRequestException
+        {
+            ByteBuffer value = values.get(bindIndex);
+            return Value.fromSerialized(value, (SetType)receiver.type);
+        }
+    }
+
+    public static class Setter extends Operation
+    {
+        public Setter(ColumnIdentifier column, Term t)
+        {
+            super(column, t);
+        }
+
+        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
+        {
+            // delete + add
+            ColumnNameBuilder column = prefix.add(columnName.key);
+            cf.addAtom(params.makeTombstoneForOverwrite(column.build(), column.buildAsEndOfRange()));
+            Adder.doAdd(t, cf, column, params);
+        }
+    }
+
+    public static class Adder extends Operation
+    {
+        public Adder(ColumnIdentifier column, Term t)
+        {
+            super(column, t);
+        }
+
+        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
+        {
+            doAdd(t, cf, prefix.add(columnName.key), params);
+        }
+
+        static void doAdd(Term t, ColumnFamily cf, ColumnNameBuilder columnName, UpdateParameters params) throws InvalidRequestException
+        {
+            Term.Terminal value = t.bind(params.variables);
+            assert value instanceof Sets.Value;
+
+            Set<ByteBuffer> toAdd = ((Sets.Value)value).elements;
+            for (ByteBuffer bb : toAdd)
+            {
+                ByteBuffer cellName = columnName.copy().add(bb).build();
+                cf.addColumn(params.makeColumn(cellName, ByteBufferUtil.EMPTY_BYTE_BUFFER));
+            }
+        }
+    }
+
+    public static class Discarder extends Operation
+    {
+        public Discarder(ColumnIdentifier column, Term t)
+        {
+            super(column, t);
+        }
+
+        public void execute(ByteBuffer rowKey, ColumnFamily cf, ColumnNameBuilder prefix, UpdateParameters params) throws InvalidRequestException
+        {
+            Term.Terminal value = t.bind(params.variables);
+
+            // This can be either a set or a single element
+            Set<ByteBuffer> toDiscard = value instanceof Constants.Value
+                                      ? Collections.singleton(((Constants.Value)value).bytes)
+                                      : ((Sets.Value)value).elements;
+
+            ColumnNameBuilder column = prefix.add(columnName.key);
+            for (ByteBuffer bb : toDiscard)
+            {
+                ByteBuffer cellName = column.copy().add(bb).build();
+                cf.addColumn(params.makeTombstone(cellName));
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/Term.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Term.java b/src/java/org/apache/cassandra/cql3/Term.java
index fb797d5..3cdb70f 100644
--- a/src/java/org/apache/cassandra/cql3/Term.java
+++ b/src/java/org/apache/cassandra/cql3/Term.java
@@ -22,236 +22,110 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.MarshalException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
-/** A term parsed from a CQL statement. */
-public class Term
+/**
+ * A CQL3 term, i.e. a column value with or without bind variables.
+ *
+ * A Term can be either terminal or non terminal. A term object is one that is typed and is obtained
+ * from a raw term (Term.Raw) by poviding the actual receiver to which the term is supposed to be a
+ * value of.
+ */
+public interface Term
 {
-    private static final Logger logger = LoggerFactory.getLogger(Term.class);
-
-    public enum Type
-    {
-        STRING, INTEGER, UUID, FLOAT, BOOLEAN, HEX, QMARK;
-
-        static Type forInt(int type)
-        {
-            if (type == CqlParser.STRING_LITERAL)
-                return STRING;
-            else if (type == CqlParser.INTEGER)
-                return INTEGER;
-            else if (type == CqlParser.UUID)
-                return UUID;
-            else if (type == CqlParser.FLOAT)
-                return FLOAT;
-            else if (type == CqlParser.BOOLEAN)
-                return BOOLEAN;
-            else if (type == CqlParser.HEXNUMBER)
-                return HEX;
-            else if (type == CqlParser.QMARK)
-                return QMARK;
-
-            throw new AssertionError();
-        }
-    }
-
-    private final String text;
-    private final Type type;
-    public final int bindIndex;
-    public final boolean isToken;
-
-    // For transition post-5198, see below
-    private static volatile boolean stringAsBlobWarningLogged = false;
-
-    // This is a hack for the timeuuid functions (minTimeuuid, maxTimeuuid, now) because instead of handling them as
-    // true function we let the TimeUUID.fromString() method handle it. We should probably clean that up someday
-    private final boolean skipTypeValidation;
-
-    private Term(String text, Type type, int bindIndex, boolean isToken, boolean skipTypeValidation)
-    {
-        this.text = text;
-        this.type = type;
-        this.bindIndex = bindIndex;
-        this.isToken = isToken;
-        this.skipTypeValidation = skipTypeValidation;
-    }
-
-    public Term(String text, Type type, boolean skipTypeValidation)
-    {
-        this(text, type, -1, false, skipTypeValidation);
-    }
-
-    public Term(String text, Type type)
-    {
-        this(text, type, -1, false, false);
-    }
-
     /**
-     * Create new Term instance from a string, and an integer that corresponds
-     * with the token ID from CQLParser.
+     * Collects the column specification for the bind variables in this Term.
+     * This is obviously a no-op if the term is Terminal.
      *
-     * @param text the text representation of the term.
-     * @param type the term's type as an integer token ID.
+     * @param boundNames the list of column specification where to collect the
+     * bind variables of this term in.
      */
-    public Term(String text, int type)
-    {
-        this(text, Type.forInt(type));
-    }
-
-    public Term(long value, Type type)
-    {
-        this(String.valueOf(value), type);
-    }
+    public void collectMarkerSpecification(ColumnSpecification[] boundNames);
 
-    public Term(String text, int type, int index)
-    {
-        this(text, Type.forInt(type), index, false, false);
-    }
+    /**
+     * Bind the values in this term to the values contained in {@code values}.
+     * This is obviously a no-op if the term is Terminal.
+     *
+     * @param values the values to bind markers to.
+     * @return the result of binding all the variables of this NonTerminal (or
+     * 'this' if the term is terminal).
+     */
+    public Terminal bind(List<ByteBuffer> values) throws InvalidRequestException;
 
-    public static Term tokenOf(Term t)
-    {
-        return new Term(t.text, t.type, t.bindIndex, true, false);
-    }
+    /**
+     * A shorter for bind(values).get().
+     * We expose it mainly because for constants it can avoids allocating a temporary
+     * object between the bind and the get (note that we still want to be able
+     * to separate bind and get for collections).
+     */
+    public ByteBuffer bindAndGet(List<ByteBuffer> values) throws InvalidRequestException;
 
     /**
-     * Returns the text parsed to create this term.
+     * A parsed, non prepared (thus untyped) term.
      *
-     * @return the string term acquired from a CQL statement.
+     * This can be one of:
+     *   - a constant
+     *   - a collection literal
+     *   - a function call
+     *   - a marker
      */
-    public String getText()
+    public interface Raw extends AssignementTestable
     {
-        return isToken ? "token(" + text + ")" : text;
+        /**
+         * This method validates this RawTerm is valid for provided column
+         * specification and "prepare" this RawTerm, returning the resulting
+         * prepared Term.
+         *
+         * @param receiver the "column" this RawTerm is supposed to be a value of. Note
+         * that the ColumnSpecification may not correspond to a real column in the
+         * case this RawTerm describe a list index or a map key, etc...
+         * @return the prepared term.
+         */
+        public Term prepare(ColumnSpecification receiver) throws InvalidRequestException;
     }
 
     /**
-     * Returns the typed value, serialized to a ByteBuffer according to a
-     * comparator/validator.
+     * A terminal term, i.e. one without any bind marker.
+     *
+     * This can be only one of:
+     *   - a constant value
+     *   - a collection value
      *
-     * @return a ByteBuffer of the value.
-     * @throws InvalidRequestException if unable to coerce the string to its type.
+     * Note that a terminal term will always have been type checked, and thus
+     * consumer can (and should) assume so.
      */
-    public ByteBuffer getByteBuffer(AbstractType<?> validator, List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        try
-        {
-            if (!isBindMarker())
-            {
-                // BytesType doesn't want it's input prefixed by '0x'.
-                if (type == Type.HEX && validator instanceof BytesType)
-                    return validator.fromString(text.substring(2));
-                return validator.fromString(text);
-            }
-
-            // must be a marker term so check for a CqlBindValue stored in the term
-            if (bindIndex == -1)
-                throw new AssertionError("a marker Term was encountered with no index value");
-
-            ByteBuffer value = variables.get(bindIndex);
-            // We don't yet support null values in prepared statements
-            if (value == null)
-                throw new InvalidRequestException("Invalid null value for prepared variable " + bindIndex);
-            validator.validate(value);
-            return value;
-        }
-        catch (MarshalException e)
-        {
-            throw new InvalidRequestException(e.getMessage());
-        }
-    }
-
-    public void validateType(String identifier, AbstractType<?> validator) throws InvalidRequestException
+    public abstract class Terminal implements Term
     {
-        if (skipTypeValidation)
-            return;
+        public void collectMarkerSpecification(ColumnSpecification[] boundNames) {}
+        public Terminal bind(List<ByteBuffer> values) { return this; }
 
-        Set<Type> supported = validator.supportedCQL3Constants();
-        // Treat null specially as this mean "I don't have a supportedCQL3Type method"
-        if (supported == null)
-            return;
+        /**
+         * @return the serialized value of this terminal.
+         */
+        public abstract ByteBuffer get();
 
-        if (!supported.contains(type))
+        public ByteBuffer bindAndGet(List<ByteBuffer> values) throws InvalidRequestException
         {
-            // Blobs should now be inputed as hexadecimal constants. However, to allow people to upgrade, we still allow
-            // blob-as-strings, even though it is deprecated (see #5198).
-            if (type == Type.STRING && validator instanceof BytesType)
-            {
-                if (!stringAsBlobWarningLogged)
-                {
-                    stringAsBlobWarningLogged = true;
-                    logger.warn("Inputing CLQ3 blobs as strings (like %s = '%s') is now deprecated and will be removed in a future version. "
-                              + "You should convert client code to use a blob constant (%s = %s) instead (see http://cassandra.apache.org/doc/cql3/CQL.html changelog section for more info).",
-                              identifier, text, identifier, "0x" + text);
-                }
-                return;
-            }
-
-            // TODO: Ideallly we'd keep the declared CQL3 type of columns and use that in the following message, instead of the AbstracType class name.
-            throw new InvalidRequestException(String.format("Invalid %s constant for %s of type %s", type, identifier, validator.asCQL3Type()));
+            return get();
         }
     }
 
     /**
-     * Obtain the term's type.
+     * A non terminal term, i.e. one that contains at least one bind marker.
      *
-     * @return the type
+     * We distinguish between the following type of NonTerminal:
+     *   - marker for a constant value
+     *   - marker for a collection value (list, set, map)
+     *   - a function having bind marker
      */
-    public Type getType()
+    public abstract class NonTerminal implements Term
     {
-        return type;
-    }
-
-    public boolean isBindMarker()
-    {
-        return type == Type.QMARK;
-    }
-
-    public List<Term> asList()
-    {
-        return Collections.singletonList(this);
-    }
-
-    @Override
-    public String toString()
-    {
-        return String.format("Term(%s, type=%s%s)", getText(), type, isToken ? ", isToken" : "");
-    }
-
-    @Override
-    public int hashCode()
-    {
-        final int prime = 31;
-        int result = 1 + (isToken ? 1 : 0);
-        result = prime * result + ((text == null) ? 0 : text.hashCode());
-        result = prime * result + ((type == null) ? 0 : type.hashCode());
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj)
-    {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        Term other = (Term) obj;
-        if (type==Type.QMARK) return false; // markers are never equal
-        if (text == null)
+        public ByteBuffer bindAndGet(List<ByteBuffer> values) throws InvalidRequestException
         {
-            if (other.text != null)
-                return false;
-        } else if (!text.equals(other.text))
-            return false;
-        if (type != other.type)
-            return false;
-        if (isToken != other.isToken)
-            return false;
-        return true;
+            return bind(values).get();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/TypeCast.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/TypeCast.java b/src/java/org/apache/cassandra/cql3/TypeCast.java
new file mode 100644
index 0000000..2fa322a
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/TypeCast.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public class TypeCast implements Term.Raw
+{
+    private final CQL3Type type;
+    private final Term.Raw term;
+
+    public TypeCast(CQL3Type type, Term.Raw term)
+    {
+        this.type = type;
+        this.term = term;
+    }
+
+    public Term prepare(ColumnSpecification receiver) throws InvalidRequestException
+    {
+        if (!term.isAssignableTo(castedSpecOf(receiver)))
+            throw new InvalidRequestException(String.format("Cannot cast value %s to type %s", term, type));
+
+        if (!isAssignableTo(receiver))
+            throw new InvalidRequestException(String.format("Cannot assign value %s to %s of type %s", this, receiver, receiver.type.asCQL3Type()));
+
+        return term.prepare(receiver);
+    }
+
+    private ColumnSpecification castedSpecOf(ColumnSpecification receiver)
+    {
+        return new ColumnSpecification(receiver.ksName, receiver.cfName, new ColumnIdentifier(toString(), true), type.getType());
+    }
+
+    public boolean isAssignableTo(ColumnSpecification receiver)
+    {
+        return receiver.type.equals(type.getType());
+    }
+
+    @Override
+    public String toString()
+    {
+        return "(" + type + ")" + term;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/UpdateParameters.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
index 1372c87..e8da34c 100644
--- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java
+++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
@@ -18,9 +18,13 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
+import org.apache.cassandra.cql3.statements.ColumnGroupMap;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * A simple container that simplify passing parameters for collections methods.
@@ -32,12 +36,16 @@ public class UpdateParameters
     private final int ttl;
     public final int localDeletionTime;
 
-    public UpdateParameters(List<ByteBuffer> variables, long timestamp, int ttl)
+    // For lists operation that require a read-before-write. Will be null otherwise.
+    private final Map<ByteBuffer, ColumnGroupMap> prefetchedLists;
+
+    public UpdateParameters(List<ByteBuffer> variables, long timestamp, int ttl, Map<ByteBuffer, ColumnGroupMap> prefetchedLists)
     {
         this.variables = variables;
         this.timestamp = timestamp;
         this.ttl = ttl;
         this.localDeletionTime = (int)(System.currentTimeMillis() / 1000);
+        this.prefetchedLists = prefetchedLists;
     }
 
     public Column makeColumn(ByteBuffer name, ByteBuffer value)
@@ -61,4 +69,13 @@ public class UpdateParameters
     {
         return new RangeTombstone(start, end, timestamp - 1, localDeletionTime);
     }
+
+    public List<Pair<ByteBuffer, IColumn>> getPrefetchedList(ByteBuffer rowKey, ByteBuffer cql3ColumnName)
+    {
+        if (prefetchedLists == null)
+            return Collections.emptyList();
+
+        ColumnGroupMap m = prefetchedLists.get(rowKey);
+        return m == null ? Collections.<Pair<ByteBuffer, IColumn>>emptyList() : m.getCollection(cql3ColumnName);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
new file mode 100644
index 0000000..dbdecf4
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
@@ -0,0 +1,68 @@
+/*
+ * 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.Arrays;
+import java.util.List;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public abstract class AbstractFunction implements Function
+{
+    public final String name;
+    public final List<AbstractType<?>> argsType;
+    public final AbstractType<?> returnType;
+
+    protected AbstractFunction(String name, AbstractType<?> returnType, AbstractType<?>... argsType)
+    {
+        this.name = name;
+        this.argsType = Arrays.asList(argsType);
+        this.returnType = returnType;
+    }
+
+    public String name()
+    {
+        return name;
+    }
+
+    public List<AbstractType<?>> argsType()
+    {
+        return argsType;
+    }
+
+    public AbstractType<?> returnType()
+    {
+        return returnType;
+    }
+
+    /**
+     * Creates a trivial factory that always return the provided function.
+     */
+    public static Function.Factory factory(final Function fun)
+    {
+        return new Function.Factory()
+        {
+            public Function create(String ksName, String cfName)
+            {
+                return fun;
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/functions/BytesConversionFcts.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/BytesConversionFcts.java b/src/java/org/apache/cassandra/cql3/functions/BytesConversionFcts.java
new file mode 100644
index 0000000..b30b5e7
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/BytesConversionFcts.java
@@ -0,0 +1,70 @@
+/*
+ * 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.List;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+
+public abstract class BytesConversionFcts
+{
+    // Most of the XAsBlob and blobAsX functions are basically no-op since everything is
+    // bytes internally. They only "trick" the type system.
+    public static Function makeToBlobFunction(AbstractType<?> fromType)
+    {
+        String name = fromType.asCQL3Type() + "asblob";
+        return new AbstractFunction(name, BytesType.instance, fromType)
+        {
+            public ByteBuffer execute(List<ByteBuffer> parameters)
+            {
+                return parameters.get(0);
+            }
+        };
+    }
+
+    public static Function makeFromBlobFunction(AbstractType<?> toType)
+    {
+        String name = "blobas" + toType.asCQL3Type();
+        return new AbstractFunction(name, toType, BytesType.instance)
+        {
+            public ByteBuffer execute(List<ByteBuffer> parameters)
+            {
+                return parameters.get(0);
+            }
+        };
+    }
+
+    public static final Function VarcharAsBlobFct = new AbstractFunction("varcharasblob", BytesType.instance, UTF8Type.instance)
+    {
+        public ByteBuffer execute(List<ByteBuffer> parameters)
+        {
+            return parameters.get(0);
+        }
+    };
+
+    public static final Function BlobAsVarcharFact = new AbstractFunction("blobasvarchar", UTF8Type.instance, BytesType.instance)
+    {
+        public ByteBuffer execute(List<ByteBuffer> parameters)
+        {
+            return parameters.get(0);
+        }
+    };
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/functions/Function.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/Function.java b/src/java/org/apache/cassandra/cql3/functions/Function.java
new file mode 100644
index 0000000..bae5593
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/Function.java
@@ -0,0 +1,42 @@
+/*
+ * 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.List;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public interface Function
+{
+    public String name();
+    public List<AbstractType<?>> argsType();
+    public AbstractType<?> returnType();
+
+    public ByteBuffer execute(List<ByteBuffer> parameters) throws InvalidRequestException;
+
+    public interface Factory
+    {
+        // We allow the function to be parametered by the table it is part of because the
+        // "token" function needs it (the argument depends on the keyValidator). However,
+        // for most function, the factory will just always the same function object (see
+        // AbstractFunction).
+        public Function create(String ksName, String cfName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java b/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
new file mode 100644
index 0000000..daaaad9
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
@@ -0,0 +1,144 @@
+/*
+ * 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.List;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public class FunctionCall extends Term.NonTerminal
+{
+    private final Function fun;
+    private final List<Term> terms;
+
+    private FunctionCall(Function fun, List<Term> terms)
+    {
+        this.fun = fun;
+        this.terms = terms;
+    }
+
+    public void collectMarkerSpecification(ColumnSpecification[] boundNames)
+    {
+        for (Term t : terms)
+            t.collectMarkerSpecification(boundNames);
+    }
+
+    public Term.Terminal bind(List<ByteBuffer> values) throws InvalidRequestException
+    {
+        return makeTerminal(fun, bindAndGet(values));
+    }
+
+    public ByteBuffer bindAndGet(List<ByteBuffer> values) throws InvalidRequestException
+    {
+        List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(terms.size());
+        for (Term t : terms)
+            buffers.add(t.bindAndGet(values));
+
+        return fun.execute(buffers);
+    }
+
+    private static Term.Terminal makeTerminal(Function fun, ByteBuffer result) throws InvalidRequestException
+    {
+        if (!(fun.returnType() instanceof CollectionType))
+            return new Constants.Value(result);
+
+        switch (((CollectionType)fun.returnType()).kind)
+        {
+            case LIST: return Lists.Value.fromSerialized(result, (ListType)fun.returnType());
+            case SET:  return Sets.Value.fromSerialized(result, (SetType)fun.returnType());
+            case MAP:  return Maps.Value.fromSerialized(result, (MapType)fun.returnType());
+        }
+        throw new AssertionError();
+    }
+
+    public static class Raw implements Term.Raw
+    {
+        private final String functionName;
+        private final List<Term.Raw> terms;
+
+        public Raw(String functionName, List<Term.Raw> terms)
+        {
+            this.functionName = functionName;
+            this.terms = terms;
+        }
+
+        public Term prepare(ColumnSpecification receiver) throws InvalidRequestException
+        {
+            Function fun = Functions.get(functionName, terms, receiver);
+
+            List<Term> parameters = new ArrayList<Term>(terms.size());
+            boolean allTerminal = true;
+            for (int i = 0; i < terms.size(); i++)
+            {
+                Term t = terms.get(i).prepare(Functions.makeArgSpec(receiver, fun, i));
+                if (t instanceof NonTerminal)
+                    allTerminal = false;
+                parameters.add(t);
+            }
+
+            return allTerminal
+                ? makeTerminal(fun, execute(fun, parameters))
+                : new FunctionCall(fun, parameters);
+        }
+
+        // All parameters must be terminal
+        private static ByteBuffer execute(Function fun, List<Term> parameters) throws InvalidRequestException
+        {
+            List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(parameters.size());
+            for (Term t : parameters)
+            {
+                assert t instanceof Term.Terminal;
+                buffers.add(((Term.Terminal)t).get());
+            }
+            return fun.execute(buffers);
+        }
+
+        public boolean isAssignableTo(ColumnSpecification receiver)
+        {
+            AbstractType<?> returnType = Functions.getReturnType(functionName, receiver.ksName, receiver.cfName);
+            return receiver.type.equals(returnType);
+        }
+
+        @Override
+        public String toString()
+        {
+            StringBuilder sb = new StringBuilder();
+            sb.append(functionName).append("(");
+            for (int i = 0; i < terms.size(); i++)
+            {
+                if (i > 0) sb.append(", ");
+                sb.append(terms.get(i));
+            }
+            return sb.append(")").toString();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/functions/Functions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/Functions.java b/src/java/org/apache/cassandra/cql3/functions/Functions.java
new file mode 100644
index 0000000..5b5e721
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/Functions.java
@@ -0,0 +1,185 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+import java.util.List;
+
+import com.google.common.collect.ArrayListMultimap;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.AssignementTestable;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public abstract class Functions
+{
+    private Functions() {}
+
+    // If we ever allow this to be populated at runtime, this will need to be thread safe.
+    private static final ArrayListMultimap<String, Function.Factory> declared = ArrayListMultimap.create();
+    static
+    {
+        // All method sharing the same name must have the same returnType. We could find a way to make that clear.
+        declared.put("token", TokenFct.factory);
+
+        declared.put("now", AbstractFunction.factory(TimeuuidFcts.nowFct));
+        declared.put("mintimeuuid", AbstractFunction.factory(TimeuuidFcts.minTimeuuidFct));
+        declared.put("maxtimeuuid", AbstractFunction.factory(TimeuuidFcts.maxTimeuuidFct));
+        declared.put("dateof", AbstractFunction.factory(TimeuuidFcts.dateOfFct));
+        declared.put("unixtimestampof", AbstractFunction.factory(TimeuuidFcts.unixTimestampOfFct));
+
+        for (CQL3Type type : CQL3Type.Native.values())
+        {
+            // Note: because text and varchar ends up being synonimous, our automatic makeToBlobFunction doesn't work
+            // for varchar, so we special case it below. We also skip blob for obvious reasons.
+            if (type == CQL3Type.Native.VARCHAR || type == CQL3Type.Native.BLOB)
+                continue;
+
+            Function toBlob = BytesConversionFcts.makeToBlobFunction(type.getType());
+            Function fromBlob = BytesConversionFcts.makeFromBlobFunction(type.getType());
+            declared.put(toBlob.name(), AbstractFunction.factory(toBlob));
+            declared.put(fromBlob.name(), AbstractFunction.factory(fromBlob));
+        }
+        declared.put("varcharasblob", AbstractFunction.factory(BytesConversionFcts.VarcharAsBlobFct));
+        declared.put("blobasvarchar", AbstractFunction.factory(BytesConversionFcts.BlobAsVarcharFact));
+    }
+
+    public static AbstractType<?> getReturnType(String functionName, String ksName, String cfName)
+    {
+        List<Function.Factory> factories = declared.get(functionName.toLowerCase());
+        return factories.isEmpty()
+             ? null // That's ok, we'll complain later
+             : factories.get(0).create(ksName, cfName).returnType();
+    }
+
+    public static ColumnSpecification makeArgSpec(ColumnSpecification receiver, Function fun, int i)
+    {
+        return new ColumnSpecification(receiver.ksName,
+                receiver.cfName,
+                new ColumnIdentifier("arg" + i +  "(" + fun.name() + ")", true),
+                fun.argsType().get(i));
+    }
+
+    public static Function get(String name, List<? extends AssignementTestable> providedArgs, ColumnSpecification receiver) throws InvalidRequestException
+    {
+        List<Function.Factory> factories = declared.get(name.toLowerCase());
+        if (factories.isEmpty())
+            throw new InvalidRequestException(String.format("Unknown CQL3 function %s called", name));
+
+        // Fast path if there is not choice
+        if (factories.size() == 1)
+        {
+            Function fun = factories.get(0).create(receiver.ksName, receiver.cfName);
+            validateTypes(fun, providedArgs, receiver);
+            return fun;
+        }
+
+        Function candidate = null;
+        for (Function.Factory factory : factories)
+        {
+            Function toTest = factory.create(receiver.ksName, receiver.cfName);
+            if (!isValidType(toTest, providedArgs, receiver))
+                continue;
+
+            if (candidate == null)
+                candidate = toTest;
+            else
+                throw new InvalidRequestException(String.format("Ambiguous call to function %s (can match both type signature %s and %s): use type casts to disambiguate", name, signature(candidate), signature(toTest)));
+        }
+        if (candidate == null)
+            throw new InvalidRequestException(String.format("Invalid call to function %s, none of its type signature matches (known type signatures: %s)", name, signatures(factories, receiver)));
+        return candidate;
+    }
+
+    private static void validateTypes(Function fun, List<? extends AssignementTestable> providedArgs, ColumnSpecification receiver) throws InvalidRequestException
+    {
+        if (!receiver.type.equals(fun.returnType()))
+            throw new InvalidRequestException(String.format("Type error: cannot assign result of function %s (type %s) to %s (type %s)", fun.name(), fun.returnType().asCQL3Type(), receiver, receiver.type.asCQL3Type()));
+
+        if (providedArgs.size() != fun.argsType().size())
+            throw new InvalidRequestException(String.format("Invalid number of arguments in call to function %s: %d required but % provided", fun.name(), fun.argsType().size(), providedArgs.size()));
+
+        for (int i = 0; i < providedArgs.size(); i++)
+        {
+            AssignementTestable provided = providedArgs.get(i);
+
+            // If the concrete argument is a bind variables, it can have any type.
+            // We'll validate the actually provided value at execution time.
+            if (provided == null)
+                continue;
+
+            ColumnSpecification expected = makeArgSpec(receiver, fun, i);
+            if (!provided.isAssignableTo(expected))
+                throw new InvalidRequestException(String.format("Type error: %s cannot be passed as argument %d of function %s of type %s", provided, i, fun.name(), expected.type.asCQL3Type()));
+        }
+    }
+
+    private static boolean isValidType(Function fun, List<? extends AssignementTestable> providedArgs, ColumnSpecification receiver)
+    {
+        if (!receiver.type.equals(fun.returnType()))
+            return false;
+
+        if (providedArgs.size() != fun.argsType().size())
+            return false;
+
+        for (int i = 0; i < providedArgs.size(); i++)
+        {
+            AssignementTestable provided = providedArgs.get(i);
+
+            // If the concrete argument is a bind variables, it can have any type.
+            // We'll validate the actually provided value at execution time.
+            if (provided == null)
+                continue;
+
+            ColumnSpecification expected = makeArgSpec(receiver, fun, i);
+            if (!provided.isAssignableTo(expected))
+                return false;
+        }
+        return true;
+    }
+
+    private static String signature(Function fun)
+    {
+        List<AbstractType<?>> args = fun.argsType();
+        StringBuilder sb = new StringBuilder();
+        sb.append("(");
+        for (int i = 0; i < args.size(); i++)
+        {
+            if (i > 0) sb.append(", ");
+            sb.append(args.get(i).asCQL3Type());
+        }
+        sb.append(") -> ");
+        sb.append(fun.returnType().asCQL3Type());
+        return sb.toString();
+    }
+
+    private static String signatures(List<Function.Factory> factories, ColumnSpecification receiver)
+    {
+        StringBuilder sb = new StringBuilder();
+        for (int i = 0; i < factories.size(); i++)
+        {
+            if (i > 0) sb.append(", ");
+            sb.append(signature(factories.get(i).create(receiver.ksName, receiver.cfName)));
+        }
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java b/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java
new file mode 100644
index 0000000..169babc
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java
@@ -0,0 +1,75 @@
+/*
+ * 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.Collections;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.DateType;
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
+
+public abstract class TimeuuidFcts
+{
+    public static final Function nowFct = new AbstractFunction("now", TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(List<ByteBuffer> parameters)
+        {
+            return ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
+        }
+    };
+
+    public static final Function minTimeuuidFct = new AbstractFunction("mintimeuuid", TimeUUIDType.instance, DateType.instance)
+    {
+        public ByteBuffer execute(List<ByteBuffer> parameters)
+        {
+            return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.minTimeUUID(DateType.instance.compose(parameters.get(0)).getTime())));
+        }
+    };
+
+    public static final Function maxTimeuuidFct = new AbstractFunction("maxtimeuuid", TimeUUIDType.instance, DateType.instance)
+    {
+        public ByteBuffer execute(List<ByteBuffer> parameters)
+        {
+            return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.maxTimeUUID(DateType.instance.compose(parameters.get(0)).getTime())));
+        }
+    };
+
+    public static final Function dateOfFct = new AbstractFunction("dateof", DateType.instance, TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(List<ByteBuffer> parameters)
+        {
+            return DateType.instance.decompose(new Date(UUIDGen.unixTimestamp(UUIDGen.getUUID(parameters.get(0)))));
+        }
+    };
+
+    public static final Function unixTimestampOfFct = new AbstractFunction("unixtimestampof", LongType.instance, TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(List<ByteBuffer> parameters)
+        {
+            return ByteBufferUtil.bytes(UUIDGen.unixTimestamp(UUIDGen.getUUID(parameters.get(0))));
+        }
+    };
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/31e669ab/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/TokenFct.java b/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
new file mode 100644
index 0000000..f1d8605
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
@@ -0,0 +1,71 @@
+/*
+ * 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.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.CFDefinition;
+import org.apache.cassandra.cql3.ColumnNameBuilder;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.StorageService;
+
+public class TokenFct extends AbstractFunction
+{
+    // The actual token function depends on the partitioner used
+    private static final IPartitioner partitioner = StorageService.instance.getPartitioner();
+
+    public static final Function.Factory factory = new Function.Factory()
+    {
+        public Function create(String ksName, String cfName)
+        {
+            return new TokenFct(Schema.instance.getCFMetaData(ksName, cfName));
+        }
+    };
+
+    private final CFDefinition cfDef;
+
+    public TokenFct(CFMetaData cfm)
+    {
+        super("token", partitioner.getTokenValidator(), getKeyTypes(cfm));
+        this.cfDef = cfm.getCfDef();
+    }
+
+    private static AbstractType[] getKeyTypes(CFMetaData cfm)
+    {
+        AbstractType[] types = new AbstractType[cfm.getCfDef().keys.size()];
+        int i = 0;
+        for (CFDefinition.Name name : cfm.getCfDef().keys.values())
+            types[i++] = name.type;
+        return types;
+    }
+
+    public ByteBuffer execute(List<ByteBuffer> parameters) throws InvalidRequestException
+    {
+        ColumnNameBuilder builder = cfDef.getKeyNameBuilder();
+        for (ByteBuffer bb : parameters)
+            builder.add(bb);
+        return partitioner.getTokenFactory().toByteArray(partitioner.getToken(builder.build()));
+    }
+}


Mime
View raw message