You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2013/06/18 16:29:36 UTC

[1/2] git commit: Updated CREATE CUSTOM INDEX syntax

Updated Branches:
  refs/heads/trunk 26018be22 -> 7edd0e0c7


Updated CREATE CUSTOM INDEX syntax

patch by Aleksey Yeschenko; reviewed by Sylvain Lebresne for
CASSANDRA-5639


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

Branch: refs/heads/trunk
Commit: 2397bc8c334142ddaa6ef8e34f18bbecffba4f4f
Parents: e5c34d7
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Jun 18 17:20:59 2013 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Jun 18 17:20:59 2013 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  3 +
 bin/cqlsh                                       |  2 +-
 doc/cql3/CQL.textile                            | 10 ++-
 pylib/cqlshlib/cql3handling.py                  |  2 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |  8 +--
 .../apache/cassandra/cql3/IndexPropDefs.java    | 68 --------------------
 .../apache/cassandra/cql3/QueryProcessor.java   |  2 +-
 .../cql3/statements/CreateIndexStatement.java   | 23 +++----
 9 files changed, 26 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/2397bc8c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 65f66de..0d42c13 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -27,6 +27,7 @@
  * don't throw away initial causes exceptions for internode encryption issues (CASSANDRA-5644)
  * Fix message spelling errors for cql select statements (CASSANDRA-5647)
  * Suppress custom exceptions thru jmx (CASSANDRA-5652)
+ * Update CREATE CUSTOM INDEX syntax (CASSANDRA-5639)
 Merged from 1.1:
  * Remove buggy thrift max message length option (CASSANDRA-5529)
  * Fix NPE in Pig's widerow mode (CASSANDRA-5488)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2397bc8c/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 099e366..5cb06da 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -17,6 +17,9 @@ Upgrading
       proportional to the number of nodes in the cluster (see
       https://issues.apache.org/jira/browse/CASSANDRA-5272).
 
+    - CQL3 syntax for CREATE CUSTOM INDEX has been updated. See CQL3
+      documentation for details.
+
 
 1.2.5
 =====

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2397bc8c/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index dd4c00d..70b70f5 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -32,7 +32,7 @@ exit 1
 from __future__ import with_statement
 
 description = "CQL Shell for Apache Cassandra"
-version = "3.1.1"
+version = "3.1.2"
 
 from StringIO import StringIO
 from itertools import groupby

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2397bc8c/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 13b6f70..5fa36ab 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1,6 +1,6 @@
 <link rel="StyleSheet" href="CQL.css" type="text/css" media="screen">
 
-h1. Cassandra Query Language (CQL) v3.0.3
+h1. Cassandra Query Language (CQL) v3.0.4
 
 
  <span id="tableOfContents">
@@ -392,14 +392,14 @@ h3(#createIndexStmt). CREATE INDEX
 __Syntax:__
 
 bc(syntax). <create-index-stmt> ::= CREATE ( CUSTOM )? INDEX <identifier>? ON <tablename> '(' <identifier> ')'
-                                        ( WITH <properties> )?
+                                        ( USING <string> )?
 
 __Sample:__
 
 bc(sample). 
 CREATE INDEX userIndex ON NerdMovies (user);
 CREATE INDEX ON Mutants (abilityId);
-CREATE CUSTOM INDEX ON users (email) WITH options = {'class': 'path.to.the.IndexClass'};
+CREATE CUSTOM INDEX ON users (email) USING 'path.to.the.IndexClass';
 
 The @CREATE INDEX@ statement is used to create a new (automatic) secondary index for a given (existing) column in a given table. A name for the index itself can be specified before the @ON@ keyword, if desired. If data already exists for the column, it will be indexed during the execution of this statement. After the index is created, new data for the column is indexed automatically at insertion time.
 
@@ -1048,6 +1048,10 @@ h2(#changes). Changes
 
 The following describes the addition/changes brought for each version of CQL.
 
+h3. 3.0.4
+
+* Updated the syntax for custom "secondary indexes":#createIndexStmt.
+
 h3. 3.0.3
 
 * Support for custom "secondary indexes":#createIndexStmt has been added.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2397bc8c/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 5f4db2c..958b9ae 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -1204,7 +1204,7 @@ def create_cf_composite_primary_key_comma_completer(ctxt, cass):
 syntax_rules += r'''
 <createIndexStatement> ::= "CREATE" "CUSTOM"? "INDEX" indexname=<identifier>? "ON"
                                cf=<columnFamilyName> "(" col=<cident> ")"
-                               ( "WITH" "options = {'class': " <stringLiteral> "}" )?
+                               ( "USING" <stringLiteral> )?
                          ;
 '''
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2397bc8c/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 34e88a1..3ee30d1 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -451,16 +451,16 @@ cfamOrdering[CreateColumnFamilyStatement.RawStatement expr]
     ;
 
 /**
- * CREATE INDEX [indexName] ON columnFamily (columnName);
+ * CREATE INDEX [indexName] ON <columnFamily> (<columnName>);
+ * CREATE CUSTOM INDEX [indexName] ON <columnFamily> (<columnName>) USING <indexClass>;
  */
 createIndexStatement returns [CreateIndexStatement expr]
     @init {
         boolean isCustom = false;
-        IndexPropDefs props = new IndexPropDefs();
     }
     : K_CREATE (K_CUSTOM { isCustom = true; })? K_INDEX (idxName=IDENT)? K_ON cf=columnFamilyName '(' id=cident ')'
-        ( K_WITH properties[props] )?
-      { $expr = new CreateIndexStatement(cf, $idxName.text, id, isCustom, props); }
+        ( K_USING cls=STRING_LITERAL )?
+      { $expr = new CreateIndexStatement(cf, $idxName.text, id, isCustom, $cls.text); }
     ;
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2397bc8c/src/java/org/apache/cassandra/cql3/IndexPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/IndexPropDefs.java b/src/java/org/apache/cassandra/cql3/IndexPropDefs.java
deleted file mode 100644
index fca12c8..0000000
--- a/src/java/org/apache/cassandra/cql3/IndexPropDefs.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.cql3;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.cassandra.db.index.SecondaryIndex;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.SyntaxException;
-
-public class IndexPropDefs extends PropertyDefinitions
-{
-    public static final String KW_OPTIONS = "options";
-
-    public static final Set<String> keywords = new HashSet<String>();
-    public static final Set<String> obsoleteKeywords = new HashSet<String>();
-
-    public static final String INDEX_CLASS_KEY = "class";
-
-    static
-    {
-        keywords.add(KW_OPTIONS);
-    }
-
-    public void validate(boolean isCustom) throws RequestValidationException
-    {
-        validate(keywords, obsoleteKeywords);
-        if (isCustom && !getOptions().containsKey(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME))
-            throw new InvalidRequestException(String.format("Custom index requires '%s' option to be specified", INDEX_CLASS_KEY));
-        if (!isCustom && !getOptions().isEmpty())
-            throw new InvalidRequestException(String.format("Only custom indexes can currently be parametrized"));
-    }
-
-    public Map<String, String> getOptions() throws SyntaxException
-    {
-        Map<String, String> options = getMap(KW_OPTIONS);
-
-        if (options == null)
-            return Collections.emptyMap();
-
-        if (!options.isEmpty() && options.containsKey(INDEX_CLASS_KEY))
-        {
-            options.put(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME, options.get(INDEX_CLASS_KEY));
-            options.remove(INDEX_CLASS_KEY);
-        }
-
-        return options;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2397bc8c/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 513c96e..5ad6e77 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -41,7 +41,7 @@ import org.apache.cassandra.utils.SemanticVersion;
 
 public class QueryProcessor
 {
-    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.0.3");
+    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.0.4");
 
     private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/2397bc8c/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 549fab8..b79a255 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -47,15 +47,15 @@ public class CreateIndexStatement extends SchemaAlteringStatement
     private final String indexName;
     private final ColumnIdentifier columnName;
     private final boolean isCustom;
-    private final IndexPropDefs props;
+    private final String indexClass;
 
-    public CreateIndexStatement(CFName name, String indexName, ColumnIdentifier columnName, boolean isCustom, IndexPropDefs props)
+    public CreateIndexStatement(CFName name, String indexName, ColumnIdentifier columnName, boolean isCustom, String indexClass)
     {
         super(name);
         this.indexName = indexName;
         this.columnName = columnName;
         this.isCustom = isCustom;
-        this.props = props;
+        this.indexClass = indexClass;
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
@@ -81,14 +81,14 @@ public class CreateIndexStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException(String.format("Cannot create index on column %s of compact CF", columnName));
             case COLUMN_METADATA:
                 ColumnDefinition cd = cfm.getColumnDefinition(columnName.key);
-
                 if (cd.getIndexType() != null)
                     throw new InvalidRequestException("Index already exists");
-
+                if (isCustom && indexClass == null)
+                    throw new InvalidRequestException("CUSTOM index requires specifiying the index class");
+                if (!isCustom && indexClass != null)
+                    throw new InvalidRequestException("Cannot specify index class for a non-CUSTOM index");
                 if (cd.getValidator().isCollection() && !isCustom)
                     throw new InvalidRequestException("Indexes on collections are no yet supported");
-
-                props.validate(isCustom);
                 break;
             default:
                 throw new AssertionError();
@@ -104,14 +104,7 @@ public class CreateIndexStatement extends SchemaAlteringStatement
 
         if (isCustom)
         {
-            try
-            {
-                cd.setIndexType(IndexType.CUSTOM, props.getOptions());
-            }
-            catch (SyntaxException e)
-            {
-                throw new AssertionError(); // can't happen after validation.
-            }
+            cd.setIndexType(IndexType.CUSTOM, Collections.singletonMap(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME, indexClass));
         }
         else if (cfDef.isComposite)
         {


[2/2] git commit: Merge branch 'cassandra-1.2' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-1.2' into trunk

Conflicts:
	bin/cqlsh
	doc/cql3/CQL.textile
	src/java/org/apache/cassandra/cql3/QueryProcessor.java
	src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java


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

Branch: refs/heads/trunk
Commit: 7edd0e0c7e05e87a1a48b81a6add08a6e65d73f1
Parents: 26018be 2397bc8
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Jun 18 17:29:23 2013 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Jun 18 17:29:23 2013 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 NEWS.txt                                        |  3 +
 doc/cql3/CQL.textile                            |  8 ++-
 pylib/cqlshlib/cql3handling.py                  |  2 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |  8 +--
 .../apache/cassandra/cql3/IndexPropDefs.java    | 68 --------------------
 .../cql3/statements/CreateIndexStatement.java   | 30 ++++-----
 7 files changed, 26 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7edd0e0c/NEWS.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7edd0e0c/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --cc doc/cql3/CQL.textile
index 169cf2b,5fa36ab..1648cc8
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@@ -1056,11 -1048,10 +1056,15 @@@ h2(#changes). Change
  
  The following describes the addition/changes brought for each version of CQL.
  
 +h3. 3.1.0
 +
 +* "ALTER TABLE":#alterTableStmt @DROP@ option has been reenabled for CQL3 tables and has new semantics now: the space formerly used by dropped columns will now be eventually reclaimed (post-compaction). You should not readd previously dropped columns unless you use timestamps with microsecond precision (see "CASSANDRA-3919":https://issues.apache.org/jira/browse/CASSANDRA-3919 for more details).
 +* SELECT statement now supports aliases in select clause. Aliases in WHERE and ORDER BY clauses are not supported. See the "section on select"#selectStmt for details.
 +
+ h3. 3.0.4
+ 
+ * Updated the syntax for custom "secondary indexes":#createIndexStmt.
+ 
  h3. 3.0.3
  
  * Support for custom "secondary indexes":#createIndexStmt has been added.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7edd0e0c/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7edd0e0c/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7edd0e0c/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 4b61ab3,b79a255..12f762f
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@@ -26,8 -28,11 +26,9 @@@ import org.apache.cassandra.auth.Permis
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.ColumnDefinition;
  import org.apache.cassandra.config.Schema;
+ import org.apache.cassandra.db.index.SecondaryIndex;
  import org.apache.cassandra.exceptions.*;
  import org.apache.cassandra.cql3.*;
 -import org.apache.cassandra.db.index.composites.CompositesIndex;
 -import org.apache.cassandra.db.marshal.CompositeType;
  import org.apache.cassandra.service.ClientState;
  import org.apache.cassandra.service.MigrationManager;
  import org.apache.cassandra.thrift.IndexType;
@@@ -62,25 -67,32 +63,29 @@@ public class CreateIndexStatement exten
      public void validate(ClientState state) throws RequestValidationException
      {
          CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
 -        CFDefinition.Name name = cfm.getCfDef().get(columnName);
 +        ColumnDefinition cd = cfm.getColumnDefinition(columnName.key);
  
 -        if (name == null)
 +        if (cd == null)
              throw new InvalidRequestException("No column definition found for column " + columnName);
  
 -        switch (name.kind)
 -        {
 -            case KEY_ALIAS:
 -            case COLUMN_ALIAS:
 -                throw new InvalidRequestException(String.format("Cannot create index on PRIMARY KEY part %s", columnName));
 -            case VALUE_ALIAS:
 -                throw new InvalidRequestException(String.format("Cannot create index on column %s of compact CF", columnName));
 -            case COLUMN_METADATA:
 -                ColumnDefinition cd = cfm.getColumnDefinition(columnName.key);
 -                if (cd.getIndexType() != null)
 -                    throw new InvalidRequestException("Index already exists");
 -                if (isCustom && indexClass == null)
 -                    throw new InvalidRequestException("CUSTOM index requires specifiying the index class");
 -                if (!isCustom && indexClass != null)
 -                    throw new InvalidRequestException("Cannot specify index class for a non-CUSTOM index");
 -                if (cd.getValidator().isCollection() && !isCustom)
 -                    throw new InvalidRequestException("Indexes on collections are no yet supported");
 -                break;
 -            default:
 -                throw new AssertionError();
 -        }
 +        if (cd.getIndexType() != null)
 +            throw new InvalidRequestException("Index already exists");
 +
++        if (isCustom && indexClass == null)
++            throw new InvalidRequestException("CUSTOM index requires specifiying the index class");
++
++        if (!isCustom && indexClass != null)
++            throw new InvalidRequestException("Cannot specify index class for a non-CUSTOM index");
++
 +        // TODO: we could lift that limitation
 +        if (cfm.getCfDef().isCompact && cd.type != ColumnDefinition.Type.REGULAR)
 +            throw new InvalidRequestException(String.format("Secondary index on %s column %s is not yet supported for compact table", cd.type, columnName));
 +
 +        if (cd.getValidator().isCollection() && !isCustom)
 +            throw new InvalidRequestException("Indexes on collections are no yet supported");
 +
 +        if (cd.type == ColumnDefinition.Type.PARTITION_KEY && (cd.componentIndex == null || cd.componentIndex == 0))
 +            throw new InvalidRequestException(String.format("Cannot add secondary index to already primarily indexed column %s", columnName));
- 
-         props.validate(isCustom);
      }
  
      public void announceMigration() throws InvalidRequestException, ConfigurationException
@@@ -90,24 -102,21 +95,11 @@@
          ColumnDefinition cd = cfm.getColumnDefinition(columnName.key);
  
          if (isCustom)
--        {
-             try
-             {
-                 cd.setIndexType(IndexType.CUSTOM, props.getOptions());
-             }
-             catch (SyntaxException e)
-             {
-                 throw new AssertionError(); // can't happen after validation.
-             }
-         }
+             cd.setIndexType(IndexType.CUSTOM, Collections.singletonMap(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME, indexClass));
 -        }
 -        else if (cfDef.isComposite)
 -        {
 -            CompositeType composite = (CompositeType)cfm.comparator;
 -            Map<String, String> opts = new HashMap<String, String>();
 -            opts.put(CompositesIndex.PREFIX_SIZE_OPTION, String.valueOf(composite.types.size() - (cfDef.hasCollections ? 2 : 1)));
 -            cd.setIndexType(IndexType.COMPOSITES, opts);
 -        }
 +        else if (cfm.getCfDef().isComposite)
-         {
 +            cd.setIndexType(IndexType.COMPOSITES, Collections.<String, String>emptyMap());
-         }
          else
--        {
              cd.setIndexType(IndexType.KEYS, Collections.<String, String>emptyMap());
--        }
  
          cd.setIndexName(indexName);
          cfm.addDefaultIndexNames();