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 2017/01/27 14:51:44 UTC

[7/7] cassandra git commit: Refactor ColumnCondition

Refactor ColumnCondition

patch by Benjamin Lerer; reviewed by Alex Petrov for CASSANDRA-12981


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e71a49e8
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e71a49e8
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e71a49e8

Branch: refs/heads/trunk
Commit: e71a49e81f97864641f406461425a74ca4c56df1
Parents: 36375f9
Author: Benjamin Lerer <b....@gmail.com>
Authored: Fri Jan 27 15:50:06 2017 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Fri Jan 27 15:50:06 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |    4 +-
 src/antlr/Cql.g                                 |    1 +
 .../cassandra/cql3/AbstractConditions.java      |   64 --
 .../apache/cassandra/cql3/AbstractMarker.java   |    4 +-
 .../apache/cassandra/cql3/ColumnCondition.java  | 1042 ------------------
 .../apache/cassandra/cql3/ColumnConditions.java |  164 ---
 .../org/apache/cassandra/cql3/Conditions.java   |  102 --
 .../cassandra/cql3/IfExistsCondition.java       |   36 -
 .../cassandra/cql3/IfNotExistsCondition.java    |   36 -
 .../org/apache/cassandra/cql3/Operator.java     |  180 ++-
 src/java/org/apache/cassandra/cql3/Terms.java   |  241 +++-
 .../cql3/conditions/AbstractConditions.java     |   64 ++
 .../cql3/conditions/ColumnCondition.java        |  825 ++++++++++++++
 .../cql3/conditions/ColumnConditions.java       |  165 +++
 .../cassandra/cql3/conditions/Conditions.java   |  103 ++
 .../cql3/conditions/IfExistsCondition.java      |   37 +
 .../cql3/conditions/IfNotExistsCondition.java   |   37 +
 .../cql3/statements/CQL3CasRequest.java         |    1 +
 .../cql3/statements/DeleteStatement.java        |    2 +
 .../cql3/statements/ModificationStatement.java  |    3 +
 .../cql3/statements/UpdateStatement.java        |    2 +
 .../cassandra/cql3/ColumnConditionTest.java     |  589 ----------
 .../cql3/conditions/ColumnConditionTest.java    |  557 ++++++++++
 .../operations/InsertUpdateIfConditionTest.java |  192 +++-
 24 files changed, 2356 insertions(+), 2095 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ee2196d..d113645 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 4.0
- * Parallelize streaming of different keyspaces (4663)
+ * Refactor ColumnCondition (CASSANDRA-12981)
+ * Parallelize streaming of different keyspaces (CASSANDRA-4663)
  * Improved compactions metrics (CASSANDRA-13015)
  * Speed-up start-up sequence by avoiding un-needed flushes (CASSANDRA-13031)
  * Use Caffeine (W-TinyLFU) for on-heap caches (CASSANDRA-10855)
@@ -216,6 +217,7 @@ Merged from 3.0:
  * Correct log message for statistics of offheap memtable flush (CASSANDRA-12776)
  * Explicitly set locale for string validation (CASSANDRA-12541,CASSANDRA-12542,CASSANDRA-12543,CASSANDRA-12545)
 Merged from 2.2:
+ * Fix handling of nulls and unsets in IN conditions (CASSANDRA-12981) 
  * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
  * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
  * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/antlr/Cql.g
----------------------------------------------------------------------
diff --git a/src/antlr/Cql.g b/src/antlr/Cql.g
index a11f2fd..8b26426 100644
--- a/src/antlr/Cql.g
+++ b/src/antlr/Cql.g
@@ -46,6 +46,7 @@ import Parser,Lexer;
     import org.apache.cassandra.cql3.statements.*;
     import org.apache.cassandra.cql3.selection.*;
     import org.apache.cassandra.cql3.functions.*;
+    import org.apache.cassandra.cql3.conditions.*;
     import org.apache.cassandra.db.marshal.CollectionType;
     import org.apache.cassandra.exceptions.ConfigurationException;
     import org.apache.cassandra.exceptions.InvalidRequestException;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/AbstractConditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/AbstractConditions.java b/src/java/org/apache/cassandra/cql3/AbstractConditions.java
deleted file mode 100644
index 530d2b1..0000000
--- a/src/java/org/apache/cassandra/cql3/AbstractConditions.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.util.List;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.functions.Function;
-
-/**
- * Base class for <code>Conditions</code> classes.
- *
- */
-abstract class AbstractConditions implements Conditions
-{
-    public void addFunctionsTo(List<Function> functions)
-    {
-    }
-
-    public Iterable<ColumnDefinition> getColumns()
-    {
-        return null;
-    }
-
-    public boolean isEmpty()
-    {
-        return false;
-    }
-
-    public boolean appliesToStaticColumns()
-    {
-        return false;
-    }
-
-    public boolean appliesToRegularColumns()
-    {
-        return false;
-    }
-
-    public boolean isIfExists()
-    {
-        return false;
-    }
-
-    public boolean isIfNotExists()
-    {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/AbstractMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/AbstractMarker.java b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
index 7ffedef..19ce26c 100644
--- a/src/java/org/apache/cassandra/cql3/AbstractMarker.java
+++ b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
@@ -140,7 +140,7 @@ public abstract class AbstractMarker extends Term.NonTerminal
      *
      * Because a single type is used, a List is used to represent the values.
      */
-    public static class INRaw extends Raw
+    public static final class INRaw extends Raw
     {
         public INRaw(int bindIndex)
         {
@@ -154,7 +154,7 @@ public abstract class AbstractMarker extends Term.NonTerminal
         }
 
         @Override
-        public AbstractMarker prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
+        public Lists.Marker prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
             return new Lists.Marker(bindIndex, makeInReceiver(receiver));
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/ColumnCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
deleted file mode 100644
index 07f9f60..0000000
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ /dev/null
@@ -1,1042 +0,0 @@
-/*
- * 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.*;
-
-import com.google.common.collect.Iterators;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.transport.ProtocolVersion;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-/**
- * A CQL3 condition on the value of a column or collection element.  For example, "UPDATE .. IF a = 0".
- */
-public class ColumnCondition
-{
-    public final ColumnDefinition column;
-
-    // For collection, when testing the equality of a specific element, null otherwise.
-    private final Term collectionElement;
-
-    // For UDT, when testing the equality of a specific field, null otherwise.
-    private final FieldIdentifier field;
-
-    private final Term value;  // a single value or a marker for a list of IN values
-    private final List<Term> inValues;
-
-    public final Operator operator;
-
-    private ColumnCondition(ColumnDefinition column, Term collectionElement, FieldIdentifier field, Term value, List<Term> inValues, Operator op)
-    {
-        this.column = column;
-        this.collectionElement = collectionElement;
-        this.field = field;
-        this.value = value;
-        this.inValues = inValues;
-        this.operator = op;
-
-        assert field == null || collectionElement == null;
-        if (operator != Operator.IN)
-            assert this.inValues == null;
-    }
-
-    public static ColumnCondition condition(ColumnDefinition column, Term value, Operator op)
-    {
-        return new ColumnCondition(column, null, null, value, null, op);
-    }
-
-    public static ColumnCondition condition(ColumnDefinition column, Term collectionElement, Term value, Operator op)
-    {
-        return new ColumnCondition(column, collectionElement, null, value, null, op);
-    }
-
-    public static ColumnCondition condition(ColumnDefinition column, FieldIdentifier udtField, Term value, Operator op)
-    {
-        return new ColumnCondition(column, null, udtField, value, null, op);
-    }
-
-    public static ColumnCondition inCondition(ColumnDefinition column, List<Term> inValues)
-    {
-        return new ColumnCondition(column, null, null, null, inValues, Operator.IN);
-    }
-
-    public static ColumnCondition inCondition(ColumnDefinition column, Term collectionElement, List<Term> inValues)
-    {
-        return new ColumnCondition(column, collectionElement, null, null, inValues, Operator.IN);
-    }
-
-    public static ColumnCondition inCondition(ColumnDefinition column, FieldIdentifier udtField, List<Term> inValues)
-    {
-        return new ColumnCondition(column, null, udtField, null, inValues, Operator.IN);
-    }
-
-    public static ColumnCondition inCondition(ColumnDefinition column, Term inMarker)
-    {
-        return new ColumnCondition(column, null, null, inMarker, null, Operator.IN);
-    }
-
-    public static ColumnCondition inCondition(ColumnDefinition column, Term collectionElement, Term inMarker)
-    {
-        return new ColumnCondition(column, collectionElement, null, inMarker, null, Operator.IN);
-    }
-
-    public static ColumnCondition inCondition(ColumnDefinition column, FieldIdentifier udtField, Term inMarker)
-    {
-        return new ColumnCondition(column, null, udtField, inMarker, null, Operator.IN);
-    }
-
-    public void addFunctionsTo(List<Function> functions)
-    {
-        if (collectionElement != null)
-           collectionElement.addFunctionsTo(functions);
-        if (value != null)
-           value.addFunctionsTo(functions);
-        if (inValues != null)
-            for (Term value : inValues)
-                if (value != null)
-                    value.addFunctionsTo(functions);
-    }
-
-    /**
-     * 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(VariableSpecifications boundNames)
-    {
-        if (collectionElement != null)
-            collectionElement.collectMarkerSpecification(boundNames);
-
-        if ((operator == Operator.IN) && inValues != null)
-        {
-            for (Term value : inValues)
-                value.collectMarkerSpecification(boundNames);
-        }
-        else
-        {
-            value.collectMarkerSpecification(boundNames);
-        }
-    }
-
-    public ColumnCondition.Bound bind(QueryOptions options) throws InvalidRequestException
-    {
-        boolean isInCondition = operator == Operator.IN;
-        if (column.type instanceof CollectionType)
-        {
-            if (collectionElement != null)
-                return isInCondition ? new ElementAccessInBound(this, options) : new ElementAccessBound(this, options);
-            else
-                return isInCondition ? new CollectionInBound(this, options) : new CollectionBound(this, options);
-        }
-        else if (column.type.isUDT())
-        {
-            if (field != null)
-                return isInCondition ? new UDTFieldAccessInBound(this, options) : new UDTFieldAccessBound(this, options);
-            else
-                return isInCondition ? new UDTInBound(this, options) : new UDTBound(this, options);
-        }
-
-        return isInCondition ? new SimpleInBound(this, options) : new SimpleBound(this, options);
-    }
-
-    public static abstract class Bound
-    {
-        public final ColumnDefinition column;
-        public final Operator operator;
-
-        protected Bound(ColumnDefinition column, Operator operator)
-        {
-            this.column = column;
-            this.operator = operator;
-        }
-
-        /**
-         * Validates whether this condition applies to {@code current}.
-         */
-        public abstract boolean appliesTo(Row row) throws InvalidRequestException;
-
-        public ByteBuffer getCollectionElementValue()
-        {
-            return null;
-        }
-
-        protected boolean isSatisfiedByValue(ByteBuffer value, Cell c, AbstractType<?> type, Operator operator) throws InvalidRequestException
-        {
-            return compareWithOperator(operator, type, value, c == null ? null : c.value());
-        }
-
-        /** Returns true if the operator is satisfied (i.e. "otherValue operator value == true"), false otherwise. */
-        protected boolean compareWithOperator(Operator operator, AbstractType<?> type, ByteBuffer value, ByteBuffer otherValue) throws InvalidRequestException
-        {
-            if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
-                throw new InvalidRequestException("Invalid 'unset' value in condition");
-            if (value == null)
-            {
-                switch (operator)
-                {
-                    case EQ:
-                        return otherValue == null;
-                    case NEQ:
-                        return otherValue != null;
-                    default:
-                        throw new InvalidRequestException(String.format("Invalid comparison with null for operator \"%s\"", operator));
-                }
-            }
-            else if (otherValue == null)
-            {
-                // the condition value is not null, so only NEQ can return true
-                return operator == Operator.NEQ;
-            }
-            return operator.isSatisfiedBy(type, otherValue, value);
-        }
-    }
-
-    private static Cell getCell(Row row, ColumnDefinition column)
-    {
-        // If we're asking for a given cell, and we didn't got any row from our read, it's
-        // the same as not having said cell.
-        return row == null ? null : row.getCell(column);
-    }
-
-    private static Cell getCell(Row row, ColumnDefinition column, CellPath path)
-    {
-        // If we're asking for a given cell, and we didn't got any row from our read, it's
-        // the same as not having said cell.
-        return row == null ? null : row.getCell(column, path);
-    }
-
-    private static Iterator<Cell> getCells(Row row, ColumnDefinition column)
-    {
-        // If we're asking for a complex cells, and we didn't got any row from our read, it's
-        // the same as not having any cells for that column.
-        if (row == null)
-            return Collections.<Cell>emptyIterator();
-
-        ComplexColumnData complexData = row.getComplexColumnData(column);
-        return complexData == null ? Collections.<Cell>emptyIterator() : complexData.iterator();
-    }
-
-    private static boolean evaluateComparisonWithOperator(int comparison, Operator operator)
-    {
-        // called when comparison != 0
-        switch (operator)
-        {
-            case EQ:
-                return false;
-            case LT:
-            case LTE:
-                return comparison < 0;
-            case GT:
-            case GTE:
-                return comparison > 0;
-            case NEQ:
-                return true;
-            default:
-                throw new AssertionError();
-        }
-    }
-
-    private static ByteBuffer cellValueAtIndex(Iterator<Cell> iter, int index)
-    {
-        int adv = Iterators.advance(iter, index);
-        if (adv == index && iter.hasNext())
-            return iter.next().value();
-        else
-            return null;
-    }
-
-    /**
-     * A condition on a single non-collection column. This does not support IN operators (see SimpleInBound).
-     */
-    static class SimpleBound extends Bound
-    {
-        public final ByteBuffer value;
-
-        private SimpleBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
-        {
-            super(condition.column, condition.operator);
-            assert !(column.type instanceof CollectionType) && condition.field == null;
-            assert condition.operator != Operator.IN;
-            this.value = condition.value.bindAndGet(options);
-        }
-
-        public boolean appliesTo(Row row) throws InvalidRequestException
-        {
-            return isSatisfiedByValue(value, getCell(row, column), column.type, operator);
-        }
-    }
-
-    /**
-     * An IN condition on a single non-collection column.
-     */
-    static class SimpleInBound extends Bound
-    {
-        public final List<ByteBuffer> inValues;
-
-        private SimpleInBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
-        {
-            super(condition.column, condition.operator);
-            assert !(column.type instanceof CollectionType) && condition.field == null;
-            assert condition.operator == Operator.IN;
-            if (condition.inValues == null)
-                this.inValues = ((Lists.Value) condition.value.bind(options)).getElements();
-            else
-            {
-                this.inValues = new ArrayList<>(condition.inValues.size());
-                for (Term value : condition.inValues)
-                    this.inValues.add(value.bindAndGet(options));
-            }
-        }
-
-        public boolean appliesTo(Row row) throws InvalidRequestException
-        {
-            Cell c = getCell(row, column);
-            for (ByteBuffer value : inValues)
-            {
-                if (isSatisfiedByValue(value, c, column.type, Operator.EQ))
-                    return true;
-            }
-            return false;
-        }
-    }
-
-    /** A condition on an element of a collection column. IN operators are not supported here, see ElementAccessInBound. */
-    static class ElementAccessBound extends Bound
-    {
-        public final ByteBuffer collectionElement;
-        public final ByteBuffer value;
-
-        private ElementAccessBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
-        {
-            super(condition.column, condition.operator);
-            assert column.type instanceof CollectionType && condition.collectionElement != null;
-            assert condition.operator != Operator.IN;
-            this.collectionElement = condition.collectionElement.bindAndGet(options);
-            this.value = condition.value.bindAndGet(options);
-        }
-
-        public boolean appliesTo(Row row) throws InvalidRequestException
-        {
-            if (collectionElement == null)
-                throw new InvalidRequestException("Invalid null value for " + (column.type instanceof MapType ? "map" : "list") + " element access");
-
-            if (column.type instanceof MapType)
-            {
-                MapType mapType = (MapType) column.type;
-                if (column.type.isMultiCell())
-                {
-                    Cell cell = getCell(row, column, CellPath.create(collectionElement));
-                    return isSatisfiedByValue(value, cell, ((MapType) column.type).getValuesType(), operator);
-                }
-                else
-                {
-                    Cell cell = getCell(row, column);
-                    ByteBuffer mapElementValue = mapType.getSerializer().getSerializedValue(cell.value(), collectionElement, mapType.getKeysType());
-                    return compareWithOperator(operator, mapType.getValuesType(), value, mapElementValue);
-                }
-            }
-
-            // sets don't have element access, so it's a list
-            ListType listType = (ListType) column.type;
-            if (column.type.isMultiCell())
-            {
-                ByteBuffer columnValue = cellValueAtIndex(getCells(row, column), getListIndex(collectionElement));
-                return compareWithOperator(operator, ((ListType)column.type).getElementsType(), value, columnValue);
-            }
-            else
-            {
-                Cell cell = getCell(row, column);
-                ByteBuffer listElementValue = listType.getSerializer().getElement(cell.value(), getListIndex(collectionElement));
-                return compareWithOperator(operator, listType.getElementsType(), value, listElementValue);
-            }
-        }
-
-        static int getListIndex(ByteBuffer collectionElement) throws InvalidRequestException
-        {
-            int idx = ByteBufferUtil.toInt(collectionElement);
-            if (idx < 0)
-                throw new InvalidRequestException(String.format("Invalid negative list index %d", idx));
-            return idx;
-        }
-
-        public ByteBuffer getCollectionElementValue()
-        {
-            return collectionElement;
-        }
-    }
-
-    static class ElementAccessInBound extends Bound
-    {
-        public final ByteBuffer collectionElement;
-        public final List<ByteBuffer> inValues;
-
-        private ElementAccessInBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
-        {
-            super(condition.column, condition.operator);
-            assert column.type instanceof CollectionType && condition.collectionElement != null;
-            this.collectionElement = condition.collectionElement.bindAndGet(options);
-
-            if (condition.inValues == null)
-                this.inValues = ((Lists.Value) condition.value.bind(options)).getElements();
-            else
-            {
-                this.inValues = new ArrayList<>(condition.inValues.size());
-                for (Term value : condition.inValues)
-                    this.inValues.add(value.bindAndGet(options));
-            }
-        }
-
-        public boolean appliesTo(Row row) throws InvalidRequestException
-        {
-            if (collectionElement == null)
-                throw new InvalidRequestException("Invalid null value for " + (column.type instanceof MapType ? "map" : "list") + " element access");
-
-            ByteBuffer cellValue;
-            AbstractType<?> valueType;
-            if (column.type instanceof MapType)
-            {
-                MapType mapType = (MapType) column.type;
-                valueType = mapType.getValuesType();
-                if (column.type.isMultiCell())
-                {
-                    Cell cell = getCell(row, column, CellPath.create(collectionElement));
-                    cellValue = cell == null ? null : cell.value();
-                }
-                else
-                {
-                    Cell cell = getCell(row, column);
-                    cellValue = cell == null
-                              ? null
-                              : mapType.getSerializer().getSerializedValue(cell.value(), collectionElement, mapType.getKeysType());
-                }
-            }
-            else // ListType
-            {
-                ListType listType = (ListType) column.type;
-                valueType = listType.getElementsType();
-                if (column.type.isMultiCell())
-                {
-                    cellValue = cellValueAtIndex(getCells(row, column), ElementAccessBound.getListIndex(collectionElement));
-                }
-                else
-                {
-                    Cell cell = getCell(row, column);
-                    cellValue = cell == null
-                              ? null
-                              : listType.getSerializer().getElement(cell.value(), ElementAccessBound.getListIndex(collectionElement));
-                }
-            }
-
-            for (ByteBuffer value : inValues)
-            {
-                if (compareWithOperator(Operator.EQ, valueType, value, cellValue))
-                    return true;
-            }
-            return false;
-        }
-    }
-
-    /** A condition on an entire collection column. IN operators are not supported here, see CollectionInBound. */
-    static class CollectionBound extends Bound
-    {
-        private final Term.Terminal value;
-
-        private CollectionBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
-        {
-            super(condition.column, condition.operator);
-            assert column.type.isCollection() && condition.collectionElement == null;
-            assert condition.operator != Operator.IN;
-            this.value = condition.value.bind(options);
-        }
-
-        public boolean appliesTo(Row row) throws InvalidRequestException
-        {
-            CollectionType type = (CollectionType)column.type;
-
-            if (type.isMultiCell())
-            {
-                Iterator<Cell> iter = getCells(row, column);
-                if (value == null)
-                {
-                    if (operator == Operator.EQ)
-                        return !iter.hasNext();
-                    else if (operator == Operator.NEQ)
-                        return iter.hasNext();
-                    else
-                        throw new InvalidRequestException(String.format("Invalid comparison with null for operator \"%s\"", operator));
-                }
-
-                return valueAppliesTo(type, iter, value, operator);
-            }
-
-            // frozen collections
-            Cell cell = getCell(row, column);
-            if (value == null)
-            {
-                if (operator == Operator.EQ)
-                    return cell == null;
-                else if (operator == Operator.NEQ)
-                    return cell != null;
-                else
-                    throw new InvalidRequestException(String.format("Invalid comparison with null for operator \"%s\"", operator));
-            }
-
-            // make sure we use v3 serialization format for comparison
-            ByteBuffer conditionValue;
-            if (type.kind == CollectionType.Kind.LIST)
-                conditionValue = ((Lists.Value) value).get(ProtocolVersion.V3);
-            else if (type.kind == CollectionType.Kind.SET)
-                conditionValue = ((Sets.Value) value).get(ProtocolVersion.V3);
-            else
-                conditionValue = ((Maps.Value) value).get(ProtocolVersion.V3);
-
-            return compareWithOperator(operator, type, conditionValue, cell.value());
-        }
-
-        static boolean valueAppliesTo(CollectionType type, Iterator<Cell> iter, Term.Terminal value, Operator operator)
-        {
-            if (value == null)
-                return !iter.hasNext();
-
-            switch (type.kind)
-            {
-                case LIST:
-                    List<ByteBuffer> valueList = ((Lists.Value) value).elements;
-                    return listAppliesTo((ListType)type, iter, valueList, operator);
-                case SET:
-                    Set<ByteBuffer> valueSet = ((Sets.Value) value).elements;
-                    return setAppliesTo((SetType)type, iter, valueSet, operator);
-                case MAP:
-                    Map<ByteBuffer, ByteBuffer> valueMap = ((Maps.Value) value).map;
-                    return mapAppliesTo((MapType)type, iter, valueMap, operator);
-            }
-            throw new AssertionError();
-        }
-
-        private static boolean setOrListAppliesTo(AbstractType<?> type, Iterator<Cell> iter, Iterator<ByteBuffer> conditionIter, Operator operator, boolean isSet)
-        {
-            while(iter.hasNext())
-            {
-                if (!conditionIter.hasNext())
-                    return (operator == Operator.GT) || (operator == Operator.GTE) || (operator == Operator.NEQ);
-
-                // for lists we use the cell value; for sets we use the cell name
-                ByteBuffer cellValue = isSet ? iter.next().path().get(0) : iter.next().value();
-                int comparison = type.compare(cellValue, conditionIter.next());
-                if (comparison != 0)
-                    return evaluateComparisonWithOperator(comparison, operator);
-            }
-
-            if (conditionIter.hasNext())
-                return (operator == Operator.LT) || (operator == Operator.LTE) || (operator == Operator.NEQ);
-
-            // they're equal
-            return operator == Operator.EQ || operator == Operator.LTE || operator == Operator.GTE;
-        }
-
-        static boolean listAppliesTo(ListType type, Iterator<Cell> iter, List<ByteBuffer> elements, Operator operator)
-        {
-            return setOrListAppliesTo(type.getElementsType(), iter, elements.iterator(), operator, false);
-        }
-
-        static boolean setAppliesTo(SetType type, Iterator<Cell> iter, Set<ByteBuffer> elements, Operator operator)
-        {
-            ArrayList<ByteBuffer> sortedElements = new ArrayList<>(elements.size());
-            sortedElements.addAll(elements);
-            Collections.sort(sortedElements, type.getElementsType());
-            return setOrListAppliesTo(type.getElementsType(), iter, sortedElements.iterator(), operator, true);
-        }
-
-        static boolean mapAppliesTo(MapType type, Iterator<Cell> iter, Map<ByteBuffer, ByteBuffer> elements, Operator operator)
-        {
-            Iterator<Map.Entry<ByteBuffer, ByteBuffer>> conditionIter = elements.entrySet().iterator();
-            while(iter.hasNext())
-            {
-                if (!conditionIter.hasNext())
-                    return (operator == Operator.GT) || (operator == Operator.GTE) || (operator == Operator.NEQ);
-
-                Map.Entry<ByteBuffer, ByteBuffer> conditionEntry = conditionIter.next();
-                Cell c = iter.next();
-
-                // compare the keys
-                int comparison = type.getKeysType().compare(c.path().get(0), conditionEntry.getKey());
-                if (comparison != 0)
-                    return evaluateComparisonWithOperator(comparison, operator);
-
-                // compare the values
-                comparison = type.getValuesType().compare(c.value(), conditionEntry.getValue());
-                if (comparison != 0)
-                    return evaluateComparisonWithOperator(comparison, operator);
-            }
-
-            if (conditionIter.hasNext())
-                return (operator == Operator.LT) || (operator == Operator.LTE) || (operator == Operator.NEQ);
-
-            // they're equal
-            return operator == Operator.EQ || operator == Operator.LTE || operator == Operator.GTE;
-        }
-    }
-
-    public static class CollectionInBound extends Bound
-    {
-        private final List<Term.Terminal> inValues;
-
-        private CollectionInBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
-        {
-            super(condition.column, condition.operator);
-            assert column.type instanceof CollectionType && condition.collectionElement == null;
-            assert condition.operator == Operator.IN;
-            inValues = new ArrayList<>();
-            if (condition.inValues == null)
-            {
-                // We have a list of serialized collections that need to be deserialized for later comparisons
-                CollectionType collectionType = (CollectionType) column.type;
-                Lists.Marker inValuesMarker = (Lists.Marker) condition.value;
-                if (column.type instanceof ListType)
-                {
-                    ListType deserializer = ListType.getInstance(collectionType.valueComparator(), false);
-                    for (ByteBuffer buffer : ((Lists.Value)inValuesMarker.bind(options)).elements)
-                    {
-                        if (buffer == null)
-                            this.inValues.add(null);
-                        else
-                            this.inValues.add(Lists.Value.fromSerialized(buffer, deserializer, options.getProtocolVersion()));
-                    }
-                }
-                else if (column.type instanceof MapType)
-                {
-                    MapType deserializer = MapType.getInstance(collectionType.nameComparator(), collectionType.valueComparator(), false);
-                    for (ByteBuffer buffer : ((Lists.Value)inValuesMarker.bind(options)).elements)
-                    {
-                        if (buffer == null)
-                            this.inValues.add(null);
-                        else
-                            this.inValues.add(Maps.Value.fromSerialized(buffer, deserializer, options.getProtocolVersion()));
-                    }
-                }
-                else if (column.type instanceof SetType)
-                {
-                    SetType deserializer = SetType.getInstance(collectionType.valueComparator(), false);
-                    for (ByteBuffer buffer : ((Lists.Value)inValuesMarker.bind(options)).elements)
-                    {
-                        if (buffer == null)
-                            this.inValues.add(null);
-                        else
-                            this.inValues.add(Sets.Value.fromSerialized(buffer, deserializer, options.getProtocolVersion()));
-                    }
-                }
-            }
-            else
-            {
-                for (Term value : condition.inValues)
-                    this.inValues.add(value.bind(options));
-            }
-        }
-
-        public boolean appliesTo(Row row) throws InvalidRequestException
-        {
-            CollectionType type = (CollectionType)column.type;
-            if (type.isMultiCell())
-            {
-                // copy iterator contents so that we can properly reuse them for each comparison with an IN value
-                for (Term.Terminal value : inValues)
-                {
-                    if (CollectionBound.valueAppliesTo(type, getCells(row, column), value, Operator.EQ))
-                        return true;
-                }
-                return false;
-            }
-            else
-            {
-                Cell cell = getCell(row, column);
-                for (Term.Terminal value : inValues)
-                {
-                    if (value == null)
-                    {
-                        if (cell == null)
-                            return true;
-                    }
-                    else if (type.compare(value.get(ProtocolVersion.V3), cell.value()) == 0)
-                    {
-                        return true;
-                    }
-                }
-                return false;
-            }
-        }
-    }
-
-    /** A condition on a UDT field. IN operators are not supported here, see UDTFieldAccessInBound. */
-    static class UDTFieldAccessBound extends Bound
-    {
-        public final FieldIdentifier field;
-        public final ByteBuffer value;
-
-        private UDTFieldAccessBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
-        {
-            super(condition.column, condition.operator);
-            assert column.type.isUDT() && condition.field != null;
-            assert condition.operator != Operator.IN;
-            this.field = condition.field;
-            this.value = condition.value.bindAndGet(options);
-        }
-
-        public boolean appliesTo(Row row) throws InvalidRequestException
-        {
-            UserType userType = (UserType) column.type;
-            int fieldPosition = userType.fieldPosition(field);
-            assert fieldPosition >= 0;
-
-            ByteBuffer cellValue;
-            if (column.type.isMultiCell())
-            {
-                Cell cell = getCell(row, column, userType.cellPathForField(field));
-                cellValue = cell == null ? null : cell.value();
-            }
-            else
-            {
-                Cell cell = getCell(row, column);
-                cellValue = cell == null
-                          ? null
-                          : userType.split(cell.value())[fieldPosition];
-            }
-            return compareWithOperator(operator, userType.fieldType(fieldPosition), value, cellValue);
-        }
-    }
-
-    /** An IN condition on a UDT field.  For example: IF user.name IN ('a', 'b') */
-    static class UDTFieldAccessInBound extends Bound
-    {
-        public final FieldIdentifier field;
-        public final List<ByteBuffer> inValues;
-
-        private UDTFieldAccessInBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
-        {
-            super(condition.column, condition.operator);
-            assert column.type.isUDT() && condition.field != null;
-            this.field = condition.field;
-
-            if (condition.inValues == null)
-                this.inValues = ((Lists.Value) condition.value.bind(options)).getElements();
-            else
-            {
-                this.inValues = new ArrayList<>(condition.inValues.size());
-                for (Term value : condition.inValues)
-                    this.inValues.add(value.bindAndGet(options));
-            }
-        }
-
-        public boolean appliesTo(Row row) throws InvalidRequestException
-        {
-            UserType userType = (UserType) column.type;
-            int fieldPosition = userType.fieldPosition(field);
-            assert fieldPosition >= 0;
-
-            ByteBuffer cellValue;
-            if (column.type.isMultiCell())
-            {
-                Cell cell = getCell(row, column, userType.cellPathForField(field));
-                cellValue = cell == null ? null : cell.value();
-            }
-            else
-            {
-                Cell cell = getCell(row, column);
-                cellValue = cell == null ? null : userType.split(getCell(row, column).value())[fieldPosition];
-            }
-
-            AbstractType<?> valueType = userType.fieldType(fieldPosition);
-            for (ByteBuffer value : inValues)
-            {
-                if (compareWithOperator(Operator.EQ, valueType, value, cellValue))
-                    return true;
-            }
-            return false;
-        }
-    }
-
-    /** A non-IN condition on an entire UDT.  For example: IF user = {name: 'joe', age: 42}). */
-    static class UDTBound extends Bound
-    {
-        private final ByteBuffer value;
-        private final ProtocolVersion protocolVersion;
-
-        private UDTBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
-        {
-            super(condition.column, condition.operator);
-            assert column.type.isUDT() && condition.field == null;
-            assert condition.operator != Operator.IN;
-            protocolVersion = options.getProtocolVersion();
-            value = condition.value.bindAndGet(options);
-        }
-
-        public boolean appliesTo(Row row) throws InvalidRequestException
-        {
-            UserType userType = (UserType) column.type;
-            ByteBuffer rowValue;
-            if (userType.isMultiCell())
-            {
-                Iterator<Cell> iter = getCells(row, column);
-                rowValue = iter.hasNext() ? userType.serializeForNativeProtocol(iter, protocolVersion) : null;
-            }
-            else
-            {
-                Cell cell = getCell(row, column);
-                rowValue = cell == null ? null : cell.value();
-            }
-
-            if (value == null)
-            {
-                if (operator == Operator.EQ)
-                    return rowValue == null;
-                else if (operator == Operator.NEQ)
-                    return rowValue != null;
-                else
-                    throw new InvalidRequestException(String.format("Invalid comparison with null for operator \"%s\"", operator));
-            }
-
-            return compareWithOperator(operator, userType, value, rowValue);
-        }
-    }
-
-    /** An IN condition on an entire UDT.  For example: IF user IN ({name: 'joe', age: 42}, {name: 'bob', age: 23}). */
-    public static class UDTInBound extends Bound
-    {
-        private final List<ByteBuffer> inValues;
-        private final ProtocolVersion protocolVersion;
-
-        private UDTInBound(ColumnCondition condition, QueryOptions options) throws InvalidRequestException
-        {
-            super(condition.column, condition.operator);
-            assert column.type.isUDT() && condition.field == null;
-            assert condition.operator == Operator.IN;
-            protocolVersion = options.getProtocolVersion();
-            inValues = new ArrayList<>();
-            if (condition.inValues == null)
-            {
-                Lists.Marker inValuesMarker = (Lists.Marker) condition.value;
-                for (ByteBuffer buffer : ((Lists.Value)inValuesMarker.bind(options)).elements)
-                    this.inValues.add(buffer);
-            }
-            else
-            {
-                for (Term value : condition.inValues)
-                    this.inValues.add(value.bindAndGet(options));
-            }
-        }
-
-        public boolean appliesTo(Row row) throws InvalidRequestException
-        {
-            UserType userType = (UserType) column.type;
-            ByteBuffer rowValue;
-            if (userType.isMultiCell())
-            {
-                Iterator<Cell> cells = getCells(row, column);
-                rowValue = cells.hasNext() ? userType.serializeForNativeProtocol(cells, protocolVersion) : null;
-            }
-            else
-            {
-                Cell cell = getCell(row, column);
-                rowValue = cell == null ? null : cell.value();
-            }
-
-            for (ByteBuffer value : inValues)
-            {
-                if (value == null || rowValue == null)
-                {
-                    if (value == rowValue) // both null
-                        return true;
-                }
-                else if (userType.compare(value, rowValue) == 0)
-                {
-                    return true;
-                }
-            }
-            return false;
-        }
-    }
-
-    public static class Raw
-    {
-        private final Term.Raw value;
-        private final List<Term.Raw> inValues;
-        private final AbstractMarker.INRaw inMarker;
-
-        // Can be null, only used with the syntax "IF m[e] = ..." (in which case it's 'e')
-        private final Term.Raw collectionElement;
-
-        // Can be null, only used with the syntax "IF udt.field = ..." (in which case it's 'field')
-        private final FieldIdentifier udtField;
-
-        private final Operator operator;
-
-        private Raw(Term.Raw value, List<Term.Raw> inValues, AbstractMarker.INRaw inMarker, Term.Raw collectionElement,
-                    FieldIdentifier udtField, Operator op)
-        {
-            this.value = value;
-            this.inValues = inValues;
-            this.inMarker = inMarker;
-            this.collectionElement = collectionElement;
-            this.udtField = udtField;
-            this.operator = op;
-        }
-
-        /** A condition on a column. For example: "IF col = 'foo'" */
-        public static Raw simpleCondition(Term.Raw value, Operator op)
-        {
-            return new Raw(value, null, null, null, null, op);
-        }
-
-        /** An IN condition on a column. For example: "IF col IN ('foo', 'bar', ...)" */
-        public static Raw simpleInCondition(List<Term.Raw> inValues)
-        {
-            return new Raw(null, inValues, null, null, null, Operator.IN);
-        }
-
-        /** An IN condition on a column with a single marker. For example: "IF col IN ?" */
-        public static Raw simpleInCondition(AbstractMarker.INRaw inMarker)
-        {
-            return new Raw(null, null, inMarker, null, null, Operator.IN);
-        }
-
-        /** A condition on a collection element. For example: "IF col['key'] = 'foo'" */
-        public static Raw collectionCondition(Term.Raw value, Term.Raw collectionElement, Operator op)
-        {
-            return new Raw(value, null, null, collectionElement, null, op);
-        }
-
-        /** An IN condition on a collection element. For example: "IF col['key'] IN ('foo', 'bar', ...)" */
-        public static Raw collectionInCondition(Term.Raw collectionElement, List<Term.Raw> inValues)
-        {
-            return new Raw(null, inValues, null, collectionElement, null, Operator.IN);
-        }
-
-        /** An IN condition on a collection element with a single marker. For example: "IF col['key'] IN ?" */
-        public static Raw collectionInCondition(Term.Raw collectionElement, AbstractMarker.INRaw inMarker)
-        {
-            return new Raw(null, null, inMarker, collectionElement, null, Operator.IN);
-        }
-
-        /** A condition on a UDT field. For example: "IF col.field = 'foo'" */
-        public static Raw udtFieldCondition(Term.Raw value, FieldIdentifier udtField, Operator op)
-        {
-            return new Raw(value, null, null, null, udtField, op);
-        }
-
-        /** An IN condition on a collection element. For example: "IF col.field IN ('foo', 'bar', ...)" */
-        public static Raw udtFieldInCondition(FieldIdentifier udtField, List<Term.Raw> inValues)
-        {
-            return new Raw(null, inValues, null, null, udtField, Operator.IN);
-        }
-
-        /** An IN condition on a collection element with a single marker. For example: "IF col.field IN ?" */
-        public static Raw udtFieldInCondition(FieldIdentifier udtField, AbstractMarker.INRaw inMarker)
-        {
-            return new Raw(null, null, inMarker, null, udtField, Operator.IN);
-        }
-
-        public ColumnCondition prepare(String keyspace, ColumnDefinition receiver, CFMetaData cfm) throws InvalidRequestException
-        {
-            if (receiver.type instanceof CounterColumnType)
-                throw new InvalidRequestException("Conditions on counters are not supported");
-
-            if (collectionElement != null)
-            {
-                if (!(receiver.type.isCollection()))
-                    throw new InvalidRequestException(String.format("Invalid element access syntax for non-collection column %s", receiver.name));
-
-                ColumnSpecification elementSpec, valueSpec;
-                switch ((((CollectionType) receiver.type).kind))
-                {
-                    case LIST:
-                        elementSpec = Lists.indexSpecOf(receiver);
-                        valueSpec = Lists.valueSpecOf(receiver);
-                        break;
-                    case MAP:
-                        elementSpec = Maps.keySpecOf(receiver);
-                        valueSpec = Maps.valueSpecOf(receiver);
-                        break;
-                    case SET:
-                        throw new InvalidRequestException(String.format("Invalid element access syntax for set column %s", receiver.name));
-                    default:
-                        throw new AssertionError();
-                }
-                if (operator == Operator.IN)
-                {
-                    if (inValues == null)
-                        return ColumnCondition.inCondition(receiver, collectionElement.prepare(keyspace, elementSpec), inMarker.prepare(keyspace, valueSpec));
-                    List<Term> terms = new ArrayList<>(inValues.size());
-                    for (Term.Raw value : inValues)
-                        terms.add(value.prepare(keyspace, valueSpec));
-                    return ColumnCondition.inCondition(receiver, collectionElement.prepare(keyspace, elementSpec), terms);
-                }
-                else
-                {
-                    return ColumnCondition.condition(receiver, collectionElement.prepare(keyspace, elementSpec), value.prepare(keyspace, valueSpec), operator);
-                }
-            }
-            else if (udtField != null)
-            {
-                UserType userType = (UserType) receiver.type;
-                int fieldPosition = userType.fieldPosition(udtField);
-                if (fieldPosition == -1)
-                    throw new InvalidRequestException(String.format("Unknown field %s for column %s", udtField, receiver.name));
-
-                ColumnSpecification fieldReceiver = UserTypes.fieldSpecOf(receiver, fieldPosition);
-                if (operator == Operator.IN)
-                {
-                    if (inValues == null)
-                        return ColumnCondition.inCondition(receiver, udtField, inMarker.prepare(keyspace, fieldReceiver));
-
-                    List<Term> terms = new ArrayList<>(inValues.size());
-                    for (Term.Raw value : inValues)
-                        terms.add(value.prepare(keyspace, fieldReceiver));
-                    return ColumnCondition.inCondition(receiver, udtField, terms);
-                }
-                else
-                {
-                    return ColumnCondition.condition(receiver, udtField, value.prepare(keyspace, fieldReceiver), operator);
-                }
-            }
-            else
-            {
-                if (operator == Operator.IN)
-                {
-                    if (inValues == null)
-                        return ColumnCondition.inCondition(receiver, inMarker.prepare(keyspace, receiver));
-                    List<Term> terms = new ArrayList<>(inValues.size());
-                    for (Term.Raw value : inValues)
-                        terms.add(value.prepare(keyspace, receiver));
-                    return ColumnCondition.inCondition(receiver, terms);
-                }
-                else
-                {
-                    return ColumnCondition.condition(receiver, value.prepare(keyspace, receiver), operator);
-                }
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/ColumnConditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnConditions.java b/src/java/org/apache/cassandra/cql3/ColumnConditions.java
deleted file mode 100644
index cb09b1a..0000000
--- a/src/java/org/apache/cassandra/cql3/ColumnConditions.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.cql3.statements.CQL3CasRequest;
-import org.apache.cassandra.db.Clustering;
-
-/**
- * A set of <code>ColumnCondition</code>s.
- *
- */
-public final class ColumnConditions extends AbstractConditions
-{
-    /**
-     * The conditions on regular columns.
-     */
-    private final List<ColumnCondition> columnConditions;
-
-    /**
-     * The conditions on static columns
-     */
-    private final List<ColumnCondition> staticConditions;
-
-    /**
-     * Creates a new <code>ColumnConditions</code> instance for the specified builder.
-     */
-    private ColumnConditions(Builder builder)
-    {
-        this.columnConditions = builder.columnConditions;
-        this.staticConditions = builder.staticConditions;
-    }
-
-    @Override
-    public boolean appliesToStaticColumns()
-    {
-        return !staticConditions.isEmpty();
-    }
-
-    @Override
-    public boolean appliesToRegularColumns()
-    {
-        return !columnConditions.isEmpty();
-    }
-
-    @Override
-    public Collection<ColumnDefinition> getColumns()
-    {
-        return Stream.concat(columnConditions.stream(), staticConditions.stream())
-                     .map(e -> e.column)
-                     .collect(Collectors.toList());
-    }
-
-    @Override
-    public boolean isEmpty()
-    {
-        return columnConditions.isEmpty() && staticConditions.isEmpty();
-    }
-
-    /**
-     * Adds the conditions to the specified CAS request.
-     *
-     * @param request the request
-     * @param clustering the clustering prefix
-     * @param options the query options
-     */
-    public void addConditionsTo(CQL3CasRequest request,
-                                Clustering clustering,
-                                QueryOptions options)
-    {
-        if (!columnConditions.isEmpty())
-            request.addConditions(clustering, columnConditions, options);
-        if (!staticConditions.isEmpty())
-            request.addConditions(Clustering.STATIC_CLUSTERING, staticConditions, options);
-    }
-
-    @Override
-    public void addFunctionsTo(List<Function> functions)
-    {
-        columnConditions.forEach(p -> p.addFunctionsTo(functions));
-        staticConditions.forEach(p -> p.addFunctionsTo(functions));
-    }
-
-    /**
-     * Creates a new <code>Builder</code> for <code>ColumnConditions</code>.
-     * @return a new <code>Builder</code> for <code>ColumnConditions</code>
-     */
-    public static Builder newBuilder()
-    {
-        return new Builder();
-    }
-
-    /**
-     * A <code>Builder</code> for <code>ColumnConditions</code>.
-     *
-     */
-    public static final class Builder
-    {
-        /**
-         * The conditions on regular columns.
-         */
-        private List<ColumnCondition> columnConditions = Collections.emptyList();
-
-        /**
-         * The conditions on static columns
-         */
-        private List<ColumnCondition> staticConditions = Collections.emptyList();
-
-        /**
-         * Adds the specified <code>ColumnCondition</code> to this set of conditions.
-         * @param condition the condition to add
-         */
-        public Builder add(ColumnCondition condition)
-        {
-            List<ColumnCondition> conds = null;
-            if (condition.column.isStatic())
-            {
-                if (staticConditions.isEmpty())
-                    staticConditions = new ArrayList<>();
-                conds = staticConditions;
-            }
-            else
-            {
-                if (columnConditions.isEmpty())
-                    columnConditions = new ArrayList<>();
-                conds = columnConditions;
-            }
-            conds.add(condition);
-            return this;
-        }
-
-        public ColumnConditions build()
-        {
-            return new ColumnConditions(this);
-        }
-
-        private Builder()
-        {
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/Conditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Conditions.java b/src/java/org/apache/cassandra/cql3/Conditions.java
deleted file mode 100644
index 16fa4aa..0000000
--- a/src/java/org/apache/cassandra/cql3/Conditions.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.util.List;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.cql3.statements.CQL3CasRequest;
-import org.apache.cassandra.db.Clustering;
-
-/**
- * Conditions that can be applied to a mutation statement.
- *
- */
-public interface Conditions
-{
-    /**
-     * An EMPTY condition
-     */
-    static final Conditions EMPTY_CONDITION = ColumnConditions.newBuilder().build();
-
-    /**
-     * IF EXISTS condition
-     */
-    static final Conditions IF_EXISTS_CONDITION = new IfExistsCondition();
-
-    /**
-     * IF NOT EXISTS condition
-     */
-    static final Conditions IF_NOT_EXISTS_CONDITION = new IfNotExistsCondition();
-
-    /**
-     * Adds the functions used by the conditions to the specified list.
-     * @param functions the list to add to
-     */
-    void addFunctionsTo(List<Function> functions);
-
-    /**
-     * Returns the column definitions to which apply the conditions.
-     * @return the column definitions to which apply the conditions.
-     */
-    Iterable<ColumnDefinition> getColumns();
-
-    /**
-     * Checks if this <code>Conditions</code> is empty.
-     * @return <code>true</code> if this <code>Conditions</code> is empty, <code>false</code> otherwise.
-     */
-    boolean isEmpty();
-
-    /**
-     * Checks if this is a IF EXIST condition.
-     * @return <code>true</code> if this is a IF EXIST condition, <code>false</code> otherwise.
-     */
-    boolean isIfExists();
-
-    /**
-     * Checks if this is a IF NOT EXIST condition.
-     * @return <code>true</code> if this is a IF NOT EXIST condition, <code>false</code> otherwise.
-     */
-    boolean isIfNotExists();
-
-    /**
-     * Checks if some of the conditions apply to static columns.
-     *
-     * @return <code>true</code> if some of the conditions apply to static columns, <code>false</code> otherwise.
-     */
-    boolean appliesToStaticColumns();
-
-    /**
-     * Checks if some of the conditions apply to regular columns.
-     *
-     * @return <code>true</code> if some of the conditions apply to regular columns, <code>false</code> otherwise.
-     */
-    boolean appliesToRegularColumns();
-
-    /**
-     * Adds the conditions to the specified CAS request.
-     *
-     * @param request the request
-     * @param clustering the clustering prefix
-     * @param options the query options
-     */
-    public void addConditionsTo(CQL3CasRequest request,
-                                Clustering clustering,
-                                QueryOptions options);
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/IfExistsCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/IfExistsCondition.java b/src/java/org/apache/cassandra/cql3/IfExistsCondition.java
deleted file mode 100644
index a24d8c0..0000000
--- a/src/java/org/apache/cassandra/cql3/IfExistsCondition.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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.cql3.statements.CQL3CasRequest;
-import org.apache.cassandra.db.Clustering;
-
-final class IfExistsCondition extends AbstractConditions
-{
-    @Override
-    public void addConditionsTo(CQL3CasRequest request, Clustering clustering, QueryOptions options)
-    {
-        request.addExist(clustering);
-    }
-
-    @Override
-    public boolean isIfExists()
-    {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/IfNotExistsCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/IfNotExistsCondition.java b/src/java/org/apache/cassandra/cql3/IfNotExistsCondition.java
deleted file mode 100644
index 05cb864..0000000
--- a/src/java/org/apache/cassandra/cql3/IfNotExistsCondition.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 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.cql3.statements.CQL3CasRequest;
-import org.apache.cassandra.db.Clustering;
-
-final class IfNotExistsCondition extends AbstractConditions
-{
-    @Override
-    public void addConditionsTo(CQL3CasRequest request, Clustering clustering, QueryOptions options)
-    {
-        request.addNotExist(clustering);
-    }
-
-    @Override
-    public boolean isIfNotExists()
-    {
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/Operator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Operator.java b/src/java/org/apache/cassandra/cql3/Operator.java
index 07c92f0..f2a6c6b 100644
--- a/src/java/org/apache/cassandra/cql3/Operator.java
+++ b/src/java/org/apache/cassandra/cql3/Operator.java
@@ -37,6 +37,12 @@ public enum Operator
         {
             return "=";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            return type.compareForCQL(leftOperand, rightOperand) == 0;
+        }
     },
     LT(4)
     {
@@ -45,6 +51,12 @@ public enum Operator
         {
             return "<";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            return type.compareForCQL(leftOperand, rightOperand) < 0;
+        }
     },
     LTE(3)
     {
@@ -53,6 +65,12 @@ public enum Operator
         {
             return "<=";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            return type.compareForCQL(leftOperand, rightOperand) <= 0;
+        }
     },
     GTE(1)
     {
@@ -61,6 +79,12 @@ public enum Operator
         {
             return ">=";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            return type.compareForCQL(leftOperand, rightOperand) >= 0;
+        }
     },
     GT(2)
     {
@@ -69,12 +93,56 @@ public enum Operator
         {
             return ">";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            return type.compareForCQL(leftOperand, rightOperand) > 0;
+        }
     },
     IN(7)
     {
+        @Override
+        public String toString()
+        {
+            return "IN";
+        }
+
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            List<?> inValues = ListType.getInstance(type, false).getSerializer().deserialize(rightOperand);
+            return inValues.contains(type.getSerializer().deserialize(leftOperand));
+        }
     },
     CONTAINS(5)
     {
+        @Override
+        public String toString()
+        {
+            return "CONTAINS";
+        }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            switch(((CollectionType<?>) type).kind)
+            {
+                case LIST :
+                    ListType<?> listType = (ListType<?>) type;
+                    List<?> list = listType.getSerializer().deserialize(leftOperand);
+                    return list.contains(listType.getElementsType().getSerializer().deserialize(rightOperand));
+                case SET:
+                    SetType<?> setType = (SetType<?>) type;
+                    Set<?> set = setType.getSerializer().deserialize(leftOperand);
+                    return set.contains(setType.getElementsType().getSerializer().deserialize(rightOperand));
+                case MAP:
+                    MapType<?, ?> mapType = (MapType<?, ?>) type;
+                    Map<?, ?> map = mapType.getSerializer().deserialize(leftOperand);
+                    return map.containsValue(mapType.getValuesType().getSerializer().deserialize(rightOperand));
+                default:
+                    throw new AssertionError();
+            }
+        }
     },
     CONTAINS_KEY(6)
     {
@@ -83,6 +151,14 @@ public enum Operator
         {
             return "CONTAINS KEY";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            MapType<?, ?> mapType = (MapType<?, ?>) type;
+            Map<?, ?> map = mapType.getSerializer().deserialize(leftOperand);
+            return map.containsKey(mapType.getKeysType().getSerializer().deserialize(rightOperand));
+        }
     },
     NEQ(8)
     {
@@ -91,6 +167,13 @@ public enum Operator
         {
             return "!=";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            return type.compareForCQL(leftOperand, rightOperand) != 0;
+
+        }
     },
     IS_NOT(9)
     {
@@ -99,6 +182,12 @@ public enum Operator
         {
             return "IS NOT";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            throw new UnsupportedOperationException();
+        }
     },
     LIKE_PREFIX(10)
     {
@@ -107,6 +196,12 @@ public enum Operator
         {
             return "LIKE '<term>%'";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            return ByteBufferUtil.startsWith(leftOperand, rightOperand);
+        }
     },
     LIKE_SUFFIX(11)
     {
@@ -115,6 +210,12 @@ public enum Operator
         {
             return "LIKE '%<term>'";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            return ByteBufferUtil.endsWith(leftOperand, rightOperand);
+        }
     },
     LIKE_CONTAINS(12)
     {
@@ -123,6 +224,12 @@ public enum Operator
         {
             return "LIKE '%<term>%'";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            return ByteBufferUtil.contains(leftOperand, rightOperand);
+        }
     },
     LIKE_MATCHES(13)
     {
@@ -131,6 +238,11 @@ public enum Operator
         {
             return "LIKE '<term>'";
         }
+
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            return ByteBufferUtil.contains(leftOperand, rightOperand);
+        }
     },
     LIKE(14)
     {
@@ -139,6 +251,12 @@ public enum Operator
         {
             return "LIKE";
         }
+
+        @Override
+        public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
+        {
+            throw new UnsupportedOperationException();
+        }
     };
 
     /**
@@ -190,59 +308,8 @@ public enum Operator
 
     /**
      * Whether 2 values satisfy this operator (given the type they should be compared with).
-     *
-     * @throws AssertionError for CONTAINS and CONTAINS_KEY as this doesn't support those operators yet
      */
-    public boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand)
-    {
-        switch (this)
-        {
-            case EQ:
-                return type.compareForCQL(leftOperand, rightOperand) == 0;
-            case LT:
-                return type.compareForCQL(leftOperand, rightOperand) < 0;
-            case LTE:
-                return type.compareForCQL(leftOperand, rightOperand) <= 0;
-            case GT:
-                return type.compareForCQL(leftOperand, rightOperand) > 0;
-            case GTE:
-                return type.compareForCQL(leftOperand, rightOperand) >= 0;
-            case NEQ:
-                return type.compareForCQL(leftOperand, rightOperand) != 0;
-            case IN:
-                List inValues = ((List) ListType.getInstance(type, false).getSerializer().deserialize(rightOperand));
-                return inValues.contains(type.getSerializer().deserialize(leftOperand));
-            case CONTAINS:
-                if (type instanceof ListType)
-                {
-                    List list = (List) type.getSerializer().deserialize(leftOperand);
-                    return list.contains(((ListType) type).getElementsType().getSerializer().deserialize(rightOperand));
-                }
-                else if (type instanceof SetType)
-                {
-                    Set set = (Set) type.getSerializer().deserialize(leftOperand);
-                    return set.contains(((SetType) type).getElementsType().getSerializer().deserialize(rightOperand));
-                }
-                else  // MapType
-                {
-                    Map map = (Map) type.getSerializer().deserialize(leftOperand);
-                    return map.containsValue(((MapType) type).getValuesType().getSerializer().deserialize(rightOperand));
-                }
-            case CONTAINS_KEY:
-                Map map = (Map) type.getSerializer().deserialize(leftOperand);
-                return map.containsKey(((MapType) type).getKeysType().getSerializer().deserialize(rightOperand));
-            case LIKE_PREFIX:
-                return ByteBufferUtil.startsWith(leftOperand, rightOperand);
-            case LIKE_SUFFIX:
-                return ByteBufferUtil.endsWith(leftOperand, rightOperand);
-            case LIKE_MATCHES:
-            case LIKE_CONTAINS:
-                return ByteBufferUtil.contains(leftOperand, rightOperand);
-            default:
-                // we shouldn't get LIKE, CONTAINS, CONTAINS KEY, or IS NOT here
-                throw new AssertionError();
-        }
-    }
+    public abstract boolean isSatisfiedBy(AbstractType<?> type, ByteBuffer leftOperand, ByteBuffer rightOperand);
 
     public int serializedSize()
     {
@@ -254,4 +321,13 @@ public enum Operator
     {
          return this.name();
     }
+
+    /**
+     * Checks if this operator is an IN operator.
+     * @return {@code true} if this operator is an IN operator, {@code false} otherwise.
+     */
+    public boolean isIN()
+    {
+        return this == IN;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/Terms.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Terms.java b/src/java/org/apache/cassandra/cql3/Terms.java
index 7d3948a..7478366 100644
--- a/src/java/org/apache/cassandra/cql3/Terms.java
+++ b/src/java/org/apache/cassandra/cql3/Terms.java
@@ -18,17 +18,248 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.List;
+import java.util.*;
 
+import org.apache.cassandra.cql3.Term.MultiItemTerminal;
+import org.apache.cassandra.cql3.Term.Terminal;
 import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.transport.ProtocolVersion;
 
-public class Terms
+/**
+ * A set of {@code Terms}
+ */
+public interface Terms
 {
+    /**
+     * The {@code List} returned when the list was not set.
+     */
+    @SuppressWarnings("rawtypes")
+    public static final List UNSET_LIST = new AbstractList()
+    {
+        @Override
+        public Object get(int index)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int size()
+        {
+            return 0;
+        }
+    };
+
+    /**
+     * Adds all functions (native and user-defined) used by any of the terms to the specified list.
+     * @param functions the list to add to
+     */
+    public void addFunctionsTo(List<Function> functions);
+
+    /**
+     * Collects the column specifications for the bind variables in the terms.
+     * This is obviously a no-op if the terms are Terminal.
+     *
+     * @param boundNames the variables specification where to collect the
+     * bind variables of the terms in.
+     */
+    public void collectMarkerSpecification(VariableSpecifications boundNames);
+
+    /**
+     * Bind the values in these terms to the values contained in {@code options}.
+     * This is obviously a no-op if the term is Terminal.
+     *
+     * @param options the values to bind markers to.
+     * @return the result of binding all the variables of these NonTerminals or an {@code UNSET_LIST} if the term
+     * was unset.
+     */
+    public List<Terminal> bind(QueryOptions options);
+
+
+    public List<ByteBuffer> bindAndGet(QueryOptions options);
+
+    /**
+     * Creates a {@code Terms} for the specified list marker.
+     *
+     * @param marker the list  marker
+     * @param type the element type
+     * @return a {@code Terms} for the specified list marker
+     */
+    public static Terms ofListMarker(final Lists.Marker marker, final AbstractType<?> type)
+    {
+        return new Terms()
+        {
+            @Override
+            public void addFunctionsTo(List<Function> functions)
+            {
+            }
+
+            @Override
+            public void collectMarkerSpecification(VariableSpecifications boundNames)
+            {
+                marker.collectMarkerSpecification(boundNames);
+            }
+
+            @Override
+            public List<ByteBuffer> bindAndGet(QueryOptions options)
+            {
+                Terminal terminal = marker.bind(options);
+
+                if (terminal == null)
+                    return null;
+
+                if (terminal == Constants.UNSET_VALUE)
+                    return UNSET_LIST;
+
+                return ((MultiItemTerminal) terminal).getElements();
+            }
+
+            @Override
+            public List<Terminal> bind(QueryOptions options)
+            {
+                Terminal terminal = marker.bind(options);
+
+                if (terminal == null)
+                    return null;
+
+                if (terminal == Constants.UNSET_VALUE)
+                    return UNSET_LIST;
+
+                java.util.function.Function<ByteBuffer, Term.Terminal> deserializer = deserializer(options.getProtocolVersion());
+
+                List<ByteBuffer> boundValues = ((MultiItemTerminal) terminal).getElements();
+                List<Term.Terminal> values = new ArrayList<>(boundValues.size());
+                for (int i = 0, m = boundValues.size(); i < m; i++)
+                {
+                    ByteBuffer buffer = boundValues.get(i);
+                    Term.Terminal value = buffer == null ? null : deserializer.apply(buffer);
+                    values.add(value);
+                }
+                return values;
+            }
+
+            public java.util.function.Function<ByteBuffer, Term.Terminal> deserializer(ProtocolVersion version)
+            {
+                if (type.isCollection())
+                {
+                    switch (((CollectionType<?>) type).kind)
+                    {
+                        case LIST:
+                            return e -> Lists.Value.fromSerialized(e, (ListType<?>) type, version);
+                        case SET:
+                            return e -> Sets.Value.fromSerialized(e, (SetType<?>) type, version);
+                        case MAP:
+                            return e -> Maps.Value.fromSerialized(e, (MapType<?, ?>) type, version);
+                    }
+                    throw new AssertionError();
+                }
+                return e -> new Constants.Value(e);
+            }
+        };
+    }
+
+    /**
+     * Creates a {@code Terms} containing a single {@code Term}.
+     *
+     * @param term the {@code Term}
+     * @return a {@code Terms} containing a single {@code Term}.
+     */
+    public static Terms of(final Term term)
+    {
+        assert !(term instanceof Lists.Marker);
+        return new Terms()
+                {
+                    @Override
+                    public void addFunctionsTo(List<Function> functions)
+                    {
+                        term.addFunctionsTo(functions);
+                    }
+
+                    @Override
+                    public void collectMarkerSpecification(VariableSpecifications boundNames)
+                    {
+                        term.collectMarkerSpecification(boundNames);
+                    }
+
+                    @Override
+                    public List<ByteBuffer> bindAndGet(QueryOptions options)
+                    {
+                        return Collections.singletonList(term.bindAndGet(options));
+                    }
+
+                    @Override
+                    public List<Terminal> bind(QueryOptions options)
+                    {
+                        return Collections.singletonList(term.bind(options));
+                    }
+                };
+    }
+
+    /**
+     * Creates a {@code Terms} containing a set of {@code Term}.
+     *
+     * @param term the {@code Term}
+     * @return a {@code Terms} containing a set of {@code Term}.
+     */
+    public static Terms of(final List<Term> terms)
+    {
+        return new Terms()
+                {
+                    @Override
+                    public void addFunctionsTo(List<Function> functions)
+                    {
+                        addFunctions(terms, functions);
+                    }
+
+                    @Override
+                    public void collectMarkerSpecification(VariableSpecifications boundNames)
+                    {
+                        for (int i = 0, m = terms.size(); i <m; i++)
+                        {
+                            Term term = terms.get(i);
+                            term.collectMarkerSpecification(boundNames);
+                        }
+                    }
+
+                    @Override
+                    public List<Terminal> bind(QueryOptions options)
+                    {
+                        int size = terms.size();
+                        List<Terminal> terminals = new ArrayList<>(size);
+                        for (int i = 0; i < size; i++)
+                        {
+                            Term term = terms.get(i);
+                            terminals.add(term.bind(options));
+                        }
+                        return terminals;
+                    }
+
+                    @Override
+                    public List<ByteBuffer> bindAndGet(QueryOptions options)
+                    {
+                        int size = terms.size();
+                        List<ByteBuffer> buffers = new ArrayList<>(size);
+                        for (int i = 0; i < size; i++)
+                        {
+                            Term term = terms.get(i);
+                            buffers.add(term.bindAndGet(options));
+                        }
+                        return buffers;
+                    }
+                };
+    }
+
+    /**
+     * Adds all functions (native and user-defined) of the specified terms to the list.
+     * @param functions the list to add to
+     */
     public static void addFunctions(Iterable<Term> terms, List<Function> functions)
     {
-        if (terms != null)
-            terms.forEach(t -> t.addFunctionsTo(functions));
+        for (Term term : terms)
+        {
+            if (term != null)
+                term.addFunctionsTo(functions);
+        }
     }
 
     public static ByteBuffer asBytes(String keyspace, String term, AbstractType type)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/conditions/AbstractConditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/conditions/AbstractConditions.java b/src/java/org/apache/cassandra/cql3/conditions/AbstractConditions.java
new file mode 100644
index 0000000..9b6dd7e
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/conditions/AbstractConditions.java
@@ -0,0 +1,64 @@
+/*
+ * 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.conditions;
+
+import java.util.List;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.functions.Function;
+
+/**
+ * Base class for <code>Conditions</code> classes.
+ *
+ */
+abstract class AbstractConditions implements Conditions
+{
+    public void addFunctionsTo(List<Function> functions)
+    {
+    }
+
+    public Iterable<ColumnDefinition> getColumns()
+    {
+        return null;
+    }
+
+    public boolean isEmpty()
+    {
+        return false;
+    }
+
+    public boolean appliesToStaticColumns()
+    {
+        return false;
+    }
+
+    public boolean appliesToRegularColumns()
+    {
+        return false;
+    }
+
+    public boolean isIfExists()
+    {
+        return false;
+    }
+
+    public boolean isIfNotExists()
+    {
+        return false;
+    }
+}