cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ble...@apache.org
Subject [3/3] cassandra git commit: Allow selecting Map values and Set elements
Date Thu, 01 Jun 2017 15:07:37 GMT
Allow selecting Map values and Set elements

patch and review by Robert Stupp, Sylvain Lebresne and Benjamin Lerer for CASSANDRA-7396


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

Branch: refs/heads/trunk
Commit: 4ebab66163e136773ae68216585a5c587eba7006
Parents: 06da35f
Author: Sylvain Lebresne <sylvain@datastax.com>
Authored: Thu Jun 1 16:57:13 2017 +0200
Committer: Benjamin Lerer <b.lerer@gmail.com>
Committed: Thu Jun 1 16:57:13 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   1 +
 doc/cql3/CQL.textile                            |   9 +
 pylib/cqlshlib/cql3handling.py                  |   2 +-
 src/antlr/Lexer.g                               |   9 +-
 src/antlr/Parser.g                              |  25 +-
 .../org/apache/cassandra/cql3/ResultSet.java    |   9 +-
 src/java/org/apache/cassandra/cql3/Term.java    |  14 +
 .../selection/AbstractFunctionSelector.java     |  23 +
 .../selection/AggregateFunctionSelector.java    |   1 -
 .../cql3/selection/AliasedSelectable.java       |  94 +++
 .../cql3/selection/CollectionFactory.java       |  13 +
 .../cql3/selection/ColumnFilterFactory.java     | 134 ++++
 .../cql3/selection/ElementsSelector.java        | 325 ++++++++
 .../cassandra/cql3/selection/FieldSelector.java |  17 +-
 .../cql3/selection/ForwardingFactory.java       |  23 +-
 .../cassandra/cql3/selection/ListSelector.java  |   9 +-
 .../cassandra/cql3/selection/MapSelector.java   |  34 +-
 .../cassandra/cql3/selection/RawSelector.java   |   7 +-
 .../cql3/selection/ResultSetBuilder.java        | 165 ++++
 .../cql3/selection/ScalarFunctionSelector.java  |   1 -
 .../cassandra/cql3/selection/Selectable.java    | 289 ++++++-
 .../cassandra/cql3/selection/Selection.java     | 463 +++++------
 .../cassandra/cql3/selection/Selector.java      |  42 +-
 .../cql3/selection/SelectorFactories.java       |  33 +
 .../cassandra/cql3/selection/SetSelector.java   |   9 +-
 .../cql3/selection/SimpleSelector.java          | 119 ++-
 .../cassandra/cql3/selection/TermSelector.java  |  16 +-
 .../cassandra/cql3/selection/TupleSelector.java |   9 +-
 .../cql3/selection/UserTypeSelector.java        |  27 +-
 .../cql3/selection/WritetimeOrTTLSelector.java  |  28 +-
 .../cql3/statements/CreateViewStatement.java    |   4 +
 .../cql3/statements/ModificationStatement.java  |   7 +-
 .../cql3/statements/SelectStatement.java        | 245 +++---
 .../cassandra/db/filter/ColumnFilter.java       |  59 +-
 .../cassandra/db/marshal/AbstractType.java      |  45 +-
 .../apache/cassandra/db/marshal/ListType.java   |   6 +
 .../apache/cassandra/schema/ColumnMetadata.java |  19 +-
 .../serializers/CollectionSerializer.java       |  56 +-
 .../cassandra/serializers/ListSerializer.java   |  17 +-
 .../cassandra/serializers/MapSerializer.java    |  80 +-
 .../cassandra/serializers/SetSerializer.java    |  87 ++
 .../org/apache/cassandra/cql3/CQLTester.java    |   2 +-
 .../validation/entities/CollectionsTest.java    | 791 +++++++++++++++++++
 .../validation/entities/StaticColumnsTest.java  |   3 +-
 .../operations/SelectOrderByTest.java           |  17 +
 46 files changed, 2908 insertions(+), 481 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 48ca381..277b846 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Allow selecting Map values and Set elements (CASSANDRA-7396)
  * Fast and garbage-free Streaming Histogram (CASSANDRA-13444)
  * Update repairTime for keyspaces on completion (CASSANDRA-13539)
  * Add configurable upper bound for validation executor threads (CASSANDRA-13521)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 4ef3e85..e31601a 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -25,6 +25,7 @@ New features
    - Support for arithmetic operations on number has been added. See CASSANDRA-11935
    - Preview expected streaming required for a repair (nodetool repair --preview), and validate the
      consistency of repaired data between nodes (nodetool repair --validate). See CASSANDRA-13257
+   - Support for selecting Map values and Set elements has been added for SELECT queries. See CASSANDRA-7396 
 
 Upgrading
 ---------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index f2f9bd8..88d6694 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1070,6 +1070,9 @@ bc(syntax)..
              | TTL '(' <identifier> ')'
              | CAST '(' <selector> AS <type> ')'
              | <function> '(' (<selector> (',' <selector>)*)? ')'
+             | <selector> '.' <identifier>
+             | <selector> '[' <term> ']'
+             | <selector> '[' <term>? .. <term>? ']'
 
 <where-clause> ::= <relation> ( AND <relation> )*
 
@@ -1113,6 +1116,8 @@ The @<select-clause>@ determines which columns needs to be queried and returned
 
 A @<selector>@ is either a column name to retrieve or a @<function>@ of one or more @<term>@s. The function allowed are the same as for @<term>@ and are described in the "function section":#functions. In addition to these generic functions, the @WRITETIME@ (resp. @TTL@) function allows to select the timestamp of when the column was inserted (resp. the time to live (in seconds) for the column (or null if the column has no expiration set)) and the "@CAST@":#castFun function can be used to convert one data type to another.
 
+Additionally, individual values of maps and sets can be selected using @[ <term> ]@. For maps, this will return the value corresponding to the key, if such entry exists. For sets, this will return the key that is selected if it exists and is thus mainly a way to check element existence. It is also possible to select a slice of a set or map with @[ <term> ... <term> @], where both bound can be omitted.
+
 Any @<selector>@ can be aliased using @AS@ keyword (see examples). Please note that @<where-clause>@ and @<order-by>@ clause should refer to the columns by their original names and not by their aliases.
 
 The @COUNT@ keyword can be used with parenthesis enclosing @*@. If so, the query will return a single result: the number of rows matching the query. Note that @COUNT(1)@ is supported as an alias.
@@ -2422,6 +2427,10 @@ h3. 3.4.3
 
 h3. 3.4.2
 
+* Support for selecting elements and slices of a collection ("CASSANDRA-7396":https://issues.apache.org/jira/browse/CASSANDRA-7396).
+
+h3. 3.4.2
+
 * "@INSERT/UPDATE options@":#updateOptions for tables having a default_time_to_live specifying a TTL of 0 will remove the TTL from the inserted or updated values
 * "@ALTER TABLE@":#alterTableStmt @ADD@ and @DROP@ now allow mutiple columns to be added/removed
 * New "@PER PARTITION LIMIT@":#selectLimit option (see "CASSANDRA-7017":https://issues.apache.org/jira/browse/CASSANDRA-7017).

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index f635480..7b63aa4 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -716,7 +716,7 @@ syntax_rules += r'''
                  ;
 <udtSubfieldSelection> ::= <identifier> "." <identifier>
                          ;
-<selector> ::= [colname]=<cident>
+<selector> ::= [colname]=<cident> ( "[" ( <term> ( ".." <term> "]" )? | <term> ".." ) )?
              | <udtSubfieldSelection>
              | "WRITETIME" "(" [colname]=<cident> ")"
              | "TTL" "(" [colname]=<cident> ")"

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/antlr/Lexer.g
----------------------------------------------------------------------
diff --git a/src/antlr/Lexer.g b/src/antlr/Lexer.g
index 1685964..8b0aca4 100644
--- a/src/antlr/Lexer.g
+++ b/src/antlr/Lexer.g
@@ -300,13 +300,18 @@ QMARK
     : '?'
     ;
 
+RANGE
+    : '..'
+    ;
+
 /*
  * Normally a lexer only emits one token at a time, but ours is tricked out
  * to support multiple (see @lexer::members near the top of the grammar).
  */
 FLOAT
-    : INTEGER EXPONENT
-    | INTEGER '.' DIGIT* EXPONENT?
+    : (INTEGER '.' RANGE) => INTEGER '.'
+    | (INTEGER RANGE) => INTEGER {$type = INTEGER;}
+    | INTEGER ('.' DIGIT*)? EXPONENT?
     ;
 
 /*

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/antlr/Parser.g
----------------------------------------------------------------------
diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index a96bb2b..185f968 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -334,11 +334,30 @@ selectionGroup returns [Selectable.Raw s]
     ;
 
 selectionGroupWithField returns [Selectable.Raw s]
-    @init { Selectable.Raw tmp = null; }
-    @after { $s = tmp; }
-    : g=selectionGroupWithoutField {tmp=g;} ( '.' fi=fident { tmp = new Selectable.WithFieldSelection.Raw(tmp, fi); } )+
+    : g=selectionGroupWithoutField m=selectorModifier[g] {$s = m;}
+    ;
+
+selectorModifier[Selectable.Raw receiver] returns [Selectable.Raw s]
+    : f=fieldSelectorModifier[receiver] m=selectorModifier[f] { $s = m; }
+    | '[' ss=collectionSubSelection[receiver] ']' m=selectorModifier[ss] { $s = m; }
+    | { $s = receiver; }
     ;
 
+fieldSelectorModifier[Selectable.Raw receiver] returns [Selectable.Raw s]
+    : '.' fi=fident { $s = new Selectable.WithFieldSelection.Raw(receiver, fi); }
+    ;
+
+collectionSubSelection [Selectable.Raw receiver] returns [Selectable.Raw s]
+    @init { boolean isSlice=false; }
+    : ( t1=term ( { isSlice=true; } RANGE (t2=term)? )?
+      | RANGE { isSlice=true; } t2=term
+      ) {
+          $s = isSlice
+             ? new Selectable.WithSliceSelection.Raw(receiver, t1, t2)
+             : new Selectable.WithElementSelection.Raw(receiver, t1);
+      }
+     ;
+
 selectionGroupWithoutField returns [Selectable.Raw s]
     @init { Selectable.Raw tmp = null; }
     @after { $s = tmp; }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/ResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index 82590fb..5ed4eee 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -226,15 +226,16 @@ public class ResultSet
         }
 
         /**
-         * Adds the specified column which will not be serialized.
+         * Adds the specified columns which will not be serialized.
          *
-         * @param name the column
+         * @param columns the columns
          */
-        public void addNonSerializedColumn(ColumnSpecification name)
+        public ResultMetadata addNonSerializedColumns(Collection<? extends ColumnSpecification> columns)
         {
             // See comment above. Because columnCount doesn't account the newly added name, it
             // won't be serialized.
-            names.add(name);
+            names.addAll(columns);
+            return this;
         }
 
         public void setHasMorePages(PagingState pagingState)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/Term.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Term.java b/src/java/org/apache/cassandra/cql3/Term.java
index 11b9860..f1737d1 100644
--- a/src/java/org/apache/cassandra/cql3/Term.java
+++ b/src/java/org/apache/cassandra/cql3/Term.java
@@ -70,6 +70,14 @@ public interface Term
      */
     public abstract boolean containsBindMarker();
 
+    /**
+     * Whether that term is terminal (this is a shortcut for {@code this instanceof Term.Terminal}).
+     */
+    default public boolean isTerminal()
+    {
+        return false; // overriden below by Terminal
+    }
+
     public void addFunctionsTo(List<Function> functions);
 
     /**
@@ -153,6 +161,12 @@ public interface Term
             return false;
         }
 
+        @Override
+        public boolean isTerminal()
+        {
+            return true;
+        }
+
         /**
          * @return the serialized value of this terminal.
          * @param protocolVersion

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
index 115729a..d420857 100644
--- a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
@@ -21,12 +21,15 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.List;
 
+import com.google.common.collect.Iterables;
+
 import org.apache.commons.lang3.text.StrBuilder;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.statements.RequestValidations;
+import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
@@ -102,6 +105,19 @@ abstract class AbstractFunctionSelector<T extends Function> extends Selector
             {
                 return fun.isAggregate() || factories.doesAggregation();
             }
+
+            @Override
+            public boolean areAllFetchedColumnsKnown()
+            {
+                return Iterables.all(factories, f -> f.areAllFetchedColumnsKnown());
+            }
+
+            @Override
+            public void addFetchedColumns(ColumnFilter.Builder builder)
+            {
+                for (Selector.Factory factory : factories)
+                    factory.addFetchedColumns(builder);
+            }
         };
     }
 
@@ -112,6 +128,13 @@ abstract class AbstractFunctionSelector<T extends Function> extends Selector
         this.args = Arrays.asList(new ByteBuffer[argSelectors.size()]);
     }
 
+    @Override
+    public void addFetchedColumns(ColumnFilter.Builder builder)
+    {
+        for (Selector selector : argSelectors)
+            selector.addFetchedColumns(builder);
+    }
+
     // Sets a given arg value. We should use that instead of directly setting the args list for the
     // sake of validation.
     protected void setArg(int i, ByteBuffer value) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java
index e3e5328..a9df220 100644
--- a/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java
@@ -22,7 +22,6 @@ import java.util.List;
 
 import org.apache.cassandra.cql3.functions.AggregateFunction;
 import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.transport.ProtocolVersion;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/AliasedSelectable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/AliasedSelectable.java b/src/java/org/apache/cassandra/cql3/selection/AliasedSelectable.java
new file mode 100644
index 0000000..3cf0029
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/AliasedSelectable.java
@@ -0,0 +1,94 @@
+/*
+ * 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.selection;
+
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.cql3.selection.Selector.Factory;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
+/**
+ * A {@code Selectable} with an alias.
+ */
+final class AliasedSelectable implements Selectable
+{
+    /**
+     * The selectable
+     */
+    private final Selectable selectable;
+
+    /**
+     * The alias associated to the selectable.
+     */
+    private final ColumnIdentifier alias;
+
+    public AliasedSelectable(Selectable selectable, ColumnIdentifier alias)
+    {
+        this.selectable = selectable;
+        this.alias = alias;
+    }
+
+    @Override
+    public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+    {
+        return selectable.testAssignment(keyspace, receiver);
+    }
+
+    @Override
+    public Factory newSelectorFactory(TableMetadata table,
+                                      AbstractType<?> expectedType,
+                                      List<ColumnMetadata> defs,
+                                      VariableSpecifications boundNames)
+    {
+        final Factory delegate = selectable.newSelectorFactory(table, expectedType, defs, boundNames);
+        final ColumnSpecification columnSpec = delegate.getColumnSpecification(table).withAlias(alias);
+
+        return new ForwardingFactory()
+        {
+            @Override
+            protected Factory delegate()
+            {
+                return delegate;
+            }
+
+            @Override
+            public ColumnSpecification getColumnSpecification(TableMetadata table)
+            {
+                return columnSpec;
+            }
+        };
+    }
+
+    @Override
+    public AbstractType<?> getExactTypeIfKnown(String keyspace)
+    {
+        return selectable.getExactTypeIfKnown(keyspace);
+    }
+
+    @Override
+    public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+    {
+        return selectable.selectColumns(predicate);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java b/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
index 88885dd..816980d 100644
--- a/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
+++ b/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.selection.Selector.Factory;
+import org.apache.cassandra.db.filter.ColumnFilter.Builder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.schema.ColumnMetadata;
 
@@ -75,6 +76,18 @@ abstract class CollectionFactory extends Factory
         return factories.containsTTLSelectorFactory();
     }
 
+    @Override
+    boolean areAllFetchedColumnsKnown()
+    {
+        return factories.areAllFetchedColumnsKnown();
+    }
+
+    @Override
+    void addFetchedColumns(Builder builder)
+    {
+        factories.addFetchedColumns(builder);
+    }
+
     protected final void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
     {
         SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/ColumnFilterFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ColumnFilterFactory.java b/src/java/org/apache/cassandra/cql3/selection/ColumnFilterFactory.java
new file mode 100644
index 0000000..2e5d0df
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/ColumnFilterFactory.java
@@ -0,0 +1,134 @@
+/*
+ * 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.selection;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
+/**
+ * Factory for {@code ColumnFilter} instances.
+ * <p>This class is used to abstract the fact that depending on the selection clause the {@code ColumnFilter} instances
+ * can be computed at prepartion time (if all the requested columns are known) or must be computed at execution time.</p>
+ */
+abstract class ColumnFilterFactory
+{
+    /**
+     * Returns the {@code ColumnFilter} instance corresponding to the specified selectors.
+     * @param selectors the selectors for which the {@code ColumnFilter} must be created.
+     * @return the {@code ColumnFilter} instance corresponding to the specified selectors
+     */
+    abstract ColumnFilter newInstance(List<Selector> selectors);
+
+    public static ColumnFilterFactory wildcard(TableMetadata table)
+    {
+        return new PrecomputedColumnFilter(ColumnFilter.all(table));
+    }
+
+    public static ColumnFilterFactory fromColumns(TableMetadata table,
+                                                  List<ColumnMetadata> selectedColumns,
+                                                  Set<ColumnMetadata> orderingColumns,
+                                                  Set<ColumnMetadata> nonPKRestrictedColumns)
+    {
+        ColumnFilter.Builder builder = ColumnFilter.allRegularColumnsBuilder(table);
+        builder.addAll(selectedColumns);
+        builder.addAll(orderingColumns);
+        // we'll also need to fetch any column on which we have a restriction (so we can apply said restriction)
+        builder.addAll(nonPKRestrictedColumns);
+        return new PrecomputedColumnFilter(builder.build());
+    }
+
+    /**
+     * Creates a new {@code ColumnFilterFactory} instance from the specified {@code SelectorFactories}.
+     *
+     * @param table the table metadata
+     * @param factories the {@code SelectorFactories}
+     * @param orderingColumns the columns used for ordering
+     * @param nonPKRestrictedColumns the non primary key columns that have been resticted in the WHERE clause
+     * @return a new {@code ColumnFilterFactory} instance
+     */
+    public static ColumnFilterFactory fromSelectorFactories(TableMetadata table,
+                                                            SelectorFactories factories,
+                                                            Set<ColumnMetadata> orderingColumns,
+                                                            Set<ColumnMetadata> nonPKRestrictedColumns)
+    {
+        if (factories.areAllFetchedColumnsKnown())
+        {
+            ColumnFilter.Builder builder = ColumnFilter.allRegularColumnsBuilder(table);
+            factories.addFetchedColumns(builder);
+            builder.addAll(orderingColumns);
+            // we'll also need to fetch any column on which we have a restriction (so we can apply said restriction)
+            builder.addAll(nonPKRestrictedColumns);
+            return new PrecomputedColumnFilter(builder.build());
+        }
+
+        return new OnRequestColumnFilterFactory(table, nonPKRestrictedColumns);
+    }
+
+    /**
+     * A factory that always return the same pre-computed {@code ColumnFilter}.
+     */
+    private static class PrecomputedColumnFilter extends ColumnFilterFactory
+    {
+        /**
+         * The precomputed {@code ColumnFilter}
+         */
+        private final ColumnFilter columnFilter;
+
+        public PrecomputedColumnFilter(ColumnFilter columnFilter)
+        {
+            this.columnFilter = columnFilter;
+        }
+
+        @Override
+        public ColumnFilter newInstance(List<Selector> selectors)
+        {
+            return columnFilter;
+        }
+    }
+
+    /**
+     * A factory that will computed the {@code ColumnFilter} on request.
+     */
+    private static class OnRequestColumnFilterFactory extends ColumnFilterFactory
+    {
+        private final TableMetadata table;
+        private final Set<ColumnMetadata> nonPKRestrictedColumns;
+
+        public OnRequestColumnFilterFactory(TableMetadata table, Set<ColumnMetadata> nonPKRestrictedColumns)
+        {
+            this.table = table;
+            this.nonPKRestrictedColumns = nonPKRestrictedColumns;
+        }
+
+        @Override
+        public ColumnFilter newInstance(List<Selector> selectors)
+        {
+            ColumnFilter.Builder builder = ColumnFilter.allRegularColumnsBuilder(table);
+            for (int i = 0, m = selectors.size(); i < m; i++)
+                selectors.get(i).addFetchedColumns(builder);
+
+            // we'll also need to fetch any column on which we have a restriction (so we can apply said restriction)
+            builder.addAll(nonPKRestrictedColumns);
+            return builder.build();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/ElementsSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ElementsSelector.java b/src/java/org/apache/cassandra/cql3/selection/ElementsSelector.java
new file mode 100644
index 0000000..f94b5c8
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/ElementsSelector.java
@@ -0,0 +1,325 @@
+/*
+ * 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.selection;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.selection.SimpleSelector.SimpleSelectorFactory;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * Selector class handling element (c[x]) and slice (c[x..y]) selections over collections.
+ */
+abstract class ElementsSelector extends Selector
+{
+    protected final Selector selected;
+
+    protected ElementsSelector(Selector selected)
+    {
+        this.selected = selected;
+    }
+
+    private static boolean isUnset(ByteBuffer bb)
+    {
+        return bb == ByteBufferUtil.UNSET_BYTE_BUFFER;
+    }
+
+    // For sets and maps, return the type corresponding to the element of a selection (that is, x in c[x]).
+    private static AbstractType<?> keyType(CollectionType<?> type)
+    {
+        return type.nameComparator();
+    }
+
+    // For sets and maps, return the type corresponding to the result of a selection (that is, c[x] in c[x]).
+    public static AbstractType<?> valueType(CollectionType<?> type)
+    {
+        return type instanceof MapType ? type.valueComparator() : type.nameComparator();
+    }
+
+    private static abstract class AbstractFactory extends Factory
+    {
+        protected final String name;
+        protected final Selector.Factory factory;
+        protected final CollectionType<?> type;
+
+        protected AbstractFactory(String name, Selector.Factory factory, CollectionType<?> type)
+        {
+            this.name = name;
+            this.factory = factory;
+            this.type = type;
+        }
+
+        protected String getColumnName()
+        {
+            return name;
+        }
+
+        protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+        {
+            factory.addColumnMapping(mapping, resultsColumn);
+        }
+
+        public boolean isAggregateSelectorFactory()
+        {
+            return factory.isAggregateSelectorFactory();
+        }
+    }
+
+    /**
+     * Creates a {@code Selector.Factory} for the selection of an element of a collection.
+     *
+     * @param name a string representing the selection the factory is for. Something like "c[x]".
+     * @param factory the {@code Selector.Factory} corresponding to the collection on which an element
+     * is selected.
+     * @param type the type of the collection.
+     * @param key the element within the value represented by {@code factory} that is selected.
+     * @return the created factory.
+     */
+    public static Factory newElementFactory(String name, Selector.Factory factory, CollectionType<?> type, final Term key)
+    {
+        return new AbstractFactory(name, factory, type)
+        {
+            protected AbstractType<?> getReturnType()
+            {
+                return valueType(type);
+            }
+
+            public Selector newInstance(QueryOptions options) throws InvalidRequestException
+            {
+                ByteBuffer keyValue = key.bindAndGet(options);
+                if (keyValue == null)
+                    throw new InvalidRequestException("Invalid null value for element selection on " + factory.getColumnName());
+                if (keyValue == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                    throw new InvalidRequestException("Invalid unset value for element selection on " + factory.getColumnName());
+                return new ElementSelector(factory.newInstance(options), keyValue);
+            }
+
+            public boolean areAllFetchedColumnsKnown()
+            {
+                // If we known all the fetched columns, it means that we don't have to wait execution to create
+                // the ColumnFilter (through addFetchedColumns below).
+                // That's the case if either there is no particular subselection
+                // to add, or if there is one but the selected key is terminal. In other words,
+                // we known all the fetched columns if all the feched columns of the factory are known and either:
+                //  1) the type is frozen (in which case there isn't subselection to do).
+                //  2) the factory (the left-hand-side) isn't a simple column selection (here again, no
+                //     subselection we can do).
+                //  3) the element selected is terminal.
+                return factory.areAllFetchedColumnsKnown()
+                        && (!type.isMultiCell() || !factory.isSimpleSelectorFactory() || key.isTerminal());
+            }
+
+            public void addFetchedColumns(ColumnFilter.Builder builder)
+            {
+                if (!type.isMultiCell() || !factory.isSimpleSelectorFactory())
+                {
+                    factory.addFetchedColumns(builder);
+                    return;
+                }
+
+                ColumnMetadata column = ((SimpleSelectorFactory) factory).getColumn();
+                builder.select(column, CellPath.create(((Term.Terminal)key).get(ProtocolVersion.V3)));
+            }
+        };
+    }
+
+    /**
+     * Creates a {@code Selector.Factory} for the selection of a slice of a collection.
+     *
+     * @param name a string representing the selection the factory is for. Something like "c[x..y]".
+     * @param factory the {@code Selector.Factory} corresponding to the collection on which a slice
+     * is selected.
+     * @param type the type of the collection.
+     * @param from the starting bound of the selected slice. This cannot be {@code null} but can be
+     * {@code Constants.UNSET_VALUE} if the slice doesn't have a start.
+     * @param to the ending bound of the selected slice. This cannot be {@code null} but can be
+     * {@code Constants.UNSET_VALUE} if the slice doesn't have an end.
+     * @return the created factory.
+     */
+    public static Factory newSliceFactory(String name, Selector.Factory factory, CollectionType<?> type, final Term from, final Term to)
+    {
+        return new AbstractFactory(name, factory, type)
+        {
+            protected AbstractType<?> getReturnType()
+            {
+                return type;
+            }
+
+            public Selector newInstance(QueryOptions options) throws InvalidRequestException
+            {
+                ByteBuffer fromValue = from.bindAndGet(options);
+                ByteBuffer toValue = to.bindAndGet(options);
+                // Note that we use UNSET values to represent no bound, so null is truly invalid
+                if (fromValue == null || toValue == null)
+                    throw new InvalidRequestException("Invalid null value for slice selection on " + factory.getColumnName());
+                return new SliceSelector(factory.newInstance(options), from.bindAndGet(options), to.bindAndGet(options));
+            }
+
+            public boolean areAllFetchedColumnsKnown()
+            {
+                // If we known all the fetched columns, it means that we don't have to wait execution to create
+                // the ColumnFilter (through addFetchedColumns below).
+                // That's the case if either there is no particular subselection
+                // to add, or if there is one but the selected bound are terminal. In other words,
+                // we known all the fetched columns if all the feched columns of the factory are known and either:
+                //  1) the type is frozen (in which case there isn't subselection to do).
+                //  2) the factory (the left-hand-side) isn't a simple column selection (here again, no
+                //     subselection we can do).
+                //  3) the bound of the selected slice are terminal.
+                return factory.areAllFetchedColumnsKnown()
+                        && (!type.isMultiCell() || !factory.isSimpleSelectorFactory() || (from.isTerminal() && to.isTerminal()));
+            }
+
+            public void addFetchedColumns(ColumnFilter.Builder builder)
+            {
+                if (!type.isMultiCell() || !factory.isSimpleSelectorFactory())
+                {
+                    factory.addFetchedColumns(builder);
+                    return;
+                }
+
+                ColumnMetadata column = ((SimpleSelectorFactory) factory).getColumn();
+                ByteBuffer fromBB = ((Term.Terminal)from).get(ProtocolVersion.V3);
+                ByteBuffer toBB = ((Term.Terminal)to).get(ProtocolVersion.V3);
+                builder.slice(column, isUnset(fromBB) ? CellPath.BOTTOM : CellPath.create(fromBB), isUnset(toBB) ? CellPath.TOP  : CellPath.create(toBB));
+            }
+        };
+    }
+
+    public ByteBuffer getOutput(ProtocolVersion protocolVersion) throws InvalidRequestException
+    {
+        ByteBuffer value = selected.getOutput(protocolVersion);
+        return value == null ? null : extractSelection(value);
+    }
+
+    protected abstract ByteBuffer extractSelection(ByteBuffer collection);
+
+    public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        selected.addInput(protocolVersion, rs);
+    }
+
+    public void reset()
+    {
+        selected.reset();
+    }
+
+    private static class ElementSelector extends ElementsSelector
+    {
+        private final CollectionType<?> type;
+        private final ByteBuffer key;
+
+        private ElementSelector(Selector selected, ByteBuffer key)
+        {
+            super(selected);
+            assert selected.getType() instanceof MapType || selected.getType() instanceof SetType : "this shouldn't have passed validation in Selectable";
+            this.type = (CollectionType<?>) selected.getType();
+            this.key = key;
+        }
+
+        public void addFetchedColumns(ColumnFilter.Builder builder)
+        {
+            if (type.isMultiCell() && selected instanceof SimpleSelector)
+            {
+                ColumnMetadata column = ((SimpleSelector)selected).column;
+                builder.select(column, CellPath.create(key));
+            }
+            else
+            {
+                selected.addFetchedColumns(builder);
+            }
+        }
+
+        protected ByteBuffer extractSelection(ByteBuffer collection)
+        {
+            return type.getSerializer().getSerializedValue(collection, key, keyType(type));
+        }
+
+        public AbstractType<?> getType()
+        {
+            return valueType(type);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("%s[%s]", selected, keyType(type).getString(key));
+        }
+    }
+
+    private static class SliceSelector extends ElementsSelector
+    {
+        private final CollectionType<?> type;
+
+        // Note that neither from nor to can be null, but they can both be ByteBufferUtil.UNSET_BYTE_BUFFER to represent no particular bound
+        private final ByteBuffer from;
+        private final ByteBuffer to;
+
+        private SliceSelector(Selector selected, ByteBuffer from, ByteBuffer to)
+        {
+            super(selected);
+            assert selected.getType() instanceof MapType || selected.getType() instanceof SetType : "this shouldn't have passed validation in Selectable";
+            assert from != null && to != null : "We can have unset buffers, but not nulls";
+            this.type = (CollectionType<?>) selected.getType();
+            this.from = from;
+            this.to = to;
+        }
+
+        public void addFetchedColumns(ColumnFilter.Builder builder)
+        {
+            if (type.isMultiCell() && selected instanceof SimpleSelector)
+            {
+                ColumnMetadata column = ((SimpleSelector)selected).column;
+                builder.slice(column, isUnset(from) ? CellPath.BOTTOM : CellPath.create(from), isUnset(to) ? CellPath.TOP  : CellPath.create(to));
+            }
+            else
+            {
+                selected.addFetchedColumns(builder);
+            }
+        }
+
+        protected ByteBuffer extractSelection(ByteBuffer collection)
+        {
+            return type.getSerializer().getSliceFromSerialized(collection, from, to, type.nameComparator());
+        }
+
+        public AbstractType<?> getType()
+        {
+            return type;
+        }
+
+        @Override
+        public String toString()
+        {
+            boolean fromUnset = isUnset(from);
+            boolean toUnset = isUnset(to);
+            return fromUnset && toUnset
+                 ? selected.toString()
+                 : String.format("%s[%s..%s]", selected, fromUnset ? "" : keyType(type).getString(from), toUnset ? "" : keyType(type).getString(to));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
index d4b74ae..c67fc03 100644
--- a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
@@ -21,7 +21,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -61,9 +61,24 @@ final class FieldSelector extends Selector
             {
                 return factory.isAggregateSelectorFactory();
             }
+
+            public boolean areAllFetchedColumnsKnown()
+            {
+                return factory.areAllFetchedColumnsKnown();
+            }
+
+            public void addFetchedColumns(ColumnFilter.Builder builder)
+            {
+                factory.addFetchedColumns(builder);
+            }
         };
     }
 
+    public void addFetchedColumns(ColumnFilter.Builder builder)
+    {
+        selected.addFetchedColumns(builder);
+    }
+
     public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
     {
         selected.addInput(protocolVersion, rs);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java b/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java
index 50b3ded..cf41e31 100644
--- a/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java
+++ b/src/java/org/apache/cassandra/cql3/selection/ForwardingFactory.java
@@ -23,6 +23,7 @@ import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.selection.Selector.Factory;
+import org.apache.cassandra.db.filter.ColumnFilter.Builder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
@@ -83,8 +84,26 @@ abstract class ForwardingFactory extends Factory
     }
 
     @Override
-    public boolean isSimpleSelectorFactory(int index)
+    public boolean isSimpleSelectorFactory()
     {
-        return delegate().isSimpleSelectorFactory(index);
+        return delegate().isSimpleSelectorFactory();
+    }
+
+    @Override
+    public boolean isSimpleSelectorFactoryFor(int index)
+    {
+        return delegate().isSimpleSelectorFactoryFor(index);
+    }
+
+    @Override
+    boolean areAllFetchedColumnsKnown()
+    {
+        return delegate().areAllFetchedColumnsKnown();
+    }
+
+    @Override
+    void addFetchedColumns(Builder builder)
+    {
+        delegate().addFetchedColumns(builder);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ListSelector.java b/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
index b540ec9..a8c5d5c 100644
--- a/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
@@ -23,7 +23,7 @@ import java.util.List;
 
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.Lists;
-import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.filter.ColumnFilter.Builder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.CollectionSerializer;
@@ -61,6 +61,13 @@ final class ListSelector extends Selector
         };
     }
 
+    @Override
+    public void addFetchedColumns(Builder builder)
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+            elements.get(i).addFetchedColumns(builder);
+    }
+
     public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
     {
         for (int i = 0, m = elements.size(); i < m; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/MapSelector.java b/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
index 97dfc30..09424bd 100644
--- a/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
@@ -28,7 +28,7 @@ import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.Maps;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.filter.ColumnFilter.Builder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -135,9 +135,41 @@ final class MapSelector extends Selector
                 }
                 return false;
             }
+
+            @Override
+            boolean areAllFetchedColumnsKnown()
+            {
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    if (!entry.left.areAllFetchedColumnsKnown() || !entry.right.areAllFetchedColumnsKnown())
+                        return false;
+                }
+                return true;
+            }
+
+            @Override
+            void addFetchedColumns(Builder builder)
+            {
+                for (Pair<Factory, Factory> entry : factories)
+                {
+                    entry.left.addFetchedColumns(builder);
+                    entry.right.addFetchedColumns(builder);
+                }
+            }
         };
     }
 
+    @Override
+    public void addFetchedColumns(Builder builder)
+    {
+        for (int i = 0, m = elements.size(); i < m; i++)
+        {
+            Pair<Selector, Selector> pair = elements.get(i);
+            pair.left.addFetchedColumns(builder);
+            pair.right.addFetchedColumns(builder);
+        }
+    }
+
     public void addInput(ProtocolVersion protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
     {
         for (int i = 0, m = elements.size(); i < m; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/RawSelector.java b/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
index fbf7c30..910b1d1 100644
--- a/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
@@ -45,11 +45,12 @@ public class RawSelector
      */
     public static List<Selectable> toSelectables(List<RawSelector> raws, final TableMetadata table)
     {
-        return Lists.transform(raws, raw -> raw.selectable.prepare(table));
+        return Lists.transform(raws, raw -> raw.prepare(table));
     }
 
-    public boolean processesSelection()
+    private Selectable prepare(TableMetadata table)
     {
-        return selectable.processesSelection();
+        Selectable s = selectable.prepare(table);
+        return alias != null ? new AliasedSelectable(s, alias) : s;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java b/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java
new file mode 100644
index 0000000..d179f93
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.ResultSet.ResultMetadata;
+import org.apache.cassandra.cql3.selection.Selection.Selectors;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.aggregation.GroupMaker;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public final class ResultSetBuilder
+{
+    private final ResultSet resultSet;
+
+    /**
+     * As multiple thread can access a <code>Selection</code> instance each <code>ResultSetBuilder</code> will use
+     * its own <code>Selectors</code> instance.
+     */
+    private final Selectors selectors;
+
+    /**
+     * The <code>GroupMaker</code> used to build the aggregates.
+     */
+    private final GroupMaker groupMaker;
+
+    /*
+     * We'll build CQL3 row one by one.
+     * The currentRow is the values for the (CQL3) columns we've fetched.
+     * We also collect timestamps and ttls for the case where the writetime and
+     * ttl functions are used. Note that we might collect timestamp and/or ttls
+     * we don't care about, but since the array below are allocated just once,
+     * it doesn't matter performance wise.
+     */
+    List<ByteBuffer> current;
+    final long[] timestamps;
+    final int[] ttls;
+
+    public ResultSetBuilder(ResultMetadata metadata, Selectors selectors)
+    {
+        this(metadata, selectors, null);
+    }
+
+    public ResultSetBuilder(ResultMetadata metadata, Selectors selectors, GroupMaker groupMaker)
+    {
+        this.resultSet = new ResultSet(metadata.copy(), new ArrayList<List<ByteBuffer>>());
+        this.selectors = selectors;
+        this.groupMaker = groupMaker;
+        this.timestamps = selectors.collectTimestamps() ? new long[selectors.numberOfFetchedColumns()] : null;
+        this.ttls = selectors.collectTTLs() ? new int[selectors.numberOfFetchedColumns()] : null;
+
+        // We use MIN_VALUE to indicate no timestamp and -1 for no ttl
+        if (timestamps != null)
+            Arrays.fill(timestamps, Long.MIN_VALUE);
+        if (ttls != null)
+            Arrays.fill(ttls, -1);
+    }
+
+    public void add(ByteBuffer v)
+    {
+        current.add(v);
+    }
+
+    public void add(Cell c, int nowInSec)
+    {
+        if (c == null)
+        {
+            current.add(null);
+            return;
+        }
+
+        current.add(value(c));
+
+        if (timestamps != null)
+            timestamps[current.size() - 1] = c.timestamp();
+
+        if (ttls != null)
+            ttls[current.size() - 1] = remainingTTL(c, nowInSec);
+    }
+
+    private int remainingTTL(Cell c, int nowInSec)
+    {
+        if (!c.isExpiring())
+            return -1;
+
+        int remaining = c.localDeletionTime() - nowInSec;
+        return remaining >= 0 ? remaining : -1;
+    }
+
+    private ByteBuffer value(Cell c)
+    {
+        return c.isCounterCell()
+             ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
+             : c.value();
+    }
+
+    /**
+     * Notifies this <code>Builder</code> that a new row is being processed.
+     *
+     * @param partitionKey the partition key of the new row
+     * @param clustering the clustering of the new row
+     */
+    public void newRow(DecoratedKey partitionKey, Clustering clustering)
+    {
+        // The groupMaker needs to be called for each row
+        boolean isNewAggregate = groupMaker == null || groupMaker.isNewGroup(partitionKey, clustering);
+        if (current != null)
+        {
+            selectors.addInputRow(this);
+            if (isNewAggregate)
+            {
+                resultSet.addRow(getOutputRow());
+                selectors.reset();
+            }
+        }
+        current = new ArrayList<>(selectors.numberOfFetchedColumns());
+    }
+
+    /**
+     * Builds the <code>ResultSet</code>
+     */
+    public ResultSet build()
+    {
+        if (current != null)
+        {
+            selectors.addInputRow(this);
+            resultSet.addRow(getOutputRow());
+            selectors.reset();
+            current = null;
+        }
+
+        // For aggregates we need to return a row even it no records have been found
+        if (resultSet.isEmpty() && groupMaker != null && groupMaker.returnAtLeastOneRow())
+            resultSet.addRow(getOutputRow());
+        return resultSet;
+    }
+
+    private List<ByteBuffer> getOutputRow()
+    {
+        return selectors.getOutputRow();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
index 15e0210..de74678 100644
--- a/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
@@ -22,7 +22,6 @@ import java.util.List;
 
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.functions.ScalarFunction;
-import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.transport.ProtocolVersion;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4ebab661/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
index b2526a5..998baca 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.cql3.selection;
 
 import java.util.*;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.text.StrBuilder;
@@ -51,6 +52,39 @@ public interface Selectable extends AssignmentTestable
      */
     public AbstractType<?> getExactTypeIfKnown(String keyspace);
 
+    /**
+     * Checks if this {@code Selectable} select columns matching the specified predicate.
+     * @return {@code true} if this {@code Selectable} select columns matching the specified predicate,
+     * {@code false} otherwise.
+     */
+    public boolean selectColumns(Predicate<ColumnMetadata> predicate);
+
+    /**
+     * Checks if the specified Selectables select columns matching the specified predicate.
+     * @param selectables the selectables to check.
+     * @return {@code true} if the specified Selectables select columns matching the specified predicate,
+      {@code false} otherwise.
+     */
+    public static boolean selectColumns(List<Selectable> selectables, Predicate<ColumnMetadata> predicate)
+    {
+        for (Selectable selectable : selectables)
+        {
+            if (selectable.selectColumns(predicate))
+                return true;
+        }
+        return false;
+    }
+
+    /**
+     * Checks if any processing is performed on the selected columns, {@code false} otherwise.
+     * @return {@code true} if any processing is performed on the selected columns, {@code false} otherwise.
+     */
+    public default boolean processesSelection()
+    {
+        // ColumnMetadata is the only case that returns false and override this
+        return true;
+    }
+
     // Term.Raw overrides this since some literals can be WEAKLY_ASSIGNABLE
     default public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
     {
@@ -69,20 +103,22 @@ public interface Selectable extends AssignmentTestable
         return idx;
     }
 
-    public static abstract class Raw
+    default ColumnSpecification specForElementOrSlice(Selectable selected, ColumnSpecification receiver, String selectionType)
     {
-        public abstract Selectable prepare(TableMetadata table);
-
-        /**
-         * Returns true if any processing is performed on the selected column.
-         **/
-        public boolean processesSelection()
+        switch (((CollectionType)receiver.type).kind)
         {
-            // ColumnIdentifier is the only case that returns false and override this
-            return true;
+            case LIST: throw new InvalidRequestException(String.format("%s selection is only allowed on sets and maps, but %s is a list", selectionType, selected));
+            case SET: return Sets.valueSpecOf(receiver);
+            case MAP: return Maps.keySpecOf(receiver);
+            default: throw new AssertionError();
         }
     }
 
+    public static abstract class Raw
+    {
+        public abstract Selectable prepare(TableMetadata table);
+    }
+
     public static class WithTerm implements Selectable
     {
         /**
@@ -157,6 +193,12 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return false;
+        }
+
+        @Override
         public String toString()
         {
             return rawTerm.getText();
@@ -217,6 +259,12 @@ public interface Selectable extends AssignmentTestable
             return isWritetime ? LongType.instance : Int32Type.instance;
         }
 
+        @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return predicate.test(column);
+        }
+
         public static class Raw extends Selectable.Raw
         {
             private final ColumnMetadata.Raw id;
@@ -258,6 +306,12 @@ public interface Selectable extends AssignmentTestable
             return AbstractFunctionSelector.newFactory(function, factories);
         }
 
+        @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return Selectable.selectColumns(args, predicate);
+        }
+
         public AbstractType<?> getExactTypeIfKnown(String keyspace)
         {
             return function.returnType();
@@ -362,6 +416,12 @@ public interface Selectable extends AssignmentTestable
         {
             return UTF8Type.instance;
         }
+
+        @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return Selectable.selectColumns(args, predicate);
+        }
     }
 
     public static class WithCast implements Selectable
@@ -409,6 +469,12 @@ public interface Selectable extends AssignmentTestable
             return type.getType();
         }
 
+        @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return arg.selectColumns(predicate);
+        }
+
         public static class Raw extends Selectable.Raw
         {
             private final CQL3Type type;
@@ -427,6 +493,9 @@ public interface Selectable extends AssignmentTestable
         }
     }
 
+    /**
+     * Represents the selection of the field of a UDT (eg. t.f).
+     */
     public static class WithFieldSelection implements Selectable
     {
         public final Selectable selected;
@@ -456,7 +525,7 @@ public interface Selectable extends AssignmentTestable
             }
 
             Selector.Factory factory = selected.newSelectorFactory(table, expectedUdtType, defs, boundNames);
-            AbstractType<?> type = factory.getColumnSpecification(table).type;
+            AbstractType<?> type = factory.getReturnType();
             if (!type.isUDT())
             {
                 throw new InvalidRequestException(
@@ -490,6 +559,12 @@ public interface Selectable extends AssignmentTestable
             return ut.fieldType(fieldIndex);
         }
 
+        @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return selected.selectColumns(predicate);
+        }
+
         public static class Raw extends Selectable.Raw
         {
             private final Selectable.Raw selected;
@@ -603,6 +678,12 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return Selectable.selectColumns(selectables, predicate);
+        }
+
+        @Override
         public String toString()
         {
             return Tuples.tupleToString(selectables);
@@ -681,6 +762,12 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return Selectable.selectColumns(selectables, predicate);
+        }
+
+        @Override
         public String toString()
         {
             return Lists.listToString(selectables);
@@ -767,6 +854,12 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return Selectable.selectColumns(selectables, predicate);
+        }
+
+        @Override
         public String toString()
         {
             return Sets.setToString(selectables);
@@ -896,6 +989,20 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            for (Pair<Selectable.Raw, Selectable.Raw> raw : raws)
+            {
+                if (!(raw.left instanceof RawIdentifier) && raw.left.prepare(cfm).selectColumns(predicate))
+                    return true;
+
+                if (!raw.right.prepare(cfm).selectColumns(predicate))
+                    return true;
+            }
+            return false;
+        }
+
+        @Override
         public String toString()
         {
             return raws.stream()
@@ -1031,6 +1138,12 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return selectable.selectColumns(predicate);
+        }
+
+        @Override
         public String toString()
         {
             return String.format("(%s)%s", typeName, selectable);
@@ -1112,4 +1225,160 @@ public interface Selectable extends AssignmentTestable
             return text;
         }
     }
+
+    /**
+     * Represents the selection of an element of a collection (eg. c[x]).
+     */
+    public static class WithElementSelection implements Selectable
+    {
+        public final Selectable selected;
+        // Note that we can't yet prepare the Term.Raw yet as we need the ColumnSpecificiation corresponding to Selectable, which
+        // we'll only know in newSelectorFactory due to functions (which needs the defs passed to newSelectorFactory to resolve which
+        // function is called). Note that this doesn't really matter performance wise since the factories are still created during
+        // preparation of the corresponding SelectStatement.
+        public final Term.Raw element;
+
+        private WithElementSelection(Selectable selected, Term.Raw element)
+        {
+            assert element != null;
+            this.selected = selected;
+            this.element = element;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("%s[%s]", selected, element);
+        }
+
+        public Selector.Factory newSelectorFactory(TableMetadata cfm, AbstractType<?> expectedType, List<ColumnMetadata> defs, VariableSpecifications boundNames)
+        {
+            Selector.Factory factory = selected.newSelectorFactory(cfm, null, defs, boundNames);
+            ColumnSpecification receiver = factory.getColumnSpecification(cfm);
+
+            if (!(receiver.type instanceof CollectionType))
+                throw new InvalidRequestException(String.format("Invalid element selection: %s is of type %s is not a collection", selected, receiver.type.asCQL3Type()));
+
+            ColumnSpecification boundSpec = specForElementOrSlice(selected, receiver, "Element");
+
+            Term elt = element.prepare(cfm.keyspace, boundSpec);
+            elt.collectMarkerSpecification(boundNames);
+            return ElementsSelector.newElementFactory(toString(), factory, (CollectionType)receiver.type, elt);
+        }
+
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            AbstractType<?> selectedType = selected.getExactTypeIfKnown(keyspace);
+            if (selectedType == null || !(selectedType instanceof CollectionType))
+                return null;
+
+            return ElementsSelector.valueType((CollectionType) selectedType);
+        }
+
+        @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return selected.selectColumns(predicate);
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final Selectable.Raw selected;
+            private final Term.Raw element;
+
+            public Raw(Selectable.Raw selected, Term.Raw element)
+            {
+                this.selected = selected;
+                this.element = element;
+            }
+
+            public WithElementSelection prepare(TableMetadata cfm)
+            {
+                return new WithElementSelection(selected.prepare(cfm), element);
+            }
+        }
+    }
+
+    /**
+     * Represents the selection of a slice of a collection (eg. c[x..y]).
+     */
+    public static class WithSliceSelection implements Selectable
+    {
+        public final Selectable selected;
+        // Note that we can't yet prepare the Term.Raw yet as we need the ColumnSpecificiation corresponding to Selectable, which
+        // we'll only know in newSelectorFactory due to functions (which needs the defs passed to newSelectorFactory to resolve which
+        // function is called). Note that this doesn't really matter performance wise since the factories are still created during
+        // preparation of the corresponding SelectStatement.
+        // Both from and to can be null if they haven't been provided
+        public final Term.Raw from;
+        public final Term.Raw to;
+
+        private WithSliceSelection(Selectable selected, Term.Raw from, Term.Raw to)
+        {
+            this.selected = selected;
+            this.from = from;
+            this.to = to;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("%s[%s..%s]", selected, from == null ? "" : from, to == null ? "" : to);
+        }
+
+        public Selector.Factory newSelectorFactory(TableMetadata cfm, AbstractType<?> expectedType, List<ColumnMetadata> defs, VariableSpecifications boundNames)
+        {
+            // Note that a slice gives you the same type as the collection you applied it to, so we can pass expectedType for selected directly
+            Selector.Factory factory = selected.newSelectorFactory(cfm, expectedType, defs, boundNames);
+            ColumnSpecification receiver = factory.getColumnSpecification(cfm);
+
+            if (!(receiver.type instanceof CollectionType))
+                throw new InvalidRequestException(String.format("Invalid slice selection: %s of type %s is not a collection", selected, receiver.type.asCQL3Type()));
+
+            ColumnSpecification boundSpec = specForElementOrSlice(selected, receiver, "Slice");
+
+            // If from or to are null, this means the user didn't provide on in the syntax (we had c[x..] or c[..x]).
+            // The equivalent of doing this when preparing values would be to use UNSET.
+            Term f = from == null ? Constants.UNSET_VALUE : from.prepare(cfm.keyspace, boundSpec);
+            Term t = to == null ? Constants.UNSET_VALUE : to.prepare(cfm.keyspace, boundSpec);
+            f.collectMarkerSpecification(boundNames);
+            t.collectMarkerSpecification(boundNames);
+            return ElementsSelector.newSliceFactory(toString(), factory, (CollectionType)receiver.type, f, t);
+        }
+
+        public AbstractType<?> getExactTypeIfKnown(String keyspace)
+        {
+            AbstractType<?> selectedType = selected.getExactTypeIfKnown(keyspace);
+            if (selectedType == null || !(selectedType instanceof CollectionType))
+                return null;
+
+            return selectedType;
+        }
+
+        @Override
+        public boolean selectColumns(Predicate<ColumnMetadata> predicate)
+        {
+            return selected.selectColumns(predicate);
+        }
+
+        public static class Raw extends Selectable.Raw
+        {
+            private final Selectable.Raw selected;
+            // Both from and to can be null if they haven't been provided
+            private final Term.Raw from;
+            private final Term.Raw to;
+
+            public Raw(Selectable.Raw selected, Term.Raw from, Term.Raw to)
+            {
+                this.selected = selected;
+                this.from = from;
+                this.to = to;
+            }
+
+            public WithSliceSelection prepare(TableMetadata cfm)
+            {
+                return new WithSliceSelection(selected.prepare(cfm), from, to);
+            }
+        }
+    }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


Mime
View raw message