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 2014/01/07 16:03:55 UTC

git commit: Allow specifying custom secondary index options in CQL3

Updated Branches:
  refs/heads/cassandra-2.0 ae0a1e0f5 -> 5fa605510


Allow specifying custom secondary index options in CQL3

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


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

Branch: refs/heads/cassandra-2.0
Commit: 5fa6055107e5d54eff1e2bbdfdbbd511cd555a4b
Parents: ae0a1e0
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Jan 7 18:01:32 2014 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Jan 7 18:03:37 2014 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |  3 +-
 bin/cqlsh                                       |  2 +-
 doc/cql3/CQL.textile                            |  9 ++-
 pylib/cqlshlib/cql3handling.py                  |  2 +-
 src/java/org/apache/cassandra/cql3/Cql.g        |  9 +--
 .../apache/cassandra/cql3/QueryProcessor.java   |  2 +-
 .../cql3/statements/CreateIndexStatement.java   | 27 ++++----
 .../cql3/statements/IndexPropDefs.java          | 70 ++++++++++++++++++++
 8 files changed, 99 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fa60551/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index df564dd..faf52a4 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,5 +1,6 @@
 2.0.5
-* Delete unfinished compaction incrementally (CASSANDRA-6086)
+ * Delete unfinished compaction incrementally (CASSANDRA-6086)
+ * Allow specifying custom secondary index options in CQL3 (CASSANDRA-6480)
 Merged from 1.2:
  * fsync compression metadata (CASSANDRA-6531)
  * Validate CF existence on execution for prepared statement (CASSANDRA-6535)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fa60551/bin/cqlsh
----------------------------------------------------------------------
diff --git a/bin/cqlsh b/bin/cqlsh
index 094aa84..0add46c 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 = "4.1.0"
+version = "4.1.1"
 
 from StringIO import StringIO
 from itertools import groupby

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fa60551/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index f31c65a..5fff402 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.1.3
+h1. Cassandra Query Language (CQL) v3.1.4
 
 
  <span id="tableOfContents">
@@ -405,7 +405,7 @@ h3(#createIndexStmt). CREATE INDEX
 __Syntax:__
 
 bc(syntax). <create-index-stmt> ::= CREATE ( CUSTOM )? INDEX ( IF NOT EXISTS )? <identifier>? ON <tablename> '(' <identifier> ')'
-                                        ( USING <string> )?
+                                        ( USING <string> ( WITH OPTIONS = <map-literal> )? )?
 
 __Sample:__
 
@@ -413,6 +413,7 @@ bc(sample).
 CREATE INDEX userIndex ON NerdMovies (user);
 CREATE INDEX ON Mutants (abilityId);
 CREATE CUSTOM INDEX ON users (email) USING 'path.to.the.IndexClass';
+CREATE CUSTOM INDEX ON users (email) USING 'path.to.the.IndexClass' WITH OPTIONS = {'storage': '/mnt/ssd/indexes/'};
 
 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 asynchronously. After the index is created, new data for the column is indexed automatically at insertion time.
 
@@ -1102,6 +1103,10 @@ h2(#changes). Changes
 
 The following describes the addition/changes brought for each version of CQL.
 
+h3. 3.1.4
+
+* @CREATE INDEX@ now allows specifying options when creating CUSTOM indexes (see "CREATE INDEX reference":#createIndexStmt).
+
 h3. 3.1.3
 
 * Millisecond precision formats have been added to the timestamp parser (see "working with dates":#usingdates).

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fa60551/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index e7aa9e1..421ab27 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -927,7 +927,7 @@ def create_cf_composite_primary_key_comma_completer(ctxt, cass):
 syntax_rules += r'''
 <createIndexStatement> ::= "CREATE" "CUSTOM"? "INDEX" ("IF" "NOT" "EXISTS")? indexname=<identifier>? "ON"
                                cf=<columnFamilyName> "(" col=<cident> ")"
-                               ( "USING" <stringLiteral> )?
+                               ( "USING" <stringLiteral> ( "WITH" "OPTIONS" "=" <mapLiteral> )? )?
                          ;
 '''
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fa60551/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 bdaf79a..53aebe7 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -510,13 +510,14 @@ cfamOrdering[CreateTableStatement.RawStatement expr]
  */
 createIndexStatement returns [CreateIndexStatement expr]
     @init {
-        boolean isCustom = false;
+        IndexPropDefs props = new IndexPropDefs();
         boolean ifNotExists = false;
     }
-    : K_CREATE (K_CUSTOM { isCustom = true; })? K_INDEX (K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
+    : K_CREATE (K_CUSTOM { props.isCustom = true; })? K_INDEX (K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
         (idxName=IDENT)? K_ON cf=columnFamilyName '(' id=cident ')'
-        ( K_USING cls=STRING_LITERAL )?
-      { $expr = new CreateIndexStatement(cf, $idxName.text, id, ifNotExists, isCustom, $cls.text); }
+        (K_USING cls=STRING_LITERAL { props.customClass = $cls.text; })?
+        (K_WITH properties[props])?
+      { $expr = new CreateIndexStatement(cf, $idxName.text, id, props, ifNotExists); }
     ;
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fa60551/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 02361a8..94c6da2 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -44,7 +44,7 @@ import org.apache.cassandra.utils.SemanticVersion;
 
 public class QueryProcessor
 {
-    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.1.3");
+    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.1.4");
 
     private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
     private static final MemoryMeter meter = new MemoryMeter();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fa60551/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 ba88e28..3ef6f5a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -26,7 +26,6 @@ import org.apache.cassandra.auth.Permission;
 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.service.ClientState;
@@ -42,18 +41,20 @@ public class CreateIndexStatement extends SchemaAlteringStatement
 
     private final String indexName;
     private final ColumnIdentifier columnName;
+    private final IndexPropDefs properties;
     private final boolean ifNotExists;
-    private final boolean isCustom;
-    private final String indexClass;
 
-    public CreateIndexStatement(CFName name, String indexName, ColumnIdentifier columnName, boolean ifNotExists, boolean isCustom, String indexClass)
+    public CreateIndexStatement(CFName name,
+                                String indexName,
+                                ColumnIdentifier columnName,
+                                IndexPropDefs properties,
+                                boolean ifNotExists)
     {
         super(name);
         this.indexName = indexName;
         this.columnName = columnName;
+        this.properties = properties;
         this.ifNotExists = ifNotExists;
-        this.isCustom = isCustom;
-        this.indexClass = indexClass;
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
@@ -80,24 +81,20 @@ public class CreateIndexStatement extends SchemaAlteringStatement
                 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");
+        properties.validate();
 
         // 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)
+        if (cd.getValidator().isCollection() && !properties.isCustom)
             throw new InvalidRequestException("Indexes on collections are no yet supported");
 
         if (cd.type == ColumnDefinition.Type.PARTITION_KEY && cd.componentIndex == null)
             throw new InvalidRequestException(String.format("Cannot add secondary index to already primarily indexed column %s", columnName));
     }
 
-    public void announceMigration() throws InvalidRequestException, ConfigurationException
+    public void announceMigration() throws RequestValidationException
     {
         logger.debug("Updating column {} definition for index {}", columnName, indexName);
         CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).clone();
@@ -106,8 +103,8 @@ public class CreateIndexStatement extends SchemaAlteringStatement
         if (cd.getIndexType() != null && ifNotExists)
             return;
 
-        if (isCustom)
-            cd.setIndexType(IndexType.CUSTOM, Collections.singletonMap(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME, indexClass));
+        if (properties.isCustom)
+            cd.setIndexType(IndexType.CUSTOM, properties.getOptions());
         else if (cfm.getCfDef().isComposite)
             cd.setIndexType(IndexType.COMPOSITES, Collections.<String, String>emptyMap());
         else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/5fa60551/src/java/org/apache/cassandra/cql3/statements/IndexPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/IndexPropDefs.java b/src/java/org/apache/cassandra/cql3/statements/IndexPropDefs.java
new file mode 100644
index 0000000..6790611
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/IndexPropDefs.java
@@ -0,0 +1,70 @@
+/*
+ * 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.statements;
+
+import java.util.*;
+
+import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.exceptions.*;
+
+public class IndexPropDefs extends PropertyDefinitions
+{
+    public static final String KW_OPTIONS = "options";
+
+    public static final Set<String> keywords = new HashSet<>();
+    public static final Set<String> obsoleteKeywords = new HashSet<>();
+
+    public boolean isCustom;
+    public String customClass;
+
+    static
+    {
+        keywords.add(KW_OPTIONS);
+    }
+
+    public void validate() throws RequestValidationException
+    {
+        validate(keywords, obsoleteKeywords);
+
+        if (isCustom && customClass == null)
+            throw new InvalidRequestException("CUSTOM index requires specifiying the index class");
+
+        if (!isCustom && customClass != null)
+            throw new InvalidRequestException("Cannot specify index class for a non-CUSTOM index");
+
+        if (!isCustom && !properties.isEmpty())
+            throw new InvalidRequestException("Cannot specify options for a non-CUSTOM index");
+
+        if (getRawOptions().containsKey(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME))
+            throw new InvalidRequestException(String.format("Cannot specify %s as a CUSTOM option",
+                                                            SecondaryIndex.CUSTOM_INDEX_OPTION_NAME));
+    }
+
+    public Map<String, String> getRawOptions() throws SyntaxException
+    {
+        Map<String, String> options = getMap(KW_OPTIONS);
+        return options == null ? Collections.<String, String>emptyMap() : options;
+    }
+
+    public Map<String, String> getOptions() throws SyntaxException
+    {
+        Map<String, String> options = new HashMap<>(getRawOptions());
+        options.put(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME, customClass);
+        return options;
+    }
+}