cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tylerho...@apache.org
Subject [3/3] git commit: Merge branch 'cassandra-2.1' into trunk
Date Tue, 04 Nov 2014 23:27:05 GMT
Merge branch 'cassandra-2.1' into trunk

Conflicts:
	src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
	src/java/org/apache/cassandra/cql3/Cql.g
	src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
	src/java/org/apache/cassandra/cql3/statements/Selectable.java
	src/java/org/apache/cassandra/cql3/statements/Selection.java


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

Branch: refs/heads/trunk
Commit: 9bf17e15a8c98a457a4855e3bb5d61dfeebf986a
Parents: 254cd85 a446e80
Author: Tyler Hobbs <tyler@datastax.com>
Authored: Tue Nov 4 17:26:11 2014 -0600
Committer: Tyler Hobbs <tyler@datastax.com>
Committed: Tue Nov 4 17:26:11 2014 -0600

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/cql3/ColumnIdentifier.java |  61 ++++++++-
 src/java/org/apache/cassandra/cql3/Cql.g        | 127 ++++++++++---------
 .../cassandra/cql3/MultiColumnRelation.java     |  12 +-
 .../org/apache/cassandra/cql3/Operation.java    |  15 ++-
 .../cassandra/cql3/SingleColumnRelation.java    |  12 +-
 .../cassandra/cql3/selection/RawSelector.java   |   9 +-
 .../cassandra/cql3/selection/Selectable.java    |  60 +++++++++
 .../cassandra/cql3/selection/Selection.java     |   4 +-
 .../cql3/statements/AlterTableStatement.java    |  31 +++--
 .../cql3/statements/CreateIndexStatement.java   |  10 +-
 .../cql3/statements/DeleteStatement.java        |   7 +-
 .../cassandra/cql3/statements/IndexTarget.java  |  30 ++++-
 .../cql3/statements/ModificationStatement.java  |  21 +--
 .../cql3/statements/SelectStatement.java        |  26 ++--
 .../cql3/statements/UpdateStatement.java        |  17 +--
 .../unit/org/apache/cassandra/SchemaLoader.java |   6 +-
 .../cassandra/cql3/ThriftCompatibilityTest.java |  72 +++++++++++
 18 files changed, 382 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index 9b4f29f,c1dcd87..e1454b1
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@@ -24,11 -23,10 +24,13 @@@ import java.nio.ByteBuffer
  import org.apache.cassandra.cache.IMeasurableMemory;
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.ColumnDefinition;
 -import org.apache.cassandra.cql3.statements.Selectable;
 +import org.apache.cassandra.cql3.selection.Selectable;
 +import org.apache.cassandra.cql3.selection.Selector;
 +import org.apache.cassandra.cql3.selection.SimpleSelector;
  import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.db.marshal.CompositeType;
+ import org.apache.cassandra.db.marshal.UTF8Type;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.ObjectSizes;
  import org.apache.cassandra.utils.memory.AbstractAllocator;
@@@ -37,7 -35,7 +39,7 @@@
   * Represents an identifer for a CQL column definition.
   * TODO : should support light-weight mode without text representation for when not interned
   */
- public class ColumnIdentifier extends Selectable implements IMeasurableMemory
 -public class ColumnIdentifier implements Selectable, IMeasurableMemory
++public class ColumnIdentifier extends org.apache.cassandra.cql3.selection.Selectable implements
IMeasurableMemory
  {
      public final ByteBuffer bytes;
      private final String text;
@@@ -107,12 -105,61 +109,69 @@@
          return new ColumnIdentifier(allocator.clone(bytes), text);
      }
  
 +    public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition>
defs) throws InvalidRequestException
 +    {
 +        ColumnDefinition def = cfm.getColumnDefinition(this);
 +        if (def == null)
 +            throw new InvalidRequestException(String.format("Undefined name %s in selection
clause", this));
 +
 +        return SimpleSelector.newFactory(def.name.toString(), addAndGetIndex(def, defs),
def.type);
 +    }
+ 
+     /**
+      * Because Thrift-created tables may have a non-text comparator, we cannot determine
the proper 'key' until
+      * we know the comparator. ColumnIdentifier.Raw is a placeholder that can be converted
to a real ColumnIdentifier
+      * once the comparator is known with prepare(). This should only be used with identifiers
that are actual
+      * column names. See CASSANDRA-8178 for more background.
+      */
+     public static class Raw implements Selectable.Raw
+     {
+         private final String rawText;
+         private final String text;
+ 
+         public Raw(String rawText, boolean keepCase)
+         {
+             this.rawText = rawText;
+             this.text =  keepCase ? rawText : rawText.toLowerCase(Locale.US);
+         }
+ 
+         public ColumnIdentifier prepare(CFMetaData cfm)
+         {
+             AbstractType<?> comparator = cfm.comparator.asAbstractType();
+             if (cfm.getIsDense() || comparator instanceof CompositeType || comparator instanceof
UTF8Type)
+                 return new ColumnIdentifier(text, true);
+ 
+             // We have a Thrift-created table with a non-text comparator.  We need to parse
column names with the comparator
+             // to get the correct ByteBuffer representation.  However, this doesn't apply
to key aliases, so we need to
+             // make a special check for those and treat them normally.  See CASSANDRA-8178.
+             ByteBuffer bufferName = ByteBufferUtil.bytes(text);
+             for (ColumnDefinition def : cfm.partitionKeyColumns())
+             {
+                 if (def.name.bytes.equals(bufferName))
+                     return new ColumnIdentifier(text, true);
+             }
+             return new ColumnIdentifier(comparator.fromString(rawText), text);
+         }
+ 
+         @Override
+         public final int hashCode()
+         {
+             return text.hashCode();
+         }
+ 
+         @Override
+         public final boolean equals(Object o)
+         {
+             if(!(o instanceof ColumnIdentifier.Raw))
+                 return false;
+             ColumnIdentifier.Raw that = (ColumnIdentifier.Raw)o;
+             return text.equals(that.text);
+         }
+ 
+         @Override
+         public String toString()
+         {
+             return text;
+         }
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Cql.g
index 89dde62,b1c598b..adfe7a3
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@@ -263,8 -260,10 +263,8 @@@ useStatement returns [UseStatement stmt
  selectStatement returns [SelectStatement.RawStatement expr]
      @init {
          boolean isDistinct = false;
 -        boolean isCount = false;
 -        ColumnIdentifier countAlias = null;
          Term.Raw limit = null;
-         Map<ColumnIdentifier, Boolean> orderings = new LinkedHashMap<ColumnIdentifier,
Boolean>();
+         Map<ColumnIdentifier.Raw, Boolean> orderings = new LinkedHashMap<ColumnIdentifier.Raw,
Boolean>();
          boolean allowFiltering = false;
      }
      : K_SELECT ( ( K_DISTINCT { isDistinct = true; } )? sclause=selectClause
@@@ -309,13 -310,8 +309,13 @@@ selectionFunctionArgs returns [List<Sel
      ;
  
  selectCountClause returns [List<RawSelector> expr]
 -    : '\*'           { $expr = Collections.<RawSelector>emptyList();}
 -    | i=INTEGER      { if (!i.getText().equals("1")) addRecognitionError("Only COUNT(1)
is supported, got COUNT(" + i.getText() + ")"); $expr = Collections.<RawSelector>emptyList();}
 +    @init{ ColumnIdentifier alias = new ColumnIdentifier("count", false); }
-     : K_COUNT '(' countArgument ')' (K_AS c=cident { alias = c; })? { $expr = new ArrayList<RawSelector>();
$expr.add( new RawSelector(new Selectable.WithFunction(new FunctionName("countRows"), Collections.<Selectable>emptyList()),
alias));}
++    : K_COUNT '(' countArgument ')' (K_AS c=ident { alias = c; })? { $expr = new ArrayList<RawSelector>();
$expr.add( new RawSelector(new Selectable.WithFunction.Raw(new FunctionName("countRows"),
Collections.<Selectable.Raw>emptyList()), alias));}
 +    ;
 +
 +countArgument
 +    : '\*'
 +    | i=INTEGER { if (!i.getText().equals("1")) addRecognitionError("Only COUNT(1) is supported,
got COUNT(" + i.getText() + ")");}
      ;
  
  whereClause returns [List<Relation> clause]
@@@ -323,11 -319,12 +323,11 @@@
      : relation[$clause] (K_AND relation[$clause])*
      ;
  
- orderByClause[Map<ColumnIdentifier, Boolean> orderings]
+ orderByClause[Map<ColumnIdentifier.Raw, Boolean> orderings]
      @init{
 -        ColumnIdentifier.Raw orderBy = null;
          boolean reversed = false;
      }
 -    : c=cident { orderBy = c; } (K_ASC | K_DESC { reversed = true; })? { orderings.put(c,
reversed); }
 +    : c=cident (K_ASC | K_DESC { reversed = true; })? { orderings.put(c, reversed); }
      ;
  
  /**
@@@ -488,53 -485,6 +488,53 @@@ batchStatementObjective returns [Modifi
      | d=deleteStatement  { $statement = d; }
      ;
  
 +createFunctionStatement returns [CreateFunctionStatement expr]
 +    @init {
 +        boolean orReplace = false;
 +        boolean ifNotExists = false;
 +
 +        boolean deterministic = true;
 +        List<ColumnIdentifier> argsNames = new ArrayList<>();
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +    }
 +    : K_CREATE (K_OR K_REPLACE { orReplace = true; })?
 +      ((K_NON { deterministic = false; })? K_DETERMINISTIC)?
 +      K_FUNCTION
 +      (K_IF K_NOT K_EXISTS { ifNotExists = true; })?
 +      fn=functionName
 +      '('
 +        (
-           k=cident v=comparatorType { argsNames.add(k); argsTypes.add(v); }
-           ( ',' k=cident v=comparatorType { argsNames.add(k); argsTypes.add(v); } )*
++          k=ident v=comparatorType { argsNames.add(k); argsTypes.add(v); }
++          ( ',' k=ident v=comparatorType { argsNames.add(k); argsTypes.add(v); } )*
 +        )?
 +      ')'
 +      K_RETURNS rt = comparatorType
 +      K_LANGUAGE language = IDENT
 +      K_AS body = STRING_LITERAL
 +      { $expr = new CreateFunctionStatement(fn, $language.text.toLowerCase(), $body.text,
deterministic, argsNames, argsTypes, rt, orReplace, ifNotExists); }
 +    ;
 +
 +dropFunctionStatement returns [DropFunctionStatement expr]
 +    @init {
 +        boolean ifExists = false;
 +        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
 +        boolean argsPresent = false;
 +    }
 +    : K_DROP K_FUNCTION
 +      (K_IF K_EXISTS { ifExists = true; } )?
 +      fn=functionName
 +      (
 +        '('
 +          (
 +            v=comparatorType { argsTypes.add(v); }
 +            ( ',' v=comparatorType { argsTypes.add(v); } )*
 +          )?
 +        ')'
 +        { argsPresent = true; }
 +      )?
 +      { $expr = new DropFunctionStatement(fn, argsTypes, argsPresent, ifExists); }
 +    ;
 +
  /**
   * CREATE KEYSPACE [IF NOT EXISTS] <KEYSPACE> WITH attr1 = value1 AND attr2 = value2;
   */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/RawSelector.java
index 372c4b3,0000000..c7e2658
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,55 -1,0 +1,56 @@@
 +/*
 + * 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 selectable;
++    public final Selectable.Raw selectable;
 +    public final ColumnIdentifier alias;
 +
-     public RawSelector(Selectable selectable, 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)
++    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;
++                return raw.selectable.prepare(cfm);
 +            }
 +        });
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selectable.java
index 9324647,0000000..48ce11a
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,166 -1,0 +1,226 @@@
 +/*
 + * 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);
++    }
++
 +    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 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 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));
++            }
++        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/selection/Selection.java
index cd5e2a8,0000000..17e2e92
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,390 -1,0 +1,390 @@@
 +/*
 + * 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;
 +    }
 +
 +    private static boolean isUsingFunction(List<RawSelector> rawSelectors)
 +    {
 +        for (RawSelector rawSelector : rawSelectors)
 +        {
 +            if (!(rawSelector.selectable instanceof ColumnIdentifier))
 +                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, defs);
++                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);
 +    }
 +
 +    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 msgTemplate the error message template
++     * @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;
 +            }
 +            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 final SelectorFactories factories;
 +
 +        public SelectionWithFunctions(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 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/9bf17e15/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index de9da86,2632ee2..686f50f
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -101,7 -98,7 +101,7 @@@ public class SelectStatement implement
      private boolean selectsOnlyStaticColumns;
  
      // Used by forSelection below
-     private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier,
Boolean>emptyMap(), false, false);
 -    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier.Raw,
Boolean>emptyMap(), false, false, null, false);
++    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier.Raw,
Boolean>emptyMap(), false, false);
  
      private static final Predicate<ColumnDefinition> isStaticFilter = new Predicate<ColumnDefinition>()
      {
@@@ -2120,12 -2166,16 +2122,12 @@@
  
      public static class Parameters
      {
-         private final Map<ColumnIdentifier, Boolean> orderings;
+         private final Map<ColumnIdentifier.Raw, Boolean> orderings;
          private final boolean isDistinct;
 -        private final boolean isCount;
 -        private final ColumnIdentifier countAlias;
          private final boolean allowFiltering;
  
-         public Parameters(Map<ColumnIdentifier, Boolean> orderings,
+         public Parameters(Map<ColumnIdentifier.Raw, Boolean> orderings,
                            boolean isDistinct,
 -                          boolean isCount,
 -                          ColumnIdentifier countAlias,
                            boolean allowFiltering)
          {
              this.orderings = orderings;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/SchemaLoader.java
index bb862e3,4a1c104..2fcfd55
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@@ -299,33 -309,10 +299,33 @@@ public class SchemaLoade
          if (Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false")))
              useCompression(schema);
  
 -        return schema;
 +        // if you're messing with low-level sstable stuff, it can be useful to inject the
schema directly
 +        // Schema.instance.load(schemaDefinition());
 +        for (KSMetaData ksm : schema)
 +            MigrationManager.announceNewKeyspace(ksm, false);
 +    }
 +
 +    public static void createKeyspace(String keyspaceName,
 +                                      Class<? extends AbstractReplicationStrategy>
strategy,
 +                                      Map<String, String> options,
 +                                      CFMetaData... cfmetas) throws ConfigurationException
 +    {
 +        createKeyspace(keyspaceName, true, true, strategy, options, cfmetas);
 +    }
 +
 +    public static void createKeyspace(String keyspaceName,
 +                                      boolean durable,
 +                                      boolean announceLocally,
 +                                      Class<? extends AbstractReplicationStrategy>
strategy,
 +                                      Map<String, String> options,
 +                                      CFMetaData... cfmetas) throws ConfigurationException
 +    {
 +        KSMetaData ksm = durable ? KSMetaData.testMetadata(keyspaceName, strategy, options,
cfmetas)
 +                                 : KSMetaData.testMetadataNotDurable(keyspaceName, strategy,
options, cfmetas);
 +        MigrationManager.announceNewKeyspace(ksm, announceLocally);
      }
  
--    private static ColumnDefinition integerColumn(String ksName, String cfName)
++    public static ColumnDefinition integerColumn(String ksName, String cfName)
      {
          return new ColumnDefinition(ksName,
                                      cfName,
@@@ -412,7 -399,7 +412,7 @@@
          return CFMetaData.denseCFMetaData(ksName, cfName, comp).defaultValidator(comp);
      }
  
--    private static CFMetaData jdbcSparseCFMD(String ksName, String cfName, AbstractType
comp)
++    public static CFMetaData jdbcSparseCFMD(String ksName, String cfName, AbstractType comp)
      {
          return CFMetaData.sparseCFMetaData(ksName, cfName, comp).defaultValidator(comp);
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9bf17e15/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
index 0000000,662800b..deb3082
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
@@@ -1,0 -1,57 +1,72 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.cassandra.cql3;
+ 
++import org.apache.cassandra.config.KSMetaData;
++import org.apache.cassandra.db.marshal.Int32Type;
++import org.apache.cassandra.locator.SimpleStrategy;
+ import org.apache.cassandra.utils.ByteBufferUtil;
++import org.junit.BeforeClass;
+ import org.junit.Test;
+ 
+ import org.apache.cassandra.SchemaLoader;
+ 
+ import static org.junit.Assert.assertEquals;
+ 
+ public class ThriftCompatibilityTest extends SchemaLoader
+ {
++    @BeforeClass
++    public static void defineSchema() throws Exception
++    {
++        SchemaLoader.prepareServer();
++        SchemaLoader.createKeyspace("thriftcompat",
++                                    SimpleStrategy.class,
++                                    KSMetaData.optsWithRF(1),
++                                    jdbcSparseCFMD("thriftcompat", "JdbcInteger", Int32Type.instance)
++                                            .addColumnDefinition(integerColumn("thriftcompat",
"JdbcInteger")));
++    }
++
+     private static UntypedResultSet execute(String query) throws Throwable
+     {
+         try
+         {
+             return QueryProcessor.executeInternal(String.format(query));
+         }
+         catch (RuntimeException exc)
+         {
+             if (exc.getCause() != null)
+                 throw exc.getCause();
+             throw exc;
+         }
+     }
+ 
+     /** Test For CASSANDRA-8178 */
+     @Test
+     public void testNonTextComparator() throws Throwable
+     {
+         // the comparator is IntegerType, and there is a column named 42 with a UTF8Type
validation type
 -        execute("INSERT INTO \"Keyspace1\".\"JdbcInteger\" (key, \"42\") VALUES (0x00000001,
'abc')");
 -        execute("UPDATE \"Keyspace1\".\"JdbcInteger\" SET \"42\" = 'abc' WHERE key = 0x00000001");
 -        execute("DELETE \"42\" FROM \"Keyspace1\".\"JdbcInteger\" WHERE key = 0x00000000");
 -        UntypedResultSet results = execute("SELECT key, \"42\" FROM \"Keyspace1\".\"JdbcInteger\"");
++        execute("INSERT INTO \"thriftcompat\".\"JdbcInteger\" (key, \"42\") VALUES (0x00000001,
'abc')");
++        execute("UPDATE \"thriftcompat\".\"JdbcInteger\" SET \"42\" = 'abc' WHERE key =
0x00000001");
++        execute("DELETE \"42\" FROM \"thriftcompat\".\"JdbcInteger\" WHERE key = 0x00000000");
++        UntypedResultSet results = execute("SELECT key, \"42\" FROM \"thriftcompat\".\"JdbcInteger\"");
+         assertEquals(1, results.size());
+         UntypedResultSet.Row row = results.iterator().next();
+         assertEquals(ByteBufferUtil.bytes(1), row.getBytes("key"));
+         assertEquals("abc", row.getString("42"));
+     }
+ }


Mime
View raw message