You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2013/06/21 12:02:35 UTC

git commit: Conditional create/drop ks/table/index statements

Updated Branches:
  refs/heads/trunk 0ba5cf12b -> b73f9d423


Conditional create/drop ks/table/index statements

patch by michalm; reviewed by slebresne for CASSANDRA-2737


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

Branch: refs/heads/trunk
Commit: b73f9d42315311e3573018749aeba34f77f1419e
Parents: 0ba5cf1
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Jun 21 11:21:54 2013 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Jun 21 12:02:25 2013 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 doc/cql3/CQL.textile                            | 26 ++++++++---
 pylib/cqlshlib/cql3handling.py                  | 16 +++----
 src/java/org/apache/cassandra/cql3/Cql.g        | 45 ++++++++++++--------
 .../statements/CreateColumnFamilyStatement.java | 22 ++++++++--
 .../cql3/statements/CreateIndexStatement.java   | 14 +++++-
 .../statements/CreateKeyspaceStatement.java     | 15 ++++++-
 .../statements/DropColumnFamilyStatement.java   | 15 ++++++-
 .../cql3/statements/DropIndexStatement.java     | 22 ++++++++--
 .../cql3/statements/DropKeyspaceStatement.java  | 14 +++++-
 10 files changed, 141 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b73f9d42/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ca27791..044add4 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -61,6 +61,7 @@
  * Include a timestamp with all read commands to determine column expiration
    (CASSANDRA-5149)
  * Streaming 2.0 (CASSANDRA-5286)
+ * Conditional create/drop ks/table/index statements in CQL3 (CASSANDRA-2737)
 
 1.2.6
  * Fix cross-DC mutation forwarding (CASSANDRA-5632)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b73f9d42/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 287a480..34a0c12 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -134,7 +134,7 @@ h3(#createKeyspaceStmt). CREATE KEYSPACE
 __Syntax:__
 
 bc(syntax).. 
-<create-keyspace-stmt> ::= CREATE KEYSPACE <identifier> WITH <properties>
+<create-keyspace-stmt> ::= CREATE KEYSPACE (IF NOT EXISTS)? <identifier> WITH <properties>
 p. 
 __Sample:__
 
@@ -159,6 +159,8 @@ The @replication@ @<property>@ is mandatory. It must at least contains the @'cla
 * @'NetworkTopologyStrategy'@: A replication strategy that allows to set the replication factor independently for each data-center. The rest of the sub-options are key-value pairs where each time the key is the name of a datacenter and the value the replication factor for that data-center.
 * @'OldNetworkTopologyStrategy'@: A legacy replication strategy. You should avoid this strategy for new keyspaces and prefer @'NetworkTopologyStrategy'@.
 
+Attempting to create an already existing keyspace will return an error unless the @IF NOT EXISTS@ option is used. If it is used, the statement will be a no-op if the keyspace already exists.
+
 h3(#useStmt). USE
 
 __Syntax:__
@@ -192,7 +194,7 @@ h3(#dropKeyspaceStmt). DROP KEYSPACE
 
 __Syntax:__
 
-bc(syntax).  <drop-keyspace-stmt> ::= DROP KEYSPACE <identifier>
+bc(syntax).  <drop-keyspace-stmt> ::= DROP KEYSPACE ( IF EXISTS )? <identifier>
 
 __Sample:__
 
@@ -200,13 +202,15 @@ bc(sample). DROP KEYSPACE myApp;
 
 A @DROP KEYSPACE@ statement results in the immediate, irreversible removal of an existing keyspace, including all column families in it, and all data contained in those column families.
 
+If the keyspace does not exists, the statement will return an error, unless @IF EXISTS@ is used in which case the operation is a no-op.
+
 
 h3(#createTableStmt). CREATE TABLE
 
 __Syntax:__
 
 bc(syntax).. 
-<create-table-stmt> ::= CREATE (TABLE | COLUMNFAMILY) <tablename>
+<create-table-stmt> ::= CREATE ( TABLE | COLUMNFAMILY ) ( IF NOT EXISTS )? <tablename>
                           '(' <definition> ( ',' <definition> )* ')'
                           ( WITH <option> ( AND <option>)* )?
 
@@ -245,6 +249,8 @@ CREATE TABLE timeline (
 p. 
 The @CREATE TABLE@ statement creates a new table. Each such table is a set of _rows_ (usually representing related entities) for which it defines a number of properties. A table is defined by a "name":#createTableName, it defines the <a href="#createTableColumn"><it>columns</it></a> composing rows of the table and have a number of "options":#createTableOptions. Note that the @CREATE COLUMNFAMILY@ syntax is supported as an alias for @CREATE TABLE@ (for historical reasons).
 
+Attempting to create an already existing table will return an error unless the @IF NOT EXISTS@ option is used. If it is used, the statement will be a no-op if the table already exists.
+
 h4(#createTableName). @<tablename>@
 
 Valid table names are the same than valid "keyspace names":#createKeyspaceStmt (up to 32 characters long alphanumerical identifiers). If the table name is provided alone, the table is created within the current keyspace (see <a href="#useStmt"><tt>USE</tt></a>), but if it is prefixed by an existing keyspace name (see "@<tablename>@":#statements grammar), it is created in the specified keyspace (but does *not* change the current keyspace).
@@ -366,7 +372,7 @@ h3(#dropTableStmt). DROP TABLE
 
 __Syntax:__
 
-bc(syntax). <drop-table-stmt> ::= DROP TABLE <tablename>
+bc(syntax). <drop-table-stmt> ::= DROP TABLE ( IF EXISTS )? <tablename>
 
 __Sample:__
 
@@ -374,6 +380,8 @@ bc(sample). DROP TABLE worldSeriesAttendees;
 
 The @DROP TABLE@ statement results in the immediate, irreversible removal of a table, including all data contained in it. As for table creation, @DROP COLUMNFAMILY@ is allowed as an alias for @DROP TABLE@.
 
+If the table does not exists, the statement will return an error, unless @IF EXISTS@ is used in which case the operation is a no-op.
+
 h3(#truncateStmt). TRUNCATE
 
 __Syntax:__
@@ -391,7 +399,7 @@ h3(#createIndexStmt). CREATE INDEX
 
 __Syntax:__
 
-bc(syntax). <create-index-stmt> ::= CREATE ( CUSTOM )? INDEX <identifier>? ON <tablename> '(' <identifier> ')'
+bc(syntax). <create-index-stmt> ::= CREATE ( CUSTOM )? INDEX ( IF NOT EXISTS )? <identifier>? ON <tablename> '(' <identifier> ')'
                                         ( USING <string> )?
 
 __Sample:__
@@ -403,11 +411,13 @@ 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.
 
+Attempting to create an already existing index will return an error unless the @IF NOT EXISTS@ option is used. If it is used, the statement will be a no-op if the index already exists.
+
 h3(#dropIndexStmt). DROP INDEX
 
 __Syntax:__
 
-bc(syntax).  <drop-index-stmt> ::= DROP INDEX <identifier>
+bc(syntax).  <drop-index-stmt> ::= DROP INDEX ( IF EXISTS )? <identifier>
 
 __Sample:__
 
@@ -415,6 +425,7 @@ bc(sample). DROP INDEX userIndex;
 
 The @DROP INDEX@ statement is used to drop an existing secondary index. The argument of the statement is the index name.
 
+If the index does not exists, the statement will return an error, unless @IF EXISTS@ is used in which case the operation is a no-op.
 
 h2(#dataManipulation). Data Manipulation
 
@@ -1059,7 +1070,8 @@ 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.
+* @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.
+* @CREATE@ statements for @KEYSPACE@, @TABLE@ and @INDEX@ now supports an @IF NOT EXISTS@ condition. Similarly, @DROP@ statements support a @IF EXISTS@ condition.
 
 h3. 3.0.4
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b73f9d42/pylib/cqlshlib/cql3handling.py
----------------------------------------------------------------------
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 8e4aee1..e68bf15 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -49,11 +49,11 @@ class Cql3ParsingRuleSet(CqlParsingRuleSet):
         'keyspace', 'schema', 'columnfamily', 'table', 'index', 'on', 'drop',
         'primary', 'into', 'values', 'timestamp', 'ttl', 'alter', 'add', 'type',
         'compact', 'storage', 'order', 'by', 'asc', 'desc', 'clustering',
-        'token', 'writetime', 'map', 'list', 'to', 'custom'
+        'token', 'writetime', 'map', 'list', 'to', 'custom', 'if', 'not'
     ))
 
     unreserved_keywords = set((
-        'key', 'clustering', 'ttl', 'compact', 'storage', 'type', 'values', 'custom'
+        'key', 'clustering', 'ttl', 'compact', 'storage', 'type', 'values', 'custom', 'exists'
     ))
 
     columnfamily_layout_options = (
@@ -834,7 +834,7 @@ syntax_rules += r'''
 '''
 
 syntax_rules += r'''
-<createKeyspaceStatement> ::= "CREATE" wat=( "KEYSPACE" | "SCHEMA" ) ksname=<cfOrKsName>
+<createKeyspaceStatement> ::= "CREATE" wat=( "KEYSPACE" | "SCHEMA" ) ("IF" "NOT" "EXISTS")?  ksname=<cfOrKsName>
                                 "WITH" <property> ( "AND" <property> )*
                             ;
 '''
@@ -913,7 +913,7 @@ def keyspace_properties_map_ender_completer(ctxt, cass):
     return ['}']
 
 syntax_rules += r'''
-<createColumnFamilyStatement> ::= "CREATE" wat=( "COLUMNFAMILY" | "TABLE" )
+<createColumnFamilyStatement> ::= "CREATE" wat=( "COLUMNFAMILY" | "TABLE" ) ("IF" "NOT" "EXISTS")?
                                     ( ks=<nonSystemKeyspaceName> dot="." )? cf=<cfOrKsName>
                                     "(" ( <singleKeyCfSpec> | <compositeKeyCfSpec> ) ")"
                                    ( "WITH" <cfamProperty> ( "AND" <cfamProperty> )* )?
@@ -1007,7 +1007,7 @@ def create_cf_composite_primary_key_comma_completer(ctxt, cass):
     return [',']
 
 syntax_rules += r'''
-<createIndexStatement> ::= "CREATE" "CUSTOM"? "INDEX" indexname=<identifier>? "ON"
+<createIndexStatement> ::= "CREATE" "CUSTOM"? "INDEX" ("IF" "NOT" "EXISTS")? indexname=<identifier>? "ON"
                                cf=<columnFamilyName> "(" col=<cident> ")"
                                ( "USING" <stringLiteral> )?
                          ;
@@ -1022,13 +1022,13 @@ def create_index_col_completer(ctxt, cass):
     return map(maybe_escape_name, colnames)
 
 syntax_rules += r'''
-<dropKeyspaceStatement> ::= "DROP" "KEYSPACE" ksname=<nonSystemKeyspaceName>
+<dropKeyspaceStatement> ::= "DROP" "KEYSPACE" ("IF" "EXISTS")? ksname=<nonSystemKeyspaceName>
                           ;
 
-<dropColumnFamilyStatement> ::= "DROP" ( "COLUMNFAMILY" | "TABLE" ) cf=<columnFamilyName>
+<dropColumnFamilyStatement> ::= "DROP" ( "COLUMNFAMILY" | "TABLE" ) ("IF" "EXISTS")? cf=<columnFamilyName>
                               ;
 
-<dropIndexStatement> ::= "DROP" "INDEX" indexname=<identifier>
+<dropIndexStatement> ::= "DROP" "INDEX" ("IF" "EXISTS")? indexname=<identifier>
                        ;
 '''
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b73f9d42/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 84ca9ee..1e6a2c6 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -430,23 +430,28 @@ batchStatementObjective returns [ModificationStatement.Parsed statement]
     ;
 
 /**
- * CREATE KEYSPACE <KEYSPACE> WITH attr1 = value1 AND attr2 = value2;
+ * CREATE KEYSPACE [IF NOT EXISTS] <KEYSPACE> WITH attr1 = value1 AND attr2 = value2;
  */
 createKeyspaceStatement returns [CreateKeyspaceStatement expr]
-    @init { KSPropDefs attrs = new KSPropDefs(); }
-    : K_CREATE K_KEYSPACE ks=keyspaceName
-      K_WITH properties[attrs] { $expr = new CreateKeyspaceStatement(ks, attrs); }
+    @init {
+        KSPropDefs attrs = new KSPropDefs();
+        boolean ifNotExists = false;
+    }
+    : K_CREATE K_KEYSPACE (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? ks=keyspaceName
+      K_WITH properties[attrs] { $expr = new CreateKeyspaceStatement(ks, attrs, ifNotExists); }
     ;
 
 /**
- * CREATE COLUMNFAMILY <CF> (
+ * CREATE COLUMNFAMILY [IF NOT EXISTS] <CF> (
  *     <name1> <type>,
  *     <name2> <type>,
  *     <name3> <type>
  * ) WITH <property> = <value> AND ...;
  */
 createColumnFamilyStatement returns [CreateColumnFamilyStatement.RawStatement expr]
-    : K_CREATE K_COLUMNFAMILY cf=columnFamilyName { $expr = new CreateColumnFamilyStatement.RawStatement(cf); }
+    @init { boolean ifNotExists = false; }
+    : K_CREATE K_COLUMNFAMILY (K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
+      cf=columnFamilyName { $expr = new CreateColumnFamilyStatement.RawStatement(cf, ifNotExists); }
       cfamDefinition[expr]
     ;
 
@@ -477,16 +482,18 @@ cfamOrdering[CreateColumnFamilyStatement.RawStatement expr]
     ;
 
 /**
- * CREATE INDEX [indexName] ON <columnFamily> (<columnName>);
- * CREATE CUSTOM INDEX [indexName] ON <columnFamily> (<columnName>) USING <indexClass>;
+ * CREATE INDEX [IF NOT EXISTS] [indexName] ON <columnFamily> (<columnName>);
+ * CREATE CUSTOM INDEX [IF NOT EXISTS] [indexName] ON <columnFamily> (<columnName>) USING <indexClass>;
  */
 createIndexStatement returns [CreateIndexStatement expr]
     @init {
         boolean isCustom = false;
+        boolean ifNotExists = false;
     }
-    : K_CREATE (K_CUSTOM { isCustom = true; })? K_INDEX (idxName=IDENT)? K_ON cf=columnFamilyName '(' id=cident ')'
+    : K_CREATE (K_CUSTOM { 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, isCustom, $cls.text); }
+      { $expr = new CreateIndexStatement(cf, $idxName.text, id, ifNotExists, isCustom, $cls.text); }
     ;
 
 /**
@@ -543,26 +550,28 @@ alterTableStatement returns [AlterTableStatement expr]
     ;
 
 /**
- * DROP KEYSPACE <KSP>;
+ * DROP KEYSPACE [IF EXISTS] <KSP>;
  */
 dropKeyspaceStatement returns [DropKeyspaceStatement ksp]
-    : K_DROP K_KEYSPACE ks=keyspaceName { $ksp = new DropKeyspaceStatement(ks); }
+    @init { boolean ifExists = false; }
+    : K_DROP K_KEYSPACE (K_IF K_EXISTS { ifExists = true; } )? ks=keyspaceName { $ksp = new DropKeyspaceStatement(ks, ifExists); }
     ;
 
 /**
- * DROP COLUMNFAMILY <CF>;
+ * DROP COLUMNFAMILY [IF EXISTS] <CF>;
  */
 dropColumnFamilyStatement returns [DropColumnFamilyStatement stmt]
-    : K_DROP K_COLUMNFAMILY cf=columnFamilyName { $stmt = new DropColumnFamilyStatement(cf); }
+    @init { boolean ifExists = false; }
+    : K_DROP K_COLUMNFAMILY (K_IF K_EXISTS { ifExists = true; } )? cf=columnFamilyName { $stmt = new DropColumnFamilyStatement(cf, ifExists); }
     ;
 
 /**
- * DROP INDEX <INDEX_NAME>
+ * DROP INDEX [IF EXISTS] <INDEX_NAME>
  */
 dropIndexStatement returns [DropIndexStatement expr]
-    :
-      K_DROP K_INDEX index=IDENT
-      { $expr = new DropIndexStatement($index.text); }
+    @init { boolean ifExists = false; }
+    : K_DROP K_INDEX (K_IF K_EXISTS { ifExists = true; } )? index=IDENT
+      { $expr = new DropIndexStatement($index.text, ifExists); }
     ;
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b73f9d42/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
index 71a3545..d5855be 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.ColumnFamilyType;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.AlreadyExistsException;
 import org.apache.cassandra.io.compress.CompressionParameters;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.MigrationManager;
@@ -52,11 +53,13 @@ public class CreateColumnFamilyStatement extends SchemaAlteringStatement
 
     private final Map<ColumnIdentifier, AbstractType> columns = new HashMap<ColumnIdentifier, AbstractType>();
     private final CFPropDefs properties;
+    private final boolean ifNotExists;
 
-    public CreateColumnFamilyStatement(CFName name, CFPropDefs properties)
+    public CreateColumnFamilyStatement(CFName name, CFPropDefs properties, boolean ifNotExists)
     {
         super(name);
         this.properties = properties;
+        this.ifNotExists = ifNotExists;
 
         try
         {
@@ -101,7 +104,15 @@ public class CreateColumnFamilyStatement extends SchemaAlteringStatement
 
     public void announceMigration() throws RequestValidationException
     {
-        MigrationManager.announceNewColumnFamily(getCFMetaData());
+        try
+        {
+           MigrationManager.announceNewColumnFamily(getCFMetaData());
+        }
+        catch (AlreadyExistsException e)
+        {
+            if (!ifNotExists)
+                throw e;
+        }
     }
 
     public ResultMessage.SchemaChange.Change changeType()
@@ -154,9 +165,12 @@ public class CreateColumnFamilyStatement extends SchemaAlteringStatement
         private boolean useCompactStorage;
         private final Multiset<ColumnIdentifier> definedNames = HashMultiset.create(1);
 
-        public RawStatement(CFName name)
+        private final boolean ifNotExists;
+
+        public RawStatement(CFName name, boolean ifNotExists)
         {
             super(name);
+            this.ifNotExists = ifNotExists;
         }
 
         /**
@@ -176,7 +190,7 @@ public class CreateColumnFamilyStatement extends SchemaAlteringStatement
 
             properties.validate();
 
-            CreateColumnFamilyStatement stmt = new CreateColumnFamilyStatement(cfName, properties);
+            CreateColumnFamilyStatement stmt = new CreateColumnFamilyStatement(cfName, properties, ifNotExists);
             stmt.setBoundTerms(getBoundsTerms());
 
             Map<ByteBuffer, CollectionType> definedCollections = null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b73f9d42/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 12f762f..c8fca42 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -42,14 +42,16 @@ public class CreateIndexStatement extends SchemaAlteringStatement
 
     private final String indexName;
     private final ColumnIdentifier columnName;
+    private final boolean ifNotExists;
     private final boolean isCustom;
     private final String indexClass;
 
-    public CreateIndexStatement(CFName name, String indexName, ColumnIdentifier columnName, boolean isCustom, String indexClass)
+    public CreateIndexStatement(CFName name, String indexName, ColumnIdentifier columnName, boolean ifNotExists, boolean isCustom, String indexClass)
     {
         super(name);
         this.indexName = indexName;
         this.columnName = columnName;
+        this.ifNotExists = ifNotExists;
         this.isCustom = isCustom;
         this.indexClass = indexClass;
     }
@@ -69,7 +71,12 @@ public class CreateIndexStatement extends SchemaAlteringStatement
             throw new InvalidRequestException("No column definition found for column " + columnName);
 
         if (cd.getIndexType() != null)
-            throw new InvalidRequestException("Index already exists");
+        {
+            if (ifNotExists)
+                return;
+            else
+                throw new InvalidRequestException("Index already exists");
+        }
 
         if (isCustom && indexClass == null)
             throw new InvalidRequestException("CUSTOM index requires specifiying the index class");
@@ -94,6 +101,9 @@ public class CreateIndexStatement extends SchemaAlteringStatement
         CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).clone();
         ColumnDefinition cd = cfm.getColumnDefinition(columnName.key);
 
+        if (cd.getIndexType() != null && ifNotExists)
+            return;
+
         if (isCustom)
             cd.setIndexType(IndexType.CUSTOM, Collections.singletonMap(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME, indexClass));
         else if (cfm.getCfDef().isComposite)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b73f9d42/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
index 71b22d8..d8840b3 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
@@ -22,6 +22,7 @@ import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.KSPropDefs;
+import org.apache.cassandra.exceptions.AlreadyExistsException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
@@ -37,6 +38,7 @@ public class CreateKeyspaceStatement extends SchemaAlteringStatement
 {
     private final String name;
     private final KSPropDefs attrs;
+    private final boolean ifNotExists;
 
     /**
      * Creates a new <code>CreateKeyspaceStatement</code> instance for a given
@@ -45,11 +47,12 @@ public class CreateKeyspaceStatement extends SchemaAlteringStatement
      * @param name the name of the keyspace to create
      * @param attrs map of the raw keyword arguments that followed the <code>WITH</code> keyword.
      */
-    public CreateKeyspaceStatement(String name, KSPropDefs attrs)
+    public CreateKeyspaceStatement(String name, KSPropDefs attrs, boolean ifNotExists)
     {
         super();
         this.name = name;
         this.attrs = attrs;
+        this.ifNotExists = ifNotExists;
     }
 
     @Override
@@ -99,7 +102,15 @@ public class CreateKeyspaceStatement extends SchemaAlteringStatement
 
     public void announceMigration() throws RequestValidationException
     {
-        MigrationManager.announceNewKeyspace(attrs.asKSMetadata(name));
+        try
+        {
+            MigrationManager.announceNewKeyspace(attrs.asKSMetadata(name));
+        }
+        catch (AlreadyExistsException e)
+        {
+            if (!ifNotExists)
+                throw e;
+        }
     }
 
     public ResultMessage.SchemaChange.Change changeType()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b73f9d42/src/java/org/apache/cassandra/cql3/statements/DropColumnFamilyStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropColumnFamilyStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropColumnFamilyStatement.java
index 7321642..3a3d436 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropColumnFamilyStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropColumnFamilyStatement.java
@@ -28,9 +28,12 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 
 public class DropColumnFamilyStatement extends SchemaAlteringStatement
 {
-    public DropColumnFamilyStatement(CFName name)
+    private final boolean ifExists;
+
+    public DropColumnFamilyStatement(CFName name, boolean ifExists)
     {
         super(name);
+        this.ifExists = ifExists;
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
@@ -40,7 +43,15 @@ public class DropColumnFamilyStatement extends SchemaAlteringStatement
 
     public void announceMigration() throws ConfigurationException
     {
-        MigrationManager.announceColumnFamilyDrop(keyspace(), columnFamily());
+        try
+        {
+            MigrationManager.announceColumnFamilyDrop(keyspace(), columnFamily());
+        }
+        catch (ConfigurationException e)
+        {
+            if (!ifExists)
+                throw e;
+        }
     }
 
     public ResultMessage.SchemaChange.Change changeType()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b73f9d42/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
index 7cc3b16..e530017 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
@@ -31,16 +31,22 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 public class DropIndexStatement extends SchemaAlteringStatement
 {
     public final String indexName;
+    public final boolean ifExists;
 
-    public DropIndexStatement(String indexName)
+    public DropIndexStatement(String indexName, boolean ifExists)
     {
         super(new CFName());
         this.indexName = indexName;
+        this.ifExists = ifExists;
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
     {
-        state.hasColumnFamilyAccess(keyspace(), findIndexedCF().cfName, Permission.ALTER);
+        CFMetaData cfm = findIndexedCF();
+        if (cfm == null)
+            return;
+
+        state.hasColumnFamilyAccess(cfm.ksName, cfm.cfName, Permission.ALTER);
     }
 
     public ResultMessage.SchemaChange.Change changeType()
@@ -51,7 +57,11 @@ public class DropIndexStatement extends SchemaAlteringStatement
 
     public void announceMigration() throws InvalidRequestException, ConfigurationException
     {
-        CFMetaData updatedCfm = updateCFMetadata(findIndexedCF());
+        CFMetaData cfm = findIndexedCF();
+        if (cfm == null)
+            return;
+
+        CFMetaData updatedCfm = updateCFMetadata(cfm);
         MigrationManager.announceColumnFamilyUpdate(updatedCfm, false);
     }
 
@@ -75,7 +85,11 @@ public class DropIndexStatement extends SchemaAlteringStatement
             if (findIndexedColumn(cfm) != null)
                 return cfm;
         }
-        throw new InvalidRequestException("Index '" + indexName + "' could not be found in any of the column families of keyspace '" + keyspace() + "'");
+
+        if (ifExists)
+            return null;
+        else
+            throw new InvalidRequestException("Index '" + indexName + "' could not be found in any of the column families of keyspace '" + keyspace() + "'");
     }
 
     private ColumnDefinition findIndexedColumn(CFMetaData cfm)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b73f9d42/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
index 710e750..ca66ac7 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
@@ -30,11 +30,13 @@ import org.apache.cassandra.transport.messages.ResultMessage;
 public class DropKeyspaceStatement extends SchemaAlteringStatement
 {
     private final String keyspace;
+    private final boolean ifExists;
 
-    public DropKeyspaceStatement(String keyspace)
+    public DropKeyspaceStatement(String keyspace, boolean ifExists)
     {
         super();
         this.keyspace = keyspace;
+        this.ifExists = ifExists;
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
@@ -57,7 +59,15 @@ public class DropKeyspaceStatement extends SchemaAlteringStatement
 
     public void announceMigration() throws ConfigurationException
     {
-        MigrationManager.announceKeyspaceDrop(keyspace);
+        try
+        {
+            MigrationManager.announceKeyspaceDrop(keyspace);
+        }
+        catch(ConfigurationException e)
+        {
+            if (!ifExists)
+                throw e;
+        }
     }
 
     public ResultMessage.SchemaChange.Change changeType()