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/12/02 20:09:07 UTC

[3/5] cassandra git commit: Refactor SelectStatement and Restrictions

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java
index 6ad36e9..e44a39f 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@ -35,17 +35,36 @@ import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 
 public abstract class Selection
 {
+    /**
+     * A predicate that returns <code>true</code> for static columns.
+     */
+    private static final Predicate<ColumnDefinition> STATIC_COLUMN_FILTER = new Predicate<ColumnDefinition>()
+    {
+        public boolean apply(ColumnDefinition def)
+        {
+            return def.isStatic();
+        }
+    };
+
+    private final CFMetaData cfm;
     private final Collection<ColumnDefinition> columns;
     private final ResultSet.Metadata metadata;
     private final boolean collectTimestamps;
     private final boolean collectTTLs;
 
-    protected Selection(Collection<ColumnDefinition> columns, List<ColumnSpecification> metadata, boolean collectTimestamps, boolean collectTTLs)
+    protected Selection(CFMetaData cfm,
+                        Collection<ColumnDefinition> columns,
+                        List<ColumnSpecification> metadata,
+                        boolean collectTimestamps,
+                        boolean collectTTLs)
     {
+        this.cfm = cfm;
         this.columns = columns;
         this.metadata = new ResultSet.Metadata(metadata);
         this.collectTimestamps = collectTimestamps;
@@ -56,6 +75,76 @@ public abstract class Selection
     public boolean isWildcard()
     {
         return false;
+    }    
+
+    /**
+     * Checks if this selection contains static columns.
+     * @return <code>true</code> if this selection contains static columns, <code>false</code> otherwise;
+     */
+    public boolean containsStaticColumns()
+    {
+        if (!cfm.hasStaticColumns())
+            return false;
+
+        if (isWildcard())
+            return true;
+
+        return !Iterables.isEmpty(Iterables.filter(columns, STATIC_COLUMN_FILTER));
+    }
+
+    /**
+     * Checks if this selection contains only static columns.
+     * @return <code>true</code> if this selection contains only static columns, <code>false</code> otherwise;
+     */
+    public boolean containsOnlyStaticColumns()
+    {
+        if (!containsStaticColumns())
+            return false;
+
+        if (isWildcard())
+            return false;
+
+        for (ColumnDefinition def : getColumns())
+        {
+            if (!def.isPartitionKey() && !def.isStatic())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Checks if this selection contains a collection.
+     *
+     * @return <code>true</code> if this selection contains a collection, <code>false</code> otherwise.
+     */
+    public boolean containsACollection()
+    {
+        if (!cfm.comparator.hasCollections())
+            return false;
+
+        for (ColumnDefinition def : getColumns())
+            if (def.type.isCollection() && def.type.isMultiCell())
+                return true;
+
+        return false;
+    }
+
+    /**
+     * Returns the index of the specified column.
+     *
+     * @param def the column definition
+     * @return the index of the specified column
+     */
+    public int indexOf(final ColumnDefinition def)
+    {
+        return Iterators.indexOf(getColumns().iterator(), new Predicate<ColumnDefinition>()
+           {
+               public boolean apply(ColumnDefinition n)
+               {
+                   return def.name.equals(n.name);
+               }
+           });
     }
 
     public ResultSet.Metadata getResultMetadata()
@@ -67,12 +156,12 @@ public abstract class Selection
     {
         List<ColumnDefinition> all = new ArrayList<ColumnDefinition>(cfm.allColumns().size());
         Iterators.addAll(all, cfm.allColumnsInSelectOrder());
-        return new SimpleSelection(all, true);
+        return new SimpleSelection(cfm, all, true);
     }
 
-    public static Selection forColumns(Collection<ColumnDefinition> columns)
+    public static Selection forColumns(CFMetaData cfm, Collection<ColumnDefinition> columns)
     {
-        return new SimpleSelection(columns, false);
+        return new SimpleSelection(cfm, columns, false);
     }
 
     public int addColumnForOrdering(ColumnDefinition c)
@@ -105,8 +194,8 @@ public abstract class Selection
                 SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, cfm), cfm, defs);
         List<ColumnSpecification> metadata = collectMetadata(cfm, rawSelectors, factories);
 
-        return processesSelection(rawSelectors) ? new SelectionWithProcessing(defs, metadata, factories)
-                                                : new SimpleSelection(defs, metadata, false);
+        return processesSelection(rawSelectors) ? new SelectionWithProcessing(cfm, defs, metadata, factories)
+                                                : new SimpleSelection(cfm, defs, metadata, false);
     }
 
     private static List<ColumnSpecification> collectMetadata(CFMetaData cfm,
@@ -280,19 +369,22 @@ public abstract class Selection
     {
         private final boolean isWildcard;
 
-        public SimpleSelection(Collection<ColumnDefinition> columns, boolean isWildcard)
+        public SimpleSelection(CFMetaData cfm, Collection<ColumnDefinition> columns, boolean isWildcard)
         {
-            this(columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
+            this(cfm, columns, new ArrayList<ColumnSpecification>(columns), isWildcard);
         }
 
-        public SimpleSelection(Collection<ColumnDefinition> columns, List<ColumnSpecification> metadata, boolean isWildcard)
+        public SimpleSelection(CFMetaData cfm,
+                               Collection<ColumnDefinition> columns,
+                               List<ColumnSpecification> metadata,
+                               boolean isWildcard)
         {
             /*
              * In theory, even a simple selection could have multiple time the same column, so we
              * could filter those duplicate out of columns. But since we're very unlikely to
              * get much duplicate in practice, it's more efficient not to bother.
              */
-            super(columns, metadata, false, false);
+            super(cfm, columns, metadata, false, false);
             this.isWildcard = isWildcard;
         }
 
@@ -340,17 +432,24 @@ public abstract class Selection
     {
         private final SelectorFactories factories;
 
-        public SelectionWithProcessing(Collection<ColumnDefinition> columns,
+        public SelectionWithProcessing(CFMetaData cfm,
+                                       Collection<ColumnDefinition> columns,
                                        List<ColumnSpecification> metadata,
                                        SelectorFactories factories) throws InvalidRequestException
         {
-            super(columns, metadata, factories.containsWritetimeSelectorFactory(), factories.containsTTLSelectorFactory());
+            super(cfm,
+                  columns,
+                  metadata,
+                  factories.containsWritetimeSelectorFactory(),
+                  factories.containsTTLSelectorFactory());
+
             this.factories = factories;
 
             if (factories.doesAggregation() && !factories.containsOnlyAggregateFunctions())
                 throw new InvalidRequestException("the select clause must either contains only aggregates or none");
         }
 
+        @Override
         public boolean usesFunction(String ksName, String functionName)
         {
             return factories.usesFunction(ksName, functionName);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/statements/Bound.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Bound.java b/src/java/org/apache/cassandra/cql3/statements/Bound.java
index 8aa3a37..7742642 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Bound.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Bound.java
@@ -28,8 +28,18 @@ public enum Bound
         this.idx = idx;
     }
 
-    public static Bound reverse(Bound b)
+    public Bound reverse()
     {
-        return b == START ? END : START;
+        return isStart() ? END : START;
+    }
+
+    public boolean isStart()
+    {
+        return this == START;
+    }
+
+    public boolean isEnd()
+    {
+        return this == END;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index 33c61e7..ff685cf 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -23,6 +23,7 @@ import java.util.*;
 import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.restrictions.Restriction;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/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 4e39614..8347ef5 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -27,6 +27,8 @@ import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction;
 import org.apache.cassandra.cql3.selection.Selection;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CBuilder;
@@ -237,7 +239,7 @@ public abstract class ModificationStatement implements CQLStatement
 
     public void addKeyValue(ColumnDefinition def, Term value) throws InvalidRequestException
     {
-        addKeyValues(def, new SingleColumnRestriction.EQ(value, false));
+        addKeyValues(def, new SingleColumnRestriction.EQ(def, value));
     }
 
     public void processWhereClause(List<Relation> whereClause, VariableSpecifications names) throws InvalidRequestException
@@ -251,7 +253,7 @@ public abstract class ModificationStatement implements CQLStatement
             }
             SingleColumnRelation rel = (SingleColumnRelation) relation;
 
-            if (rel.onToken)
+            if (rel.onToken())
                 throw new InvalidRequestException(String.format("The token function cannot be used in WHERE clauses for UPDATE and DELETE statements: %s", relation));
 
             ColumnIdentifier id = rel.getEntity().prepare(cfm);
@@ -265,31 +267,9 @@ public abstract class ModificationStatement implements CQLStatement
                 case CLUSTERING_COLUMN:
                     Restriction restriction;
 
-                    if (rel.operator() == Operator.EQ)
+                    if (rel.isEQ() || (def.isPartitionKey() && rel.isIN()))
                     {
-                        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() == Operator.IN)
-                    {
-                        if (rel.getValue() != null)
-                        {
-                            Term t = rel.getValue().prepare(keyspace(), def);
-                            t.collectMarkerSpecification(names);
-                            restriction = new SingleColumnRestriction.InWithMarker((Lists.Marker)t);
-                        }
-                        else
-                        {
-                            List<Term> values = new ArrayList<Term>(rel.getInValues().size());
-                            for (Term.Raw raw : rel.getInValues())
-                            {
-                                Term t = raw.prepare(keyspace(), def);
-                                t.collectMarkerSpecification(names);
-                                values.add(t);
-                            }
-                            restriction = new SingleColumnRestriction.InWithValues(values);
-                        }
+                        restriction = rel.toRestriction(cfm, names);
                     }
                     else
                     {
@@ -623,7 +603,8 @@ public abstract class ModificationStatement implements CQLStatement
             }
             for (ColumnDefinition def : columnsWithConditions)
                 defs.add(def);
-            selection = Selection.forColumns(new ArrayList<>(defs));
+            selection = Selection.forColumns(cfm, new ArrayList<>(defs));
+
         }
 
         long now = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
deleted file mode 100644
index 96cb905..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3.statements;
-
-import org.apache.cassandra.cql3.AbstractMarker;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.cql3.Tuples;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-public interface MultiColumnRestriction extends Restriction
-{
-    public static class EQ extends SingleColumnRestriction.EQ implements MultiColumnRestriction
-    {
-        public EQ(Term value, boolean onToken)
-        {
-            super(value, onToken);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            Tuples.Value t = (Tuples.Value)value.bind(options);
-            return t.getElements();
-        }
-    }
-
-    public interface IN extends MultiColumnRestriction
-    {
-        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException;
-    }
-
-    /**
-     * An IN restriction that has a set of terms for in values.
-     * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)"
-     */
-    public static class InWithValues extends SingleColumnRestriction.InWithValues implements MultiColumnRestriction.IN
-    {
-        public InWithValues(List<Term> values)
-        {
-            super(values);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
-        {
-            List<List<ByteBuffer>> buffers = new ArrayList<>(values.size());
-            for (Term value : values)
-            {
-                Term.MultiItemTerminal term = (Term.MultiItemTerminal)value.bind(options);
-                buffers.add(term.getElements());
-            }
-            return buffers;
-        }
-    }
-
-    /**
-     * An IN restriction that uses a single marker for a set of IN values that are tuples.
-     * For example: "SELECT ... WHERE (a, b, c) IN ?"
-     */
-    public static class InWithMarker extends SingleColumnRestriction.InWithMarker implements MultiColumnRestriction.IN
-    {
-        public InWithMarker(AbstractMarker marker)
-        {
-            super(marker);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
-        {
-            Tuples.InMarker inMarker = (Tuples.InMarker)marker;
-            Tuples.InValue inValue = inMarker.bind(options);
-            if (inValue == null)
-                throw new InvalidRequestException("Invalid null value for IN restriction");
-            return inValue.getSplitValues();
-        }
-    }
-
-    public static class Slice extends SingleColumnRestriction.Slice implements MultiColumnRestriction
-    {
-        public Slice(boolean onToken)
-        {
-            super(onToken);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException
-        {
-            throw new UnsupportedOperationException("Multicolumn slice restrictions do not support bound()");
-        }
-
-        /**
-         * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single
-         * ByteBuffer to represent the entire bound.
-         */
-        public List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
-        {
-            Tuples.Value value = (Tuples.Value)bounds[b.idx].bind(options);
-            return value.getElements();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java b/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java
new file mode 100644
index 0000000..ce15e15
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java
@@ -0,0 +1,194 @@
+/*
+ * 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.statements;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.commons.lang3.ArrayUtils.EMPTY_OBJECT_ARRAY;
+
+/**
+ * Utility methods use to perform request validation.
+ */
+public final class RequestValidations
+{
+    /**
+     * Checks that the specified expression is <code>true</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param messageTemplate the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified expression is <code>false</code>.
+     */
+    public static void checkTrue(boolean expression, String message) throws InvalidRequestException
+    {
+        checkTrue(expression, message, EMPTY_OBJECT_ARRAY);
+    }
+
+    /**
+     * Checks that the specified expression is <code>true</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified expression is <code>false</code>.
+     */
+    public static void checkTrue(boolean expression,
+                                 String messageTemplate,
+                                 Object... messageArgs)
+                                 throws InvalidRequestException
+    {
+        if (!expression)
+            throw invalidRequest(messageTemplate, messageArgs);
+    }
+
+    /**
+     * Checks that the specified list does not contains duplicates.
+     *
+     * @param list the list to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified list contains duplicates.
+     */
+    public static void checkContainsNoDuplicates(List<?> list, String message) throws InvalidRequestException
+    {
+        if (new HashSet<>(list).size() != list.size())
+            throw invalidRequest(message);
+    }
+
+    /**
+     * Checks that the specified list contains only the specified elements.
+     *
+     * @param list the list to test
+     * @param expectedElements the expected elements
+     * @param message the error message
+     * @throws InvalidRequestException if the specified list contains duplicates.
+     */
+    public static <E> void checkContainsOnly(List<E> list,
+                                             List<E> expectedElements,
+                                             String message) throws InvalidRequestException
+    {
+        List<E> copy = new ArrayList<>(list);
+        copy.removeAll(expectedElements);
+        if (!copy.isEmpty())
+            throw invalidRequest(message);
+    }
+
+    /**
+     * Checks that the specified expression is <code>false</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified expression is <code>true</code>.
+     */
+    public static void checkFalse(boolean expression,
+                                  String messageTemplate,
+                                  Object... messageArgs)
+                                  throws InvalidRequestException
+    {
+        checkTrue(!expression, messageTemplate, messageArgs);
+    }
+
+    /**
+     * Checks that the specified expression is <code>false</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param message the error message
+     * @throws InvalidRequestException if the specified expression is <code>true</code>.
+     */
+    public static void checkFalse(boolean expression, String message) throws InvalidRequestException
+    {
+        checkTrue(!expression, message);
+    }
+
+    /**
+     * Checks that the specified object is NOT <code>null</code>.
+     * If it is an <code>InvalidRequestException</code> will be throws.
+     *
+     * @param object the object to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @return the object
+     * @throws InvalidRequestException if the specified object is <code>null</code>.
+     */
+    public static <T> T checkNotNull(T object, String messageTemplate, Object... messageArgs)
+            throws InvalidRequestException
+    {
+        checkTrue(object != null, messageTemplate, messageArgs);
+        return object;
+    }
+
+    /**
+     * Checks that the specified object is <code>null</code>.
+     * If it is not an <code>InvalidRequestException</code> will be throws.
+     *
+     * @param object the object to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @return the object
+     * @throws InvalidRequestException if the specified object is not <code>null</code>.
+     */
+    public static <T> T checkNull(T object, String messageTemplate, Object... messageArgs)
+            throws InvalidRequestException
+    {
+        checkTrue(object == null, messageTemplate, messageArgs);
+        return object;
+    }
+
+    /**
+     * Checks that the specified object is <code>null</code>.
+     * If it is not an <code>InvalidRequestException</code> will be throws.
+     *
+     * @param object the object to test
+     * @param message the error message
+     * @return the object
+     * @throws InvalidRequestException if the specified object is not <code>null</code>.
+     */
+    public static <T> T checkNull(T object, String message) throws InvalidRequestException
+    {
+        return checkNull(object, message, EMPTY_OBJECT_ARRAY);
+    }
+
+    /**
+     * Returns an <code>InvalidRequestException</code> with the specified message.
+     *
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @return an <code>InvalidRequestException</code> with the specified message.
+     */
+    public static InvalidRequestException invalidRequest(String messageTemplate, Object... messageArgs)
+    {
+        return new InvalidRequestException(String.format(messageTemplate, messageArgs));
+    }
+
+    /**
+     * This class must not be instantiated as it only contains static methods.
+     */
+    private RequestValidations()
+    {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/65a7088e/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
deleted file mode 100644
index b264156..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3.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;
-
-/**
- * A restriction/clause on a column.
- * The goal of this class being to group all conditions for a column in a SELECT.
- */
-public interface Restriction
-{
-    public boolean isOnToken();
-
-    public boolean isSlice();
-    public boolean isEQ();
-    public boolean isIN();
-    public boolean isContains();
-    public boolean isMultiColumn();
-
-    /**
-     * Returns true if, when applied to a clustering column, this restriction can be handled through one or more slices
-     * alone without filtering.  For example, EQ restrictions can be represented as a slice, but CONTAINS cannot.
-     */
-    public boolean canEvaluateWithSlices();
-
-    // Not supported by Slice, but it's convenient to have here
-    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
-
-    boolean usesFunction(String ksName, String functionName);
-
-    public static interface EQ extends Restriction {}
-
-    public static interface IN extends Restriction
-    {
-        public boolean canHaveOnlyOneValue();
-    }
-
-    public static interface Slice extends Restriction
-    {
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
-
-        /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */
-        public boolean hasBound(Bound b);
-
-        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException;
-
-        /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
-        public boolean isInclusive(Bound b);
-
-        public Operator getRelation(Bound eocBound, Bound inclusiveBound);
-
-        public Operator getIndexOperator(Bound b);
-
-        public void setBound(ColumnIdentifier name, Operator type, Term t) throws InvalidRequestException;
-    }
-}