You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/11/04 17:32:24 UTC

[1/3] git commit: Fix issues w/ CONTAINS (KEY) queries on 2ary indexes

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 684b4f96f -> 5117a5d5b


Fix issues w/ CONTAINS (KEY) queries on 2ary indexes

Patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-8147


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

Branch: refs/heads/cassandra-2.1
Commit: 52a701f295490cf10850b2a6ad8fb3fdcbc57211
Parents: 684b4f9
Author: blerer <b_...@hotmail.com>
Authored: Tue Nov 4 10:16:57 2014 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Tue Nov 4 10:16:57 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 pylib/cqlshlib/cql3handling.py                  |  4 +-
 .../org/apache/cassandra/config/CFMetaData.java |  2 +-
 .../cassandra/config/ColumnDefinition.java      | 12 +++++
 .../org/apache/cassandra/cql3/Relation.java     | 57 ++++++++++++++++----
 .../cql3/statements/CreateIndexStatement.java   |  6 ++-
 .../cql3/statements/SelectStatement.java        |  2 +-
 .../apache/cassandra/db/IndexExpression.java    | 44 +++++++++++++++
 .../cassandra/db/index/SecondaryIndex.java      | 23 +++++++-
 .../db/index/SecondaryIndexManager.java         |  2 +-
 .../db/index/SecondaryIndexSearcher.java        |  7 +--
 .../db/index/composites/CompositesIndex.java    |  9 ++--
 .../CompositesIndexOnCollectionKey.java         |  7 +++
 .../CompositesIndexOnCollectionValue.java       |  6 +++
 .../cassandra/db/marshal/CollectionType.java    |  9 ++++
 .../cassandra/cql3/ContainsRelationTest.java    | 44 +++++++++++++++
 16 files changed, 211 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 37b3f83..7fb32fe 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 2.1.2
+ * Fix issues with CONTAINS (KEY) queries on secondary indexes
+   (CASSANDRA-8147)
  * Fix read-rate tracking of sstables for some queries (CASSANDRA-8239)
  * Fix default timestamp in QueryOptions (CASSANDRA-8246)
  * Set socket timeout when reading remote version (CASSANDRA-8188)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 8d2fec5..e12e7e1 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -599,10 +599,10 @@ syntax_rules += r'''
                     ;
 <whereClause> ::= <relation> ( "AND" <relation> )*
                 ;
-<relation> ::= [rel_lhs]=<cident> ( "=" | "<" | ">" | "<=" | ">=" | "CONTAINS" ) <term>
+<relation> ::= [rel_lhs]=<cident> ( "=" | "<" | ">" | "<=" | ">=" | "CONTAINS" ( "KEY" )? ) <term>
              | token="TOKEN" "(" [rel_tokname]=<cident>
                                  ( "," [rel_tokname]=<cident> )*
-                             ")" ("=" | "<" | ">" | "<=" | ">=" | "CONTAINS") <tokenDefinition>
+                             ")" ("=" | "<" | ">" | "<=" | ">=") <tokenDefinition>
              | [rel_lhs]=<cident> "IN" "(" <term> ( "," <term> )* ")"
              ;
 <selectClause> ::= "DISTINCT"? <selector> ("AS" <cident>)? ("," <selector> ("AS" <cident>)?)*

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index b5784ed..d986c40 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -1540,7 +1540,7 @@ public final class CFMetaData
 
                 if (c.getIndexType() == IndexType.CUSTOM)
                 {
-                    if (c.getIndexOptions() == null || !c.getIndexOptions().containsKey(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME))
+                    if (c.getIndexOptions() == null || !c.hasIndexOption(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME))
                         throw new ConfigurationException("Required index option missing: " + SecondaryIndex.CUSTOM_INDEX_OPTION_NAME);
                 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index cbb3e75..ff66162 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -476,4 +476,16 @@ public class ColumnDefinition extends ColumnSpecification
     {
         return indexOptions;
     }
+
+    /**
+     * Checks if the index option with the specified name has been specified.
+     *
+     * @param name index option name
+     * @return <code>true</code> if the index option with the specified name has been specified, <code>false</code>
+     * otherwise.
+     */
+    public boolean hasIndexOption(String name)
+    {
+        return indexOptions.containsKey(name);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/cql3/Relation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Relation.java b/src/java/org/apache/cassandra/cql3/Relation.java
index 42373c3..c6a7c65 100644
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@ -17,6 +17,10 @@
  */
 package org.apache.cassandra.cql3;
 
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.marshal.CollectionType;
+
 
 public abstract class Relation {
 
@@ -24,19 +28,50 @@ public abstract class Relation {
 
     public static enum Type
     {
-        EQ, LT, LTE, GTE, GT, IN, CONTAINS, CONTAINS_KEY, NEQ;
-
-        public boolean allowsIndexQuery()
+        EQ
         {
-            switch (this)
+            public boolean allowsIndexQueryOn(ColumnDefinition columnDef)
             {
-                case EQ:
-                case CONTAINS:
-                case CONTAINS_KEY:
-                    return true;
-                default:
-                    return false;
+                return columnDef.isIndexed();
             }
+        },
+        LT,
+        LTE,
+        GTE,
+        GT,
+        IN,
+        CONTAINS
+        {
+            public boolean allowsIndexQueryOn(ColumnDefinition columnDef)
+            {
+                return columnDef.isIndexed()
+                        && columnDef.type.isCollection()
+                        && (!((CollectionType<?>) columnDef.type).isMap()
+                                || columnDef.hasIndexOption(SecondaryIndex.INDEX_VALUES_OPTION_NAME));
+            }
+        },
+        CONTAINS_KEY
+        {
+            public boolean allowsIndexQueryOn(ColumnDefinition columnDef)
+            {
+                return columnDef.isIndexed()
+                        && columnDef.type.isCollection()
+                        && (!((CollectionType<?>) columnDef.type).isMap()
+                                || columnDef.hasIndexOption(SecondaryIndex.INDEX_KEYS_OPTION_NAME));
+            }
+        },
+        NEQ;
+
+        /**
+         * Checks if this relation type allow index queries on the specified column
+         *
+         * @param columnDef the column definition.
+         * @return <code>true</code> if this relation type allow index queries on the specified column,
+         * <code>false</code> otherwise.
+         */
+        public boolean allowsIndexQueryOn(ColumnDefinition columnDef)
+        {
+            return false;
         }
 
         @Override
@@ -56,6 +91,8 @@ public abstract class Relation {
                     return ">=";
                 case NEQ:
                     return "!=";
+                case CONTAINS_KEY:
+                    return "CONTAINS KEY";
                 default:
                     return this.name();
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 4809187..fc5c4d1 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -29,6 +29,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.IndexType;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.cql3.*;
@@ -82,7 +83,7 @@ public class CreateIndexStatement extends SchemaAlteringStatement
 
         if (cd.getIndexType() != null)
         {
-            boolean previousIsKeys = cd.getIndexOptions().containsKey("index_keys");
+            boolean previousIsKeys = cd.hasIndexOption(SecondaryIndex.INDEX_KEYS_OPTION_NAME);
             if (isMap && target.isCollectionKeys != previousIsKeys)
             {
                 String msg = "Cannot create index on %s %s, an index on %s %s already exists and indexing "
@@ -137,7 +138,8 @@ public class CreateIndexStatement extends SchemaAlteringStatement
             // to also index map keys, so we record that this is the values we index to make our
             // lives easier then.
             if (cd.type.isCollection())
-                options = ImmutableMap.of(target.isCollectionKeys ? "index_keys" : "index_values", "");
+                options = ImmutableMap.of(target.isCollectionKeys ? SecondaryIndex.INDEX_KEYS_OPTION_NAME
+                                                                  : SecondaryIndex.INDEX_VALUES_OPTION_NAME, "");
             cd.setIndexType(IndexType.COMPOSITES, options);
         }
         else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 3d57f4c..3354c0b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -1496,7 +1496,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 handleUnrecognizedEntity(entity, relation);
 
             stmt.restrictedColumns.add(def);
-            if (def.isIndexed() && relation.operator().allowsIndexQuery())
+            if (relation.operator().allowsIndexQueryOn(def))
                 return new boolean[]{true, def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN};
             return new boolean[]{false, false};
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/db/IndexExpression.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/IndexExpression.java b/src/java/org/apache/cassandra/db/IndexExpression.java
index b57890a..910bfbc 100644
--- a/src/java/org/apache/cassandra/db/IndexExpression.java
+++ b/src/java/org/apache/cassandra/db/IndexExpression.java
@@ -76,6 +76,50 @@ public class IndexExpression
                     return false;
             }
         }
+
+        @Override
+        public String toString()
+        {
+            switch (this)
+            {
+                case EQ:
+                    return "=";
+                case LT:
+                    return "<";
+                case LTE:
+                    return "<=";
+                case GT:
+                    return ">";
+                case GTE:
+                    return ">=";
+                case CONTAINS_KEY:
+                    return "CONTAINS KEY";
+                default:
+                    return this.name();
+            }
+        }
+    }
+
+    /**
+     * Checks if the operator of this <code>IndexExpression</code> is a <code>CONTAINS</code> operator.
+     *
+     * @return <code>true</code> if the operator of this <code>IndexExpression</code> is a <code>CONTAINS</code>
+     * operator, <code>false</code> otherwise.
+     */
+    public boolean isContains()
+    {
+        return Operator.CONTAINS == operator;
+    }
+
+    /**
+     * Checks if the operator of this <code>IndexExpression</code> is a <code>CONTAINS_KEY</code> operator.
+     *
+     * @return <code>true</code> if the operator of this <code>IndexExpression</code> is a <code>CONTAINS_KEY</code>
+     * operator, <code>false</code> otherwise.
+     */
+    public boolean isContainsKey()
+    {
+        return Operator.CONTAINS_KEY == operator;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
index 4d50fa6..e3ed73c 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
@@ -32,11 +32,14 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.IndexExpression;
 import org.apache.cassandra.db.SystemKeyspace;
+
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -45,6 +48,7 @@ import org.apache.cassandra.db.index.composites.CompositesIndex;
 import org.apache.cassandra.db.index.keys.KeysIndex;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.LocalByPartionerType;
 import org.apache.cassandra.dht.LocalToken;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -52,7 +56,8 @@ import org.apache.cassandra.io.sstable.ReducingKeyIterator;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.memory.MemtableAllocator;
+
+import com.google.common.collect.Iterables;
 
 /**
  * Abstract base class for different types of secondary indexes.
@@ -65,6 +70,16 @@ public abstract class SecondaryIndex
 
     public static final String CUSTOM_INDEX_OPTION_NAME = "class_name";
 
+    /**
+     * The name of the option used to specify that the index is on the collection keys.
+     */
+    public static final String INDEX_KEYS_OPTION_NAME = "index_keys";
+
+    /**
+     * The name of the option used to specify that the index is on the collection values.
+     */
+    public static final String INDEX_VALUES_OPTION_NAME = "index_values";
+
     public static final AbstractType<?> keyComparator = StorageService.getPartitioner().preservesOrder()
                                                       ? BytesType.instance
                                                       : new LocalByPartionerType(StorageService.getPartitioner());
@@ -281,6 +296,12 @@ public abstract class SecondaryIndex
         }
     }
 
+    /** Returns true if the index supports lookups for the given operator, false otherwise. */
+    public boolean supportsOperator(IndexExpression.Operator operator)
+    {
+        return operator == IndexExpression.Operator.EQ;
+    }
+
     /**
      * Returns the decoratedKey for a column value
      * @param value column value

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index c2d481b..976bbb8 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -528,7 +528,7 @@ public class SecondaryIndexManager
         {
             SecondaryIndex index = getIndexForColumn(ix.column);
 
-            if (index == null)
+            if (index == null || !index.supportsOperator(ix.operator))
                 continue;
 
             Set<ByteBuffer> columns = groupByIndexType.get(index.indexTypeForGrouping());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
index 1239c29..b9ccd8e 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
@@ -54,11 +54,11 @@ public abstract class SecondaryIndexSearcher
     {
         for (IndexExpression expression : clause)
         {
-            if (!columns.contains(expression.column) || !expression.operator.allowsIndexQuery())
+            if (!columns.contains(expression.column))
                 continue;
 
             SecondaryIndex index = indexManager.getIndexForColumn(expression.column);
-            if (index != null && index.getIndexCfs() != null)
+            if (index != null && index.getIndexCfs() != null && index.supportsOperator(expression.operator))
                 return true;
         }
         return false;
@@ -88,8 +88,9 @@ public abstract class SecondaryIndexSearcher
                 continue;
 
             SecondaryIndex index = indexManager.getIndexForColumn(expression.column);
-            if (index == null || index.getIndexCfs() == null || !expression.operator.allowsIndexQuery())
+            if (index == null || index.getIndexCfs() == null || !index.supportsOperator(expression.operator))
                 continue;
+
             int columns = index.getIndexCfs().getMeanColumns();
             candidates.put(index, columns);
             if (columns < bestMeanCount)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
index f69f716..410ea83 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
@@ -30,6 +30,7 @@ import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.index.AbstractSimplePerColumnSecondaryIndex;
+import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.db.index.SecondaryIndexSearcher;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -66,7 +67,7 @@ public abstract class CompositesIndex extends AbstractSimplePerColumnSecondaryIn
                 case SET:
                     return new CompositesIndexOnCollectionKey();
                 case MAP:
-                    return cfDef.getIndexOptions().containsKey("index_keys")
+                    return cfDef.hasIndexOption(SecondaryIndex.INDEX_KEYS_OPTION_NAME)
                          ? new CompositesIndexOnCollectionKey()
                          : new CompositesIndexOnCollectionValue();
             }
@@ -98,7 +99,7 @@ public abstract class CompositesIndex extends AbstractSimplePerColumnSecondaryIn
                 case SET:
                     return CompositesIndexOnCollectionKey.buildIndexComparator(baseMetadata, cfDef);
                 case MAP:
-                    return cfDef.getIndexOptions().containsKey("index_keys")
+                    return cfDef.hasIndexOption(SecondaryIndex.INDEX_KEYS_OPTION_NAME)
                          ? CompositesIndexOnCollectionKey.buildIndexComparator(baseMetadata, cfDef)
                          : CompositesIndexOnCollectionValue.buildIndexComparator(baseMetadata, cfDef);
             }
@@ -159,8 +160,8 @@ public abstract class CompositesIndex extends AbstractSimplePerColumnSecondaryIn
 
         if (columnDef.type.isCollection())
         {
-            options.remove("index_values");
-            options.remove("index_keys");
+            options.remove(SecondaryIndex.INDEX_VALUES_OPTION_NAME);
+            options.remove(SecondaryIndex.INDEX_KEYS_OPTION_NAME);
         }
 
         if (!options.isEmpty())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
index c252546..1067a94 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
@@ -89,6 +89,13 @@ public class CompositesIndexOnCollectionKey extends CompositesIndex
     }
 
     @Override
+    public boolean supportsOperator(IndexExpression.Operator operator)
+    {
+        return operator == IndexExpression.Operator.CONTAINS_KEY ||
+                operator == IndexExpression.Operator.CONTAINS && columnDef.type instanceof SetType;
+    }
+
+    @Override
     public boolean indexes(CellName name)
     {
         // We index if the CQL3 column name is the one of the collection we index

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
index 7a8c552..5b4aa64 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
@@ -95,6 +95,12 @@ public class CompositesIndexOnCollectionValue extends CompositesIndex
     }
 
     @Override
+    public boolean supportsOperator(IndexExpression.Operator operator)
+    {
+        return operator == IndexExpression.Operator.CONTAINS;
+    }
+
+    @Override
     public boolean indexes(CellName name)
     {
         AbstractType<?> comp = baseCfs.metadata.getColumnDefinitionComparator(columnDef);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/src/java/org/apache/cassandra/db/marshal/CollectionType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
index 8a5fe5c..e63f2a5 100644
--- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
@@ -115,6 +115,15 @@ public abstract class CollectionType<T> extends AbstractType<T>
         return true;
     }
 
+    /**
+     * Checks if this collection is Map.
+     * @return <code>true</code> if this collection is a Map, <code>false</code> otherwise.
+     */
+    public boolean isMap()
+    {
+        return kind == Kind.MAP;
+    }
+
     protected List<Cell> enforceLimit(List<Cell> cells, int version)
     {
         if (version >= 3 || cells.size() <= MAX_ELEMENTS)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/52a701f2/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java b/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
index f854ec6..335636b 100644
--- a/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ContainsRelationTest.java
@@ -204,4 +204,48 @@ public class ContainsRelationTest extends CQLTester
             row(0, 1, 1, set(3, 4, 5))
         );
     }
+
+    @Test
+    public void testContainsKeyAndContainsWithIndexOnMapKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(keys(categories))");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn", "foo2"));
+
+        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo");
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
+                   row("test", 5, map("lmn", "foo")),
+                   row("test", 6, map("lmn", "foo2")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ? AND categories CONTAINS ? ALLOW FILTERING",
+                           "test", "lmn", "foo"),
+                   row("test", 5, map("lmn", "foo")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS KEY ? ALLOW FILTERING",
+                           "test", "foo", "lmn"),
+                   row("test", 5, map("lmn", "foo")));
+    }
+
+    @Test
+    public void testContainsKeyAndContainsWithIndexOnMapValue() throws Throwable
+    {
+        createTable("CREATE TABLE %s (account text, id int, categories map<text,text>, PRIMARY KEY (account, id))");
+        createIndex("CREATE INDEX ON %s(categories)");
+
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
+        execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn2", "foo"));
+
+        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn");
+
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
+                   row("test", 5, map("lmn", "foo")),
+                   row("test", 6, map("lmn2", "foo")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ? AND categories CONTAINS ? ALLOW FILTERING",
+                           "test", "lmn", "foo"),
+                   row("test", 5, map("lmn", "foo")));
+        assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS KEY ? ALLOW FILTERING",
+                           "test", "foo", "lmn"),
+                   row("test", 5, map("lmn", "foo")));
+    }
 }


[2/3] Consolidate Relation.Type and IndexExpression.Operator

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/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
index 2071a33..205babc 100644
--- a/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
@@ -71,7 +71,7 @@ public class ColumnConditionTest
         ColumnDefinition definition = new ColumnDefinition("ks", "cf", new ColumnIdentifier("c", true), Int32Type.instance, null, null, null, null, null);
 
         // EQ
-        ColumnCondition condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Relation.Type.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));
@@ -84,7 +84,7 @@ public class ColumnConditionTest
         assertFalse(isSatisfiedBy(bound, null, ONE));
 
         // NEQ
-        condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Relation.Type.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));
@@ -97,7 +97,7 @@ public class ColumnConditionTest
         assertTrue(isSatisfiedBy(bound, null, ONE));
 
         // LT
-        condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Relation.Type.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));
@@ -109,7 +109,7 @@ public class ColumnConditionTest
         assertFalse(isSatisfiedBy(bound, ONE, null));
 
         // LTE
-        condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Relation.Type.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));
@@ -121,7 +121,7 @@ public class ColumnConditionTest
         assertFalse(isSatisfiedBy(bound, ONE, null));
 
         // GT
-        condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Relation.Type.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));
@@ -133,7 +133,7 @@ public class ColumnConditionTest
         assertFalse(isSatisfiedBy(bound, ONE, null));
 
         // GT
-        condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Relation.Type.GTE);
+        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));
@@ -179,7 +179,7 @@ public class ColumnConditionTest
         ColumnDefinition definition = new ColumnDefinition("ks", "cf", new ColumnIdentifier("c", true), ListType.getInstance(Int32Type.instance), null, null, null, null, null);
 
         // EQ
-        ColumnCondition condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Relation.Type.EQ);
+        ColumnCondition condition = ColumnCondition.condition(definition, null, 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()));
@@ -195,7 +195,7 @@ public class ColumnConditionTest
         assertTrue(listAppliesTo(bound, list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // NEQ
-        condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Relation.Type.NEQ);
+        condition = ColumnCondition.condition(definition, null, 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()));
@@ -211,7 +211,7 @@ public class ColumnConditionTest
         assertFalse(listAppliesTo(bound, list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // LT
-        condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Relation.Type.LT);
+        condition = ColumnCondition.condition(definition, null, 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()));
@@ -227,7 +227,7 @@ public class ColumnConditionTest
         assertFalse(listAppliesTo(bound, list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // LTE
-        condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Relation.Type.LTE);
+        condition = ColumnCondition.condition(definition, null, 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()));
@@ -243,7 +243,7 @@ public class ColumnConditionTest
         assertTrue(listAppliesTo(bound, list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // GT
-        condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Relation.Type.GT);
+        condition = ColumnCondition.condition(definition, null, 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()));
@@ -259,7 +259,7 @@ public class ColumnConditionTest
         assertFalse(listAppliesTo(bound, list(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // GTE
-        condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Relation.Type.GTE);
+        condition = ColumnCondition.condition(definition, null, 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()));
@@ -309,7 +309,7 @@ public class ColumnConditionTest
         ColumnDefinition definition = new ColumnDefinition("ks", "cf", new ColumnIdentifier("c", true), SetType.getInstance(Int32Type.instance), null, null, null, null, null);
 
         // EQ
-        ColumnCondition condition = ColumnCondition.condition(definition, null, new Sets.Value(set(ONE)), Relation.Type.EQ);
+        ColumnCondition condition = ColumnCondition.condition(definition, null, 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()));
@@ -325,7 +325,7 @@ public class ColumnConditionTest
         assertTrue(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // NEQ
-        condition = ColumnCondition.condition(definition, null, new Sets.Value(set(ONE)), Relation.Type.NEQ);
+        condition = ColumnCondition.condition(definition, null, 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()));
@@ -341,7 +341,7 @@ public class ColumnConditionTest
         assertFalse(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // LT
-        condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Relation.Type.LT);
+        condition = ColumnCondition.condition(definition, null, 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()));
@@ -357,7 +357,7 @@ public class ColumnConditionTest
         assertFalse(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // LTE
-        condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Relation.Type.LTE);
+        condition = ColumnCondition.condition(definition, null, 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()));
@@ -373,7 +373,7 @@ public class ColumnConditionTest
         assertTrue(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // GT
-        condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Relation.Type.GT);
+        condition = ColumnCondition.condition(definition, null, 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()));
@@ -389,7 +389,7 @@ public class ColumnConditionTest
         assertFalse(setAppliesTo(bound, set(ByteBufferUtil.EMPTY_BYTE_BUFFER), list(ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // GTE
-        condition = ColumnCondition.condition(definition, null, new Lists.Value(Arrays.asList(ONE)), Relation.Type.GTE);
+        condition = ColumnCondition.condition(definition, null, 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()));
@@ -443,7 +443,7 @@ public class ColumnConditionTest
         Maps.Value placeholder = new Maps.Value(placeholderMap);
 
         // EQ
-        ColumnCondition condition = ColumnCondition.condition(definition, null, placeholder, Relation.Type.EQ);
+        ColumnCondition condition = ColumnCondition.condition(definition, null, placeholder, Operator.EQ);
         ColumnCondition.CollectionBound bound = (ColumnCondition.CollectionBound) condition.bind(QueryOptions.DEFAULT);
 
         assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
@@ -465,7 +465,7 @@ public class ColumnConditionTest
         assertTrue(mapAppliesTo(bound, map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // NEQ
-        condition = ColumnCondition.condition(definition, null, placeholder, Relation.Type.NEQ);
+        condition = ColumnCondition.condition(definition, null, placeholder, Operator.NEQ);
         bound = (ColumnCondition.CollectionBound) condition.bind(QueryOptions.DEFAULT);
 
         assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
@@ -487,7 +487,7 @@ public class ColumnConditionTest
         assertFalse(mapAppliesTo(bound, map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // LT
-        condition = ColumnCondition.condition(definition, null, placeholder, Relation.Type.LT);
+        condition = ColumnCondition.condition(definition, null, placeholder, Operator.LT);
         bound = (ColumnCondition.CollectionBound) condition.bind(QueryOptions.DEFAULT);
 
         assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
@@ -509,7 +509,7 @@ public class ColumnConditionTest
         assertFalse(mapAppliesTo(bound, map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // LTE
-        condition = ColumnCondition.condition(definition, null, placeholder, Relation.Type.LTE);
+        condition = ColumnCondition.condition(definition, null, placeholder, Operator.LTE);
         bound = (ColumnCondition.CollectionBound) condition.bind(QueryOptions.DEFAULT);
 
         assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
@@ -531,7 +531,7 @@ public class ColumnConditionTest
         assertTrue(mapAppliesTo(bound, map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // GT
-        condition = ColumnCondition.condition(definition, null, placeholder, Relation.Type.GT);
+        condition = ColumnCondition.condition(definition, null, placeholder, Operator.GT);
         bound = (ColumnCondition.CollectionBound) condition.bind(QueryOptions.DEFAULT);
 
         assertFalse(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));
@@ -553,7 +553,7 @@ public class ColumnConditionTest
         assertFalse(mapAppliesTo(bound, map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER), map(ONE, ByteBufferUtil.EMPTY_BYTE_BUFFER)));
 
         // GTE
-        condition = ColumnCondition.condition(definition, null, placeholder, Relation.Type.GTE);
+        condition = ColumnCondition.condition(definition, null, placeholder, Operator.GTE);
         bound = (ColumnCondition.CollectionBound) condition.bind(QueryOptions.DEFAULT);
 
         assertTrue(mapAppliesTo(bound, map(ONE, ONE), map(ONE, ONE)));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 79a8470..06f8997 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -18,7 +18,6 @@
 */
 package org.apache.cassandra.db;
 
-import static org.junit.Assert.assertEquals;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -29,11 +28,14 @@ import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.Test;
+
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.db.filter.IDiskAtomFilter;
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.dht.BytesToken;
 import org.apache.cassandra.dht.Range;
@@ -41,7 +43,8 @@ import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
 
 public class CleanupTest extends SchemaLoader
 {
@@ -106,7 +109,7 @@ public class CleanupTest extends SchemaLoader
             Thread.sleep(10);
 
         // verify we get it back w/ index query too
-        IndexExpression expr = new IndexExpression(COLUMN, IndexExpression.Operator.EQ, VALUE);
+        IndexExpression expr = new IndexExpression(COLUMN, Operator.EQ, VALUE);
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index 0aa33a6..243b2d2 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -56,6 +56,7 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.IndexType;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -235,7 +236,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.apply();
 
         // basic single-expression query
-        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
@@ -254,7 +255,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert ByteBufferUtil.bytes(1L).equals( rows.get(1).cf.getColumn(birthdate).value());
 
         // add a second expression
-        IndexExpression expr2 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), IndexExpression.Operator.GTE, ByteBufferUtil.bytes(2L));
+        IndexExpression expr2 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), Operator.GTE, ByteBufferUtil.bytes(2L));
         clause = Arrays.asList(expr, expr2);
         rows = cfs.search(range, clause, filter, 100);
 
@@ -283,7 +284,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // query with index hit but rejected by secondary clause, with a small enough count that just checking count
         // doesn't tell the scan loop that it's done
-        IndexExpression expr3 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(-1L));
+        IndexExpression expr3 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), Operator.EQ, ByteBufferUtil.bytes(-1L));
         clause = Arrays.asList(expr, expr3);
         rows = cfs.search(range, clause, filter, 100);
 
@@ -303,8 +304,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
             rm.applyUnsafe();
         }
 
-        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(34L));
-        IndexExpression expr2 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(34L));
+        IndexExpression expr2 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr, expr2);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
@@ -329,7 +330,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
@@ -348,7 +349,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // verify that it's not being indexed under the deletion column value either
         Cell deletion = rm.getColumnFamilies().iterator().next().iterator().next();
         ByteBuffer deletionLong = ByteBufferUtil.bytes((long) ByteBufferUtil.toInt(deletion.value()));
-        IndexExpression expr0 = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, deletionLong);
+        IndexExpression expr0 = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, deletionLong);
         List<IndexExpression> clause0 = Arrays.asList(expr0);
         rows = cfs.search(range, clause0, filter, 100);
         assert rows.isEmpty();
@@ -429,14 +430,14 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(2L), 2);
         rm.apply();
 
-        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
         List<Row> rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 0;
 
-        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(2L));
+        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(2L));
         clause = Arrays.asList(expr);
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         String key = ByteBufferUtil.string(rows.get(0).key.getKey());
@@ -465,7 +466,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
         rm.apply();
 
-        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(100L));
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(100L));
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
@@ -488,7 +489,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(101L), 1, 1000);
         rm.apply();
 
-        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(101L));
+        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(101L));
         clause = Arrays.asList(expr);
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         assertEquals(1, rows.size());
@@ -501,7 +502,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         assertEquals(0, rows.size());
         // and for the new
-        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(102L));
+        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(102L));
         clause = Arrays.asList(expr);
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         assertEquals(1, rows.size());
@@ -527,7 +528,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, colName, val1, 0);
         rm.apply();
-        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, val1);
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, val1);
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
@@ -549,7 +550,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(0, rows.size());
         // now check for the updated value
-        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, val2);
+        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, val2);
         clause = Arrays.asList(expr);
         filter = new IdentityQueryFilter();
         range = Util.range("", "");
@@ -562,7 +563,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.add(cfName, colName, ByteBufferUtil.bytes(1L), 3);
         keyspace.apply(rm, true, false);
 
-        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
+        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
         clause = Arrays.asList(expr);
         filter = new IdentityQueryFilter();
         range = Util.range("", "");
@@ -597,7 +598,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.apply();
 
         // test that the index query fetches this version
-        IndexExpression expr = new IndexExpression(colName, IndexExpression.Operator.EQ, val1);
+        IndexExpression expr = new IndexExpression(colName, Operator.EQ, val1);
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
@@ -621,7 +622,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(0, rows.size());
         // now check for the updated value
-        expr = new IndexExpression(colName, IndexExpression.Operator.EQ, val2);
+        expr = new IndexExpression(colName, Operator.EQ, val2);
         clause = Arrays.asList(expr);
         filter = new IdentityQueryFilter();
         range = Util.range("", "");
@@ -634,7 +635,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.add(cfName, compositeName, val1, 2);
         keyspace.apply(rm, true, false);
 
-        expr = new IndexExpression(colName, IndexExpression.Operator.EQ, val1);
+        expr = new IndexExpression(colName, Operator.EQ, val1);
         clause = Arrays.asList(expr);
         filter = new IdentityQueryFilter();
         range = Util.range("", "");
@@ -680,7 +681,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // Read the index and we check we do get no value (and no NPE)
         // Note: the index will return the entry because it hasn't been deleted (we
         // haven't read yet nor compacted) but the data read itself will return null
-        IndexExpression expr = new IndexExpression(colName, IndexExpression.Operator.EQ, val1);
+        IndexExpression expr = new IndexExpression(colName, Operator.EQ, val1);
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
@@ -719,8 +720,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.apply();
 
         // basic single-expression query
-        IndexExpression expr1 = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
-        IndexExpression expr2 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), IndexExpression.Operator.GT, ByteBufferUtil.bytes(1L));
+        IndexExpression expr1 = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
+        IndexExpression expr2 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), Operator.GT, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr1, expr2);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
@@ -764,7 +765,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
     private void queryBirthdate(Keyspace keyspace) throws CharacterCodingException
     {
-        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         List<Row> rows = keyspace.getColumnFamilyStore("Indexed2").search(Util.range("", ""), clause, filter, 100);
@@ -1419,7 +1420,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         store.forceBlockingFlush();
 
-        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, LongType.instance.decompose(1L));
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, LongType.instance.decompose(1L));
         // explicitly tell to the KeysSearcher to use column limiting for rowsPerQuery to trigger bogus columnsRead--; (CASSANDRA-3996)
         List<Row> rows = store.search(store.makeExtendedFilter(Util.range("", ""), new IdentityQueryFilter(), Arrays.asList(expr), 10, true, false, System.currentTimeMillis()));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index ef6b14a..a99aa0c 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -44,6 +44,7 @@ import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -456,7 +457,7 @@ public class SSTableReaderTest extends SchemaLoader
             clearAndLoad(cfs);
 
         // query using index to see if sstable for secondary index opens
-        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
         Range<RowPosition> range = Util.range("", "");
         List<Row> rows = indexedCFS.search(range, clause, new IdentityQueryFilter(), 100);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index 2b5d029..5d44210 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -36,6 +36,7 @@ import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.context.CounterContext;
@@ -241,7 +242,7 @@ public class StreamingTransferTest extends SchemaLoader
         {
             long val = key.hashCode();
             IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"),
-                                                       IndexExpression.Operator.EQ,
+                                                       Operator.EQ,
                                                        ByteBufferUtil.bytes(val));
             List<IndexExpression> clause = Arrays.asList(expr);
             IDiskAtomFilter filter = new IdentityQueryFilter();


[3/3] git commit: Consolidate Relation.Type and IndexExpression.Operator

Posted by ty...@apache.org.
Consolidate Relation.Type and IndexExpression.Operator

Patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-8147


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

Branch: refs/heads/cassandra-2.1
Commit: 5117a5d5baca409a0510dac61d479827920cdfbc
Parents: 52a701f
Author: blerer <b_...@hotmail.com>
Authored: Tue Nov 4 10:31:12 2014 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Tue Nov 4 10:31:12 2014 -0600

----------------------------------------------------------------------
 .../apache/cassandra/cql/QueryProcessor.java    |  4 +-
 .../apache/cassandra/cql3/ColumnCondition.java  | 98 ++++++++++----------
 src/java/org/apache/cassandra/cql3/Cql.g        | 18 ++--
 .../cassandra/cql3/MultiColumnRelation.java     | 14 +--
 .../org/apache/cassandra/cql3/Operator.java     | 93 +++++++++++++++++++
 .../org/apache/cassandra/cql3/Relation.java     | 82 +---------------
 .../cassandra/cql3/SingleColumnRelation.java    | 18 ++--
 .../cql3/statements/ModificationStatement.java  |  8 +-
 .../cassandra/cql3/statements/Restriction.java  | 12 ++-
 .../cql3/statements/SelectStatement.java        | 53 ++++++-----
 .../statements/SingleColumnRestriction.java     | 17 ++--
 .../apache/cassandra/db/IndexExpression.java    | 95 +++++++------------
 .../apache/cassandra/db/PagedRangeCommand.java  | 10 +-
 .../apache/cassandra/db/RangeSliceCommand.java  | 11 +--
 .../cassandra/db/filter/ExtendedFilter.java     | 31 ++++---
 .../cassandra/db/index/SecondaryIndex.java      | 12 +--
 .../CompositesIndexOnCollectionKey.java         |  7 +-
 .../CompositesIndexOnCollectionValue.java       |  5 +-
 .../cassandra/db/marshal/CompositeType.java     |  4 +-
 .../cassandra/thrift/ThriftConversion.java      |  3 +-
 .../cassandra/cql3/ColumnConditionTest.java     | 48 +++++-----
 .../org/apache/cassandra/db/CleanupTest.java    | 11 ++-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 47 +++++-----
 .../cassandra/io/sstable/SSTableReaderTest.java |  3 +-
 .../streaming/StreamingTransferTest.java        |  3 +-
 25 files changed, 351 insertions(+), 356 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/cql/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/QueryProcessor.java b/src/java/org/apache/cassandra/cql/QueryProcessor.java
index 0b9ba2e..d947cdb 100644
--- a/src/java/org/apache/cassandra/cql/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql/QueryProcessor.java
@@ -33,7 +33,7 @@ import org.apache.cassandra.cql.hooks.ExecutionContext;
 import org.apache.cassandra.cql.hooks.PostPreparationHook;
 import org.apache.cassandra.cql.hooks.PreExecutionHook;
 import org.apache.cassandra.cql.hooks.PreparationContext;
-import org.apache.cassandra.db.CounterCell;
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -193,7 +193,7 @@ public class QueryProcessor
             ByteBuffer value = columnRelation.getValue().getByteBuffer(metadata.getValueValidator(metadata.comparator.cellFromByteBuffer(entity)), variables);
 
             expressions.add(new IndexExpression(entity,
-                                                IndexExpression.Operator.valueOf(columnRelation.operator().toString()),
+                                                Operator.valueOf(columnRelation.operator().name()),
                                                 value));
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/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
index 703ac5b..25cb07d 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@ -52,9 +52,9 @@ public class ColumnCondition
     private final Term value;  // a single value or a marker for a list of IN values
     private final List<Term> inValues;
 
-    public final Relation.Type operator;
+    public final Operator operator;
 
-    private ColumnCondition(ColumnDefinition column, Term collectionElement, Term value, List<Term> inValues, Relation.Type op)
+    private ColumnCondition(ColumnDefinition column, Term collectionElement, Term value, List<Term> inValues, Operator op)
     {
         this.column = column;
         this.collectionElement = collectionElement;
@@ -62,38 +62,38 @@ public class ColumnCondition
         this.inValues = inValues;
         this.operator = op;
 
-        if (!operator.equals(Relation.Type.IN))
+        if (!operator.equals(Operator.IN))
             assert this.inValues == null;
     }
 
-    public static ColumnCondition condition(ColumnDefinition column, Term value, Relation.Type op)
+    public static ColumnCondition condition(ColumnDefinition column, Term value, Operator op)
     {
         return new ColumnCondition(column, null, value, null, op);
     }
 
-    public static ColumnCondition condition(ColumnDefinition column, Term collectionElement, Term value, Relation.Type op)
+    public static ColumnCondition condition(ColumnDefinition column, Term collectionElement, Term value, Operator op)
     {
         return new ColumnCondition(column, collectionElement, value, null, op);
     }
 
     public static ColumnCondition inCondition(ColumnDefinition column, List<Term> inValues)
     {
-        return new ColumnCondition(column, null, null, inValues, Relation.Type.IN);
+        return new ColumnCondition(column, null, null, inValues, Operator.IN);
     }
 
     public static ColumnCondition inCondition(ColumnDefinition column, Term collectionElement, List<Term> inValues)
     {
-        return new ColumnCondition(column, collectionElement, null, inValues, Relation.Type.IN);
+        return new ColumnCondition(column, collectionElement, null, inValues, Operator.IN);
     }
 
     public static ColumnCondition inCondition(ColumnDefinition column, Term inMarker)
     {
-        return new ColumnCondition(column, null, inMarker, null, Relation.Type.IN);
+        return new ColumnCondition(column, null, inMarker, null, Operator.IN);
     }
 
     public static ColumnCondition inCondition(ColumnDefinition column, Term collectionElement, Term inMarker)
     {
-        return new ColumnCondition(column, collectionElement, inMarker, null, Relation.Type.IN);
+        return new ColumnCondition(column, collectionElement, inMarker, null, Operator.IN);
     }
 
     /**
@@ -107,7 +107,7 @@ public class ColumnCondition
         if (collectionElement != null)
             collectionElement.collectMarkerSpecification(boundNames);
 
-        if (operator.equals(Relation.Type.IN) && inValues != null)
+        if (operator.equals(Operator.IN) && inValues != null)
         {
             for (Term value : inValues)
                 value.collectMarkerSpecification(boundNames);
@@ -120,7 +120,7 @@ public class ColumnCondition
 
     public ColumnCondition.Bound bind(QueryOptions options) throws InvalidRequestException
     {
-        boolean isInCondition = operator.equals(Relation.Type.IN);
+        boolean isInCondition = operator.equals(Operator.IN);
         if (column.type instanceof CollectionType)
         {
             if (collectionElement == null)
@@ -134,9 +134,9 @@ public class ColumnCondition
     public static abstract class Bound
     {
         public final ColumnDefinition column;
-        public final Relation.Type operator;
+        public final Operator operator;
 
-        protected Bound(ColumnDefinition column, Relation.Type operator)
+        protected Bound(ColumnDefinition column, Operator operator)
         {
             this.column = column;
             this.operator = operator;
@@ -152,14 +152,14 @@ public class ColumnCondition
             return null;
         }
 
-        protected boolean isSatisfiedByValue(ByteBuffer value, Cell c, AbstractType<?> type, Relation.Type operator, long now) throws InvalidRequestException
+        protected boolean isSatisfiedByValue(ByteBuffer value, Cell c, AbstractType<?> type, Operator operator, long now) throws InvalidRequestException
         {
             ByteBuffer columnValue = (c == null || !c.isLive(now)) ? null : c.value();
             return compareWithOperator(operator, type, value, columnValue);
         }
 
         /** Returns true if the operator is satisfied (i.e. "value operator otherValue == true"), false otherwise. */
-        protected boolean compareWithOperator(Relation.Type operator, AbstractType<?> type, ByteBuffer value, ByteBuffer otherValue) throws InvalidRequestException
+        protected boolean compareWithOperator(Operator operator, AbstractType<?> type, ByteBuffer value, ByteBuffer otherValue) throws InvalidRequestException
         {
             if (value == null)
             {
@@ -176,7 +176,7 @@ public class ColumnCondition
             else if (otherValue == null)
             {
                 // the condition value is not null, so only NEQ can return true
-                return operator.equals(Relation.Type.NEQ);
+                return operator.equals(Operator.NEQ);
             }
             int comparison = type.compare(otherValue, value);
             switch (operator)
@@ -226,7 +226,7 @@ public class ColumnCondition
         {
             super(condition.column, condition.operator);
             assert !(column.type instanceof CollectionType) && condition.collectionElement == null;
-            assert !condition.operator.equals(Relation.Type.IN);
+            assert !condition.operator.equals(Operator.IN);
             this.value = condition.value.bindAndGet(options);
         }
 
@@ -254,7 +254,7 @@ public class ColumnCondition
         {
             super(condition.column, condition.operator);
             assert !(column.type instanceof CollectionType) && condition.collectionElement == null;
-            assert condition.operator.equals(Relation.Type.IN);
+            assert condition.operator.equals(Operator.IN);
             if (condition.inValues == null)
                 this.inValues = ((Lists.Marker) condition.value).bind(options).getElements();
             else
@@ -270,7 +270,7 @@ public class ColumnCondition
             CellName name = current.metadata().comparator.create(rowPrefix, column);
             for (ByteBuffer value : inValues)
             {
-                if (isSatisfiedByValue(value, current.getColumn(name), column.type, Relation.Type.EQ, now))
+                if (isSatisfiedByValue(value, current.getColumn(name), column.type, Operator.EQ, now))
                     return true;
             }
             return false;
@@ -293,7 +293,7 @@ public class ColumnCondition
         {
             super(condition.column, condition.operator);
             assert column.type instanceof CollectionType && condition.collectionElement != null;
-            assert !condition.operator.equals(Relation.Type.IN);
+            assert !condition.operator.equals(Operator.IN);
             this.collectionElement = condition.collectionElement.bindAndGet(options);
             this.value = condition.value.bindAndGet(options);
         }
@@ -380,7 +380,7 @@ public class ColumnCondition
                 AbstractType<?> valueType = ((MapType) column.type).values;
                 for (ByteBuffer value : inValues)
                 {
-                    if (isSatisfiedByValue(value, item, valueType, Relation.Type.EQ, now))
+                    if (isSatisfiedByValue(value, item, valueType, Operator.EQ, now))
                         return true;
                 }
                 return false;
@@ -394,7 +394,7 @@ public class ColumnCondition
             AbstractType<?> valueType = ((ListType) column.type).elements;
             for (ByteBuffer value : inValues)
             {
-                if (compareWithOperator(Relation.Type.EQ, valueType, value, columnValue))
+                if (compareWithOperator(Operator.EQ, valueType, value, columnValue))
                     return true;
             }
             return false;
@@ -416,7 +416,7 @@ public class ColumnCondition
         {
             super(condition.column, condition.operator);
             assert column.type instanceof CollectionType && condition.collectionElement == null;
-            assert !condition.operator.equals(Relation.Type.IN);
+            assert !condition.operator.equals(Operator.IN);
             this.value = condition.value.bind(options);
         }
 
@@ -427,9 +427,9 @@ public class ColumnCondition
             Iterator<Cell> iter = collectionColumns(current.metadata().comparator.create(rowPrefix, column), current, now);
             if (value == null)
             {
-                if (operator.equals(Relation.Type.EQ))
+                if (operator.equals(Operator.EQ))
                     return !iter.hasNext();
-                else if (operator.equals(Relation.Type.NEQ))
+                else if (operator.equals(Operator.NEQ))
                     return iter.hasNext();
                 else
                     throw new InvalidRequestException(String.format("Invalid comparison with null for operator \"%s\"", operator));
@@ -438,7 +438,7 @@ public class ColumnCondition
             return valueAppliesTo(type, iter, value, operator);
         }
 
-        static boolean valueAppliesTo(CollectionType type, Iterator<Cell> iter, Term.Terminal value, Relation.Type operator)
+        static boolean valueAppliesTo(CollectionType type, Iterator<Cell> iter, Term.Terminal value, Operator operator)
         {
             if (value == null)
                 return !iter.hasNext();
@@ -452,12 +452,12 @@ public class ColumnCondition
             throw new AssertionError();
         }
 
-        private static boolean setOrListAppliesTo(AbstractType<?> type, Iterator<Cell> iter, Iterator<ByteBuffer> conditionIter, Relation.Type operator, boolean isSet)
+        private static boolean setOrListAppliesTo(AbstractType<?> type, Iterator<Cell> iter, Iterator<ByteBuffer> conditionIter, Operator operator, boolean isSet)
         {
             while(iter.hasNext())
             {
                 if (!conditionIter.hasNext())
-                    return operator.equals(Relation.Type.GT) || operator.equals(Relation.Type.GTE) || operator.equals(Relation.Type.NEQ);
+                    return operator.equals(Operator.GT) || operator.equals(Operator.GTE) || operator.equals(Operator.NEQ);
 
                 // for lists we use the cell value; for sets we use the cell name
                 ByteBuffer cellValue = isSet? iter.next().name().collectionElement() : iter.next().value();
@@ -467,13 +467,13 @@ public class ColumnCondition
             }
 
             if (conditionIter.hasNext())
-                return operator.equals(Relation.Type.LT) || operator.equals(Relation.Type.LTE) || operator.equals(Relation.Type.NEQ);
+                return operator.equals(Operator.LT) || operator.equals(Operator.LTE) || operator.equals(Operator.NEQ);
 
             // they're equal
-            return operator == Relation.Type.EQ || operator == Relation.Type.LTE || operator == Relation.Type.GTE;
+            return operator == Operator.EQ || operator == Operator.LTE || operator == Operator.GTE;
         }
 
-        private static boolean evaluateComparisonWithOperator(int comparison, Relation.Type operator)
+        private static boolean evaluateComparisonWithOperator(int comparison, Operator operator)
         {
             // called when comparison != 0
             switch (operator)
@@ -493,12 +493,12 @@ public class ColumnCondition
             }
         }
 
-        static boolean listAppliesTo(ListType type, Iterator<Cell> iter, List<ByteBuffer> elements, Relation.Type operator)
+        static boolean listAppliesTo(ListType type, Iterator<Cell> iter, List<ByteBuffer> elements, Operator operator)
         {
             return setOrListAppliesTo(type.elements, iter, elements.iterator(), operator, false);
         }
 
-        static boolean setAppliesTo(SetType type, Iterator<Cell> iter, Set<ByteBuffer> elements, Relation.Type operator)
+        static boolean setAppliesTo(SetType type, Iterator<Cell> iter, Set<ByteBuffer> elements, Operator operator)
         {
             ArrayList<ByteBuffer> sortedElements = new ArrayList<>(elements.size());
             sortedElements.addAll(elements);
@@ -506,13 +506,13 @@ public class ColumnCondition
             return setOrListAppliesTo(type.elements, iter, sortedElements.iterator(), operator, true);
         }
 
-        static boolean mapAppliesTo(MapType type, Iterator<Cell> iter, Map<ByteBuffer, ByteBuffer> elements, Relation.Type operator)
+        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.equals(Relation.Type.GT) || operator.equals(Relation.Type.GTE) || operator.equals(Relation.Type.NEQ);
+                    return operator.equals(Operator.GT) || operator.equals(Operator.GTE) || operator.equals(Operator.NEQ);
 
                 Map.Entry<ByteBuffer, ByteBuffer> conditionEntry = conditionIter.next();
                 Cell c = iter.next();
@@ -529,10 +529,10 @@ public class ColumnCondition
             }
 
             if (conditionIter.hasNext())
-                return operator.equals(Relation.Type.LT) || operator.equals(Relation.Type.LTE) || operator.equals(Relation.Type.NEQ);
+                return operator.equals(Operator.LT) || operator.equals(Operator.LTE) || operator.equals(Operator.NEQ);
 
             // they're equal
-            return operator == Relation.Type.EQ || operator == Relation.Type.LTE || operator == Relation.Type.GTE;
+            return operator == Operator.EQ || operator == Operator.LTE || operator == Operator.GTE;
         }
 
         @Override
@@ -566,7 +566,7 @@ public class ColumnCondition
         {
             super(condition.column, condition.operator);
             assert column.type instanceof CollectionType && condition.collectionElement == null;
-            assert condition.operator.equals(Relation.Type.IN);
+            assert condition.operator.equals(Operator.IN);
             inValues = new ArrayList<>();
             if (condition.inValues == null)
             {
@@ -623,7 +623,7 @@ public class ColumnCondition
             List<Cell> cells = newArrayList(collectionColumns(name, current, now));
             for (Term.Terminal value : inValues)
             {
-                if (CollectionBound.valueAppliesTo(type, cells.iterator(), value, Relation.Type.EQ))
+                if (CollectionBound.valueAppliesTo(type, cells.iterator(), value, Operator.EQ))
                     return true;
             }
             return false;
@@ -669,9 +669,9 @@ public class ColumnCondition
         // Can be null, only used with the syntax "IF m[e] = ..." (in which case it's 'e')
         private final Term.Raw collectionElement;
 
-        private final Relation.Type operator;
+        private final Operator operator;
 
-        private Raw(Term.Raw value, List<Term.Raw> inValues, AbstractMarker.INRaw inMarker, Term.Raw collectionElement, Relation.Type op)
+        private Raw(Term.Raw value, List<Term.Raw> inValues, AbstractMarker.INRaw inMarker, Term.Raw collectionElement, Operator op)
         {
             this.value = value;
             this.inValues = inValues;
@@ -681,7 +681,7 @@ public class ColumnCondition
         }
 
         /** A condition on a column. For example: "IF col = 'foo'" */
-        public static Raw simpleCondition(Term.Raw value, Relation.Type op)
+        public static Raw simpleCondition(Term.Raw value, Operator op)
         {
             return new Raw(value, null, null, null, op);
         }
@@ -689,17 +689,17 @@ public class ColumnCondition
         /** 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, Relation.Type.IN);
+            return new Raw(null, inValues, 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, Relation.Type.IN);
+            return new Raw(null, null, inMarker, 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, Relation.Type op)
+        public static Raw collectionCondition(Term.Raw value, Term.Raw collectionElement, Operator op)
         {
             return new Raw(value, null, null, collectionElement, op);
         }
@@ -707,13 +707,13 @@ public class ColumnCondition
         /** 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, Relation.Type.IN);
+            return new Raw(null, inValues, null, collectionElement, 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, Relation.Type.IN);
+            return new Raw(null, null, inMarker, collectionElement, Operator.IN);
         }
 
         public ColumnCondition prepare(String keyspace, ColumnDefinition receiver) throws InvalidRequestException
@@ -723,7 +723,7 @@ public class ColumnCondition
 
             if (collectionElement == null)
             {
-                if (operator.equals(Relation.Type.IN))
+                if (operator.equals(Operator.IN))
                 {
                     if (inValues == null)
                         return ColumnCondition.inCondition(receiver, inMarker.prepare(keyspace, receiver));
@@ -757,7 +757,7 @@ public class ColumnCondition
                 default:
                     throw new AssertionError();
             }
-            if (operator.equals(Relation.Type.IN))
+            if (operator.equals(Operator.IN))
             {
                 if (inValues == null)
                     return ColumnCondition.inCondition(receiver, collectionElement.prepare(keyspace, elementSpec), inMarker.prepare(keyspace, valueSpec));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 2c90c18..189ae7d 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -1013,13 +1013,13 @@ propertyValue returns [String str]
     | u=unreserved_keyword { $str = u; }
     ;
 
-relationType returns [Relation.Type op]
-    : '='  { $op = Relation.Type.EQ; }
-    | '<'  { $op = Relation.Type.LT; }
-    | '<=' { $op = Relation.Type.LTE; }
-    | '>'  { $op = Relation.Type.GT; }
-    | '>=' { $op = Relation.Type.GTE; }
-    | '!=' { $op = Relation.Type.NEQ; }
+relationType returns [Operator op]
+    : '='  { $op = Operator.EQ; }
+    | '<'  { $op = Operator.LT; }
+    | '<=' { $op = Operator.LTE; }
+    | '>'  { $op = Operator.GT; }
+    | '>=' { $op = Operator.GTE; }
+    | '!=' { $op = Operator.NEQ; }
     ;
 
 relation[List<Relation> clauses]
@@ -1030,10 +1030,10 @@ relation[List<Relation> clauses]
                 $clauses.add(new SingleColumnRelation(id, type, t, true));
         }
     | name=cident K_IN marker=inMarker
-        { $clauses.add(new SingleColumnRelation(name, Relation.Type.IN, marker)); }
+        { $clauses.add(new SingleColumnRelation(name, Operator.IN, marker)); }
     | name=cident K_IN inValues=singleColumnInValues
         { $clauses.add(SingleColumnRelation.createInRelation($name.id, inValues)); }
-    | name=cident K_CONTAINS { Relation.Type rt = Relation.Type.CONTAINS; } (K_KEY { rt = Relation.Type.CONTAINS_KEY; })?
+    | name=cident K_CONTAINS { Operator rt = Operator.CONTAINS; } (K_KEY { rt = Operator.CONTAINS_KEY; })?
         t=term { $clauses.add(new SingleColumnRelation(name, rt, t)); }
     | ids=tupleOfIdentifiers
       ( K_IN

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
index fda60df..9c51d89 100644
--- a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@ -39,7 +39,7 @@ public class MultiColumnRelation extends Relation
 
     private final Tuples.INRaw inMarker;
 
-    private MultiColumnRelation(List<ColumnIdentifier> entities, Type relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
+    private MultiColumnRelation(List<ColumnIdentifier> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
     {
         this.entities = entities;
         this.relationType = relationType;
@@ -56,9 +56,9 @@ public class MultiColumnRelation extends Relation
      * @param relationType the relation operator
      * @param valuesOrMarker a Tuples.Literal instance or a Tuples.Raw marker
      */
-    public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier> entities, Type relationType, Term.MultiColumnRaw valuesOrMarker)
+    public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker)
     {
-        assert relationType != Relation.Type.IN;
+        assert relationType != Operator.IN;
         return new MultiColumnRelation(entities, relationType, valuesOrMarker, null, null);
     }
 
@@ -70,7 +70,7 @@ public class MultiColumnRelation extends Relation
      */
     public static MultiColumnRelation createInRelation(List<ColumnIdentifier> entities, List<? extends Term.MultiColumnRaw> inValues)
     {
-        return new MultiColumnRelation(entities, Relation.Type.IN, null, inValues, null);
+        return new MultiColumnRelation(entities, Operator.IN, null, inValues, null);
     }
 
     /**
@@ -81,7 +81,7 @@ public class MultiColumnRelation extends Relation
      */
     public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnIdentifier> entities, Tuples.INRaw inMarker)
     {
-        return new MultiColumnRelation(entities, Relation.Type.IN, null, null, inMarker);
+        return new MultiColumnRelation(entities, Operator.IN, null, null, inMarker);
     }
 
     public List<ColumnIdentifier> getEntities()
@@ -94,7 +94,7 @@ public class MultiColumnRelation extends Relation
      */
     public Term.MultiColumnRaw getValue()
     {
-        assert relationType != Relation.Type.IN;
+        assert relationType != Operator.IN;
         return valuesOrMarker;
     }
 
@@ -124,7 +124,7 @@ public class MultiColumnRelation extends Relation
     @Override
     public String toString()
     {
-        if (relationType == Type.IN)
+        if (relationType == Operator.IN)
         {
             StringBuilder sb = new StringBuilder(Tuples.tupleToString(entities));
             sb.append(" IN ");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/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
new file mode 100644
index 0000000..359fcb8
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/Operator.java
@@ -0,0 +1,93 @@
+/*
+ * 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.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public enum Operator
+{
+    EQ(0), LT(4), LTE(3), GTE(1), GT(2), IN(7), CONTAINS(5), CONTAINS_KEY(6), NEQ(8);
+
+    /**
+     * The binary representation of this <code>Enum</code> value.
+     */
+    private final int b;
+
+    /**
+     * Creates a new <code>Operator</code> with the specified binary representation.
+     * @param b the binary representation of this <code>Enum</code> value
+     */
+    private Operator(int b)
+    {
+        this.b = b;
+    }
+
+    /**
+     * Write the serialized version of this <code>Operator</code> to the specified output.
+     *
+     * @param output the output to write to
+     * @throws IOException if an I/O problem occurs while writing to the specified output
+     */
+    public void writeTo(DataOutput output) throws IOException
+    {
+        output.writeInt(b);
+    }
+
+    /**
+     * Deserializes a <code>Operator</code> instance from the specified input.
+     *
+     * @param input the input to read from
+     * @return the <code>Operator</code> instance deserialized
+     * @throws IOException if a problem occurs while deserializing the <code>Type</code> instance.
+     */
+    public static Operator readFrom(DataInput input) throws IOException
+    {
+          int b = input.readInt();
+          for (Operator operator : values())
+              if (operator.b == b)
+                  return operator;
+
+          throw new IOException(String.format("Cannot resolve Relation.Type from binary representation: %s", b));
+    }
+
+    @Override
+    public String toString()
+    {
+        switch (this)
+        {
+            case EQ:
+                return "=";
+            case LT:
+                return "<";
+            case LTE:
+                return "<=";
+            case GT:
+                return ">";
+            case GTE:
+                return ">=";
+            case NEQ:
+                return "!=";
+            case CONTAINS_KEY:
+                return "CONTAINS KEY";
+            default:
+                return this.name();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/cql3/Relation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Relation.java b/src/java/org/apache/cassandra/cql3/Relation.java
index c6a7c65..91d4100 100644
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@ -17,89 +17,11 @@
  */
 package org.apache.cassandra.cql3;
 
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.index.SecondaryIndex;
-import org.apache.cassandra.db.marshal.CollectionType;
-
-
 public abstract class Relation {
 
-    protected Type relationType;
-
-    public static enum Type
-    {
-        EQ
-        {
-            public boolean allowsIndexQueryOn(ColumnDefinition columnDef)
-            {
-                return columnDef.isIndexed();
-            }
-        },
-        LT,
-        LTE,
-        GTE,
-        GT,
-        IN,
-        CONTAINS
-        {
-            public boolean allowsIndexQueryOn(ColumnDefinition columnDef)
-            {
-                return columnDef.isIndexed()
-                        && columnDef.type.isCollection()
-                        && (!((CollectionType<?>) columnDef.type).isMap()
-                                || columnDef.hasIndexOption(SecondaryIndex.INDEX_VALUES_OPTION_NAME));
-            }
-        },
-        CONTAINS_KEY
-        {
-            public boolean allowsIndexQueryOn(ColumnDefinition columnDef)
-            {
-                return columnDef.isIndexed()
-                        && columnDef.type.isCollection()
-                        && (!((CollectionType<?>) columnDef.type).isMap()
-                                || columnDef.hasIndexOption(SecondaryIndex.INDEX_KEYS_OPTION_NAME));
-            }
-        },
-        NEQ;
-
-        /**
-         * Checks if this relation type allow index queries on the specified column
-         *
-         * @param columnDef the column definition.
-         * @return <code>true</code> if this relation type allow index queries on the specified column,
-         * <code>false</code> otherwise.
-         */
-        public boolean allowsIndexQueryOn(ColumnDefinition columnDef)
-        {
-            return false;
-        }
-
-        @Override
-        public String toString()
-        {
-            switch (this)
-            {
-                case EQ:
-                    return "=";
-                case LT:
-                    return "<";
-                case LTE:
-                    return "<=";
-                case GT:
-                    return ">";
-                case GTE:
-                    return ">=";
-                case NEQ:
-                    return "!=";
-                case CONTAINS_KEY:
-                    return "CONTAINS KEY";
-                default:
-                    return this.name();
-            }
-        }
-    }
+    protected Operator relationType;
 
-    public Type operator()
+    public Operator operator()
     {
         return relationType;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
index ee95da0..fd427fd 100644
--- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@ -31,7 +31,7 @@ public class SingleColumnRelation extends Relation
     private final List<Term.Raw> inValues;
     public final boolean onToken;
 
-    private SingleColumnRelation(ColumnIdentifier entity, Type type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
+    private SingleColumnRelation(ColumnIdentifier entity, Operator type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
     {
         this.entity = entity;
         this.relationType = type;
@@ -47,19 +47,19 @@ public class SingleColumnRelation extends Relation
      * @param type the type that describes how this entity relates to the value.
      * @param value the value being compared.
      */
-    public SingleColumnRelation(ColumnIdentifier entity, Type type, Term.Raw value)
+    public SingleColumnRelation(ColumnIdentifier entity, Operator type, Term.Raw value)
     {
         this(entity, type, value, null, false);
     }
 
-    public SingleColumnRelation(ColumnIdentifier entity, Type type, Term.Raw value, boolean onToken)
+    public SingleColumnRelation(ColumnIdentifier entity, Operator type, Term.Raw value, boolean onToken)
     {
         this(entity, type, value, null, onToken);
     }
 
     public static SingleColumnRelation createInRelation(ColumnIdentifier entity, List<Term.Raw> inValues)
     {
-        return new SingleColumnRelation(entity, Type.IN, null, inValues, false);
+        return new SingleColumnRelation(entity, Operator.IN, null, inValues, false);
     }
 
     public ColumnIdentifier getEntity()
@@ -69,13 +69,13 @@ public class SingleColumnRelation extends Relation
 
     public Term.Raw getValue()
     {
-        assert relationType != Type.IN || value == null || value instanceof AbstractMarker.INRaw;
+        assert relationType != Operator.IN || value == null || value instanceof AbstractMarker.INRaw;
         return value;
     }
 
     public List<Term.Raw> getInValues()
     {
-        assert relationType == Type.IN;
+        assert relationType == Operator.IN;
         return inValues;
     }
 
@@ -88,8 +88,8 @@ public class SingleColumnRelation extends Relation
     {
         switch (relationType)
         {
-            case GT: return new SingleColumnRelation(entity, Type.GTE, value);
-            case LT:  return new SingleColumnRelation(entity, Type.LTE, value);
+            case GT: return new SingleColumnRelation(entity, Operator.GTE, value);
+            case LT:  return new SingleColumnRelation(entity, Operator.LTE, value);
             default: return this;
         }
     }
@@ -97,7 +97,7 @@ public class SingleColumnRelation extends Relation
     @Override
     public String toString()
     {
-        if (relationType == Type.IN)
+        if (relationType == Operator.IN)
             return String.format("%s IN %s", entity, inValues);
         else if (onToken)
             return String.format("token(%s) %s %s", entity, relationType, value);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/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 b8cb818..69984b6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -33,8 +33,6 @@ import org.apache.cassandra.db.composites.CBuilder;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
-import org.apache.cassandra.db.marshal.CompositeType;
-import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.marshal.BooleanType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
@@ -180,7 +178,7 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                                 staticConditions == null ? Collections.<ColumnDefinition>emptyList() : Iterables.transform(staticConditions, getColumnForCondition));
     }
 
-    public void addCondition(ColumnCondition cond) throws InvalidRequestException
+    public void addCondition(ColumnCondition cond)
     {
         List<ColumnCondition> conds = null;
         if (cond.column.isStatic())
@@ -257,13 +255,13 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
                 case CLUSTERING_COLUMN:
                     Restriction restriction;
 
-                    if (rel.operator() == Relation.Type.EQ)
+                    if (rel.operator() == Operator.EQ)
                     {
                         Term t = rel.getValue().prepare(keyspace(), def);
                         t.collectMarkerSpecification(names);
                         restriction = new SingleColumnRestriction.EQ(t, false);
                     }
-                    else if (def.kind == ColumnDefinition.Kind.PARTITION_KEY && rel.operator() == Relation.Type.IN)
+                    else if (def.kind == ColumnDefinition.Kind.PARTITION_KEY && rel.operator() == Operator.IN)
                     {
                         if (rel.getValue() != null)
                         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/cql3/statements/Restriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Restriction.java b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
index c529a38..5307cbb 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
@@ -20,9 +20,11 @@ package org.apache.cassandra.cql3.statements;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.db.IndexExpression;
-import org.apache.cassandra.cql3.*;
 
 /**
  * A restriction/clause on a column.
@@ -60,10 +62,10 @@ public interface Restriction
         /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
         public boolean isInclusive(Bound b);
 
-        public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound);
+        public Operator getRelation(Bound eocBound, Bound inclusiveBound);
 
-        public IndexExpression.Operator getIndexOperator(Bound b);
+        public Operator getIndexOperator(Bound b);
 
-        public void setBound(ColumnIdentifier name, Relation.Type type, Term t) throws InvalidRequestException;
+        public void setBound(ColumnIdentifier name, Operator type, Term t) throws InvalidRequestException;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 3354c0b..aa01c93 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -38,6 +38,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.*;
+import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.dht.*;
@@ -840,7 +841,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             if (r.isSlice())
             {
                 builder.add(getSliceValue(r, b, options));
-                Relation.Type relType = ((Restriction.Slice)r).getRelation(eocBound, b);
+                Operator relType = ((Restriction.Slice)r).getRelation(eocBound, b);
                 return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
             }
             else
@@ -880,7 +881,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         return Collections.singletonList(eocBound == Bound.END && builder.remainingCount() > 0 ? prefix.end() : prefix);
     }
 
-    private static Composite.EOC eocForRelation(Relation.Type op)
+    private static Composite.EOC eocForRelation(Operator op)
     {
         switch (op)
         {
@@ -942,7 +943,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 throw new InvalidRequestException("Invalid null value in condition for column " + def.name);
             builder.add(v);
         }
-        Relation.Type relType = slice.getRelation(eocBound, firstComponentBound);
+        Operator relType = slice.getRelation(eocBound, firstComponentBound);
         return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
     }
 
@@ -1052,7 +1053,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     if (slice.hasBound(b))
                     {
                         ByteBuffer value = validateIndexedValue(def, slice.bound(b, options));
-                        IndexExpression.Operator op = slice.getIndexOperator(b);
+                        Operator op = slice.getIndexOperator(b);
                         // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
                         // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
                         // use the underlying comparator as is.
@@ -1068,12 +1069,12 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 for (ByteBuffer value : contains.values(options))
                 {
                     validateIndexedValue(def, value);
-                    expressions.add(new IndexExpression(def.name.bytes, IndexExpression.Operator.CONTAINS, value));
+                    expressions.add(new IndexExpression(def.name.bytes, Operator.CONTAINS, value));
                 }
                 for (ByteBuffer key : contains.keys(options))
                 {
                     validateIndexedValue(def, key);
-                    expressions.add(new IndexExpression(def.name.bytes, IndexExpression.Operator.CONTAINS_KEY, key));
+                    expressions.add(new IndexExpression(def.name.bytes, Operator.CONTAINS_KEY, key));
                 }
             }
             else
@@ -1084,7 +1085,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     throw new InvalidRequestException("IN restrictions are not supported on indexed columns");
 
                 ByteBuffer value = validateIndexedValue(def, values.get(0));
-                expressions.add(new IndexExpression(def.name.bytes, IndexExpression.Operator.EQ, value));
+                expressions.add(new IndexExpression(def.name.bytes, Operator.EQ, value));
             }
         }
         
@@ -1136,14 +1137,14 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         };
     }
 
-    private static IndexExpression.Operator reverse(IndexExpression.Operator op)
+    private static Operator reverse(Operator op)
     {
         switch (op)
         {
-            case LT:  return IndexExpression.Operator.GT;
-            case LTE: return IndexExpression.Operator.GTE;
-            case GT:  return IndexExpression.Operator.LT;
-            case GTE: return IndexExpression.Operator.LTE;
+            case LT:  return Operator.GT;
+            case LTE: return Operator.GTE;
+            case GT:  return Operator.LT;
+            case GTE: return Operator.LTE;
             default: return op;
         }
     }
@@ -1414,6 +1415,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             boolean hasQueriableClusteringColumnIndex = false;
             boolean hasSingleColumnRelations = false;
             boolean hasMultiColumnRelations = false;
+
+            ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
+            SecondaryIndexManager indexManager = cfs.indexManager;
+
             for (Relation relation : whereClause)
             {
                 if (relation.isMultiColumn())
@@ -1423,7 +1428,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     for (ColumnIdentifier entity : rel.getEntities())
                     {
                         ColumnDefinition def = cfm.getColumnDefinition(entity);
-                        boolean[] queriable = processRelationEntity(stmt, relation, entity, def);
+                        boolean[] queriable = processRelationEntity(stmt, indexManager, relation, entity, def);
                         hasQueriableIndex |= queriable[0];
                         hasQueriableClusteringColumnIndex |= queriable[1];
                         names.add(def);
@@ -1436,7 +1441,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     SingleColumnRelation rel = (SingleColumnRelation) relation;
                     ColumnIdentifier entity = rel.getEntity();
                     ColumnDefinition def = cfm.getColumnDefinition(entity);
-                    boolean[] queriable = processRelationEntity(stmt, relation, entity, def);
+                    boolean[] queriable = processRelationEntity(stmt, indexManager, relation, entity, def);
                     hasQueriableIndex |= queriable[0];
                     hasQueriableClusteringColumnIndex |= queriable[1];
                     hasSingleColumnRelations |= ColumnDefinition.Kind.CLUSTERING_COLUMN.equals(def.kind);
@@ -1490,13 +1495,19 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         }
 
         /** Returns a pair of (hasQueriableIndex, hasQueriableClusteringColumnIndex) */
-        private boolean[] processRelationEntity(SelectStatement stmt, Relation relation, ColumnIdentifier entity, ColumnDefinition def) throws InvalidRequestException
+        private boolean[] processRelationEntity(SelectStatement stmt,
+                                                SecondaryIndexManager indexManager,
+                                                Relation relation,
+                                                ColumnIdentifier entity,
+                                                ColumnDefinition def) throws InvalidRequestException
         {
             if (def == null)
                 handleUnrecognizedEntity(entity, relation);
 
             stmt.restrictedColumns.add(def);
-            if (relation.operator().allowsIndexQueryOn(def))
+
+            SecondaryIndex index = indexManager.getIndexForColumn(def.name.bytes);
+            if (index != null && index.supportsOperator(relation.operator()))
                 return new boolean[]{true, def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN};
             return new boolean[]{false, false};
         }
@@ -1551,10 +1562,10 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 previousPosition++;
 
                 Restriction existing = getExistingRestriction(stmt, def);
-                Relation.Type operator = relation.operator();
+                Operator operator = relation.operator();
                 if (existing != null)
                 {
-                    if (operator == Relation.Type.EQ || operator == Relation.Type.IN)
+                    if (operator == Operator.EQ || operator == Operator.IN)
                         throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by more than one relation if it is in an %s relation", def, relation.operator()));
                     else if (!existing.isSlice())
                         throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by an equality relation and an inequality relation", def));
@@ -1681,7 +1692,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             }
 
             // We don't support relations against entire collections, like "numbers = {1, 2, 3}"
-            if (receiver.type.isCollection() && !(newRel.operator().equals(Relation.Type.CONTAINS_KEY) || newRel.operator() == Relation.Type.CONTAINS))
+            if (receiver.type.isCollection() && !(newRel.operator().equals(Operator.CONTAINS_KEY) || newRel.operator() == Operator.CONTAINS))
             {
                 throw new InvalidRequestException(String.format("Collection column '%s' (%s) cannot be restricted by a '%s' relation",
                                                                 def.name, receiver.type.asCQL3Type(), newRel.operator()));
@@ -1759,7 +1770,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                         existingRestriction = new SingleColumnRestriction.Contains();
                     else if (!existingRestriction.isContains())
                         throw new InvalidRequestException(String.format("Collection column %s can only be restricted by CONTAINS or CONTAINS KEY", def.name));
-                    boolean isKey = newRel.operator() == Relation.Type.CONTAINS_KEY;
+                    boolean isKey = newRel.operator() == Operator.CONTAINS_KEY;
                     receiver = makeCollectionReceiver(receiver, isKey);
                     Term t = newRel.getValue().prepare(keyspace(), receiver);
                     t.collectMarkerSpecification(boundNames);
@@ -2097,7 +2108,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 // so it can't be a MultiColumnRelation.
                 SingleColumnRelation rel = (SingleColumnRelation)r;
                 if (cfm.getColumnDefinition(rel.getEntity()).kind == ColumnDefinition.Kind.CLUSTERING_COLUMN
-                    && (rel.operator() == Relation.Type.GT || rel.operator() == Relation.Type.LT))
+                    && (rel.operator() == Operator.GT || rel.operator() == Operator.LT))
                     return rel;
             }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
index 1ee0ebe..7f8156e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.IndexExpression;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
 import java.nio.ByteBuffer;
@@ -249,35 +248,35 @@ public abstract class SingleColumnRestriction implements Restriction
             return bounds[b.idx] == null || boundInclusive[b.idx];
         }
 
-        public Relation.Type getRelation(Bound eocBound, Bound inclusiveBound)
+        public Operator getRelation(Bound eocBound, Bound inclusiveBound)
         {
             switch (eocBound)
             {
                 case START:
-                    return boundInclusive[inclusiveBound.idx] ? Relation.Type.GTE : Relation.Type.GT;
+                    return boundInclusive[inclusiveBound.idx] ? Operator.GTE : Operator.GT;
                 case END:
-                    return boundInclusive[inclusiveBound.idx] ? Relation.Type.LTE : Relation.Type.LT;
+                    return boundInclusive[inclusiveBound.idx] ? Operator.LTE : Operator.LT;
             }
             throw new AssertionError();
         }
 
-        public IndexExpression.Operator getIndexOperator(Bound b)
+        public Operator getIndexOperator(Bound b)
         {
             switch (b)
             {
                 case START:
-                    return boundInclusive[b.idx] ? IndexExpression.Operator.GTE : IndexExpression.Operator.GT;
+                    return boundInclusive[b.idx] ? Operator.GTE : Operator.GT;
                 case END:
-                    return boundInclusive[b.idx] ? IndexExpression.Operator.LTE : IndexExpression.Operator.LT;
+                    return boundInclusive[b.idx] ? Operator.LTE : Operator.LT;
             }
             throw new AssertionError();
         }
 
-        public void setBound(ColumnIdentifier name, Relation.Type type, Term t) throws InvalidRequestException
+        public void setBound(ColumnIdentifier name, Operator operator, Term t) throws InvalidRequestException
         {
             Bound b;
             boolean inclusive;
-            switch (type)
+            switch (operator)
             {
                 case GT:
                     b = Bound.START;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/db/IndexExpression.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/IndexExpression.java b/src/java/org/apache/cassandra/db/IndexExpression.java
index 910bfbc..bdb74ce 100644
--- a/src/java/org/apache/cassandra/db/IndexExpression.java
+++ b/src/java/org/apache/cassandra/db/IndexExpression.java
@@ -19,13 +19,17 @@
 
 package org.apache.cassandra.db;
 
+import java.io.DataInput;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import com.google.common.base.Objects;
 
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class IndexExpression
+public final class IndexExpression
 {
     public final ByteBuffer column;
     public final Operator operator;
@@ -38,68 +42,6 @@ public class IndexExpression
         this.value = value;
     }
 
-    public enum Operator
-    {
-        EQ, GTE, GT, LTE, LT, CONTAINS, CONTAINS_KEY;
-
-        public static Operator findByOrdinal(int ordinal)
-        {
-            switch (ordinal) {
-                case 0:
-                    return EQ;
-                case 1:
-                    return GTE;
-                case 2:
-                    return GT;
-                case 3:
-                    return LTE;
-                case 4:
-                    return LT;
-                case 5:
-                    return CONTAINS;
-                case 6:
-                    return CONTAINS_KEY;
-                default:
-                    throw new AssertionError();
-            }
-        }
-
-        public boolean allowsIndexQuery()
-        {
-            switch (this)
-            {
-                case EQ:
-                case CONTAINS:
-                case CONTAINS_KEY:
-                    return true;
-                default:
-                    return false;
-            }
-        }
-
-        @Override
-        public String toString()
-        {
-            switch (this)
-            {
-                case EQ:
-                    return "=";
-                case LT:
-                    return "<";
-                case LTE:
-                    return "<=";
-                case GT:
-                    return ">";
-                case GTE:
-                    return ">=";
-                case CONTAINS_KEY:
-                    return "CONTAINS KEY";
-                default:
-                    return this.name();
-            }
-        }
-    }
-
     /**
      * Checks if the operator of this <code>IndexExpression</code> is a <code>CONTAINS</code> operator.
      *
@@ -149,4 +91,31 @@ public class IndexExpression
     {
         return Objects.hashCode(column, operator, value);
     }
+
+    /**
+     * Write the serialized version of this <code>IndexExpression</code> to the specified output.
+     *
+     * @param output the output to write to
+     * @throws IOException if an I/O problem occurs while writing to the specified output
+     */
+    public void writeTo(DataOutputPlus output) throws IOException
+    {
+        ByteBufferUtil.writeWithShortLength(column, output);
+        operator.writeTo(output);
+        ByteBufferUtil.writeWithShortLength(value, output);
+    }
+
+    /**
+     * Deserializes an <code>IndexExpression</code> instance from the specified input. 
+     *
+     * @param input the input to read from 
+     * @return the <code>IndexExpression</code> instance deserialized
+     * @throws IOException if a problem occurs while deserializing the <code>IndexExpression</code> instance.
+     */
+    public static IndexExpression readFrom(DataInput input) throws IOException
+    {
+        return new IndexExpression(ByteBufferUtil.readWithShortLength(input),
+                                   Operator.readFrom(input),
+                                   ByteBufferUtil.readWithShortLength(input));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/db/PagedRangeCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PagedRangeCommand.java b/src/java/org/apache/cassandra/db/PagedRangeCommand.java
index f2d81b9..614f0f7 100644
--- a/src/java/org/apache/cassandra/db/PagedRangeCommand.java
+++ b/src/java/org/apache/cassandra/db/PagedRangeCommand.java
@@ -31,7 +31,6 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class PagedRangeCommand extends AbstractRangeCommand
 {
@@ -145,9 +144,7 @@ public class PagedRangeCommand extends AbstractRangeCommand
             out.writeInt(cmd.rowFilter.size());
             for (IndexExpression expr : cmd.rowFilter)
             {
-                ByteBufferUtil.writeWithShortLength(expr.column, out);
-                out.writeInt(expr.operator.ordinal());
-                ByteBufferUtil.writeWithShortLength(expr.value, out);
+                expr.writeTo(out);;
             }
 
             out.writeInt(cmd.limit);
@@ -174,10 +171,7 @@ public class PagedRangeCommand extends AbstractRangeCommand
             List<IndexExpression> rowFilter = new ArrayList<IndexExpression>(filterCount);
             for (int i = 0; i < filterCount; i++)
             {
-                IndexExpression expr = new IndexExpression(ByteBufferUtil.readWithShortLength(in),
-                                                           IndexExpression.Operator.findByOrdinal(in.readInt()),
-                                                           ByteBufferUtil.readWithShortLength(in));
-                rowFilter.add(expr);
+                rowFilter.add(IndexExpression.readFrom(in));
             }
 
             int limit = in.readInt();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/db/RangeSliceCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RangeSliceCommand.java b/src/java/org/apache/cassandra/db/RangeSliceCommand.java
index 82e892c..4d2955b 100644
--- a/src/java/org/apache/cassandra/db/RangeSliceCommand.java
+++ b/src/java/org/apache/cassandra/db/RangeSliceCommand.java
@@ -34,7 +34,6 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.pager.Pageable;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class RangeSliceCommand extends AbstractRangeCommand implements Pageable
 {
@@ -170,9 +169,7 @@ class RangeSliceCommandSerializer implements IVersionedSerializer<RangeSliceComm
             out.writeInt(sliceCommand.rowFilter.size());
             for (IndexExpression expr : sliceCommand.rowFilter)
             {
-                ByteBufferUtil.writeWithShortLength(expr.column, out);
-                out.writeInt(expr.operator.ordinal());
-                ByteBufferUtil.writeWithShortLength(expr.value, out);
+                expr.writeTo(out);
             }
         }
         AbstractBounds.serializer.serialize(sliceCommand.keyRange, out, version);
@@ -196,11 +193,7 @@ class RangeSliceCommandSerializer implements IVersionedSerializer<RangeSliceComm
         rowFilter = new ArrayList<>(filterCount);
         for (int i = 0; i < filterCount; i++)
         {
-            IndexExpression expr;
-            expr = new IndexExpression(ByteBufferUtil.readWithShortLength(in),
-                                       IndexExpression.Operator.findByOrdinal(in.readInt()),
-                                       ByteBufferUtil.readWithShortLength(in));
-            rowFilter.add(expr);
+            rowFilter.add(IndexExpression.readFrom(in));
         }
         AbstractBounds<RowPosition> range = AbstractBounds.serializer.deserialize(in, version).toRowBounds();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
index b152472..2d56bfa 100644
--- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
@@ -18,15 +18,26 @@
 package org.apache.cassandra.db.filter;
 
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
 
 import com.google.common.base.Objects;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.IndexExpression;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -130,7 +141,7 @@ public abstract class ExtendedFilter
      */
     public abstract boolean isSatisfiedBy(DecoratedKey rowKey, ColumnFamily data, Composite prefix, ByteBuffer collectionElement);
 
-    public static boolean satisfies(int comparison, IndexExpression.Operator op)
+    public static boolean satisfies(int comparison, Operator op)
     {
         switch (op)
         {
@@ -339,7 +350,7 @@ public abstract class ExtendedFilter
             assert def.type.isCollection();
             CollectionType type = (CollectionType)def.type;
 
-            if (expr.operator == IndexExpression.Operator.CONTAINS)
+            if (expr.isContains())
             {
                 // get a slice of the collection cells
                 Iterator<Cell> iter = data.iterator(new ColumnSlice[]{ data.getComparator().create(prefix, def).slice() });
@@ -369,15 +380,13 @@ public abstract class ExtendedFilter
                 case SET:
                     return data.getColumn(data.getComparator().create(prefix, def, expr.value)) != null;
                 case MAP:
-                    if (expr.operator == IndexExpression.Operator.CONTAINS_KEY)
+                    if (expr.isContainsKey())
                     {
                         return data.getColumn(data.getComparator().create(prefix, def, expr.value)) != null;
                     }
-                    else
-                    {
-                        assert collectionElement != null;
-                        return type.valueComparator().compare(data.getColumn(data.getComparator().create(prefix, def, collectionElement)).value(), expr.value) == 0;
-                    }
+
+                    assert collectionElement != null;
+                    return type.valueComparator().compare(data.getColumn(data.getComparator().create(prefix, def, collectionElement)).value(), expr.value) == 0;
             }
             throw new AssertionError();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
index e3ed73c..e2bafaa 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
@@ -27,19 +27,18 @@ import java.util.concurrent.Future;
 import java.util.concurrent.FutureTask;
 
 import org.apache.commons.lang3.StringUtils;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
-
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.IndexExpression;
 import org.apache.cassandra.db.SystemKeyspace;
-
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -48,7 +47,6 @@ import org.apache.cassandra.db.index.composites.CompositesIndex;
 import org.apache.cassandra.db.index.keys.KeysIndex;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.LocalByPartionerType;
 import org.apache.cassandra.dht.LocalToken;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -57,8 +55,6 @@ import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
-import com.google.common.collect.Iterables;
-
 /**
  * Abstract base class for different types of secondary indexes.
  *
@@ -297,9 +293,9 @@ public abstract class SecondaryIndex
     }
 
     /** Returns true if the index supports lookups for the given operator, false otherwise. */
-    public boolean supportsOperator(IndexExpression.Operator operator)
+    public boolean supportsOperator(Operator operator)
     {
-        return operator == IndexExpression.Operator.EQ;
+        return operator == Operator.EQ;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
index 1067a94..81982bb 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CBuilder;
 import org.apache.cassandra.db.composites.CellName;
@@ -89,10 +90,10 @@ public class CompositesIndexOnCollectionKey extends CompositesIndex
     }
 
     @Override
-    public boolean supportsOperator(IndexExpression.Operator operator)
+    public boolean supportsOperator(Operator operator)
     {
-        return operator == IndexExpression.Operator.CONTAINS_KEY ||
-                operator == IndexExpression.Operator.CONTAINS && columnDef.type instanceof SetType;
+        return operator == Operator.CONTAINS_KEY ||
+                operator == Operator.CONTAINS && columnDef.type instanceof SetType;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
index 5b4aa64..e69e656 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CBuilder;
 import org.apache.cassandra.db.composites.CellName;
@@ -95,9 +96,9 @@ public class CompositesIndexOnCollectionValue extends CompositesIndex
     }
 
     @Override
-    public boolean supportsOperator(IndexExpression.Operator operator)
+    public boolean supportsOperator(Operator operator)
     {
-        return operator == IndexExpression.Operator.CONTAINS;
+        return operator == Operator.CONTAINS;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/db/marshal/CompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index 1115dff..9ee9fb3 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -30,7 +30,7 @@ import com.google.common.collect.ImmutableList;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.Relation;
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -430,7 +430,7 @@ public class CompositeType extends AbstractCompositeType
             return bb;
         }
 
-        public ByteBuffer buildForRelation(Relation.Type op)
+        public ByteBuffer buildForRelation(Operator op)
         {
             /*
              * Given the rules for eoc (end-of-component, see AbstractCompositeType.compare()),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5117a5d5/src/java/org/apache/cassandra/thrift/ThriftConversion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftConversion.java b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
index 0c75d2c..2aca45a 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftConversion.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.RequestTimeoutException;
@@ -124,7 +125,7 @@ public class ThriftConversion
         for (IndexExpression expr : exprs)
         {
             converted.add(new org.apache.cassandra.db.IndexExpression(expr.column_name,
-                                                                      org.apache.cassandra.db.IndexExpression.Operator.findByOrdinal(expr.op.getValue()),
+                                                                      Operator.valueOf(expr.op.name()),
                                                                       expr.value));
         }
         return converted;