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 2012/09/12 18:06:57 UTC

git commit: Add ALTER KEYSPACE statement to CQL3

Updated Branches:
  refs/heads/trunk c64d975cd -> 1e126dada


Add ALTER KEYSPACE statement to CQL3

patch by slebresne; reviewed by xedin for CASSANDRA-4611


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

Branch: refs/heads/trunk
Commit: 1e126dadac0498a8fac9841da6d216d2510a2a1c
Parents: c64d975
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Wed Sep 12 18:03:19 2012 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Wed Sep 12 18:03:19 2012 +0200

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 .../org/apache/cassandra/config/KSMetaData.java    |    8 +-
 .../org/apache/cassandra/cql/QueryProcessor.java   |    3 +-
 src/java/org/apache/cassandra/cql3/CFPropDefs.java |  194 ++++-----------
 src/java/org/apache/cassandra/cql3/Cql.g           |   32 ++-
 src/java/org/apache/cassandra/cql3/KSPropDefs.java |   86 +++++++
 .../apache/cassandra/cql3/PropertyDefinitions.java |  148 +++++++++++
 .../cql3/statements/AlterKeyspaceStatement.java    |   86 +++++++
 .../cql3/statements/AlterTableStatement.java       |    6 +-
 .../statements/CreateColumnFamilyStatement.java    |    2 +-
 .../cql3/statements/CreateKeyspaceStatement.java   |   38 +--
 11 files changed, 412 insertions(+), 192 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 630ae18..371af7c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -61,6 +61,7 @@
  * Replace Throttle with Guava's RateLimiter for HintedHandOff (CASSANDRA-4541)
  * fix counter add/get using CQL2 and CQL3 in stress tool (CASSANDRA-4633)
  * Add sstable count per level to cfstats (CASSANDRA-4537)
+ * (cql3) Add ALTER KEYSPACE statement (CASSANDRA-4611)
 
 
 1.1.6

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/src/java/org/apache/cassandra/config/KSMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/KSMetaData.java b/src/java/org/apache/cassandra/config/KSMetaData.java
index 9feb0d3..050e32f 100644
--- a/src/java/org/apache/cassandra/config/KSMetaData.java
+++ b/src/java/org/apache/cassandra/config/KSMetaData.java
@@ -57,18 +57,18 @@ public final class KSMetaData
     }
 
     // For new user created keyspaces (through CQL)
-    public static KSMetaData newKeyspace(String name, String strategyName, Map<String, String> options) throws ConfigurationException
+    public static KSMetaData newKeyspace(String name, String strategyName, Map<String, String> options, boolean durableWrites) throws ConfigurationException
     {
         Class<? extends AbstractReplicationStrategy> cls = AbstractReplicationStrategy.getClass(strategyName);
         if (cls.equals(LocalStrategy.class))
             throw new ConfigurationException("Unable to use given strategy class: LocalStrategy is reserved for internal use.");
 
-        return newKeyspace(name, cls, options, Collections.<CFMetaData>emptyList());
+        return newKeyspace(name, cls, options, durableWrites, Collections.<CFMetaData>emptyList());
     }
 
-    public static KSMetaData newKeyspace(String name, Class<? extends AbstractReplicationStrategy> strategyClass, Map<String, String> options, Iterable<CFMetaData> cfDefs)
+    public static KSMetaData newKeyspace(String name, Class<? extends AbstractReplicationStrategy> strategyClass, Map<String, String> options, boolean durablesWrites, Iterable<CFMetaData> cfDefs)
     {
-        return new KSMetaData(name, strategyClass, options, true, cfDefs);
+        return new KSMetaData(name, strategyClass, options, durablesWrites, cfDefs);
     }
 
     public static KSMetaData cloneWith(KSMetaData ksm, Iterable<CFMetaData> cfDefs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/src/java/org/apache/cassandra/cql/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/QueryProcessor.java b/src/java/org/apache/cassandra/cql/QueryProcessor.java
index 074c50a..f3800cb 100644
--- a/src/java/org/apache/cassandra/cql/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql/QueryProcessor.java
@@ -647,7 +647,8 @@ public class QueryProcessor
                 {
                     KSMetaData ksm = KSMetaData.newKeyspace(create.getName(),
                                                             create.getStrategyClass(),
-                                                            create.getStrategyOptions());
+                                                            create.getStrategyOptions(),
+                                                            true);
                     ThriftValidation.validateKeyspaceNotYetExisting(ksm.name);
                     MigrationManager.announceNewKeyspace(ksm);
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/src/java/org/apache/cassandra/cql3/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/CFPropDefs.java
index 0d1722c..fb5f365 100644
--- a/src/java/org/apache/cassandra/cql3/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql3/CFPropDefs.java
@@ -17,20 +17,23 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
 import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.io.compress.CompressionParameters;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-public class CFPropDefs
+public class CFPropDefs extends PropertyDefinitions
 {
     private static final Logger logger = LoggerFactory.getLogger(CFPropDefs.class);
 
@@ -41,17 +44,15 @@ public class CFPropDefs
     public static final String KW_MINCOMPACTIONTHRESHOLD = "min_threshold";
     public static final String KW_MAXCOMPACTIONTHRESHOLD = "max_threshold";
     public static final String KW_REPLICATEONWRITE = "replicate_on_write";
-    public static final String KW_COMPACTION_STRATEGY_CLASS = "class";
     public static final String KW_CACHING = "caching";
     public static final String KW_BF_FP_CHANCE = "bloom_filter_fp_chance";
+    public static final String KW_COMPACTION = "compaction";
+    public static final String KW_COMPRESSION = "compression";
+
+    public static final String COMPACTION_STRATEGY_CLASS_KEY = "class";
 
-    // Maps CQL short names to the respective Cassandra comparator/validator class names
     public static final Set<String> keywords = new HashSet<String>();
     public static final Set<String> obsoleteKeywords = new HashSet<String>();
-    public static final Set<String> allowedKeywords = new HashSet<String>();
-
-    public static final String COMPACTION_PARAMETERS = "compaction";
-    public static final String COMPRESSION_PARAMETERS = "compression";
 
     static
     {
@@ -62,6 +63,8 @@ public class CFPropDefs
         keywords.add(KW_REPLICATEONWRITE);
         keywords.add(KW_CACHING);
         keywords.add(KW_BF_FP_CHANCE);
+        keywords.add(KW_COMPACTION);
+        keywords.add(KW_COMPRESSION);
 
         obsoleteKeywords.add("compaction_strategy_class");
         obsoleteKeywords.add("compaction_strategy_options");
@@ -69,170 +72,75 @@ public class CFPropDefs
         obsoleteKeywords.add("max_compaction_threshold");
         obsoleteKeywords.add("compaction_parameters");
         obsoleteKeywords.add("compression_parameters");
-
-        allowedKeywords.addAll(keywords);
-        allowedKeywords.addAll(obsoleteKeywords);
     }
 
-    public final Map<String, String> properties = new HashMap<String, String>();
     private Class<? extends AbstractCompactionStrategy> compactionStrategyClass = null;
-    public final Map<String, String> compactionStrategyOptions = new HashMap<String, String>();
-    public final Map<String, String> compressionParameters = new HashMap<String, String>()
-    {{
-        if (CFMetaData.DEFAULT_COMPRESSOR != null)
-            put(CompressionParameters.SSTABLE_COMPRESSION, CFMetaData.DEFAULT_COMPRESSOR);
-    }};
-
-    public void validate() throws ConfigurationException
+
+    public void validate() throws ConfigurationException, SyntaxException
     {
-        // Catch the case where someone passed a kwarg that is not recognized.
-        for (String bogus : Sets.difference(properties.keySet(), allowedKeywords))
-            throw new ConfigurationException(bogus + " is not a valid keyword argument for CREATE TABLE");
-        for (String obsolete : Sets.intersection(properties.keySet(), obsoleteKeywords))
-            logger.warn("Ignoring obsolete property {}", obsolete);
+        validate(keywords, obsoleteKeywords);
 
-        if (!compactionStrategyOptions.isEmpty())
+        Map<String, String> compactionOptions = getCompactionOptions();
+        if (!compactionOptions.isEmpty())
         {
-            if (compactionStrategyOptions.containsKey(KW_COMPACTION_STRATEGY_CLASS))
-            {
-                compactionStrategyClass = CFMetaData.createCompactionStrategy(compactionStrategyOptions.get(KW_COMPACTION_STRATEGY_CLASS));
-                compactionStrategyOptions.remove(KW_COMPACTION_STRATEGY_CLASS);
-            }
-            else
-            {
-                throw new ConfigurationException("Missing sub-option '" + KW_COMPACTION_STRATEGY_CLASS + "' for the '" + COMPACTION_PARAMETERS + "' option.");
-            }
+            String strategy = compactionOptions.get(COMPACTION_STRATEGY_CLASS_KEY);
+            if (strategy == null)
+                throw new ConfigurationException("Missing sub-option '" + COMPACTION_STRATEGY_CLASS_KEY + "' for the '" + KW_COMPACTION + "' option.");
+
+            compactionStrategyClass = CFMetaData.createCompactionStrategy(strategy);
+            compactionOptions.remove(COMPACTION_STRATEGY_CLASS_KEY);
         }
     }
 
-    /** Map a keyword to the corresponding value */
-    public void addProperty(String name, String value)
+    public Map<String, String> getCompactionOptions() throws SyntaxException
     {
-        properties.put(name, value);
+        Map<String, String> compactionOptions = getMap(KW_COMPACTION);
+        if (compactionOptions == null)
+            return Collections.<String, String>emptyMap();
+        return compactionOptions;
     }
 
-    public void addProperty(String name, Map<String, String> value)
+    public Map<String, String> getCompressionOptions() throws SyntaxException
     {
-        if (name.equalsIgnoreCase(COMPACTION_PARAMETERS))
+        Map<String, String> compressionOptions = getMap(KW_COMPRESSION);
+        if (compressionOptions == null)
         {
-            for (Map.Entry<String, String> entry : value.entrySet())
-                compactionStrategyOptions.put(entry.getKey().toLowerCase(), entry.getValue());
+            return new HashMap<String, String>()
+            {{
+                 if (CFMetaData.DEFAULT_COMPRESSOR != null)
+                     put(CompressionParameters.SSTABLE_COMPRESSION, CFMetaData.DEFAULT_COMPRESSOR);
+            }};
         }
-        else if (name.equalsIgnoreCase(COMPRESSION_PARAMETERS))
-        {
-            for (Map.Entry<String, String> entry : value.entrySet())
-                compressionParameters.put(entry.getKey().toLowerCase(), entry.getValue());
-        }
-    }
-
-    public void addAll(Map<String, String> propertyMap)
-    {
-        for (Map.Entry<String, String> entry : propertyMap.entrySet())
-            addProperty(entry.getKey(), entry.getValue());
+        return compressionOptions;
     }
 
-    public Boolean hasProperty(String name)
-    {
-        return properties.containsKey(name);
-    }
-
-    public void applyToCFMetadata(CFMetaData cfm) throws ConfigurationException
+    public void applyToCFMetadata(CFMetaData cfm) throws ConfigurationException, SyntaxException
     {
         if (hasProperty(KW_COMMENT))
-            cfm.comment(get(KW_COMMENT));
+            cfm.comment(getString(KW_COMMENT, ""));
 
         cfm.readRepairChance(getDouble(KW_READREPAIRCHANCE, cfm.getReadRepairChance()));
         cfm.dcLocalReadRepairChance(getDouble(KW_DCLOCALREADREPAIRCHANCE, cfm.getDcLocalReadRepair()));
         cfm.gcGraceSeconds(getInt(KW_GCGRACESECONDS, cfm.getGcGraceSeconds()));
         cfm.replicateOnWrite(getBoolean(KW_REPLICATEONWRITE, cfm.getReplicateOnWrite()));
-        cfm.minCompactionThreshold(toInt(KW_MINCOMPACTIONTHRESHOLD, compactionStrategyOptions.get(KW_MINCOMPACTIONTHRESHOLD), cfm.getMinCompactionThreshold()));
-        cfm.maxCompactionThreshold(toInt(KW_MAXCOMPACTIONTHRESHOLD, compactionStrategyOptions.get(KW_MAXCOMPACTIONTHRESHOLD), cfm.getMaxCompactionThreshold()));
+        cfm.minCompactionThreshold(toInt(KW_MINCOMPACTIONTHRESHOLD, getCompactionOptions().get(KW_MINCOMPACTIONTHRESHOLD), cfm.getMinCompactionThreshold()));
+        cfm.maxCompactionThreshold(toInt(KW_MAXCOMPACTIONTHRESHOLD, getCompactionOptions().get(KW_MAXCOMPACTIONTHRESHOLD), cfm.getMaxCompactionThreshold()));
         cfm.caching(CFMetaData.Caching.fromString(getString(KW_CACHING, cfm.getCaching().toString())));
         cfm.bloomFilterFpChance(getDouble(KW_BF_FP_CHANCE, cfm.getBloomFilterFpChance()));
 
         if (compactionStrategyClass != null)
-            cfm.compactionStrategyClass(compactionStrategyClass);
-
-        if (!compactionStrategyOptions.isEmpty())
-            cfm.compactionStrategyOptions(new HashMap<String, String>(compactionStrategyOptions));
-
-        if (!compressionParameters.isEmpty())
-            cfm.compressionParameters(CompressionParameters.create(compressionParameters));
-    }
-
-    public String get(String name)
-    {
-        return properties.get(name);
-    }
-
-    public String getString(String key, String defaultValue)
-    {
-        String value = properties.get(key);
-        return value != null ? value : defaultValue;
-    }
-
-    // Return a property value, typed as a Boolean
-    public Boolean getBoolean(String key, Boolean defaultValue)
-    {
-        String value = properties.get(key);
-        return (value == null) ? defaultValue : value.toLowerCase().matches("(1|true|yes)");
-    }
-
-    // Return a property value, typed as a Double
-    public Double getDouble(String key, Double defaultValue) throws ConfigurationException
-    {
-        Double result;
-        String value = properties.get(key);
-
-        if (value == null)
-            result = defaultValue;
-        else
         {
-            try
-            {
-                result = Double.valueOf(value);
-            }
-            catch (NumberFormatException e)
-            {
-                throw new ConfigurationException(String.format("%s not valid for \"%s\"", value, key));
-            }
+            cfm.compactionStrategyClass(compactionStrategyClass);
+            cfm.compactionStrategyOptions(new HashMap<String, String>(getCompactionOptions()));
         }
-        return result;
-    }
 
-    // Return a property value, typed as an Integer
-    public Integer getInt(String key, Integer defaultValue) throws ConfigurationException
-    {
-        String value = properties.get(key);
-        return toInt(key, value, defaultValue);
-    }
-
-    public static Integer toInt(String key, String value, Integer defaultValue) throws ConfigurationException
-    {
-        Integer result;
-
-        if (value == null)
-            result = defaultValue;
-        else
-        {
-            try
-            {
-                result = Integer.valueOf(value);
-            }
-            catch (NumberFormatException e)
-            {
-                throw new ConfigurationException(String.format("%s not valid for \"%s\"", value, key));
-            }
-        }
-        return result;
+        if (!getCompressionOptions().isEmpty())
+            cfm.compressionParameters(CompressionParameters.create(getCompressionOptions()));
     }
 
     @Override
     public String toString()
     {
-        return String.format("CFPropDefs(%s, compaction: %s, compression: %s)",
-                             properties.toString(),
-                             compactionStrategyOptions.toString(),
-                             compressionParameters.toString());
+        return String.format("CFPropDefs(%s)", properties.toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/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 acb6cb1..79c5b9f 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -169,6 +169,7 @@ cqlStatement returns [ParsedStatement stmt]
     | st15=grantStatement              { $stmt = st15; }
     | st16=revokeStatement             { $stmt = st16; }
     | st17=listGrantsStatement         { $stmt = st17; }
+    | st18=alterKeyspaceStatement      { $stmt = st18; }
     ;
 
 /*
@@ -373,8 +374,9 @@ batchStatementObjective returns [ModificationStatement statement]
  * CREATE KEYSPACE <KEYSPACE> WITH attr1 = value1 AND attr2 = value2;
  */
 createKeyspaceStatement returns [CreateKeyspaceStatement expr]
+    @init { KSPropDefs attrs = new KSPropDefs(); }
     : K_CREATE K_KEYSPACE ks=keyspaceName
-      K_WITH props=mapProperties { $expr = new CreateKeyspaceStatement(ks, props); }
+      K_WITH properties[attrs] { $expr = new CreateKeyspaceStatement(ks, attrs); }
     ;
 
 /**
@@ -424,6 +426,16 @@ createIndexStatement returns [CreateIndexStatement expr]
     ;
 
 /**
+ * ALTER KEYSPACE <KS> WITH <property> = <value>;
+ */
+alterKeyspaceStatement returns [AlterKeyspaceStatement expr]
+    @init { KSPropDefs attrs = new KSPropDefs(); }
+    : K_ALTER K_KEYSPACE ks=keyspaceName
+        K_WITH properties[attrs] { $expr = new AlterKeyspaceStatement(ks, attrs); }
+    ;
+
+
+/**
  * ALTER COLUMN FAMILY <CF> ALTER <column> TYPE <newtype>;
  * ALTER COLUMN FAMILY <CF> ADD <column> <newtype>;
  * ALTER COLUMN FAMILY <CF> DROP <column>;
@@ -438,8 +450,7 @@ alterTableStatement returns [AlterTableStatement expr]
           ( K_ALTER id=cident K_TYPE v=comparatorType { type = AlterTableStatement.Type.ALTER; }
           | K_ADD   id=cident v=comparatorType        { type = AlterTableStatement.Type.ADD; }
           | K_DROP  id=cident                         { type = AlterTableStatement.Type.DROP; }
-          | K_WITH  (property[props]
-                     ( K_AND property[props] )* { type = AlterTableStatement.Type.OPTS; })
+          | K_WITH  properties[props]                 { type = AlterTableStatement.Type.OPTS; }
           )
     {
         $expr = new AlterTableStatement(cf, type, id, v, props);
@@ -623,9 +634,13 @@ operation returns [Operation op]
     | '+' ml=map_literal { $op = MapOperation.Put(ml); }
     ;
 
-property[CFPropDefs props]
-    : k=cident '=' (simple=propertyValue { $props.addProperty(k.toString(), simple); }
-                   |   map=map_literal   { $props.addProperty(k.toString(), convertMap(map)); })
+properties[PropertyDefinitions props]
+    : property[props] (K_AND property[props])*
+    ;
+
+property[PropertyDefinitions props]
+    : k=cident '=' (simple=propertyValue { try { $props.addProperty(k.toString(), simple); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } }
+                   |   map=map_literal   { try { $props.addProperty(k.toString(), convertMap(map)); } catch (SyntaxException e) { addRecognitionError(e.getMessage()); } })
     ;
 
 propertyValue returns [String str]
@@ -633,11 +648,6 @@ propertyValue returns [String str]
     | u=unreserved_keyword                         { $str = u; }
     ;
 
-mapProperties returns [Map<String, Map<String, String>> props]
-    @init { $props = new HashMap<String, Map<String, String>>(); }
-    : k=cident '=' v=map_literal { $props.put(k.toString(), convertMap(v)); } (K_AND kn=cident '=' vn=map_literal { $props.put(kn.toString(), convertMap(vn)); } )*
-    ;
-
 // Either a string or a list of terms
 tokenDefinition returns [Pair<String, List<Term>> tkdef]
     : K_TOKEN { List<Term> l = new ArrayList<Term>(); }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/src/java/org/apache/cassandra/cql3/KSPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/KSPropDefs.java b/src/java/org/apache/cassandra/cql3/KSPropDefs.java
new file mode 100644
index 0000000..70df622
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/KSPropDefs.java
@@ -0,0 +1,86 @@
+/*
+ * 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.*;
+
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.*;
+
+public class KSPropDefs extends PropertyDefinitions
+{
+    public static final String KW_DURABLE_WRITES = "durable_writes";
+    public static final String KW_REPLICATION = "replication";
+
+    public static final String REPLICATION_STRATEGY_CLASS_KEY = "class";
+
+    public static final Set<String> keywords = new HashSet<String>();
+    public static final Set<String> obsoleteKeywords = new HashSet<String>();
+
+    static
+    {
+        keywords.add(KW_DURABLE_WRITES);
+        keywords.add(KW_REPLICATION);
+
+        obsoleteKeywords.add("strategy_class");
+    }
+
+    private String strategyClass;
+
+    public void validate() throws ConfigurationException, SyntaxException
+    {
+        validate(keywords, obsoleteKeywords);
+
+        Map<String, String> replicationOptions = getReplicationOptions();
+        if (!replicationOptions.isEmpty())
+        {
+            strategyClass = replicationOptions.get(REPLICATION_STRATEGY_CLASS_KEY);
+            replicationOptions.remove(REPLICATION_STRATEGY_CLASS_KEY);
+        }
+    }
+
+    public Map<String, String> getReplicationOptions() throws SyntaxException
+    {
+        Map<String, String> replicationOptions = getMap(KW_REPLICATION);
+        if (replicationOptions == null)
+            return Collections.<String, String>emptyMap();
+        return replicationOptions;
+    }
+
+    public String getReplicationStrategyClass()
+    {
+        return strategyClass;
+    }
+
+    public KSMetaData asKSMetadata(String ksName) throws RequestValidationException
+    {
+        return KSMetaData.newKeyspace(ksName, getReplicationStrategyClass(), getReplicationOptions(), getBoolean(KW_DURABLE_WRITES, true));
+    }
+
+    public KSMetaData asKSMetadataUpdate(KSMetaData old) throws RequestValidationException
+    {
+        String sClass = strategyClass;
+        Map<String, String> sOptions = getReplicationOptions();
+        if (sClass == null)
+        {
+            sClass = old.strategyClass.getName();
+            sOptions = old.strategyOptions;
+        }
+        return KSMetaData.newKeyspace(old.name, sClass, sOptions, getBoolean(KW_DURABLE_WRITES, old.durableWrites));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/src/java/org/apache/cassandra/cql3/PropertyDefinitions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/PropertyDefinitions.java b/src/java/org/apache/cassandra/cql3/PropertyDefinitions.java
new file mode 100644
index 0000000..3db84fa
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/PropertyDefinitions.java
@@ -0,0 +1,148 @@
+/*
+ * 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.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.exceptions.SyntaxException;
+
+public class PropertyDefinitions
+{
+    protected static final Logger logger = LoggerFactory.getLogger(PropertyDefinitions.class);
+
+    protected final Map<String, Object> properties = new HashMap<String, Object>();
+
+    public void addProperty(String name, String value) throws SyntaxException
+    {
+        if (properties.put(name, value) != null)
+            throw new SyntaxException(String.format("Multiple definition for property '%s'", name));
+    }
+
+    public void addProperty(String name, Map<String, String> value) throws SyntaxException
+    {
+        // Lowercase the map keys to be nice to users
+        Map<String, String> lowerCased = new HashMap<String, String>(value.size());
+        for (Map.Entry<String, String> entry : value.entrySet())
+            lowerCased.put(entry.getKey().toLowerCase(), entry.getValue());
+
+        if (properties.put(name, lowerCased) != null)
+            throw new SyntaxException(String.format("Multiple definition for property '%s'", name));
+    }
+
+    public void validate(Set<String> keywords, Set<String> obsolete) throws SyntaxException
+    {
+        for (String name : properties.keySet())
+        {
+            if (keywords.contains(name))
+                continue;
+
+            if (obsolete.contains(name))
+                logger.warn("Ignoring obsolete property {}", name);
+            else
+                throw new SyntaxException(String.format("Unknown property '%s'", name));
+        }
+    }
+
+    protected String getSimple(String name) throws SyntaxException
+    {
+        Object val = properties.get(name);
+        if (val == null)
+            return null;
+        if (!(val instanceof String))
+            throw new SyntaxException(String.format("Invalid value for property '%s'", name));
+        return (String)val;
+    }
+
+    protected Map<String, String> getMap(String name) throws SyntaxException
+    {
+        Object val = properties.get(name);
+        if (val == null)
+            return null;
+        if (!(val instanceof Map))
+            throw new SyntaxException(String.format("Invalid value for property '%s'", name));
+        return (Map<String, String>)val;
+    }
+
+    public Boolean hasProperty(String name)
+    {
+        return properties.containsKey(name);
+    }
+
+    public String getString(String key, String defaultValue) throws SyntaxException
+    {
+        String value = getSimple(key);
+        return value != null ? value : defaultValue;
+    }
+
+    // Return a property value, typed as a Boolean
+    public Boolean getBoolean(String key, Boolean defaultValue) throws SyntaxException
+    {
+        String value = getSimple(key);
+        return (value == null) ? defaultValue : value.toLowerCase().matches("(1|true|yes)");
+    }
+
+    // Return a property value, typed as a Double
+    public Double getDouble(String key, Double defaultValue) throws SyntaxException
+    {
+        String value = getSimple(key);
+        if (value == null)
+        {
+            return defaultValue;
+        }
+        else
+        {
+            try
+            {
+                return Double.valueOf(value);
+            }
+            catch (NumberFormatException e)
+            {
+                throw new SyntaxException(String.format("Invalid double value %s for '%s'", value, key));
+            }
+        }
+    }
+
+    // Return a property value, typed as an Integer
+    public Integer getInt(String key, Integer defaultValue) throws SyntaxException
+    {
+        String value = getSimple(key);
+        return toInt(key, value, defaultValue);
+    }
+
+    public static Integer toInt(String key, String value, Integer defaultValue) throws SyntaxException
+    {
+        if (value == null)
+        {
+            return defaultValue;
+        }
+        else
+        {
+            try
+            {
+                return Integer.valueOf(value);
+            }
+            catch (NumberFormatException e)
+            {
+                throw new SyntaxException(String.format("Invalid integer value %s for '%s'", value, key));
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
new file mode 100644
index 0000000..13c9e44
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
@@ -0,0 +1,86 @@
+/*
+ * 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 org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.KSPropDefs;
+import org.apache.cassandra.db.Table;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.service.StorageService;
+
+public class AlterKeyspaceStatement extends SchemaAlteringStatement
+{
+    private final String name;
+    private final KSPropDefs attrs;
+
+    public AlterKeyspaceStatement(String name, KSPropDefs attrs)
+    {
+        super();
+        this.name = name;
+        this.attrs = attrs;
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        state.hasKeyspaceAccess(name, Permission.ALTER);
+    }
+
+    @Override
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        super.validate(state);
+
+        KSMetaData ksm = Schema.instance.getKSMetaData(name);
+        if (ksm == null)
+            throw new InvalidRequestException("Unknown keyspace " + name);
+        if (ksm.name.equalsIgnoreCase(Table.SYSTEM_KS))
+            throw new InvalidRequestException("Cannot alter system keyspace");
+
+        attrs.validate();
+
+        if (attrs.getReplicationStrategyClass() == null && !attrs.getReplicationOptions().isEmpty())
+        {
+            throw new ConfigurationException("Missing replication strategy class");
+        }
+        else if (attrs.getReplicationStrategyClass() != null)
+        {
+            // trial run to let ARS validate class + per-class options
+            AbstractReplicationStrategy.createReplicationStrategy(name,
+                                                                  AbstractReplicationStrategy.getClass(attrs.getReplicationStrategyClass()),
+                                                                  StorageService.instance.getTokenMetadata(),
+                                                                  DatabaseDescriptor.getEndpointSnitch(),
+                                                                  attrs.getReplicationOptions());
+        }
+    }
+
+    public void announceMigration() throws RequestValidationException
+    {
+        KSMetaData ksm = Schema.instance.getKSMetaData(name);
+        // In the (very) unlikely case the keyspace was dropped since validate()
+        if (ksm == null)
+            throw new InvalidRequestException("Unknown keyspace " + name);
+
+        MigrationManager.announceKeyspaceUpdate(attrs.asKSMetadataUpdate(ksm));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index fe89f32..a33ff12 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -28,9 +28,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.MigrationManager;
 
@@ -62,7 +60,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
         state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.ALTER);
     }
 
-    public void announceMigration() throws InvalidRequestException, ConfigurationException
+    public void announceMigration() throws RequestValidationException
     {
         CFMetaData meta = validateColumnFamily(keyspace(), columnFamily());
         CFMetaData cfm = meta.clone();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/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 238ce67..1cf3137 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
@@ -119,7 +119,7 @@ public class CreateColumnFamilyStatement extends SchemaAlteringStatement
         return newCFMD;
     }
 
-    public void applyPropertiesTo(CFMetaData cfmd) throws InvalidRequestException, ConfigurationException
+    public void applyPropertiesTo(CFMetaData cfmd) throws RequestValidationException
     {
         cfmd.defaultValidator(defaultValidator)
             .columnMetadata(getColumns())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e126dad/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 4252dfc..5933292 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
@@ -25,6 +25,7 @@ import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.KSPropDefs;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
@@ -37,13 +38,8 @@ import org.apache.cassandra.thrift.ThriftValidation;
 /** A <code>CREATE KEYSPACE</code> statement parsed from a CQL query. */
 public class CreateKeyspaceStatement extends SchemaAlteringStatement
 {
-    private static String REPLICATION_PARAMETERS_PREFIX = "replication";
-    private static String REPLICATION_STRATEGY_CLASS_KEY = "class";
-
     private final String name;
-    private final Map<String, Map<String, String>> attrs;
-    private String strategyClass;
-    private final Map<String, String> strategyOptions = new HashMap<String, String>();
+    private final KSPropDefs attrs;
 
     /**
      * Creates a new <code>CreateKeyspaceStatement</code> instance for a given
@@ -52,7 +48,7 @@ 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, Map<String, Map<String, String>> attrs)
+    public CreateKeyspaceStatement(String name, KSPropDefs attrs)
     {
         super();
         this.name = name;
@@ -83,35 +79,21 @@ public class CreateKeyspaceStatement extends SchemaAlteringStatement
         if (name.length() > Schema.NAME_LENGTH)
             throw new InvalidRequestException(String.format("Keyspace names shouldn't be more than %s characters long (got \"%s\")", Schema.NAME_LENGTH, name));
 
-        if (!attrs.containsKey(REPLICATION_PARAMETERS_PREFIX))
-            throw new InvalidRequestException("missing required argument '" +  REPLICATION_PARAMETERS_PREFIX + "'");
-
-        Map<String, String> replication_parameters = attrs.get(REPLICATION_PARAMETERS_PREFIX);
-
-        strategyClass = replication_parameters.get(REPLICATION_STRATEGY_CLASS_KEY);
-
-        if (strategyClass == null)
-            throw new InvalidRequestException("missing required field '" + REPLICATION_STRATEGY_CLASS_KEY + "' for '" + REPLICATION_PARAMETERS_PREFIX + "' option");
-
-        for (Map.Entry<String, String> entry : replication_parameters.entrySet())
-        {
-            if (entry.getKey().equals(REPLICATION_STRATEGY_CLASS_KEY))
-                continue;
+        attrs.validate();
 
-            strategyOptions.put(entry.getKey(), entry.getValue());
-        }
+        if (attrs.getReplicationStrategyClass() == null)
+            throw new ConfigurationException("Missing mandatory replication strategy class");
 
         // trial run to let ARS validate class + per-class options
         AbstractReplicationStrategy.createReplicationStrategy(name,
-                                                              AbstractReplicationStrategy.getClass(strategyClass),
+                                                              AbstractReplicationStrategy.getClass(attrs.getReplicationStrategyClass()),
                                                               StorageService.instance.getTokenMetadata(),
                                                               DatabaseDescriptor.getEndpointSnitch(),
-                                                              strategyOptions);
+                                                              attrs.getReplicationOptions());
     }
 
-    public void announceMigration() throws InvalidRequestException, ConfigurationException
+    public void announceMigration() throws RequestValidationException
     {
-        KSMetaData ksm = KSMetaData.newKeyspace(name, strategyClass, strategyOptions);
-        MigrationManager.announceNewKeyspace(ksm);
+        MigrationManager.announceNewKeyspace(attrs.asKSMetadata(name));
     }
 }