Merge branch 'cassandra-2.1' into trunk

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

Branch: refs/heads/trunk
Commit: b8cf17284e55eb98fea61be672a0bcadb3613754
Parents: 3cc9a0c 289314a
Author: Tyler Hobbs <ty...@datastax.com>
Authored: Wed Nov 19 11:11:37 2014 -0600
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Wed Nov 19 11:11:37 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/cql3/ColumnIdentifier.java |   5 +
 .../cassandra/cql3/selection/RawSelector.java   |   5 +
 .../cassandra/cql3/selection/Selectable.java    |  20 ++
 .../cassandra/cql3/selection/Selection.java     |  24 +-
 .../cql3/selection/SelectorFactories.java       |  10 +
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |   7 +
 .../cassandra/cql3/SelectionOrderingTest.java   | 233 +++++++++++++++++++
 9 files changed, 298 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b8cf1728/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b8cf1728/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b8cf1728/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/RawSelector.java
index c7e2658,0000000..7d5543f
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
@@@ -1,56 -1,0 +1,61 @@@
 +/*
 + * 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 org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +
 +import com.google.common.base.Function;
 +import com.google.common.collect.Lists;
 +
 +public class RawSelector
 +{
 +    public final Selectable.Raw selectable;
 +    public final ColumnIdentifier alias;
 +
 +    public RawSelector(Selectable.Raw selectable, ColumnIdentifier alias)
 +    {
 +        this.selectable = selectable;
 +        this.alias = alias;
 +    }
 +
 +    /**
 +     * Converts the specified list of <code>RawSelector</code>s into a list 
of <code>Selectable</code>s.
 +     *
 +     * @param raws the <code>RawSelector</code>s to converts.
 +     * @return a list of <code>Selectable</code>s
 +     */
 +    public static List<Selectable> toSelectables(List<RawSelector> raws, 
final CFMetaData cfm)
 +    {
 +        return Lists.transform(raws, new Function<RawSelector, Selectable>()
 +        {
 +            public Selectable apply(RawSelector raw)
 +            {
 +                return raw.selectable.prepare(cfm);
 +            }
 +        });
 +    }
++
++    public boolean processesSelection()
++    {
++        return selectable.processesSelection();
++    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b8cf1728/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selectable.java
index 48ce11a,0000000..c5ef857
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@@ -1,226 -1,0 +1,246 @@@
 +/*
 + * 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.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.cql3.functions.Function;
 +import org.apache.cassandra.cql3.functions.FunctionName;
 +import org.apache.cassandra.cql3.functions.Functions;
 +import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.UserType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +import org.apache.commons.lang3.text.StrBuilder;
 +
 +public abstract class Selectable
 +{
 +    public abstract Selector.Factory newSelectorFactory(CFMetaData cfm, 
List<ColumnDefinition> defs)
 +            throws InvalidRequestException;
 +
 +    protected static int addAndGetIndex(ColumnDefinition def, 
List<ColumnDefinition> l)
 +    {
 +        int idx = l.indexOf(def);
 +        if (idx < 0)
 +        {
 +            idx = l.size();
 +            l.add(def);
 +        }
 +        return idx;
 +    }
 +
 +    public static interface Raw
 +    {
 +        public Selectable prepare(CFMetaData cfm);
++
++        /**
++         * Returns true if any processing is performed on the selected column.
++         **/
++        public boolean processesSelection();
 +    }
 +
 +    public static class WritetimeOrTTL extends Selectable
 +    {
 +        public final ColumnIdentifier id;
 +        public final boolean isWritetime;
 +
 +        public WritetimeOrTTL(ColumnIdentifier id, boolean isWritetime)
 +        {
 +            this.id = id;
 +            this.isWritetime = isWritetime;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return (isWritetime ? "writetime" : "ttl") + "(" + id + ")";
 +        }
 +
 +        public Selector.Factory newSelectorFactory(CFMetaData cfm,
 +                                                   List<ColumnDefinition> 
defs) throws InvalidRequestException
 +        {
 +            ColumnDefinition def = cfm.getColumnDefinition(id);
 +            if (def == null)
 +                throw new InvalidRequestException(String.format("Undefined 
name %s in selection clause", id));
 +            if (def.isPrimaryKeyColumn())
 +                throw new InvalidRequestException(
 +                        String.format("Cannot use selection function %s on 
PRIMARY KEY part %s",
 +                                      isWritetime ? "writeTime" : "ttl",
 +                                      def.name));
 +            if (def.type.isCollection())
 +                throw new InvalidRequestException(String.format("Cannot use 
selection function %s on collections",
 +                                                                isWritetime ? 
"writeTime" : "ttl"));
 +
 +            return WritetimeOrTTLSelector.newFactory(def.name.toString(), 
addAndGetIndex(def, defs), isWritetime);
 +        }
 +
 +        public static class Raw implements Selectable.Raw
 +        {
 +            private final ColumnIdentifier.Raw id;
 +            private final boolean isWritetime;
 +
 +            public Raw(ColumnIdentifier.Raw id, boolean isWritetime)
 +            {
 +                this.id = id;
 +                this.isWritetime = isWritetime;
 +            }
 +
 +            public WritetimeOrTTL prepare(CFMetaData cfm)
 +            {
 +                return new WritetimeOrTTL(id.prepare(cfm), isWritetime);
 +            }
++
++            public boolean processesSelection()
++            {
++                return true;
++            }
 +        }
 +    }
 +
 +    public static class WithFunction extends Selectable
 +    {
 +        public final FunctionName functionName;
 +        public final List<Selectable> args;
 +
 +        public WithFunction(FunctionName functionName, List<Selectable> args)
 +        {
 +            this.functionName = functionName;
 +            this.args = args;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return new StrBuilder().append(functionName)
 +                                   .append("(")
 +                                   .appendWithSeparators(args, ", ")
 +                                   .append(")")
 +                                   .toString();
 +        }
 +
 +        public Selector.Factory newSelectorFactory(CFMetaData cfm,
 +                                                   List<ColumnDefinition> 
defs) throws InvalidRequestException
 +        {
 +            SelectorFactories factories  =
 +                    
SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, cfm, defs);
 +
 +            // resolve built-in functions before user defined functions
 +            Function fun = Functions.get(cfm.ksName, functionName, 
factories.newInstances(), cfm.ksName, cfm.cfName);
 +            if (fun == null)
 +                throw new InvalidRequestException(String.format("Unknown 
function '%s'", functionName));
 +            if (fun.returnType() == null)
 +                throw new InvalidRequestException(String.format("Unknown 
function %s called in selection clause",
 +                                                                
functionName));
 +
 +            return AbstractFunctionSelector.newFactory(fun, factories);
 +        }
 +
 +        public static class Raw implements Selectable.Raw
 +        {
 +            private final FunctionName functionName;
 +            private final List<Selectable.Raw> args;
 +
 +            public Raw(FunctionName functionName, List<Selectable.Raw> args)
 +            {
 +                this.functionName = functionName;
 +                this.args = args;
 +            }
 +
 +            public WithFunction prepare(CFMetaData cfm)
 +            {
 +                List<Selectable> preparedArgs = new ArrayList<>(args.size());
 +                for (Selectable.Raw arg : args)
 +                    preparedArgs.add(arg.prepare(cfm));
 +                return new WithFunction(functionName, preparedArgs);
 +            }
++
++            public boolean processesSelection()
++            {
++                return true;
++            }
 +        }
 +    }
 +
 +    public static class WithFieldSelection extends Selectable
 +    {
 +        public final Selectable selected;
 +        public final ColumnIdentifier field;
 +
 +        public WithFieldSelection(Selectable selected, ColumnIdentifier field)
 +        {
 +            this.selected = selected;
 +            this.field = field;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("%s.%s", selected, field);
 +        }
 +
 +        public Selector.Factory newSelectorFactory(CFMetaData cfm,
 +                                                   List<ColumnDefinition> 
defs) throws InvalidRequestException
 +        {
 +            Selector.Factory factory = selected.newSelectorFactory(cfm, defs);
 +            AbstractType<?> type = factory.newInstance().getType();
 +            if (!(type instanceof UserType))
 +                throw new InvalidRequestException(
 +                        String.format("Invalid field selection: %s of type %s 
is not a user type",
 +                                      selected,
 +                                      type.asCQL3Type()));
 +
 +            UserType ut = (UserType) type;
 +            for (int i = 0; i < ut.size(); i++)
 +            {
 +                if (!ut.fieldName(i).equals(field.bytes))
 +                    continue;
 +                return FieldSelector.newFactory(ut, i, factory);
 +            }
 +            throw new InvalidRequestException(String.format("%s of type %s 
has no field %s",
 +                                                            selected,
 +                                                            type.asCQL3Type(),
 +                                                            field));
 +        }
 +
 +        public static class Raw implements Selectable.Raw
 +        {
 +            private final Selectable.Raw selected;
 +            private final ColumnIdentifier.Raw field;
 +
 +            public Raw(Selectable.Raw selected, ColumnIdentifier.Raw field)
 +            {
 +                this.selected = selected;
 +                this.field = field;
 +            }
 +
 +            public WithFieldSelection prepare(CFMetaData cfm)
 +            {
 +                return new WithFieldSelection(selected.prepare(cfm), 
field.prepare(cfm));
 +            }
++
++            public boolean processesSelection()
++            {
++                return true;
++            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b8cf1728/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selection.java
index 7c7dab7,0000000..888d96d
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,405 -1,0 +1,413 @@@
 +/*
 + * 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.Collection;
 +import java.util.Iterator;
 +import java.util.List;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.ColumnIdentifier;
 +import org.apache.cassandra.cql3.ColumnSpecification;
 +import org.apache.cassandra.cql3.ResultSet;
 +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.exceptions.InvalidRequestException;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +
 +import com.google.common.collect.Iterators;
 +
 +public abstract class Selection
 +{
 +    private final Collection<ColumnDefinition> columns;
 +    private final ResultSet.Metadata metadata;
 +    private final boolean collectTimestamps;
 +    private final boolean collectTTLs;
 +
 +    protected Selection(Collection<ColumnDefinition> columns, 
List<ColumnSpecification> metadata, boolean collectTimestamps, boolean 
collectTTLs)
 +    {
 +        this.columns = columns;
 +        this.metadata = new ResultSet.Metadata(metadata);
 +        this.collectTimestamps = collectTimestamps;
 +        this.collectTTLs = collectTTLs;
 +    }
 +
 +    // Overriden by SimpleSelection when appropriate.
 +    public boolean isWildcard()
 +    {
 +        return false;
 +    }
 +
 +    public ResultSet.Metadata getResultMetadata()
 +    {
 +        return metadata;
 +    }
 +
 +    public static Selection wildcard(CFMetaData cfm)
 +    {
 +        List<ColumnDefinition> all = new 
ArrayList<ColumnDefinition>(cfm.allColumns().size());
 +        Iterators.addAll(all, cfm.allColumnsInSelectOrder());
 +        return new SimpleSelection(all, true);
 +    }
 +
 +    public static Selection forColumns(Collection<ColumnDefinition> columns)
 +    {
 +        return new SimpleSelection(columns, false);
 +    }
 +
 +    public int addColumnForOrdering(ColumnDefinition c)
 +    {
 +        columns.add(c);
 +        metadata.addNonSerializedColumn(c);
 +        return columns.size() - 1;
 +    }
 +
 +    public boolean usesFunction(String ksName, String functionName)
 +    {
 +        return false;
 +    }
 +
-     private static boolean isUsingFunction(List<RawSelector> rawSelectors)
++    private static boolean processesSelection(List<RawSelector> rawSelectors)
 +    {
 +        for (RawSelector rawSelector : rawSelectors)
 +        {
-             if (!(rawSelector.selectable instanceof ColumnIdentifier))
++            if (rawSelector.processesSelection())
 +                return true;
 +        }
 +        return false;
 +    }
 +
 +    public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> 
rawSelectors) throws InvalidRequestException
 +    {
 +        List<ColumnDefinition> defs = new ArrayList<ColumnDefinition>();
 +
 +        SelectorFactories factories =
 +                
SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors,
 cfm), cfm, defs);
 +        List<ColumnSpecification> metadata = collectMetadata(cfm, 
rawSelectors, factories);
 +
-         return isUsingFunction(rawSelectors) ? new 
SelectionWithFunctions(defs, metadata, factories)
-                                              : new SimpleSelection(defs, 
metadata, false);
++        return processesSelection(rawSelectors) ? new 
SelectionWithProcessing(defs, metadata, factories)
++                                                : new SimpleSelection(defs, 
metadata, false);
 +    }
 +
 +    private static List<ColumnSpecification> collectMetadata(CFMetaData cfm,
 +                                                             
List<RawSelector> rawSelectors,
 +                                                             
SelectorFactories factories)
 +    {
 +        List<ColumnSpecification> metadata = new 
ArrayList<ColumnSpecification>(rawSelectors.size());
 +        Iterator<RawSelector> iter = rawSelectors.iterator();
 +        for (Selector.Factory factory : factories)
 +        {
 +            ColumnSpecification colSpec = factory.getColumnSpecification(cfm);
 +            ColumnIdentifier alias = iter.next().alias;
 +            metadata.add(alias == null ? colSpec : colSpec.withAlias(alias));
 +        }
 +        return metadata;
 +    }
 +
 +    protected abstract Selectors newSelectors();
 +
 +    /**
 +     * @return the list of CQL3 columns value this SelectionClause needs.
 +     */
 +    public Collection<ColumnDefinition> getColumns()
 +    {
 +        return columns;
 +    }
 +
 +    public ResultSetBuilder resultSetBuilder(long now)
 +    {
 +        return new ResultSetBuilder(now);
 +    }
 +
 +    public abstract boolean isAggregate();
 +
 +    /**
 +     * Checks that selectors are either all aggregates or that none of them 
is.
 +     *
 +     * @param selectors the selectors to test.
 +     * @param messageTemplate the error message template
 +     * @param messageArgs the error message arguments
 +     * @throws InvalidRequestException if some of the selectors are aggregate 
but not all of them
 +     */
 +    static void validateSelectors(List<Selector> selectors, String 
messageTemplate, Object... messageArgs)
 +            throws InvalidRequestException
 +    {
 +        int aggregates = 0;
 +        for (Selector s : selectors)
 +            if (s.isAggregate())
 +                ++aggregates;
 +
 +        if (aggregates != 0 && aggregates != selectors.size())
 +            throw new InvalidRequestException(String.format(messageTemplate, 
messageArgs));
 +    }
 +
 +    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 ResultSetBuilder(long now)
 +        {
 +            this.resultSet = new ResultSet(getResultMetadata().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;
 +        }
 +
 +        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() throws InvalidRequestException
 +        {
 +            if (current != null)
 +            {
 +                selectors.addInputRow(this);
 +                if (!selectors.isAggregate())
 +                {
 +                    resultSet.addRow(selectors.getOutputRow());
 +                    selectors.reset();
 +                }
 +            }
 +            current = new ArrayList<ByteBuffer>(columns.size());
 +        }
 +
 +        public ResultSet build() throws InvalidRequestException
 +        {
 +            if (current != null)
 +            {
 +                selectors.addInputRow(this);
 +                resultSet.addRow(selectors.getOutputRow());
 +                selectors.reset();
 +                current = null;
 +            }
 +
 +            if (resultSet.isEmpty() && selectors.isAggregate())
 +            {
 +                resultSet.addRow(selectors.getOutputRow());
 +            }
 +            return resultSet;
 +        }
 +
 +        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(ResultSetBuilder rs) throws 
InvalidRequestException;
 +
 +        public List<ByteBuffer> getOutputRow() 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(Collection<ColumnDefinition> columns, boolean 
isWildcard)
 +        {
 +            this(columns, new ArrayList<ColumnSpecification>(columns), 
isWildcard);
 +        }
 +
 +        public SimpleSelection(Collection<ColumnDefinition> columns, 
List<ColumnSpecification> metadata, boolean isWildcard)
 +        {
 +            /*
 +             * In theory, even a simple selection could have multiple time 
the same column, so we
 +             * could filter those duplicate out of columns. But since we're 
very unlikely to
 +             * get much duplicate in practice, it's more efficient not to 
bother.
 +             */
 +            super(columns, metadata, false, false);
 +            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()
 +                {
 +                    return current;
 +                }
 +
 +                public void addInputRow(ResultSetBuilder rs) throws 
InvalidRequestException
 +                {
 +                    current = rs.current;
 +                }
 +
 +                public boolean isAggregate()
 +                {
 +                    return false;
 +                }
 +            };
 +        }
 +    }
 +
-     private static class SelectionWithFunctions extends Selection
++    private static class SelectionWithProcessing extends Selection
 +    {
 +        private final SelectorFactories factories;
 +
-         public SelectionWithFunctions(Collection<ColumnDefinition> columns,
-                                       List<ColumnSpecification> metadata,
-                                       SelectorFactories factories) throws 
InvalidRequestException
++        public SelectionWithProcessing(Collection<ColumnDefinition> columns,
++                                       List<ColumnSpecification> metadata,
++                                       SelectorFactories factories) throws 
InvalidRequestException
 +        {
 +            super(columns, metadata, 
factories.containsWritetimeSelectorFactory(), 
factories.containsTTLSelectorFactory());
 +            this.factories = factories;
 +
 +            if (factories.doesAggregation() && 
!factories.containsOnlyAggregateFunctions())
 +                throw new InvalidRequestException("the select clause must 
either contains only aggregates or none");
 +        }
 +
 +        public boolean usesFunction(String ksName, String functionName)
 +        {
 +            return factories.usesFunction(ksName, functionName);
 +        }
 +
++        @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()
 +        {
 +            return new Selectors()
 +            {
 +                private final List<Selector> selectors = 
factories.newInstances();
 +
 +                public void reset()
 +                {
 +                    for (int i = 0, m = selectors.size(); i < m; i++)
 +                    {
 +                        selectors.get(i).reset();
 +                    }
 +                }
 +
 +                public boolean isAggregate()
 +                {
 +                    return factories.containsOnlyAggregateFunctions();
 +                }
 +
 +                public List<ByteBuffer> getOutputRow() throws 
InvalidRequestException
 +                {
 +                    List<ByteBuffer> outputRow = new 
ArrayList<>(selectors.size());
 +
 +                    for (int i = 0, m = selectors.size(); i < m; i++)
 +                    {
 +                        outputRow.add(selectors.get(i).getOutput());
 +                    }
 +                    return outputRow;
 +                }
 +
 +                public void addInputRow(ResultSetBuilder rs) throws 
InvalidRequestException
 +                {
 +                    for (int i = 0, m = selectors.size(); i < m; i++)
 +                    {
 +                        selectors.get(i).addInput(rs);
 +                    }
 +                }
 +            };
 +        }
 +
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b8cf1728/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
index 4d3e974,0000000..9f6025c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
@@@ -1,179 -1,0 +1,189 @@@
 +/*
 + * 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.ArrayList;
 +import java.util.Iterator;
 +import java.util.List;
 +
 +import com.google.common.base.Function;
 +import com.google.common.collect.Lists;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.cql3.selection.Selector.Factory;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
 +
 +/**
 + * A set of <code>Selector</code> factories.
 + */
 +final class SelectorFactories implements Iterable<Selector.Factory>
 +{
 +    /**
 +     * The <code>Selector</code> factories.
 +     */
 +    private final List<Selector.Factory> factories;
 +
 +    /**
 +     * <code>true</code> if one of the factory creates writetime selectors.
 +     */
 +    private boolean containsWritetimeFactory;
 +
 +    /**
 +     * <code>true</code> if one of the factory creates TTL selectors.
 +     */
 +    private boolean containsTTLFactory;
 +
 +    /**
 +     * The number of factories creating aggregates.
 +     */
 +    private int numberOfAggregateFactories;
 +
 +    /**
 +     * Creates a new <code>SelectorFactories</code> instance and collect the 
column definitions.
 +     *
 +     * @param selectables the <code>Selectable</code>s for which the 
factories must be created
 +     * @param cfm the Column Family Definition
 +     * @param defs the collector parameter for the column definitions
 +     * @return a new <code>SelectorFactories</code> instance
 +     * @throws InvalidRequestException if a problem occurs while creating the 
factories
 +     */
 +    public static SelectorFactories 
createFactoriesAndCollectColumnDefinitions(List<Selectable> selectables,
 +                                                                              
 CFMetaData cfm,
 +                                                                              
 List<ColumnDefinition> defs)
 +                                                                              
 throws InvalidRequestException
 +    {
 +        return new SelectorFactories(selectables, cfm, defs);
 +    }
 +
 +    private SelectorFactories(List<Selectable> selectables,
 +                              CFMetaData cfm,
 +                              List<ColumnDefinition> defs)
 +                              throws InvalidRequestException
 +    {
 +        factories = new ArrayList<>(selectables.size());
 +
 +        for (Selectable selectable : selectables)
 +        {
 +            Factory factory = selectable.newSelectorFactory(cfm, defs);
 +            containsWritetimeFactory |= factory.isWritetimeSelectorFactory();
 +            containsTTLFactory |= factory.isTTLSelectorFactory();
 +            if (factory.isAggregateSelectorFactory())
 +                ++numberOfAggregateFactories;
 +            factories.add(factory);
 +        }
 +    }
 +
 +    public boolean usesFunction(String ksName, String functionName)
 +    {
 +        for (Factory factory : factories)
 +            if (factory != null && factory.usesFunction(ksName, functionName))
 +                return true;
 +        return false;
 +    }
 +
 +    /**
++     * Adds a new <code>Selector.Factory</code> for a column that is needed 
only for ORDER BY purposes.
++     * @param def the column that is needed for ordering
++     * @param index the index of the column definition in the Selection's 
list of columns
++     */
++    public void addSelectorForOrdering(ColumnDefinition def, int index)
++    {
++        factories.add(SimpleSelector.newFactory(def.name.toString(), index, 
def.type));
++    }
++
++    /**
 +     * Checks if this <code>SelectorFactories</code> contains only factories 
for aggregates.
 +     *
 +     * @return <code>true</code> if this <code>SelectorFactories</code> 
contains only factories for aggregates,
 +     * <code>false</code> otherwise.
 +     */
 +    public boolean containsOnlyAggregateFunctions()
 +    {
 +        int size = factories.size();
 +        return  size != 0 && numberOfAggregateFactories == size;
 +    }
 +
 +    /**
 +     * Whether the selector built by this factory does aggregation or not 
(either directly or in a sub-selector).
 +     *
 +     * @return <code>true</code> if the selector built by this factor does 
aggregation, <code>false</code> otherwise.
 +     */
 +    public boolean doesAggregation()
 +    {
 +        return numberOfAggregateFactories > 0;
 +    }
 +
 +    /**
 +     * Checks if this <code>SelectorFactories</code> contains at least one 
factory for writetime selectors.
 +     *
 +     * @return <code>true</code> if this <code>SelectorFactories</code> 
contains at least one factory for writetime
 +     * selectors, <code>false</code> otherwise.
 +     */
 +    public boolean containsWritetimeSelectorFactory()
 +    {
 +        return containsWritetimeFactory;
 +    }
 +
 +    /**
 +     * Checks if this <code>SelectorFactories</code> contains at least one 
factory for TTL selectors.
 +     *
 +     * @return <code>true</code> if this <code>SelectorFactories</code> 
contains at least one factory for TTL
 +     * selectors, <code>false</code> otherwise.
 +     */
 +    public boolean containsTTLSelectorFactory()
 +    {
 +        return containsTTLFactory;
 +    }
 +
 +    /**
 +     * Creates a list of new <code>Selector</code> instances.
 +     * @return a list of new <code>Selector</code> instances.
 +     */
 +    public List<Selector> newInstances()
 +    {
 +        List<Selector> selectors = new ArrayList<>(factories.size());
 +        for (Selector.Factory factory : factories)
 +        {
 +            selectors.add(factory.newInstance());
 +        }
 +        return selectors;
 +    }
 +
 +    public Iterator<Factory> iterator()
 +    {
 +        return factories.iterator();
 +    }
 +
 +    /**
 +     * Returns the names of the columns corresponding to the output values of 
the selector instances created by
 +     * these factories.
 +     *
 +     * @return a list of column names
 +     */
 +    public List<String> getColumnNames()
 +    {
 +        return Lists.transform(factories, new Function<Selector.Factory, 
String>()
 +        {
 +            public String apply(Selector.Factory factory)
 +            {
 +                return factory.getColumnName();
 +            }
 +        });
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b8cf1728/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b8cf1728/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------

Reply via email to