http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java 
b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
new file mode 100644
index 0000000..cfd62f5
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
@@ -0,0 +1,825 @@
+/*
+ * 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.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.*;
+import org.apache.cassandra.cql3.Term.Terminal;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.*;
+
+/**
+ * A CQL3 condition on the value of a column or collection element.  For 
example, "UPDATE .. IF a = 0".
+ */
+public abstract class ColumnCondition
+{
+    public final ColumnDefinition column;
+    public final Operator operator;
+    private final Terms terms;
+
+    private ColumnCondition(ColumnDefinition column, Operator op, Terms terms)
+    {
+        this.column = column;
+        this.operator = op;
+        this.terms = terms;
+    }
+
+    /**
+     * Adds functions for the bind variables of this operation.
+     *
+     * @param functions the list of functions to get add
+     */
+    public void addFunctionsTo(List<Function> functions)
+    {
+        terms.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)
+    {
+        terms.collectMarkerSpecification(boundNames);
+    }
+
+    public abstract ColumnCondition.Bound bind(QueryOptions options);
+
+    protected final List<ByteBuffer> bindAndGetTerms(QueryOptions options)
+    {
+        return 
filterUnsetValuesIfNeeded(checkValues(terms.bindAndGet(options)));
+    }
+
+    protected final List<Terminal> bindTerms(QueryOptions options)
+    {
+        return filterUnsetValuesIfNeeded(checkValues(terms.bind(options)));
+    }
+
+    /**
+     * Checks that the output of a bind operations on {@code Terms} is a valid 
one.
+     * @param values the list to check
+     * @return the input list
+     */
+    private <T> List<T> checkValues(List<T> values)
+    {
+        checkFalse(values == null && operator.isIN(), "Invalid null list in IN 
condition");
+        checkFalse(values == Terms.UNSET_LIST, "Invalid 'unset' value in 
condition");
+        return values;
+    }
+
+    private <T> List<T> filterUnsetValuesIfNeeded(List<T> values)
+    {
+        if (!operator.isIN())
+            return values;
+
+        List<T> filtered = new ArrayList<>(values.size());
+        for (int i = 0, m = values.size(); i < m; i++)
+        {
+            T value = values.get(i);
+            // The value can be ByteBuffer or Constants.Value so we need to 
check the 2 type of UNSET
+            if (value != ByteBufferUtil.UNSET_BYTE_BUFFER && value != 
Constants.UNSET_VALUE)
+                filtered.add(value);
+        }
+        return filtered;
+    }
+
+    /**
+     * Simple condition (e.g. <pre>IF v = 1</pre>).
+     */
+    private static final class SimpleColumnCondition extends ColumnCondition
+    {
+        public SimpleColumnCondition(ColumnDefinition column, Operator op, 
Terms values)
+        {
+            super(column, op, values);
+        }
+
+        public Bound bind(QueryOptions options)
+        {
+            if (column.type.isCollection() && column.type.isMultiCell())
+                return new MultiCellCollectionBound(column, operator, 
bindTerms(options));
+
+            if (column.type.isUDT() && column.type.isMultiCell())
+                return new MultiCellUdtBound(column, operator, 
bindAndGetTerms(options), options.getProtocolVersion());
+
+            return new SimpleBound(column, operator, bindAndGetTerms(options));
+        }
+    }
+
+    /**
+     * A condition on a collection element (e.g. <pre>IF l[1] = 1</pre>).
+     */
+    private static class CollectionElementCondition extends ColumnCondition
+    {
+        private final Term collectionElement;
+
+        public CollectionElementCondition(ColumnDefinition column, Term 
collectionElement, Operator op, Terms values)
+        {
+            super(column, op, values);
+            this.collectionElement = collectionElement;
+        }
+
+        public void addFunctionsTo(List<Function> functions)
+        {
+            collectionElement.addFunctionsTo(functions);
+            super.addFunctionsTo(functions);
+        }
+
+        public void collectMarkerSpecification(VariableSpecifications 
boundNames)
+        {
+            collectionElement.collectMarkerSpecification(boundNames);
+            super.collectMarkerSpecification(boundNames);
+        }
+
+        public Bound bind(QueryOptions options)
+        {
+            return new ElementAccessBound(column, 
collectionElement.bindAndGet(options), operator, bindAndGetTerms(options));
+        }
+    }
+
+    /**
+     *  A condition on a UDT field (e.g. <pre>IF v.a = 1</pre>).
+     */
+    private final static class UDTFieldCondition extends ColumnCondition
+    {
+        private final FieldIdentifier udtField;
+
+        public UDTFieldCondition(ColumnDefinition column, FieldIdentifier 
udtField, Operator op, Terms values)
+        {
+            super(column, op, values);
+            assert udtField != null;
+            this.udtField = udtField;
+        }
+
+        public Bound bind(QueryOptions options)
+        {
+            return new UDTFieldAccessBound(column, udtField, operator, 
bindAndGetTerms(options));
+        }
+    }
+
+    /**
+     *  A regular column, simple condition.
+     */
+    public static ColumnCondition condition(ColumnDefinition column, Operator 
op, Terms terms)
+    {
+        return new SimpleColumnCondition(column, op, terms);
+    }
+
+    /**
+     * A collection column, simple condition.
+     */
+    public static ColumnCondition condition(ColumnDefinition column, Term 
collectionElement, Operator op, Terms terms)
+    {
+        return new CollectionElementCondition(column, collectionElement, op, 
terms);
+    }
+
+    /**
+     * A UDT column, simple condition.
+     */
+    public static ColumnCondition condition(ColumnDefinition column, 
FieldIdentifier udtField, Operator op, Terms terms)
+    {
+        return new UDTFieldCondition(column, udtField, op, terms);
+    }
+
+    public static abstract class Bound
+    {
+        public final ColumnDefinition column;
+        public final Operator comparisonOperator;
+
+        protected Bound(ColumnDefinition column, Operator operator)
+        {
+            this.column = column;
+            // If the operator is an IN we want to compare the value using an 
EQ.
+            this.comparisonOperator = operator.isIN() ? Operator.EQ : operator;
+        }
+
+        /**
+         * Validates whether this condition applies to {@code current}.
+         */
+        public abstract boolean appliesTo(Row row);
+
+        public ByteBuffer getCollectionElementValue()
+        {
+            return null;
+        }
+
+        /** Returns true if the operator is satisfied (i.e. "otherValue 
operator value == true"), false otherwise. */
+        protected static boolean compareWithOperator(Operator operator, 
AbstractType<?> type, ByteBuffer value, ByteBuffer otherValue)
+        {
+            if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                throw invalidRequest("Invalid 'unset' value in condition");
+
+            if (value == null)
+            {
+                switch (operator)
+                {
+                    case EQ:
+                        return otherValue == null;
+                    case NEQ:
+                        return otherValue != null;
+                    default:
+                        throw invalidRequest("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);
+        }
+    }
+
+    protected static final 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);
+    }
+
+    protected static final 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);
+    }
+
+    protected static final 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();
+    }
+
+    protected static final 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();
+        }
+    }
+
+    /**
+     * A condition on a single non-collection column.
+     */
+    private static final class SimpleBound extends Bound
+    {
+        /**
+         * The condition values
+         */
+        private final List<ByteBuffer> values;
+
+        private SimpleBound(ColumnDefinition column, Operator operator, 
List<ByteBuffer> values)
+        {
+            super(column, operator);
+            this.values = values;
+        }
+
+        @Override
+        public boolean appliesTo(Row row)
+        {
+            return isSatisfiedBy(rowValue(row));
+        }
+
+        private ByteBuffer rowValue(Row row)
+        {
+            Cell c = getCell(row, column);
+            return c == null ? null : c.value();
+        }
+
+        private boolean isSatisfiedBy(ByteBuffer rowValue)
+        {
+            for (ByteBuffer value : values)
+            {
+                if (compareWithOperator(comparisonOperator, column.type, 
value, rowValue))
+                    return true;
+            }
+            return false;
+        }
+    }
+
+    /**
+     * A condition on an element of a collection column.
+     */
+    private static final class ElementAccessBound extends Bound
+    {
+        /**
+         * The collection element
+         */
+        private final ByteBuffer collectionElement;
+
+        /**
+         * The conditions values.
+         */
+        private final List<ByteBuffer> values;
+
+        private ElementAccessBound(ColumnDefinition column,
+                                     ByteBuffer collectionElement,
+                                     Operator operator,
+                                     List<ByteBuffer> values)
+        {
+            super(column, operator);
+
+            this.collectionElement = collectionElement;
+            this.values = values;
+        }
+
+        @Override
+        public boolean appliesTo(Row row)
+        {
+            boolean isMap = column.type instanceof MapType;
+
+            if (collectionElement == null)
+                throw invalidRequest("Invalid null value for %s element 
access", isMap ? "map" : "list");
+
+            if (isMap)
+            {
+                MapType<?, ?> mapType = (MapType<?, ?>) column.type;
+                ByteBuffer rowValue = rowMapValue(mapType, row);
+                return isSatisfiedBy(mapType.getKeysType(), rowValue);
+            }
+
+            ListType<?> listType = (ListType<?>) column.type;
+            ByteBuffer rowValue = rowListValue(listType, row);
+            return isSatisfiedBy(listType.getElementsType(), rowValue);
+        }
+
+        private ByteBuffer rowMapValue(MapType<?, ?> type, Row row)
+        {
+            if (column.type.isMultiCell())
+            {
+                Cell cell = getCell(row, column, 
CellPath.create(collectionElement));
+                return cell == null ? null : cell.value();
+            }
+
+            Cell cell = getCell(row, column);
+            return cell == null
+                    ? null
+                    : type.getSerializer().getSerializedValue(cell.value(), 
collectionElement, type.getKeysType());
+        }
+
+        private ByteBuffer rowListValue(ListType<?> type, Row row)
+        {
+            if (column.type.isMultiCell())
+                return cellValueAtIndex(getCells(row, column), 
getListIndex(collectionElement));
+
+            Cell cell = getCell(row, column);
+            return cell == null
+                    ? null
+                    : type.getSerializer().getElement(cell.value(), 
getListIndex(collectionElement));
+        }
+
+        private static ByteBuffer cellValueAtIndex(Iterator<Cell> iter, int 
index)
+        {
+            int adv = Iterators.advance(iter, index);
+            if (adv == index && iter.hasNext())
+                return iter.next().value();
+
+            return null;
+        }
+
+        private boolean isSatisfiedBy(AbstractType<?> valueType, ByteBuffer 
rowValue)
+        {
+            for (ByteBuffer value : values)
+            {
+                if (compareWithOperator(comparisonOperator, valueType, value, 
rowValue))
+                    return true;
+            }
+            return false;
+        }
+
+        @Override
+        public ByteBuffer getCollectionElementValue()
+        {
+            return collectionElement;
+        }
+
+        private static int getListIndex(ByteBuffer collectionElement)
+        {
+            int idx = ByteBufferUtil.toInt(collectionElement);
+            checkFalse(idx < 0, "Invalid negative list index %d", idx);
+            return idx;
+        }
+    }
+
+    /**
+     * A condition on an entire collection column.
+     */
+    private static final class MultiCellCollectionBound extends Bound
+    {
+        private final List<Term.Terminal> values;
+
+        public MultiCellCollectionBound(ColumnDefinition column, Operator 
operator, List<Term.Terminal> values)
+        {
+            super(column, operator);
+            assert column.type.isMultiCell();
+            this.values = values;
+        }
+
+        public boolean appliesTo(Row row)
+        {
+            CollectionType<?> type = (CollectionType<?>)column.type;
+
+            // copy iterator contents so that we can properly reuse them for 
each comparison with an IN value
+            for (Term.Terminal value : values)
+            {
+                Iterator<Cell> iter = getCells(row, column);
+                if (value == null)
+                {
+                    if (comparisonOperator == Operator.EQ)
+                    {
+                        if (!iter.hasNext())
+                            return true;
+                        continue;
+                    }
+
+                    if (comparisonOperator == Operator.NEQ)
+                        return iter.hasNext();
+
+                    throw invalidRequest("Invalid comparison with null for 
operator \"%s\"", comparisonOperator);
+                }
+
+                if (valueAppliesTo(type, iter, value, comparisonOperator))
+                    return true;
+            }
+            return false;
+        }
+
+        private 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;
+        }
+
+        private static boolean listAppliesTo(ListType<?> type, Iterator<Cell> 
iter, List<ByteBuffer> elements, Operator operator)
+        {
+            return setOrListAppliesTo(type.getElementsType(), iter, 
elements.iterator(), operator, false);
+        }
+
+        private static boolean setAppliesTo(SetType<?> type, Iterator<Cell> 
iter, Set<ByteBuffer> elements, Operator operator)
+        {
+            ArrayList<ByteBuffer> sortedElements = new ArrayList<>(elements);
+            Collections.sort(sortedElements, type.getElementsType());
+            return setOrListAppliesTo(type.getElementsType(), iter, 
sortedElements.iterator(), operator, true);
+        }
+
+        private 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;
+        }
+    }
+
+    /**
+     * A condition on a UDT field
+     */
+    private static final class UDTFieldAccessBound extends Bound
+    {
+        /**
+         * The UDT field.
+         */
+        private final FieldIdentifier field;
+
+        /**
+         * The conditions values.
+         */
+        private final List<ByteBuffer> values;
+
+        private UDTFieldAccessBound(ColumnDefinition column, FieldIdentifier 
field, Operator operator, List<ByteBuffer> values)
+        {
+            super(column, operator);
+            assert column.type.isUDT() && field != null;
+            this.field = field;
+            this.values = values;
+        }
+
+        @Override
+        public boolean appliesTo(Row row)
+        {
+            return isSatisfiedBy(rowValue(row));
+        }
+
+        private ByteBuffer rowValue(Row row)
+        {
+            UserType userType = (UserType) column.type;
+
+            if (column.type.isMultiCell())
+            {
+                Cell cell = getCell(row, column, 
userType.cellPathForField(field));
+                return cell == null ? null : cell.value();
+            }
+
+            Cell cell = getCell(row, column);
+            return cell == null
+                      ? null
+                      : 
userType.split(cell.value())[userType.fieldPosition(field)];
+        }
+
+        private boolean isSatisfiedBy(ByteBuffer rowValue)
+        {
+            UserType userType = (UserType) column.type;
+            int fieldPosition = userType.fieldPosition(field);
+            AbstractType<?> valueType = userType.fieldType(fieldPosition);
+            for (ByteBuffer value : values)
+            {
+                if (compareWithOperator(comparisonOperator, valueType, value, 
rowValue))
+                    return true;
+            }
+            return false;
+        }
+    }
+
+    /**
+     * A condition on an entire UDT.
+     */
+    private static final class MultiCellUdtBound extends Bound
+    {
+        /**
+         * The conditions values.
+         */
+        private final List<ByteBuffer> values;
+
+        /**
+         * The protocol version
+         */
+        private final ProtocolVersion protocolVersion;
+
+        private MultiCellUdtBound(ColumnDefinition column, Operator op, 
List<ByteBuffer> values, ProtocolVersion protocolVersion)
+        {
+            super(column, op);
+            assert column.type.isMultiCell();
+            this.values = values;
+            this.protocolVersion = protocolVersion;
+        }
+
+        @Override
+        public boolean appliesTo(Row row)
+        {
+            return isSatisfiedBy(rowValue(row));
+        }
+
+        private final ByteBuffer rowValue(Row row)
+        {
+            UserType userType = (UserType) column.type;
+            Iterator<Cell> iter = getCells(row, column);
+            return iter.hasNext() ? userType.serializeForNativeProtocol(iter, 
protocolVersion) : null;
+        }
+
+        private boolean isSatisfiedBy(ByteBuffer rowValue)
+        {
+            for (ByteBuffer value : values)
+            {
+                if (compareWithOperator(comparisonOperator, column.type, 
value, rowValue))
+                    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)
+        {
+            if (receiver.type instanceof CounterColumnType)
+                throw invalidRequest("Conditions on counters are not 
supported");
+
+            if (collectionElement != null)
+            {
+                if (!(receiver.type.isCollection()))
+                    throw invalidRequest("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 invalidRequest("Invalid element access syntax 
for set column %s", receiver.name);
+                    default:
+                        throw new AssertionError();
+                }
+
+                return condition(receiver, collectionElement.prepare(keyspace, 
elementSpec), operator, prepareTerms(keyspace, valueSpec));
+            }
+
+            if (udtField != null)
+            {
+                UserType userType = (UserType) receiver.type;
+                int fieldPosition = userType.fieldPosition(udtField);
+                if (fieldPosition == -1)
+                    throw invalidRequest("Unknown field %s for column %s", 
udtField, receiver.name);
+
+                ColumnSpecification fieldReceiver = 
UserTypes.fieldSpecOf(receiver, fieldPosition);
+                return condition(receiver, udtField, operator, 
prepareTerms(keyspace, fieldReceiver));
+            }
+
+            return condition(receiver, operator, prepareTerms(keyspace, 
receiver));
+        }
+
+        private Terms prepareTerms(String keyspace, ColumnSpecification 
receiver)
+        {
+            if (operator.isIN())
+            {
+                return inValues == null ? 
Terms.ofListMarker(inMarker.prepare(keyspace, receiver), receiver.type)
+                                        : Terms.of(prepareTerms(keyspace, 
receiver, inValues));
+            }
+
+            return Terms.of(value.prepare(keyspace, receiver));
+        }
+
+        private static List<Term> prepareTerms(String keyspace, 
ColumnSpecification receiver, List<Term.Raw> raws)
+        {
+            List<Term> terms = new ArrayList<>(raws.size());
+            for (int i = 0, m = raws.size(); i < m; i++)
+            {
+                Term.Raw raw = raws.get(i);
+                terms.add(raw.prepare(keyspace, receiver));
+            }
+            return terms;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java 
b/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java
new file mode 100644
index 0000000..b0378b7
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java
@@ -0,0 +1,165 @@
+/*
+ * 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.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.QueryOptions;
+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/Conditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/conditions/Conditions.java 
b/src/java/org/apache/cassandra/cql3/conditions/Conditions.java
new file mode 100644
index 0000000..0214fcf
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/conditions/Conditions.java
@@ -0,0 +1,103 @@
+/*
+ * 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.QueryOptions;
+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/conditions/IfExistsCondition.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/cql3/conditions/IfExistsCondition.java 
b/src/java/org/apache/cassandra/cql3/conditions/IfExistsCondition.java
new file mode 100644
index 0000000..fa8822d
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/conditions/IfExistsCondition.java
@@ -0,0 +1,37 @@
+/*
+ * 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 org.apache.cassandra.cql3.QueryOptions;
+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/conditions/IfNotExistsCondition.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/cql3/conditions/IfNotExistsCondition.java 
b/src/java/org/apache/cassandra/cql3/conditions/IfNotExistsCondition.java
new file mode 100644
index 0000000..ede9119
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/conditions/IfNotExistsCondition.java
@@ -0,0 +1,37 @@
+/*
+ * 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 org.apache.cassandra.cql3.QueryOptions;
+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/statements/CQL3CasRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java 
b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
index c203ace..4cf2baf 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
@@ -24,6 +24,7 @@ import com.google.common.collect.*;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.conditions.ColumnCondition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.rows.Row;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java 
b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index 26b25de..10f3d6f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -22,6 +22,8 @@ import java.util.List;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.conditions.ColumnCondition;
+import org.apache.cassandra.cql3.conditions.Conditions;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.Slice;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java 
b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 6f2ab44..601ddf7 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -30,6 +30,9 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.ColumnDefinition.Raw;
 import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.conditions.ColumnCondition;
+import org.apache.cassandra.cql3.conditions.ColumnConditions;
+import org.apache.cassandra.cql3.conditions.Conditions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 import org.apache.cassandra.cql3.selection.Selection;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java 
b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index 86d7171..01d47bd 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -24,6 +24,8 @@ import java.util.List;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.conditions.ColumnCondition;
+import org.apache.cassandra.cql3.conditions.Conditions;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.CompactTables;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e71a49e8/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java 
b/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
deleted file mode 100644
index ca0c182..0000000
--- a/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
+++ /dev/null
@@ -1,589 +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 org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.rows.BufferCell;
-import org.apache.cassandra.db.rows.Cell;
-import org.apache.cassandra.db.rows.CellPath;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.serializers.Int32Serializer;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.UUIDGen;
-
-import static org.junit.Assert.*;
-
-public class ColumnConditionTest
-{
-    private static final CellPath LIST_PATH = 
CellPath.create(ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()));
-
-    public static final ByteBuffer ZERO = Int32Type.instance.fromString("0");
-    public static final ByteBuffer ONE = Int32Type.instance.fromString("1");
-    public static final ByteBuffer TWO = Int32Type.instance.fromString("2");
-
-    public static final ByteBuffer A = AsciiType.instance.fromString("a");
-    public static final ByteBuffer B = AsciiType.instance.fromString("b");
-
-    @BeforeClass
-    public static void setupDD()
-    {
-        DatabaseDescriptor.daemonInitialization();
-    }
-
-    private static boolean isSatisfiedBy(ColumnCondition.Bound bound, 
ByteBuffer conditionValue, ByteBuffer columnValue) throws 
InvalidRequestException
-    {
-        Cell cell = null;
-        if (columnValue != null)
-        {
-            ColumnDefinition definition = ColumnDefinition.regularDef("ks", 
"cf", "c", ListType.getInstance(Int32Type.instance, true));
-            cell = testCell(definition, columnValue, LIST_PATH);
-        }
-        return bound.isSatisfiedByValue(conditionValue, cell, 
Int32Type.instance, bound.operator);
-    }
-
-    private static Cell testCell(ColumnDefinition column, ByteBuffer value, 
CellPath path)
-    {
-        return new BufferCell(column, 0L, Cell.NO_TTL, Cell.NO_DELETION_TIME, 
value, path);
-    }
-
-    private static void assertThrowsIRE(ColumnCondition.Bound bound, 
ByteBuffer conditionValue, ByteBuffer columnValue)
-    {
-        try
-        {
-            isSatisfiedBy(bound, conditionValue, columnValue);
-            fail("Expected InvalidRequestException was not thrown");
-        } catch (InvalidRequestException e) { }
-    }
-
-    @Test
-    public void testSimpleBoundIsSatisfiedByValue() throws 
InvalidRequestException
-    {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", 
"c", ListType.getInstance(Int32Type.instance, true));
-
-        // EQ
-        ColumnCondition condition = ColumnCondition.condition(definition, new 
Constants.Value(ONE), Operator.EQ);
-        ColumnCondition.Bound bound = condition.bind(QueryOptions.DEFAULT);
-        assertTrue(isSatisfiedBy(bound, ONE, ONE));
-        assertFalse(isSatisfiedBy(bound, ZERO, ONE));
-        assertFalse(isSatisfiedBy(bound, TWO, ONE));
-        assertFalse(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE));
-        assertFalse(isSatisfiedBy(bound, ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertTrue(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertTrue(isSatisfiedBy(bound, null, null));
-        assertFalse(isSatisfiedBy(bound, ONE, null));
-        assertFalse(isSatisfiedBy(bound, null, ONE));
-
-        // NEQ
-        condition = ColumnCondition.condition(definition, new 
Constants.Value(ONE), Operator.NEQ);
-        bound = condition.bind(QueryOptions.DEFAULT);
-        assertFalse(isSatisfiedBy(bound, ONE, ONE));
-        assertTrue(isSatisfiedBy(bound, ZERO, ONE));
-        assertTrue(isSatisfiedBy(bound, TWO, ONE));
-        assertTrue(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE));
-        assertTrue(isSatisfiedBy(bound, ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertFalse(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertFalse(isSatisfiedBy(bound, null, null));
-        assertTrue(isSatisfiedBy(bound, ONE, null));
-        assertTrue(isSatisfiedBy(bound, null, ONE));
-
-        // LT
-        condition = ColumnCondition.condition(definition, new 
Constants.Value(ONE), Operator.LT);
-        bound = condition.bind(QueryOptions.DEFAULT);
-        assertFalse(isSatisfiedBy(bound, ONE, ONE));
-        assertFalse(isSatisfiedBy(bound, ZERO, ONE));
-        assertTrue(isSatisfiedBy(bound, TWO, ONE));
-        assertFalse(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE));
-        assertTrue(isSatisfiedBy(bound, ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertFalse(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertThrowsIRE(bound, null, ONE);
-        assertFalse(isSatisfiedBy(bound, ONE, null));
-
-        // LTE
-        condition = ColumnCondition.condition(definition, new 
Constants.Value(ONE), Operator.LTE);
-        bound = condition.bind(QueryOptions.DEFAULT);
-        assertTrue(isSatisfiedBy(bound, ONE, ONE));
-        assertFalse(isSatisfiedBy(bound, ZERO, ONE));
-        assertTrue(isSatisfiedBy(bound, TWO, ONE));
-        assertFalse(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE));
-        assertTrue(isSatisfiedBy(bound, ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertTrue(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertThrowsIRE(bound, null, ONE);
-        assertFalse(isSatisfiedBy(bound, ONE, null));
-
-        // GT
-        condition = ColumnCondition.condition(definition, new 
Constants.Value(ONE), Operator.GT);
-        bound = condition.bind(QueryOptions.DEFAULT);
-        assertFalse(isSatisfiedBy(bound, ONE, ONE));
-        assertTrue(isSatisfiedBy(bound, ZERO, ONE));
-        assertFalse(isSatisfiedBy(bound, TWO, ONE));
-        assertTrue(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE));
-        assertFalse(isSatisfiedBy(bound, ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertFalse(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertThrowsIRE(bound, null, ONE);
-        assertFalse(isSatisfiedBy(bound, ONE, null));
-
-        // GT
-        condition = ColumnCondition.condition(definition, new 
Constants.Value(ONE), Operator.GTE);
-        bound = condition.bind(QueryOptions.DEFAULT);
-        assertTrue(isSatisfiedBy(bound, ONE, ONE));
-        assertTrue(isSatisfiedBy(bound, ZERO, ONE));
-        assertFalse(isSatisfiedBy(bound, TWO, ONE));
-        assertTrue(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE));
-        assertFalse(isSatisfiedBy(bound, ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertTrue(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ByteBufferUtil.EMPTY_BYTE_BUFFER));
-        assertThrowsIRE(bound, null, ONE);
-        assertFalse(isSatisfiedBy(bound, ONE, null));
-    }
-
-    private static List<ByteBuffer> list(ByteBuffer... values)
-    {
-        return Arrays.asList(values);
-    }
-
-    private static boolean listAppliesTo(ColumnCondition.CollectionBound 
bound, List<ByteBuffer> conditionValues, List<ByteBuffer> columnValues)
-    {
-        CFMetaData cfm = CFMetaData.compile("create table foo(a int PRIMARY 
KEY, b int, c list<int>)", "ks");
-        Map<ByteBuffer, CollectionType> typeMap = new HashMap<>();
-        typeMap.put(ByteBufferUtil.bytes("c"), 
ListType.getInstance(Int32Type.instance, true));
-
-        ColumnDefinition definition = ColumnDefinition.regularDef(cfm, 
ByteBufferUtil.bytes("c"), ListType.getInstance(Int32Type.instance, true));
-
-        List<Cell> cells = new ArrayList<>(columnValues.size());
-        if (columnValues != null)
-        {
-            for (int i = 0; i < columnValues.size(); i++)
-            {
-                ByteBuffer key = Int32Serializer.instance.serialize(i);
-                ByteBuffer value = columnValues.get(i);
-                cells.add(testCell(definition, value, CellPath.create(key)));
-            };
-        }
-
-        return bound.listAppliesTo(ListType.getInstance(Int32Type.instance, 
true), cells == null ? null : cells.iterator(), conditionValues, 
bound.operator);
-    }
-
-    @Test
-    // sets use the same check as lists
-    public void testListCollectionBoundAppliesTo() throws 
InvalidRequestException
-    {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", 
"c", ListType.getInstance(Int32Type.instance, true));
-
-        // EQ
-        ColumnCondition condition = ColumnCondition.condition(definition, new 
Lists.Value(Arrays.asList(ONE)), Operator.EQ);
-        ColumnCondition.CollectionBound bound = 
(ColumnCondition.CollectionBound) condition.bind(QueryOptions.DEFAULT);
-        assertTrue(listAppliesTo(bound, list(ONE), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(), list()));
-        assertFalse(listAppliesTo(bound, list(ZERO), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE), list(ZERO)));
-        assertFalse(listAppliesTo(bound, list(ONE), list(ONE, ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE, ONE), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE), list()));
-
-        assertFalse(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertTrue(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // NEQ
-        condition = ColumnCondition.condition(definition, new 
Lists.Value(Arrays.asList(ONE)), Operator.NEQ);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-        assertFalse(listAppliesTo(bound, list(ONE), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(), list()));
-        assertTrue(listAppliesTo(bound, list(ZERO), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE), list(ZERO)));
-        assertTrue(listAppliesTo(bound, list(ONE), list(ONE, ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE, ONE), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE), list()));
-
-        assertTrue(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertFalse(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // LT
-        condition = ColumnCondition.condition(definition, new 
Lists.Value(Arrays.asList(ONE)), Operator.LT);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-        assertFalse(listAppliesTo(bound, list(ONE), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(), list()));
-        assertFalse(listAppliesTo(bound, list(ZERO), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE), list(ZERO)));
-        assertFalse(listAppliesTo(bound, list(ONE), list(ONE, ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE, ONE), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE), list()));
-
-        assertFalse(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertFalse(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // LTE
-        condition = ColumnCondition.condition(definition, new 
Lists.Value(Arrays.asList(ONE)), Operator.LTE);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-        assertTrue(listAppliesTo(bound, list(ONE), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(), list()));
-        assertFalse(listAppliesTo(bound, list(ZERO), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE), list(ZERO)));
-        assertFalse(listAppliesTo(bound, list(ONE), list(ONE, ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE, ONE), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE), list()));
-
-        assertFalse(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertTrue(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // GT
-        condition = ColumnCondition.condition(definition, new 
Lists.Value(Arrays.asList(ONE)), Operator.GT);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-        assertFalse(listAppliesTo(bound, list(ONE), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(), list()));
-        assertTrue(listAppliesTo(bound, list(ZERO), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE), list(ZERO)));
-        assertTrue(listAppliesTo(bound, list(ONE), list(ONE, ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE, ONE), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE), list()));
-
-        assertTrue(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertFalse(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // GTE
-        condition = ColumnCondition.condition(definition, new 
Lists.Value(Arrays.asList(ONE)), Operator.GTE);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-        assertTrue(listAppliesTo(bound, list(ONE), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(), list()));
-        assertTrue(listAppliesTo(bound, list(ZERO), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE), list(ZERO)));
-        assertTrue(listAppliesTo(bound, list(ONE), list(ONE, ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE, ONE), list(ONE)));
-        assertTrue(listAppliesTo(bound, list(), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE), list()));
-
-        assertTrue(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ONE)));
-        assertFalse(listAppliesTo(bound, list(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertTrue(listAppliesTo(bound, 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-    }
-
-    private static SortedSet<ByteBuffer> set(ByteBuffer... values)
-    {
-        SortedSet<ByteBuffer> results = new TreeSet<>(Int32Type.instance);
-        results.addAll(Arrays.asList(values));
-        return results;
-    }
-
-    private static boolean setAppliesTo(ColumnCondition.CollectionBound bound, 
Set<ByteBuffer> conditionValues, List<ByteBuffer> columnValues)
-    {
-        CFMetaData cfm = CFMetaData.compile("create table foo(a int PRIMARY 
KEY, b int, c set<int>)", "ks");
-        Map<ByteBuffer, CollectionType> typeMap = new HashMap<>();
-        typeMap.put(ByteBufferUtil.bytes("c"), 
SetType.getInstance(Int32Type.instance, true));
-        ColumnDefinition definition = ColumnDefinition.regularDef(cfm, 
ByteBufferUtil.bytes("c"), SetType.getInstance(Int32Type.instance, true));
-
-        List<Cell> cells = new ArrayList<>(columnValues.size());
-        if (columnValues != null)
-        {
-            for (int i = 0; i < columnValues.size(); i++)
-            {
-                ByteBuffer key = columnValues.get(i);
-                cells.add(testCell(definition, 
ByteBufferUtil.EMPTY_BYTE_BUFFER, CellPath.create(key)));
-            };
-        }
-
-        return bound.setAppliesTo(SetType.getInstance(Int32Type.instance, 
true), cells == null ? null : cells.iterator(), conditionValues, 
bound.operator);
-    }
-
-    @Test
-    public void testSetCollectionBoundAppliesTo() throws 
InvalidRequestException
-    {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", 
"c", ListType.getInstance(Int32Type.instance, true));
-
-        // EQ
-        ColumnCondition condition = ColumnCondition.condition(definition, new 
Sets.Value(set(ONE)), Operator.EQ);
-        ColumnCondition.CollectionBound bound = 
(ColumnCondition.CollectionBound) condition.bind(QueryOptions.DEFAULT);
-        assertTrue(setAppliesTo(bound, set(ONE), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(), list()));
-        assertFalse(setAppliesTo(bound, set(ZERO), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(ONE), list(ZERO)));
-        assertFalse(setAppliesTo(bound, set(ONE), list(ONE, TWO)));
-        assertFalse(setAppliesTo(bound, set(ONE, TWO), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(ONE), list()));
-
-        assertFalse(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ONE)));
-        assertFalse(setAppliesTo(bound, set(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertTrue(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // NEQ
-        condition = ColumnCondition.condition(definition, new 
Sets.Value(set(ONE)), Operator.NEQ);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-        assertFalse(setAppliesTo(bound, set(ONE), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(), list()));
-        assertTrue(setAppliesTo(bound, set(ZERO), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(ONE), list(ZERO)));
-        assertTrue(setAppliesTo(bound, set(ONE), list(ONE, TWO)));
-        assertTrue(setAppliesTo(bound, set(ONE, TWO), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(ONE), list()));
-
-        assertTrue(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ONE)));
-        assertTrue(setAppliesTo(bound, set(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertFalse(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // LT
-        condition = ColumnCondition.condition(definition, new 
Lists.Value(Arrays.asList(ONE)), Operator.LT);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-        assertFalse(setAppliesTo(bound, set(ONE), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(), list()));
-        assertFalse(setAppliesTo(bound, set(ZERO), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(ONE), list(ZERO)));
-        assertFalse(setAppliesTo(bound, set(ONE), list(ONE, TWO)));
-        assertTrue(setAppliesTo(bound, set(ONE, TWO), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(ONE), list()));
-
-        assertFalse(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ONE)));
-        assertTrue(setAppliesTo(bound, set(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertFalse(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // LTE
-        condition = ColumnCondition.condition(definition, new 
Lists.Value(Arrays.asList(ONE)), Operator.LTE);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-        assertTrue(setAppliesTo(bound, set(ONE), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(), list()));
-        assertFalse(setAppliesTo(bound, set(ZERO), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(ONE), list(ZERO)));
-        assertFalse(setAppliesTo(bound, set(ONE), list(ONE, TWO)));
-        assertTrue(setAppliesTo(bound, set(ONE, TWO), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(ONE), list()));
-
-        assertFalse(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ONE)));
-        assertTrue(setAppliesTo(bound, set(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertTrue(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // GT
-        condition = ColumnCondition.condition(definition, new 
Lists.Value(Arrays.asList(ONE)), Operator.GT);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-        assertFalse(setAppliesTo(bound, set(ONE), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(), list()));
-        assertTrue(setAppliesTo(bound, set(ZERO), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(ONE), list(ZERO)));
-        assertTrue(setAppliesTo(bound, set(ONE), list(ONE, TWO)));
-        assertFalse(setAppliesTo(bound, set(ONE, TWO), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(ONE), list()));
-
-        assertTrue(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ONE)));
-        assertFalse(setAppliesTo(bound, set(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertFalse(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // GTE
-        condition = ColumnCondition.condition(definition, new 
Lists.Value(Arrays.asList(ONE)), Operator.GTE);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-        assertTrue(setAppliesTo(bound, set(ONE), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(), list()));
-        assertTrue(setAppliesTo(bound, set(ZERO), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(ONE), list(ZERO)));
-        assertTrue(setAppliesTo(bound, set(ONE), list(ONE, TWO)));
-        assertFalse(setAppliesTo(bound, set(ONE, TWO), list(ONE)));
-        assertTrue(setAppliesTo(bound, set(), list(ONE)));
-        assertFalse(setAppliesTo(bound, set(ONE), list()));
-
-        assertTrue(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ONE)));
-        assertFalse(setAppliesTo(bound, set(ONE), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertTrue(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), 
list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-    }
-
-    // values should be a list of key, value, key, value, ...
-    private static Map<ByteBuffer, ByteBuffer> map(ByteBuffer... values)
-    {
-        Map<ByteBuffer, ByteBuffer> map = new TreeMap<>();
-        for (int i = 0; i < values.length; i += 2)
-            map.put(values[i], values[i + 1]);
-
-        return map;
-    }
-
-    private static boolean mapAppliesTo(ColumnCondition.CollectionBound bound, 
Map<ByteBuffer, ByteBuffer> conditionValues, Map<ByteBuffer, ByteBuffer> 
columnValues)
-    {
-        CFMetaData cfm = CFMetaData.compile("create table foo(a int PRIMARY 
KEY, b map<int, int>)", "ks");
-        Map<ByteBuffer, CollectionType> typeMap = new HashMap<>();
-        typeMap.put(ByteBufferUtil.bytes("b"), 
MapType.getInstance(Int32Type.instance, Int32Type.instance, true));
-        ColumnDefinition definition = ColumnDefinition.regularDef(cfm, 
ByteBufferUtil.bytes("b"), MapType.getInstance(Int32Type.instance, 
Int32Type.instance, true));
-
-        List<Cell> cells = new ArrayList<>(columnValues.size());
-        if (columnValues != null)
-        {
-            for (Map.Entry<ByteBuffer, ByteBuffer> entry : 
columnValues.entrySet())
-                cells.add(testCell(definition, entry.getValue(), 
CellPath.create(entry.getKey())));
-        }
-
-        return bound.mapAppliesTo(MapType.getInstance(Int32Type.instance, 
Int32Type.instance, true), cells.iterator(), conditionValues, bound.operator);
-    }
-
-    @Test
-    public void testMapCollectionBoundIsSatisfiedByValue() throws 
InvalidRequestException
-    {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", 
"c", ListType.getInstance(Int32Type.instance, true));
-
-        Map<ByteBuffer, ByteBuffer> placeholderMap = new TreeMap<>();
-        placeholderMap.put(ONE, ONE);
-        Maps.Value placeholder = new Maps.Value(placeholderMap);
-
-        // EQ
-        ColumnCondition condition = ColumnCondition.condition(definition, 
placeholder, Operator.EQ);
-        ColumnCondition.CollectionBound bound = 
(ColumnCondition.CollectionBound) condition.bind(QueryOptions.DEFAULT);
-
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(), map()));
-        assertFalse(mapAppliesTo(bound, map(ZERO, ONE), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ZERO, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ZERO), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ZERO)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE, TWO, 
ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE, TWO, ONE), map(ONE, 
ONE)));
-        assertFalse(mapAppliesTo(bound, map(), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map()));
-
-        assertFalse(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), 
map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertTrue(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // NEQ
-        condition = ColumnCondition.condition(definition, placeholder, 
Operator.NEQ);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(), map()));
-        assertTrue(mapAppliesTo(bound, map(ZERO, ONE), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ZERO, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ZERO), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ZERO)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE, TWO, 
ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE, TWO, ONE), map(ONE, 
ONE)));
-        assertTrue(mapAppliesTo(bound, map(), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map()));
-
-        assertTrue(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), 
map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertFalse(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // LT
-        condition = ColumnCondition.condition(definition, placeholder, 
Operator.LT);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(), map()));
-        assertFalse(mapAppliesTo(bound, map(ZERO, ONE), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ZERO, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ZERO), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ZERO)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE, TWO, 
ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE, TWO, ONE), map(ONE, 
ONE)));
-        assertFalse(mapAppliesTo(bound, map(), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map()));
-
-        assertFalse(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), 
map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertFalse(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // LTE
-        condition = ColumnCondition.condition(definition, placeholder, 
Operator.LTE);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(), map()));
-        assertFalse(mapAppliesTo(bound, map(ZERO, ONE), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ZERO, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ZERO), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ZERO)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE, TWO, 
ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE, TWO, ONE), map(ONE, 
ONE)));
-        assertFalse(mapAppliesTo(bound, map(), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map()));
-
-        assertFalse(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), 
map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertTrue(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // GT
-        condition = ColumnCondition.condition(definition, placeholder, 
Operator.GT);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(), map()));
-        assertTrue(mapAppliesTo(bound, map(ZERO, ONE), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ZERO, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ZERO), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ZERO)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE, TWO, 
ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE, TWO, ONE), map(ONE, 
ONE)));
-        assertTrue(mapAppliesTo(bound, map(), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map()));
-
-        assertTrue(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), 
map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertFalse(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-
-        // GTE
-        condition = ColumnCondition.condition(definition, placeholder, 
Operator.GTE);
-        bound = (ColumnCondition.CollectionBound) 
condition.bind(QueryOptions.DEFAULT);
-
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
-        assertTrue(mapAppliesTo(bound, map(), map()));
-        assertTrue(mapAppliesTo(bound, map(ZERO, ONE), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ZERO, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ZERO), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ZERO)));
-        assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE, TWO, 
ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE, TWO, ONE), map(ONE, 
ONE)));
-        assertTrue(mapAppliesTo(bound, map(), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map()));
-
-        assertTrue(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), 
map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ONE)));
-        assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-        assertTrue(mapAppliesTo(bound, map(ByteBufferUtil.EMPTY_BYTE_BUFFER, 
ONE), map(ByteBufferUtil.EMPTY_BYTE_BUFFER, ONE)));
-        assertTrue(mapAppliesTo(bound, map(ONE, 
ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
-    }
-}

Reply via email to