Merge branch 'cassandra-2.1' into cassandra-2.2

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

Branch: refs/heads/cassandra-2.2
Commit: 1411ad5f6ab7afd554e485534126b566806b9a96
Parents: 99f7ce9 7473877
Author: Sam Tunnicliffe <s...@beobal.com>
Authored: Thu Jul 2 11:26:05 2015 +0100
Committer: Sam Tunnicliffe <s...@beobal.com>
Committed: Thu Jul 2 11:29:20 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../selection/AbstractFunctionSelector.java     |  13 +-
 .../cassandra/cql3/selection/Selection.java     |   5 +-
 .../cql3/selection/SelectionColumnMapping.java  |  68 +++--
 .../selection/SelectionColumnMappingTest.java   | 274 ++++++++++++++-----
 5 files changed, 265 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1411ad5f/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index a282fd7,b316aa5..a734a4b
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -22,15 -3,14 +22,16 @@@ Merged from 2.1
   * Update internal python driver for cqlsh (CASSANDRA-9064)
   * Fix IndexOutOfBoundsException when inserting tuple with too many
     elements using the string literal notation (CASSANDRA-9559)
 - * Allow JMX over SSL directly from nodetool (CASSANDRA-9090)
 - * Fix incorrect result for IN queries where column not found (CASSANDRA-9540)
   * Enable describe on indices (CASSANDRA-7814)
 + * Fix incorrect result for IN queries where column not found (CASSANDRA-9540)
   * ColumnFamilyStore.selectAndReference may block during compaction 
(CASSANDRA-9637)
 + * Fix bug in cardinality check when compacting (CASSANDRA-9580)
 + * Fix memory leak in Ref due to ConcurrentLinkedQueue.remove() behaviour 
(CASSANDRA-9549)
 + * Make rebuild only run one at a time (CASSANDRA-9119)
  Merged from 2.0:
+  * Bug fixes to resultset metadata construction (CASSANDRA-9636)
   * Fix setting 'durable_writes' in ALTER KEYSPACE (CASSANDRA-9560)
 - * Avoid ballot clash in Paxos (CASSANDRA-9649)
 + * Avoids ballot clash in Paxos (CASSANDRA-9649)
   * Improve trace messages for RR (CASSANDRA-9479)
   * Fix suboptimal secondary index selection when restricted
     clustering column is also indexed (CASSANDRA-9631)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1411ad5f/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
----------------------------------------------------------------------
diff --cc 
src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
index fa40152,0000000..956efca
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
@@@ -1,133 -1,0 +1,138 @@@
 +/*
 + * 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.Arrays;
++import java.util.Collections;
 +import java.util.List;
 +
 +import com.google.common.collect.Iterables;
 +import org.apache.commons.lang3.text.StrBuilder;
 +
++import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.ColumnSpecification;
 +import org.apache.cassandra.cql3.functions.Function;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +abstract class AbstractFunctionSelector<T extends Function> extends Selector
 +{
 +    protected final T fun;
 +
 +    /**
 +     * The list used to pass the function arguments is recycled to avoid the 
cost of instantiating a new list
 +     * with each function call.
 +     */
 +    protected final List<ByteBuffer> args;
 +    protected final List<Selector> argSelectors;
 +
 +    public static Factory newFactory(final Function fun, final 
SelectorFactories factories) throws InvalidRequestException
 +    {
 +        if (fun.isAggregate())
 +        {
 +            if (factories.doesAggregation())
 +                throw new InvalidRequestException("aggregate functions cannot 
be used as arguments of aggregate functions");
 +        }
 +        else
 +        {
 +            if (factories.doesAggregation() && 
!factories.containsOnlyAggregateFunctions())
 +                throw new InvalidRequestException(String.format("arguments of 
function %s must be either all aggregates or no aggregates",
 +                                                                fun.name()));
 +        }
 +
 +        return new Factory()
 +        {
 +            protected String getColumnName()
 +            {
 +                return new StrBuilder(fun.name().toString()).append('(')
 +                                                            
.appendWithSeparators(factories.getColumnNames(), ", ")
 +                                                            .append(')')
 +                                                            .toString();
 +            }
 +
 +            protected AbstractType<?> getReturnType()
 +            {
 +                return fun.returnType();
 +            }
 +
 +            protected void addColumnMapping(SelectionColumnMapping mapping, 
ColumnSpecification resultsColumn)
 +            {
++                SelectionColumnMapping tmpMapping = 
SelectionColumnMapping.newMapping();
 +                for (Factory factory : factories)
-                    factory.addColumnMapping(mapping, resultsColumn);
++                   factory.addColumnMapping(tmpMapping, resultsColumn);
 +
-                 if (mapping.getMappings().get(resultsColumn).isEmpty())
++                if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
 +                    // add a null mapping for cases where there are no
 +                    // further selectors, such as no-arg functions and count
-                     mapping.addMapping(resultsColumn, null);
- 
++                    mapping.addMapping(resultsColumn, (ColumnDefinition)null);
++                else
++                    // collate the mapped columns from the child factories & 
add those
++                    mapping.addMapping(resultsColumn, 
tmpMapping.getMappings().values());
 +            }
 +
 +            public Iterable<Function> getFunctions()
 +            {
 +                return Iterables.concat(fun.getFunctions(), 
factories.getFunctions());
 +            }
 +
 +            public Selector newInstance() throws InvalidRequestException
 +            {
 +                return fun.isAggregate() ? new AggregateFunctionSelector(fun, 
factories.newInstances())
 +                                         : new ScalarFunctionSelector(fun, 
factories.newInstances());
 +            }
 +
 +            public boolean isWritetimeSelectorFactory()
 +            {
 +                return factories.containsWritetimeSelectorFactory();
 +            }
 +
 +            public boolean isTTLSelectorFactory()
 +            {
 +                return factories.containsTTLSelectorFactory();
 +            }
 +
 +            public boolean isAggregateSelectorFactory()
 +            {
 +                return fun.isAggregate() || 
factories.containsOnlyAggregateFunctions();
 +            }
 +        };
 +    }
 +
 +    protected AbstractFunctionSelector(T fun, List<Selector> argSelectors)
 +    {
 +        this.fun = fun;
 +        this.argSelectors = argSelectors;
 +        this.args = Arrays.asList(new ByteBuffer[argSelectors.size()]);
 +    }
 +
 +    public AbstractType<?> getType()
 +    {
 +        return fun.returnType();
 +    }
 +
 +    @Override
 +    public String toString()
 +    {
 +        return new StrBuilder().append(fun.name())
 +                               .append("(")
 +                               .appendWithSeparators(argSelectors, ", ")
 +                               .append(")")
 +                               .toString();
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1411ad5f/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selection.java
index 25278df,0000000..ea66fcc
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@@ -1,547 -1,0 +1,548 @@@
 +/*
 + * 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.*;
 +
 +import com.google.common.base.Objects;
 +import com.google.common.base.Predicate;
 +import com.google.common.collect.Iterables;
 +import com.google.common.collect.Iterators;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.functions.Function;
 +import org.apache.cassandra.db.Cell;
 +import org.apache.cassandra.db.CounterCell;
 +import org.apache.cassandra.db.ExpiringCell;
 +import org.apache.cassandra.db.context.CounterContext;
 +import org.apache.cassandra.db.marshal.UTF8Type;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +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 List<ColumnDefinition> columns;
 +    private final SelectionColumnMapping columnMapping;
 +    private final ResultSet.ResultMetadata metadata;
 +    private final boolean collectTimestamps;
 +    private final boolean collectTTLs;
 +
 +    protected Selection(CFMetaData cfm,
 +                        List<ColumnDefinition> columns,
 +                        SelectionColumnMapping columnMapping,
 +                        boolean collectTimestamps,
 +                        boolean collectTTLs)
 +    {
 +        this.cfm = cfm;
 +        this.columns = columns;
 +        this.columnMapping = columnMapping;
 +        this.metadata = new 
ResultSet.ResultMetadata(columnMapping.getColumnSpecifications());
 +        this.collectTimestamps = collectTimestamps;
 +        this.collectTTLs = collectTTLs;
 +    }
 +
 +    // Overriden by SimpleSelection when appropriate.
 +    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.ResultMetadata getResultMetadata(boolean isJson)
 +    {
 +        if (!isJson)
 +            return metadata;
 +
 +        ColumnSpecification firstColumn = metadata.names.get(0);
 +        ColumnSpecification jsonSpec = new 
ColumnSpecification(firstColumn.ksName, firstColumn.cfName, 
Json.JSON_COLUMN_ID, UTF8Type.instance);
 +        return new ResultSet.ResultMetadata(Arrays.asList(jsonSpec));
 +    }
 +
 +    public static Selection wildcard(CFMetaData cfm)
 +    {
 +        List<ColumnDefinition> all = new ArrayList<>(cfm.allColumns().size());
 +        Iterators.addAll(all, cfm.allColumnsInSelectOrder());
 +        return new SimpleSelection(cfm, all, true);
 +    }
 +
 +    public static Selection forColumns(CFMetaData cfm, List<ColumnDefinition> 
columns)
 +    {
 +        return new SimpleSelection(cfm, columns, false);
 +    }
 +
 +    public int addColumnForOrdering(ColumnDefinition c)
 +    {
 +        columns.add(c);
 +        metadata.addNonSerializedColumn(c);
 +        return columns.size() - 1;
 +    }
 +
 +    public Iterable<Function> getFunctions()
 +    {
 +        return Collections.emptySet();
 +    }
 +
 +    private static boolean processesSelection(List<RawSelector> rawSelectors)
 +    {
 +        for (RawSelector rawSelector : rawSelectors)
 +        {
 +            if (rawSelector.processesSelection())
 +                return true;
 +        }
 +        return false;
 +    }
 +
 +    public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> 
rawSelectors) throws InvalidRequestException
 +    {
 +        List<ColumnDefinition> defs = new ArrayList<>();
 +
 +        SelectorFactories factories =
 +                
SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors,
 cfm), cfm, defs);
 +        SelectionColumnMapping mapping = collectColumnMappings(cfm, 
rawSelectors, factories);
 +
-         return processesSelection(rawSelectors) ? new 
SelectionWithProcessing(cfm, defs, mapping, factories)
-                                                 : new SimpleSelection(cfm, 
defs, mapping, false);
++        return (processesSelection(rawSelectors) || rawSelectors.size() != 
defs.size())
++               ? new SelectionWithProcessing(cfm, defs, mapping, factories)
++               : new SimpleSelection(cfm, defs, mapping, false);
 +    }
 +
 +    private static SelectionColumnMapping collectColumnMappings(CFMetaData 
cfm,
 +                                                                
List<RawSelector> rawSelectors,
 +                                                                
SelectorFactories factories)
 +    {
 +        SelectionColumnMapping selectionColumns = 
SelectionColumnMapping.newMapping();
 +        Iterator<RawSelector> iter = rawSelectors.iterator();
 +        for (Selector.Factory factory : factories)
 +        {
 +            ColumnSpecification colSpec = factory.getColumnSpecification(cfm);
 +            ColumnIdentifier alias = iter.next().alias;
 +            factory.addColumnMapping(selectionColumns,
 +                                     alias == null ? colSpec : 
colSpec.withAlias(alias));
 +        }
 +        return selectionColumns;
 +    }
 +
 +    protected abstract Selectors newSelectors() throws 
InvalidRequestException;
 +
 +    /**
 +     * @return the list of CQL3 columns value this SelectionClause needs.
 +     */
 +    public List<ColumnDefinition> getColumns()
 +    {
 +        return columns;
 +    }
 +
 +    /**
 +     * @return the mappings between resultset columns and the underlying 
columns
 +     */
 +    public SelectionColumns getColumnMapping()
 +    {
 +        return columnMapping;
 +    }
 +
 +    public ResultSetBuilder resultSetBuilder(long now, boolean isJson) throws 
InvalidRequestException
 +    {
 +        return new ResultSetBuilder(now, isJson);
 +    }
 +
 +    public abstract boolean isAggregate();
 +
 +    @Override
 +    public String toString()
 +    {
 +        return Objects.toStringHelper(this)
 +                .add("columns", columns)
 +                .add("columnMapping", columnMapping)
 +                .add("metadata", metadata)
 +                .add("collectTimestamps", collectTimestamps)
 +                .add("collectTTLs", collectTTLs)
 +                .toString();
 +    }
 +
 +    public 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;
 +
 +        /*
 +         * 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;
 +        final long now;
 +
 +        private final boolean isJson;
 +
 +        private ResultSetBuilder(long now, boolean isJson) throws 
InvalidRequestException
 +        {
 +            this.resultSet = new ResultSet(getResultMetadata(isJson).copy(), 
new ArrayList<List<ByteBuffer>>());
 +            this.selectors = newSelectors();
 +            this.timestamps = collectTimestamps ? new long[columns.size()] : 
null;
 +            this.ttls = collectTTLs ? new int[columns.size()] : null;
 +            this.now = now;
 +            this.isJson = isJson;
 +        }
 +
 +        public void add(ByteBuffer v)
 +        {
 +            current.add(v);
 +        }
 +
 +        public void add(Cell c)
 +        {
 +            current.add(isDead(c) ? null : value(c));
 +            if (timestamps != null)
 +            {
 +                timestamps[current.size() - 1] = isDead(c) ? Long.MIN_VALUE : 
c.timestamp();
 +            }
 +            if (ttls != null)
 +            {
 +                int ttl = -1;
 +                if (!isDead(c) && c instanceof ExpiringCell)
 +                    ttl = c.getLocalDeletionTime() - (int) (now / 1000);
 +                ttls[current.size() - 1] = ttl;
 +            }
 +        }
 +
 +        private boolean isDead(Cell c)
 +        {
 +            return c == null || !c.isLive(now);
 +        }
 +
 +        public void newRow(int protocolVersion) throws InvalidRequestException
 +        {
 +            if (current != null)
 +            {
 +                selectors.addInputRow(protocolVersion, this);
 +                if (!selectors.isAggregate())
 +                {
 +                    resultSet.addRow(getOutputRow(protocolVersion));
 +                    selectors.reset();
 +                }
 +            }
 +            current = new ArrayList<>(columns.size());
 +        }
 +
 +        public ResultSet build(int protocolVersion) throws 
InvalidRequestException
 +        {
 +            if (current != null)
 +            {
 +                selectors.addInputRow(protocolVersion, this);
 +                resultSet.addRow(getOutputRow(protocolVersion));
 +                selectors.reset();
 +                current = null;
 +            }
 +
 +            if (resultSet.isEmpty() && selectors.isAggregate())
 +                resultSet.addRow(getOutputRow(protocolVersion));
 +            return resultSet;
 +        }
 +
 +        private List<ByteBuffer> getOutputRow(int protocolVersion)
 +        {
 +            List<ByteBuffer> outputRow = 
selectors.getOutputRow(protocolVersion);
 +            return isJson ? rowToJson(outputRow, protocolVersion)
 +                          : outputRow;
 +        }
 +
 +        private List<ByteBuffer> rowToJson(List<ByteBuffer> row, int 
protocolVersion)
 +        {
 +            StringBuilder sb = new StringBuilder("{");
 +            for (int i = 0; i < metadata.names.size(); i++)
 +            {
 +                if (i > 0)
 +                    sb.append(", ");
 +
 +                ColumnSpecification spec = metadata.names.get(i);
 +                String columnName = spec.name.toString();
 +                if (!columnName.equals(columnName.toLowerCase(Locale.US)))
 +                    columnName = "\"" + columnName + "\"";
 +
 +                ByteBuffer buffer = row.get(i);
 +                sb.append('"');
 +                sb.append(Json.JSON_STRING_ENCODER.quoteAsString(columnName));
 +                sb.append("\": ");
 +                if (buffer == null)
 +                    sb.append("null");
 +                else
 +                    sb.append(spec.type.toJSONString(buffer, 
protocolVersion));
 +            }
 +            sb.append("}");
 +            return 
Collections.singletonList(UTF8Type.instance.getSerializer().serialize(sb.toString()));
 +        }
 +
 +        private ByteBuffer value(Cell c)
 +        {
 +            return (c instanceof CounterCell)
 +                ? 
ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
 +                : c.value();
 +        }
 +    }
 +
 +    private static interface Selectors
 +    {
 +        public boolean isAggregate();
 +
 +        /**
 +         * Adds the current row of the specified 
<code>ResultSetBuilder</code>.
 +         *
 +         * @param rs the <code>ResultSetBuilder</code>
 +         * @throws InvalidRequestException
 +         */
 +        public void addInputRow(int protocolVersion, ResultSetBuilder rs) 
throws InvalidRequestException;
 +
 +        public List<ByteBuffer> getOutputRow(int protocolVersion) throws 
InvalidRequestException;
 +
 +        public void reset();
 +    }
 +
 +    // Special cased selection for when no function is used (this save some 
allocations).
 +    private static class SimpleSelection extends Selection
 +    {
 +        private final boolean isWildcard;
 +
 +        public SimpleSelection(CFMetaData cfm, List<ColumnDefinition> 
columns, boolean isWildcard)
 +        {
 +            this(cfm, columns, SelectionColumnMapping.simpleMapping(columns), 
isWildcard);
 +        }
 +
 +        public SimpleSelection(CFMetaData cfm,
 +                               List<ColumnDefinition> columns,
 +                               SelectionColumnMapping 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(cfm, columns, metadata, false, false);
 +            this.isWildcard = isWildcard;
 +        }
 +
 +        @Override
 +        public boolean isWildcard()
 +        {
 +            return isWildcard;
 +        }
 +
 +        public boolean isAggregate()
 +        {
 +            return false;
 +        }
 +
 +        protected Selectors newSelectors()
 +        {
 +            return new Selectors()
 +            {
 +                private List<ByteBuffer> current;
 +
 +                public void reset()
 +                {
 +                    current = null;
 +                }
 +
 +                public List<ByteBuffer> getOutputRow(int protocolVersion)
 +                {
 +                    return current;
 +                }
 +
 +                public void addInputRow(int protocolVersion, ResultSetBuilder 
rs) throws InvalidRequestException
 +                {
 +                    current = rs.current;
 +                }
 +
 +                public boolean isAggregate()
 +                {
 +                    return false;
 +                }
 +            };
 +        }
 +    }
 +
 +    private static class SelectionWithProcessing extends Selection
 +    {
 +        private final SelectorFactories factories;
 +
 +        public SelectionWithProcessing(CFMetaData cfm,
 +                                       List<ColumnDefinition> columns,
 +                                       SelectionColumnMapping metadata,
 +                                       SelectorFactories factories) throws 
InvalidRequestException
 +        {
 +            super(cfm,
 +                  columns,
 +                  metadata,
 +                  factories.containsWritetimeSelectorFactory(),
 +                  factories.containsTTLSelectorFactory());
 +
 +            this.factories = factories;
 +
 +            if (factories.doesAggregation() && 
!factories.containsOnlyAggregateFunctions())
 +                throw new InvalidRequestException("the select clause must 
either contain only aggregates or no aggregate");
 +        }
 +
 +        @Override
 +        public Iterable<Function> getFunctions()
 +        {
 +            return factories.getFunctions();
 +        }
 +
 +        @Override
 +        public int addColumnForOrdering(ColumnDefinition c)
 +        {
 +            int index = super.addColumnForOrdering(c);
 +            factories.addSelectorForOrdering(c, index);
 +            return index;
 +        }
 +
 +        public boolean isAggregate()
 +        {
 +            return factories.containsOnlyAggregateFunctions();
 +        }
 +
 +        protected Selectors newSelectors() throws InvalidRequestException
 +        {
 +            return new Selectors()
 +            {
 +                private final List<Selector> selectors = 
factories.newInstances();
 +
 +                public void reset()
 +                {
 +                    for (Selector selector : selectors)
 +                        selector.reset();
 +                }
 +
 +                public boolean isAggregate()
 +                {
 +                    return factories.containsOnlyAggregateFunctions();
 +                }
 +
 +                public List<ByteBuffer> getOutputRow(int protocolVersion) 
throws InvalidRequestException
 +                {
 +                    List<ByteBuffer> outputRow = new 
ArrayList<>(selectors.size());
 +
 +                    for (Selector selector: selectors)
 +                        outputRow.add(selector.getOutput(protocolVersion));
 +
 +                    return outputRow;
 +                }
 +
 +                public void addInputRow(int protocolVersion, ResultSetBuilder 
rs) throws InvalidRequestException
 +                {
 +                    for (Selector selector : selectors)
 +                        selector.addInput(protocolVersion, rs);
 +                }
 +            };
 +        }
 +
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1411ad5f/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
----------------------------------------------------------------------
diff --cc 
src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
index e6c8979,0000000..33ef0af
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
@@@ -1,118 -1,0 +1,132 @@@
 +package org.apache.cassandra.cql3.selection;
 +
- import java.util.LinkedHashSet;
- import java.util.List;
++import java.util.*;
 +
 +import com.google.common.base.Function;
 +import com.google.common.base.Joiner;
 +import com.google.common.base.Objects;
 +import com.google.common.collect.*;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.ColumnSpecification;
 +
 +/**
 + * Separately maintains the ColumnSpecifications and their mappings to 
underlying
 + * columns as we may receive null mappings. This occurs where a query result
 + * includes a column specification which does not map to any particular real
 + * column, e.g. COUNT queries or where no-arg functions like now() are used
 + */
 +public class SelectionColumnMapping implements SelectionColumns
 +{
-     // Uses a LinkedHashSet as both order and uniqueness need to be preserved
-     private final LinkedHashSet<ColumnSpecification> columnSpecifications;
++    private final ArrayList<ColumnSpecification> columnSpecifications;
 +    private final HashMultimap<ColumnSpecification, ColumnDefinition> 
columnMappings;
 +
 +    private SelectionColumnMapping()
 +    {
-         this.columnSpecifications = new LinkedHashSet<>();
++        this.columnSpecifications = new ArrayList<>();
 +        this.columnMappings = HashMultimap.create();
 +    }
 +
 +    protected static SelectionColumnMapping newMapping()
 +    {
 +        return new SelectionColumnMapping();
 +    }
 +
-     protected static SelectionColumnMapping 
simpleMapping(List<ColumnDefinition> columnDefinitions)
++    protected static SelectionColumnMapping 
simpleMapping(Iterable<ColumnDefinition> columnDefinitions)
 +    {
 +        SelectionColumnMapping mapping = new SelectionColumnMapping();
 +        for (ColumnDefinition def: columnDefinitions)
 +            mapping.addMapping(def, def);
 +        return mapping;
 +    }
 +
 +    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, 
ColumnDefinition column)
 +    {
 +        columnSpecifications.add(colSpec);
-         // some AbstractFunctionSelector impls do not map directly to an 
underlying column
-         // so don't record a mapping in that case
-         if (null != column)
++        // functions without arguments do not map to any column, so don't
++        // record any mapping in that case
++        if (column != null)
 +            columnMappings.put(colSpec, column);
 +        return this;
 +    }
 +
++    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, 
Iterable<ColumnDefinition> columns)
++    {
++        columnSpecifications.add(colSpec);
++        columnMappings.putAll(colSpec, columns);
++        return this;
++    }
++
 +    public List<ColumnSpecification> getColumnSpecifications()
 +    {
 +        // return a mutable copy as we may add extra columns
 +        // for ordering (CASSANDRA-4911 & CASSANDRA-8286)
 +        return Lists.newArrayList(columnSpecifications);
 +    }
 +
 +    public Multimap<ColumnSpecification, ColumnDefinition> getMappings()
 +    {
 +        return Multimaps.unmodifiableMultimap(columnMappings);
 +    }
 +
 +    public boolean equals(Object obj)
 +    {
 +        if (obj == null)
 +            return false;
 +
 +        if (!(obj instanceof SelectionColumnMapping))
 +            return false;
 +
-         return Objects.equal(this.columnMappings, 
((SelectionColumnMapping)obj).columnMappings);
++        SelectionColumns other = (SelectionColumns)obj;
++        return Objects.equal(columnMappings, other.getMappings())
++            && Objects.equal(columnSpecifications, 
other.getColumnSpecifications());
 +    }
 +
 +    public int hashCode()
 +    {
 +        return Objects.hashCode(columnMappings);
 +    }
 +
 +    public String toString()
 +    {
 +        final Function<ColumnDefinition, String> getDefName = new 
Function<ColumnDefinition, String>()
 +        {
 +            public String apply(ColumnDefinition def)
 +            {
 +                return def.name.toString();
 +            }
 +        };
-         final Function<ColumnSpecification, String> colSpecToMappingString = 
new Function<ColumnSpecification, String>()
-         {
-             public String apply(ColumnSpecification colSpec)
++        Function<Map.Entry<ColumnSpecification, 
Collection<ColumnDefinition>>, String> mappingEntryToString =
++        new Function<Map.Entry<ColumnSpecification, 
Collection<ColumnDefinition>>, String>(){
++            public String apply(Map.Entry<ColumnSpecification, 
Collection<ColumnDefinition>> entry)
 +            {
 +                StringBuilder builder = new StringBuilder();
-                 builder.append(colSpec.name.toString());
-                 if (columnMappings.containsKey(colSpec))
-                 {
-                     builder.append(":[");
-                     
builder.append(Joiner.on(',').join(Iterables.transform(columnMappings.get(colSpec),
 getDefName)));
-                     builder.append("]");
-                 }
-                 else
-                 {
-                     builder.append(":[]");
-                 }
++                builder.append(entry.getKey().name.toString());
++                builder.append(":[");
++                
builder.append(Joiner.on(',').join(Iterables.transform(entry.getValue(), 
getDefName)));
++                builder.append("]");
 +                return builder.toString();
 +            }
 +        };
 +
++        Function<ColumnSpecification, String> colSpecToString = new 
Function<ColumnSpecification, String>()
++        {
++            public String apply(ColumnSpecification columnSpecification)
++            {
++                return columnSpecification.name.toString();
++            }
++        };
++
 +        StringBuilder builder = new StringBuilder();
-         builder.append("{ ");
-         builder.append(Joiner.on(", 
").join(Iterables.transform(columnSpecifications, colSpecToMappingString)));
-         builder.append(" }");
++        builder.append("{ Columns:[");
++        builder.append(Joiner.on(",")
++                             .join(Iterables.transform(columnSpecifications, 
colSpecToString)));
++        builder.append("], Mappings:[");
++        builder.append(Joiner.on(", ")
++                             
.join(Iterables.transform(columnMappings.asMap().entrySet(),
++                                                       
mappingEntryToString)));
++        builder.append("] }");
 +        return builder.toString();
 +    }
++
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1411ad5f/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --cc 
test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
index 2f1d361,0000000..0cebb03
mode 100644,000000..100644
--- 
a/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
+++ 
b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
@@@ -1,362 -1,0 +1,510 @@@
 +package org.apache.cassandra.cql3.selection;
 +
- import java.util.Collections;
++import java.util.ArrayList;
++import java.util.List;
 +
- import com.google.common.collect.ImmutableList;
 +import org.junit.BeforeClass;
 +import org.junit.Test;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.cql3.*;
 +import org.apache.cassandra.cql3.statements.SelectStatement;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.dht.ByteOrderedPartitioner;
++import org.apache.cassandra.exceptions.RequestExecutionException;
 +import org.apache.cassandra.exceptions.RequestValidationException;
 +import org.apache.cassandra.service.ClientState;
++import org.apache.cassandra.service.QueryState;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +import static org.junit.Assert.assertEquals;
- import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertTrue;
 +
 +public class SelectionColumnMappingTest extends CQLTester
 +{
++    private static final ColumnDefinition NULL_DEF = null;
 +    String tableName;
 +    String typeName;
 +    UserType userType;
 +    String functionName;
 +
 +    @BeforeClass
 +    public static void setUpClass()
 +    {
 +        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
 +    }
 +
 +    @Test
 +    public void testSelectionColumnMapping() throws Throwable
 +    {
 +        // Organised as a single test to avoid the overhead of
 +        // table creation for each variant
 +
 +        typeName = createType("CREATE TYPE %s (f1 int, f2 text)");
 +        tableName = createTable("CREATE TABLE %s (" +
-                                     " k int PRIMARY KEY," +
-                                     " v1 int," +
-                                     " v2 ascii," +
-                                     " v3 frozen<" + typeName + ">)");
++                                " k int PRIMARY KEY," +
++                                " v1 int," +
++                                " v2 ascii," +
++                                " v3 frozen<" + typeName + ">)");
 +        userType = 
Schema.instance.getKSMetaData(KEYSPACE).userTypes.getType(ByteBufferUtil.bytes(typeName));
 +        functionName = createFunction(KEYSPACE, "int, ascii",
 +                                      "CREATE FUNCTION %s (i int, a ascii) " +
 +                                      "CALLED ON NULL INPUT " +
 +                                      "RETURNS int " +
 +                                      "LANGUAGE java " +
 +                                      "AS 'return Integer.valueOf(i);'");
++        execute("INSERT INTO %s (k, v1 ,v2, v3) VALUES (1, 1, 'foo', {f1:1, 
f2:'bar'})");
++
 +        testSimpleTypes();
 +        testWildcard();
 +        testSimpleTypesWithAliases();
 +        testUserTypes();
 +        testUserTypesWithAliases();
 +        testWritetimeAndTTL();
 +        testWritetimeAndTTLWithAliases();
 +        testFunction();
 +        testNoArgFunction();
 +        testUserDefinedFunction();
 +        testOverloadedFunction();
 +        testFunctionWithAlias();
-         testMultipleAliasesOnSameColumn();
++        testNoArgumentFunction();
++        testNestedFunctions();
++        testNestedFunctionsWithArguments();
 +        testCount();
++        testDuplicateFunctionsWithoutAliases();
++        testDuplicateFunctionsWithAliases();
++        testSelectDistinct();
++        testMultipleAliasesOnSameColumn();
 +        testMixedColumnTypes();
++        testMultipleUnaliasedSelectionOfSameColumn();
++        testUserDefinedAggregate();
 +    }
 +
 +    @Test
 +    public void testMultipleArgumentFunction() throws Throwable
 +    {
 +        // demonstrate behaviour of token() with composite partition key
 +        tableName = createTable("CREATE TABLE %s (a int, b text, PRIMARY KEY 
((a, b)))");
 +        ColumnSpecification tokenSpec = columnSpecification("system.token(a, 
b)", BytesType.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                 
.addMapping(tokenSpec, columnDefinition("a"))
-                                                                 
.addMapping(tokenSpec, columnDefinition("b"));
- 
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT 
token(a,b) FROM %s"));
++                                                                
.addMapping(tokenSpec, columnDefinitions("a", "b"));
++        // we don't use verify like with the other tests because this query 
will produce no results
++        SelectStatement statement = getSelect("SELECT token(a,b) FROM %s");
++        verifyColumnMapping(expected, statement);
++        statement.executeInternal(QueryState.forInternalCalls(), 
QueryOptions.DEFAULT);
 +    }
 +
 +    private void testSimpleTypes() throws Throwable
 +    {
 +        // simple column identifiers without aliases are represented in
 +        // ResultSet.Metadata by the underlying ColumnDefinition
 +        ColumnSpecification kSpec = columnSpecification("k", 
Int32Type.instance);
 +        ColumnSpecification v1Spec = columnSpecification("v1", 
Int32Type.instance);
 +        ColumnSpecification v2Spec = columnSpecification("v2", 
AsciiType.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(kSpec, columnDefinition("k"))
 +                                                                
.addMapping(v1Spec, columnDefinition("v1"))
 +                                                                
.addMapping(v2Spec, columnDefinition("v2"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT k, v1, 
v2 FROM %s"));
++        verify(expected, "SELECT k, v1, v2 FROM %s");
 +    }
 +
 +    private void testWildcard() throws Throwable
 +    {
 +        // Wildcard select represents each column in the table with a 
ColumnDefinition
 +        // in the ResultSet metadata
 +        ColumnDefinition kSpec = columnDefinition("k");
 +        ColumnDefinition v1Spec = columnDefinition("v1");
 +        ColumnDefinition v2Spec = columnDefinition("v2");
 +        ColumnDefinition v3Spec = columnDefinition("v3");
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(kSpec, columnDefinition("k"))
 +                                                                
.addMapping(v1Spec, columnDefinition("v1"))
 +                                                                
.addMapping(v2Spec, columnDefinition("v2"))
 +                                                                
.addMapping(v3Spec, columnDefinition("v3"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT * FROM 
%s"));
++        verify(expected, "SELECT * FROM %s");
 +    }
 +
 +    private void testSimpleTypesWithAliases() throws Throwable
 +    {
 +        // simple column identifiers with aliases are represented in 
ResultSet.Metadata
 +        // by a ColumnSpecification based on the underlying ColumnDefinition
 +        ColumnSpecification kSpec = columnSpecification("k_alias", 
Int32Type.instance);
 +        ColumnSpecification v1Spec = columnSpecification("v1_alias", 
Int32Type.instance);
 +        ColumnSpecification v2Spec = columnSpecification("v2_alias", 
AsciiType.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(kSpec, columnDefinition("k"))
 +                                                                
.addMapping(v1Spec, columnDefinition("v1"))
 +                                                                
.addMapping(v2Spec, columnDefinition("v2"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT k AS 
k_alias, v1 AS v1_alias, v2 AS v2_alias FROM %s"));
++        verify(expected, "SELECT k AS k_alias, v1 AS v1_alias, v2 AS v2_alias 
FROM %s");
 +    }
 +
 +    private void testUserTypes() throws Throwable
 +    {
 +        // User type fields are represented in ResultSet.Metadata by a
 +        // ColumnSpecification denoting the name and type of the particular 
field
 +        ColumnSpecification f1Spec = columnSpecification("v3.f1", 
Int32Type.instance);
 +        ColumnSpecification f2Spec = columnSpecification("v3.f2", 
UTF8Type.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(f1Spec, columnDefinition("v3"))
 +                                                                
.addMapping(f2Spec, columnDefinition("v3"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT v3.f1, 
v3.f2 FROM %s"));
++        verify(expected, "SELECT v3.f1, v3.f2 FROM %s");
 +    }
 +
 +    private void testUserTypesWithAliases() throws Throwable
 +    {
 +        // User type fields with aliases are represented in ResultSet.Metadata
 +        // by a ColumnSpecification with the alias name and the type of the 
actual field
 +        ColumnSpecification f1Spec = columnSpecification("f1_alias", 
Int32Type.instance);
 +        ColumnSpecification f2Spec = columnSpecification("f2_alias", 
UTF8Type.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(f1Spec, columnDefinition("v3"))
 +                                                                
.addMapping(f2Spec, columnDefinition("v3"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT v3.f1 
AS f1_alias, v3.f2 AS f2_alias FROM %s"));
++        verify(expected, "SELECT v3.f1 AS f1_alias, v3.f2 AS f2_alias FROM 
%s");
 +    }
 +
 +    private void testWritetimeAndTTL() throws Throwable
 +    {
 +        // writetime and ttl are represented in ResultSet.Metadata by a 
ColumnSpecification
 +        // with the function name plus argument and a long or int type 
respectively
 +        ColumnSpecification wtSpec = columnSpecification("writetime(v1)", 
LongType.instance);
 +        ColumnSpecification ttlSpec = columnSpecification("ttl(v2)", 
Int32Type.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(wtSpec, columnDefinition("v1"))
 +                                                                
.addMapping(ttlSpec, columnDefinition("v2"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT 
writetime(v1), ttl(v2) FROM %s"));
++        verify(expected, "SELECT writetime(v1), ttl(v2) FROM %s");
 +    }
 +
 +    private void testWritetimeAndTTLWithAliases() throws Throwable
 +    {
 +        // writetime and ttl with aliases are represented in 
ResultSet.Metadata
 +        // by a ColumnSpecification with the alias name and the appropriate 
numeric type
 +        ColumnSpecification wtSpec = columnSpecification("wt_alias", 
LongType.instance);
 +        ColumnSpecification ttlSpec = columnSpecification("ttl_alias", 
Int32Type.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(wtSpec, columnDefinition("v1"))
 +                                                                
.addMapping(ttlSpec, columnDefinition("v2"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT 
writetime(v1) AS wt_alias, ttl(v2) AS ttl_alias FROM %s"));
++        verify(expected, "SELECT writetime(v1) AS wt_alias, ttl(v2) AS 
ttl_alias FROM %s");
 +    }
 +
 +    private void testFunction() throws Throwable
 +    {
 +        // a function such as intasblob(<col>) is represented in 
ResultSet.Metadata
 +        // by a ColumnSpecification with the function name plus args and the 
type set
 +        // to the function's return type
 +        ColumnSpecification fnSpec = 
columnSpecification("system.intasblob(v1)", BytesType.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(fnSpec, columnDefinition("v1"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT 
intasblob(v1) FROM %s"));
++        verify(expected, "SELECT intasblob(v1) FROM %s");
 +    }
 +
 +    private void testNoArgFunction() throws Throwable
 +    {
 +        // a no-arg function such as now() is represented in 
ResultSet.Metadata
 +        // but has no mapping to any underlying column
 +        ColumnSpecification fnSpec = columnSpecification("system.now()", 
TimeUUIDType.instance);
-         SelectionColumnMapping expected = 
SelectionColumnMapping.newMapping().addMapping(fnSpec, null);
++        SelectionColumnMapping expected = 
SelectionColumnMapping.newMapping().addMapping(fnSpec, NULL_DEF);
 +
-         SelectionColumns actual = extractColumnMappingFromSelect("SELECT 
now() FROM %s");
-         assertEquals(expected, actual);
-         assertEquals(Collections.singletonList(fnSpec), 
actual.getColumnSpecifications());
-         assertTrue(actual.getMappings().isEmpty());
++        verify(expected, "SELECT now() FROM %s");
 +    }
 +
 +    private void testOverloadedFunction() throws Throwable
 +    {
 +        String fnName = createFunction(KEYSPACE, "int",
 +                                       "CREATE FUNCTION %s (input int) " +
 +                                       "RETURNS NULL ON NULL INPUT " +
 +                                       "RETURNS text " +
 +                                       "LANGUAGE java " +
 +                                       "AS 'return \"Hello World\";'");
 +        createFunctionOverload(fnName, "text",
 +                               "CREATE FUNCTION %s (input text) " +
 +                               "RETURNS NULL ON NULL INPUT " +
 +                               "RETURNS text " +
 +                               "LANGUAGE java " +
 +                               "AS 'return \"Hello World\";'");
 +
 +        createFunctionOverload(fnName, "int, text",
 +                               "CREATE FUNCTION %s (input1 int, input2 text) 
" +
 +                               "RETURNS NULL ON NULL INPUT " +
 +                               "RETURNS text " +
 +                               "LANGUAGE java " +
 +                               "AS 'return \"Hello World\";'");
 +        ColumnSpecification fnSpec1 = columnSpecification(fnName + "(v1)", 
UTF8Type.instance);
 +        ColumnSpecification fnSpec2 = columnSpecification(fnName + "(v2)", 
UTF8Type.instance);
 +        ColumnSpecification fnSpec3 = columnSpecification(fnName + "(v1, 
v2)", UTF8Type.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(fnSpec1, columnDefinition("v1"))
 +                                                                
.addMapping(fnSpec2, columnDefinition("v2"))
-                                                                 
.addMapping(fnSpec3, columnDefinition("v1"))
-                                                                 
.addMapping(fnSpec3, columnDefinition("v2"));
++                                                                
.addMapping(fnSpec3, columnDefinitions("v1", "v2"));
 +
-         String select = String.format("SELECT %1$s(v1), %1$s(v2), %1$s(v1, 
v2) FROM %%s", fnName);
-         SelectionColumns actual = extractColumnMappingFromSelect(select);
- 
-         assertEquals(expected, actual);
-         assertEquals(ImmutableList.of(fnSpec1, fnSpec2, fnSpec3), 
actual.getColumnSpecifications());
++        verify(expected, String.format("SELECT %1$s(v1), %1$s(v2), %1$s(v1, 
v2) FROM %%s", fnName));
 +    }
 +
 +    private void testCount() throws Throwable
 +    {
 +        // SELECT COUNT does not necessarily include any mappings, but it 
must always return
 +        // a singleton list from getColumnSpecifications() in order for the 
ResultSet.Metadata
 +        // to be constructed correctly:
 +        // * COUNT(*) / COUNT(1) do not generate any mappings, as no specific 
columns are referenced
 +        // * COUNT(foo) does generate a mapping from the 'system.count' 
column spec to foo
 +        ColumnSpecification count = columnSpecification("count", 
LongType.instance);
-         SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                 
.addMapping(count, null);
++        SelectionColumnMapping expected = 
SelectionColumnMapping.newMapping().addMapping(count, NULL_DEF);
++
++        verify(expected, "SELECT COUNT(*) FROM %s");
++        verify(expected, "SELECT COUNT(1) FROM %s");
 +
-         SelectionColumns actual = extractColumnMappingFromSelect("SELECT 
COUNT(*) FROM %s");
-         assertEquals(expected, actual);
-         assertEquals(Collections.singletonList(count), 
actual.getColumnSpecifications());
-         assertTrue(actual.getMappings().isEmpty());
++        ColumnSpecification aliased = columnSpecification("count_alias", 
LongType.instance);
++        expected = SelectionColumnMapping.newMapping().addMapping(aliased, 
NULL_DEF);
 +
-         actual = extractColumnMappingFromSelect("SELECT COUNT(1) FROM %s");
-         assertEquals(expected, actual);
-         assertEquals(Collections.singletonList(count), 
actual.getColumnSpecifications());
-         assertTrue(actual.getMappings().isEmpty());
++        verify(expected, "SELECT COUNT(*) AS count_alias FROM %s");
++        verify(expected, "SELECT COUNT(1) AS count_alias FROM %s");
 +
 +        ColumnSpecification countV1 = columnSpecification("system.count(v1)", 
LongType.instance);
-         expected = SelectionColumnMapping.newMapping()
-                                          .addMapping(countV1, 
columnDefinition("v1"));
-         actual = extractColumnMappingFromSelect("SELECT COUNT(v1) FROM %s");
-         assertEquals(expected, actual);
-         assertEquals(Collections.singletonList(countV1), 
actual.getColumnSpecifications());
-         assertFalse(actual.getMappings().isEmpty());
++        expected = SelectionColumnMapping.newMapping().addMapping(countV1, 
columnDefinition("v1"));
++        verify(expected, "SELECT COUNT(v1) FROM %s");
++
++        ColumnSpecification countV1Alias = columnSpecification("count_v1", 
LongType.instance);
++        expected = 
SelectionColumnMapping.newMapping().addMapping(countV1Alias, 
columnDefinition("v1"));
++        verify(expected, "SELECT COUNT(v1) AS count_v1 FROM %s");
 +    }
 +
 +    private void testUserDefinedFunction() throws Throwable
 +    {
 +        // UDFs are basically represented in the same way as system functions
 +        String functionCall = String.format("%s(v1, v2)", functionName);
 +        ColumnSpecification fnSpec = columnSpecification(functionCall, 
Int32Type.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                 
.addMapping(fnSpec, columnDefinition("v1"))
-                                                                 
.addMapping(fnSpec, columnDefinition("v2"));
- 
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT " + 
functionCall + " FROM %s"));
++                                                                
.addMapping(fnSpec, columnDefinitions("v1", "v2"));
++        verify(expected, "SELECT " + functionCall + " FROM %s");
 +    }
 +
 +    private void testFunctionWithAlias() throws Throwable
 +    {
 +        // a function with an alias is represented in ResultSet.Metadata by a
 +        // ColumnSpecification with the alias and the type set to the 
function's
 +        // return type
 +        ColumnSpecification fnSpec = columnSpecification("fn_alias", 
BytesType.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(fnSpec, columnDefinition("v1"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT 
intasblob(v1) AS fn_alias FROM %s"));
++        verify(expected, "SELECT intasblob(v1) AS fn_alias FROM %s");
++    }
++
++    public void testNoArgumentFunction() throws Throwable
++    {
++        SelectionColumns expected = SelectionColumnMapping.newMapping()
++                                                          
.addMapping(columnSpecification("system.now()",
++                                                                              
            TimeUUIDType.instance),
++                                                                      
NULL_DEF);
++        verify(expected, "SELECT now() FROM %s");
++    }
++
++    public void testNestedFunctionsWithArguments() throws Throwable
++    {
++        SelectionColumns expected = SelectionColumnMapping.newMapping()
++                                                          
.addMapping(columnSpecification("system.blobasint(system.intasblob(v1))",
++                                                                              
            Int32Type.instance),
++                                                                      
columnDefinition("v1"));
++        verify(expected, "SELECT blobasint(intasblob(v1)) FROM %s");
++    }
++
++    public void testNestedFunctions() throws Throwable
++    {
++        SelectionColumns expected = SelectionColumnMapping.newMapping()
++                                                          
.addMapping(columnSpecification("system.tounixtimestamp(system.now())",
++                                                                              
            LongType.instance),
++                                                                      
NULL_DEF);
++        verify(expected, "SELECT tounixtimestamp(now()) FROM %s");
++    }
++
++    public void testDuplicateFunctionsWithoutAliases() throws Throwable
++    {
++        // where duplicate functions are present, the ColumnSpecification 
list will
++        // contain an entry per-duplicate but the mappings will be 
deduplicated (i.e.
++        // a single mapping k/v pair regardless of the number of duplicates)
++        ColumnSpecification spec = 
columnSpecification("system.intasblob(v1)", BytesType.instance);
++        SelectionColumns expected = SelectionColumnMapping.newMapping()
++                                                          .addMapping(spec, 
columnDefinition("v1"))
++                                                          .addMapping(spec, 
columnDefinition("v1"));
++        verify(expected, "SELECT intasblob(v1), intasblob(v1) FROM %s");
++    }
++
++    public void testDuplicateFunctionsWithAliases() throws Throwable
++    {
++        // where duplicate functions are present with distinct aliases, they 
are
++        // represented as any other set of distinct columns would be - an 
entry
++        // in theColumnSpecification list and a separate k/v mapping for each
++        SelectionColumns expected = SelectionColumnMapping.newMapping()
++                                                          
.addMapping(columnSpecification("blob_1", BytesType.instance),
++                                                                      
columnDefinition("v1"))
++                                                          
.addMapping(columnSpecification("blob_2", BytesType.instance),
++                                                                      
columnDefinition("v1"));
++        verify(expected, "SELECT intasblob(v1) AS blob_1, intasblob(v1) AS 
blob_2 FROM %s");
++    }
++
++    public void testSelectDistinct() throws Throwable
++    {
++        SelectionColumns expected = 
SelectionColumnMapping.newMapping().addMapping(columnSpecification("k",
++                                                                              
                         Int32Type.instance),
++                                                                              
     columnDefinition("k"));
++        verify(expected, "SELECT DISTINCT k FROM %s");
++
 +    }
 +
 +    private void testMultipleAliasesOnSameColumn() throws Throwable
 +    {
 +        // Multiple result columns derived from the same underlying column are
 +        // represented by ColumnSpecifications
 +        ColumnSpecification alias1 = columnSpecification("alias_1", 
Int32Type.instance);
 +        ColumnSpecification alias2 = columnSpecification("alias_2", 
Int32Type.instance);
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(alias1, columnDefinition("v1"))
 +                                                                
.addMapping(alias2, columnDefinition("v1"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT v1 AS 
alias_1, v1 AS alias_2 FROM %s"));
++        verify(expected, "SELECT v1 AS alias_1, v1 AS alias_2 FROM %s");
++    }
++
++    private void testMultipleUnaliasedSelectionOfSameColumn() throws Throwable
++    {
++        // simple column identifiers without aliases are represented in
++        // ResultSet.Metadata by the underlying ColumnDefinition
++        SelectionColumns expected = SelectionColumnMapping.newMapping()
++                                                          
.addMapping(columnSpecification("v1", Int32Type.instance),
++                                                                      
columnDefinition("v1"))
++                                                          
.addMapping(columnSpecification("v1", Int32Type.instance),
++                                                                      
columnDefinition("v1"));
++
++        verify(expected, "SELECT v1, v1 FROM %s");
 +    }
 +
 +    private void testMixedColumnTypes() throws Throwable
 +    {
 +        ColumnSpecification kSpec = columnSpecification("k_alias", 
Int32Type.instance);
 +        ColumnSpecification v1Spec = columnSpecification("writetime(v1)", 
LongType.instance);
 +        ColumnSpecification v2Spec = columnSpecification("ttl_alias", 
Int32Type.instance);
 +        ColumnSpecification f1Spec = columnSpecification("v3.f1", 
Int32Type.instance);
 +        ColumnSpecification f2Spec = columnSpecification("f2_alias", 
UTF8Type.instance);
 +        ColumnSpecification f3Spec = columnSpecification("v3", userType);
 +
 +        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
 +                                                                
.addMapping(kSpec, columnDefinition("k"))
 +                                                                
.addMapping(v1Spec, columnDefinition("v1"))
 +                                                                
.addMapping(v2Spec, columnDefinition("v2"))
 +                                                                
.addMapping(f1Spec, columnDefinition("v3"))
 +                                                                
.addMapping(f2Spec, columnDefinition("v3"))
 +                                                                
.addMapping(f3Spec, columnDefinition("v3"));
 +
-         assertEquals(expected, extractColumnMappingFromSelect("SELECT k AS 
k_alias," +
-                                                               "       
writetime(v1)," +
-                                                               "       ttl(v2) 
as ttl_alias," +
-                                                               "       v3.f1," 
+
-                                                               "       v3.f2 
AS f2_alias," +
-                                                               "       v3" +
-                                                               " FROM %s"));
++
++        verify(expected, "SELECT k AS k_alias," +
++                         "       writetime(v1)," +
++                         "       ttl(v2) as ttl_alias," +
++                         "       v3.f1," +
++                         "       v3.f2 AS f2_alias," +
++                         "       v3" +
++                         " FROM %s");
++    }
++
++    private void testUserDefinedAggregate() throws Throwable
++    {
++        String sFunc = createFunction(KEYSPACE, "int",
++                                      " CREATE FUNCTION %s (a int, b int)" +
++                                      " RETURNS NULL ON NULL INPUT" +
++                                      " RETURNS int" +
++                                      " LANGUAGE javascript" +
++                                      " AS 'a + b'");
++
++        String aFunc = createAggregate(KEYSPACE, "int, int",
++                                       " CREATE AGGREGATE %s (int)" +
++                                       " SFUNC " + sFunc +
++                                       " STYPE int" +
++                                       " INITCOND 0");
++
++        String plusOne = createFunction(KEYSPACE, "int",
++                                        " CREATE FUNCTION %s (a int)" +
++                                        " RETURNS NULL ON NULL INPUT" +
++                                        " RETURNS int" +
++                                        " LANGUAGE javascript" +
++                                        " AS 'a+1'");
++
++        String sqFunc = createFunction(KEYSPACE, "int",
++                                       " CREATE FUNCTION %s (a int)" +
++                                       " RETURNS NULL ON NULL INPUT" +
++                                       " RETURNS int" +
++                                       " LANGUAGE javascript" +
++                                       " AS 'a*a'");
++
++        ColumnDefinition v1 = columnDefinition("v1");
++        SelectionColumns expected = SelectionColumnMapping.newMapping()
++                                                          
.addMapping(columnSpecification(aFunc + "(v1)",
++                                                                              
            Int32Type.instance),
++                                                                      v1);
++        verify(expected, String.format("SELECT %s(v1) FROM %%s", aFunc));
++
++        // aggregate with nested udfs as input
++        String specName = String.format("%s(%s(%s(v1)))", aFunc, sqFunc, 
plusOne);
++        expected = 
SelectionColumnMapping.newMapping().addMapping(columnSpecification(specName, 
Int32Type.instance),
++                                                                  v1);
++        verify(expected, String.format("SELECT %s FROM %%s", specName));
 +    }
 +
-     private SelectionColumns extractColumnMappingFromSelect(String query) 
throws RequestValidationException
++    private void verify(SelectionColumns expected, String query) throws 
Throwable
++    {
++        SelectStatement statement = getSelect(query);
++        verifyColumnMapping(expected, statement);
++        checkExecution(statement, expected.getColumnSpecifications());
++    }
++
++    private void checkExecution(SelectStatement statement, 
List<ColumnSpecification> expectedResultColumns)
++    throws RequestExecutionException, RequestValidationException
++    {
++        UntypedResultSet rs = 
UntypedResultSet.create(statement.executeInternal(QueryState.forInternalCalls(),
++                                                                              
  QueryOptions.DEFAULT).result);
++
++        assertEquals(expectedResultColumns, rs.one().getColumns());
++    }
++
++    private SelectStatement getSelect(String query) throws 
RequestValidationException
 +    {
 +        CQLStatement statement = 
QueryProcessor.getStatement(String.format(query, KEYSPACE + "." + tableName),
 +                                                             
ClientState.forInternalCalls()).statement;
 +        assertTrue(statement instanceof SelectStatement);
-         return ((SelectStatement)statement).getSelection().getColumnMapping();
++        return (SelectStatement)statement;
++    }
++
++    private void verifyColumnMapping(SelectionColumns expected, 
SelectStatement select)
++    {
++        assertEquals(expected, select.getSelection().getColumnMapping());
++    }
++
++    private Iterable<ColumnDefinition> columnDefinitions(String...names)
++    {
++        List<ColumnDefinition> defs = new ArrayList<>();
++        for (String n : names)
++            defs.add(columnDefinition(n));
++        return defs;
 +    }
 +
 +    private ColumnDefinition columnDefinition(String name)
 +    {
 +        return Schema.instance.getCFMetaData(KEYSPACE, tableName)
 +                              .getColumnDefinition(new ColumnIdentifier(name, 
true));
 +
 +    }
 +
 +    private ColumnSpecification columnSpecification(String name, 
AbstractType<?> type)
 +    {
 +        return new ColumnSpecification(KEYSPACE,
 +                                       tableName,
 +                                       new ColumnIdentifier(name, true),
 +                                       type);
 +    }
 +}

Reply via email to