You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ee...@apache.org on 2011/02/14 17:59:17 UTC

svn commit: r1070571 - in /cassandra/trunk: src/java/org/apache/cassandra/cql/ test/system/

Author: eevans
Date: Mon Feb 14 16:59:16 2011
New Revision: 1070571

URL: http://svn.apache.org/viewvc?rev=1070571&view=rev
Log:
CREATE COLUMNFAMILY w/ system tests

Patch by eevans for CASSANDRA-1709

Added:
    cassandra/trunk/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
Modified:
    cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g
    cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java
    cassandra/trunk/src/java/org/apache/cassandra/cql/StatementType.java
    cassandra/trunk/test/system/test_cql.py

Modified: cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g?rev=1070571&r1=1070570&r2=1070571&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cql/Cql.g Mon Feb 14 16:59:16 2011
@@ -39,6 +39,22 @@ options {
     package org.apache.cassandra.cql;
 }
 
+@lexer::members {
+    List<Token> tokens = new ArrayList<Token>();
+    
+    public void emit(Token token) {
+        state.token = token;
+        tokens.add(token);
+    }
+    
+    public Token nextToken() {
+        super.nextToken();
+        if (tokens.size() == 0)
+            return Token.EOF_TOKEN;
+        return tokens.remove(0);
+    }
+}
+
 query returns [CQLStatement stmnt]
     : selectStatement   { $stmnt = new CQLStatement(StatementType.SELECT, $selectStatement.expr); }
     | updateStatement   { $stmnt = new CQLStatement(StatementType.UPDATE, $updateStatement.expr); }
@@ -47,6 +63,7 @@ query returns [CQLStatement stmnt]
     | truncateStatement { $stmnt = new CQLStatement(StatementType.TRUNCATE, $truncateStatement.cfam); }
     | deleteStatement   { $stmnt = new CQLStatement(StatementType.DELETE, $deleteStatement.expr); }
     | createKeyspaceStatement { $stmnt = new CQLStatement(StatementType.CREATE_KEYSPACE, $createKeyspaceStatement.expr); }
+    | createColumnFamilyStatement { $stmnt = new CQLStatement(StatementType.CREATE_COLUMNFAMILY, $createColumnFamilyStatement.expr); }
     ;
 
 // USE <KEYSPACE>;
@@ -90,6 +107,27 @@ selectStatement returns [SelectStatement
       }
     ;
 
+// [FIRST n] [REVERSED] name1[[[,name2],nameN],...]
+// [FIRST n] [REVERSED] name1..nameN
+selectExpression returns [SelectExpression expr]
+    : {
+          int count = 10000;
+          boolean reversed = false;
+      }
+      ( K_FIRST cols=INTEGER { count = Integer.parseInt($cols.text); } )?
+      ( K_REVERSED { reversed = true; } )?
+      ( first=term { $expr = new SelectExpression(first, count, reversed); }
+            (',' next=term { $expr.and(next); })*
+      | start=term RANGEOP finish=term { $expr = new SelectExpression(start, finish, count, reversed); }
+      )
+    ;
+
+// relation [[AND relation] ...]
+whereClause returns [WhereClause clause]
+    : first=relation { $clause = new WhereClause(first); } 
+          (K_AND next=relation { $clause.and(next); })*
+    ;
+
 /**
  * BEGIN BATCH [USING CONSISTENCY.<LVL>]
  * UPDATE <CF> SET name1 = value1 WHERE KEY = keyname1;
@@ -175,6 +213,38 @@ createKeyspaceStatement returns [CreateK
           return new CreateKeyspaceStatement($keyspace.text, attrs);
       }
     ;
+    
+/**
+ * CREATE COLUMNFAMILY <CF> (
+ *     <name1> <type>,
+ *     <name2> <type>,
+ *     <name3> <type>
+ * ) WITH comparator = <type> [AND ...];
+ */
+createColumnFamilyStatement returns [CreateColumnFamilyStatement expr]
+    : K_CREATE K_COLUMNFAMILY name=IDENT { $expr = new CreateColumnFamilyStatement($name.text); } ( '('
+          col1=term v1=createCfamColumnValidator { $expr.addColumn(col1, $v1.validator); } ( ','
+          colN=term vN=createCfamColumnValidator { $expr.addColumn(colN, $vN.validator); } )*
+      ')' )?
+      ( K_WITH prop1=IDENT '=' arg1=createCfamKeywordArgument { $expr.addProperty($prop1.text, $arg1.arg); }
+          ( K_AND propN=IDENT '=' argN=createCfamKeywordArgument { $expr.addProperty($propN.text, $argN.arg); } )*
+      )?
+      endStmnt
+    ;
+
+createCfamColumnValidator returns [String validator]
+    : comparatorType { $validator = $comparatorType.text; }
+    | STRING_LITERAL { $validator = $STRING_LITERAL.text; }
+    ;
+
+createCfamKeywordArgument returns [String arg]
+    : comparatorType { $arg = $comparatorType.text; }
+    | value=( STRING_LITERAL | INTEGER | FLOAT ) { $arg = $value.text; }
+    ;
+
+comparatorType
+    : 'bytes' | 'ascii' | 'utf8' | 'int' | 'long' | 'uuid' | 'timeuuid'
+    ;
 
 term returns [Term item]
     : ( t=timeuuid | t=uuid | t=literal ) { $item = t; }
@@ -201,27 +271,6 @@ relation returns [Relation rel]
       { return new Relation(entity, $type.text, $t.item); }
     ;
 
-// relation [[AND relation] ...]
-whereClause returns [WhereClause clause]
-    : first=relation { $clause = new WhereClause(first); } 
-          (K_AND next=relation { $clause.and(next); })*
-    ;
-
-// [FIRST n] [REVERSED] name1[[[,name2],nameN],...]
-// [FIRST n] [REVERSED] name1..nameN
-selectExpression returns [SelectExpression expr]
-    : {
-          int count = 10000;
-          boolean reversed = false;
-      }
-      ( K_FIRST cols=INTEGER { count = Integer.parseInt($cols.text); } )?
-      ( K_REVERSED { reversed = true; } )?
-      ( first=term { $expr = new SelectExpression(first, count, reversed); }
-            (',' next=term { $expr.and(next); })*
-      | start=term '..' finish=term { $expr = new SelectExpression(start, finish, count, reversed); }
-      )
-    ;
-
 // TRUNCATE <CF>;
 truncateStatement returns [String cfam]
     : K_TRUNCATE columnFamily=IDENT { $cfam = $columnFamily.text; } endStmnt
@@ -273,6 +322,7 @@ K_DELETE:      D E L E T E;
 K_IN:          I N;
 K_CREATE:      C R E A T E;
 K_KEYSPACE:    K E Y S P A C E;
+K_COLUMNFAMILY: C O L U M N F A M I L Y;
 
 // Case-insensitive alpha characters
 fragment A: ('a'|'A');
@@ -320,6 +370,10 @@ fragment LETTER
     : ('A'..'Z' | 'a'..'z')
     ;
 
+RANGEOP
+    : '..'
+    ;
+
 INTEGER
     : DIGIT+
     ;
@@ -328,6 +382,20 @@ LONG
     : INTEGER 'L' { setText($INTEGER.text); }
     ;
 
+/* Normally a lexer only emits one token at a time, but ours is tricked out
+ * to support multiple (see @lexer::members near the top of the grammar).
+ */
+FLOAT
+    : d=INTEGER r=RANGEOP
+      {
+          $d.setType(INTEGER);
+          emit($d);
+          $r.setType(RANGEOP);
+          emit($r);
+      }
+      | INTEGER '.' INTEGER
+    ;
+
 IDENT
     : LETTER (LETTER | DIGIT | '_')*
     ;
@@ -339,7 +407,7 @@ COMPIDENT
 UNICODE
     : 'u' STRING_LITERAL
     ;
-    
+
 WS
     : (' ' | '\t' | '\n' | '\r')+ { $channel = HIDDEN; }
     ;

Added: cassandra/trunk/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java?rev=1070571&view=auto
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java (added)
+++ cassandra/trunk/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java Mon Feb 14 16:59:16 2011
@@ -0,0 +1,282 @@
+/*
+ * 
+ * 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.cql;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.ConfigurationException;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyType;
+import org.apache.cassandra.thrift.InvalidRequestException;
+
+/** A <code>CREATE COLUMNFAMILY</code> parsed from a CQL query statement. */
+public class CreateColumnFamilyStatement
+{
+    private static final String KW_COMPARATOR = "comparator";
+    private static final String KW_COMMENT = "comment";
+    private static final String KW_ROWCACHESIZE = "row_cache_size";
+    private static final String KW_KEYCACHESIZE = "key_cache_size";
+    private static final String KW_READREPAIRCHANCE = "read_repair_chance";
+    private static final String KW_GCGRACESECONDS = "gc_grace_seconds";
+    private static final String KW_DEFAULTVALIDATION = "default_validation";
+    private static final String KW_MINCOMPACTIONTHRESHOLD = "min_compaction_threshold";
+    private static final String KW_MAXCOMPACTIONTHRESHOLD = "max_compaction_threshold";
+    private static final String KW_ROWCACHESAVEPERIODSECS = "row_cache_save_period_in_seconds";
+    private static final String KW_KEYCACHESAVEPERIODSECS = "key_cache_save_period_in_seconds";
+    private static final String KW_MEMTABLEFLUSHINMINS = "memtable_flush_after_mins";
+    private static final String KW_MEMTABLESIZEINMB = "memtable_throughput_in_mb";
+    private static final String KW_MEMTABLEOPSINMILLIONS = "memtable_operations_in_millions";
+    private static final String KW_REPLICATEONWRITE = "replicate_on_write";
+    
+    // Maps CQL short names to the respective Cassandra comparator/validator class names
+    private static final Map<String, String> comparators = new HashMap<String, String>();
+    private static final Set<String> keywords = new HashSet<String>();
+    
+    static
+    {
+        comparators.put("bytes", "BytesType");
+        comparators.put("ascii", "AsciiType");
+        comparators.put("utf8", "UTF8Type");
+        comparators.put("int", "IntegerType");
+        comparators.put("long", "LongType");
+        comparators.put("uuid", "LexicalUUIDType");
+        comparators.put("timeuuid", "TimeUUIDType");
+        
+        keywords.add(KW_COMPARATOR);
+        keywords.add(KW_COMMENT);
+        keywords.add(KW_ROWCACHESIZE);
+        keywords.add(KW_KEYCACHESIZE);
+        keywords.add(KW_READREPAIRCHANCE);
+        keywords.add(KW_GCGRACESECONDS);
+        keywords.add(KW_DEFAULTVALIDATION);
+        keywords.add(KW_MINCOMPACTIONTHRESHOLD);
+        keywords.add(KW_MAXCOMPACTIONTHRESHOLD);
+        keywords.add(KW_ROWCACHESAVEPERIODSECS);
+        keywords.add(KW_KEYCACHESAVEPERIODSECS);
+        keywords.add(KW_MEMTABLEFLUSHINMINS);
+        keywords.add(KW_MEMTABLESIZEINMB);
+        keywords.add(KW_MEMTABLEOPSINMILLIONS);
+        keywords.add(KW_REPLICATEONWRITE);
+    }
+ 
+    private final String name;
+    private final Map<Term, String> columns = new HashMap<Term, String>();
+    private final Map<String, String> properties = new HashMap<String, String>();
+    
+    public CreateColumnFamilyStatement(String name)
+    {
+        this.name = name;
+    }
+    
+    /** Perform validation of parsed params */
+    private void validate() throws InvalidRequestException
+    {
+        // Catch the case where someone passed a kwarg that is not recognized.
+        Set<String> keywordsFound = new HashSet<String>(properties.keySet());
+        keywordsFound.removeAll(keywords);
+        
+        for (String bogus : keywordsFound)
+            throw new InvalidRequestException(bogus + " is not a valid keyword argument for CREATE COLUMNFAMILY");
+        
+        // Validate min/max compaction thresholds
+        Integer minCompaction = getPropertyInt(KW_MINCOMPACTIONTHRESHOLD, null);
+        Integer maxCompaction = getPropertyInt(KW_MAXCOMPACTIONTHRESHOLD, null);
+        
+        if ((minCompaction != null) && (maxCompaction != null))     // Both min and max are set
+        {
+            if ((minCompaction > maxCompaction) && (maxCompaction != 0))
+                throw new InvalidRequestException(String.format("%s cannot be larger than %s",
+                                                                KW_MINCOMPACTIONTHRESHOLD,
+                                                                KW_MAXCOMPACTIONTHRESHOLD));
+        }
+        else if (minCompaction != null)     // Only the min threshold is set
+        {
+            if (minCompaction > CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD)
+                throw new InvalidRequestException(String.format("%s cannot be larger than %s, (default %s)",
+                                                                KW_MINCOMPACTIONTHRESHOLD,
+                                                                KW_MAXCOMPACTIONTHRESHOLD,
+                                                                CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD));
+        }
+        else if (maxCompaction != null)     // Only the max threshold is set
+        {
+            if ((maxCompaction < CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD) && (maxCompaction != 0))
+                throw new InvalidRequestException(String.format("%s cannot be smaller than %s, (default %s)",
+                                                                KW_MAXCOMPACTIONTHRESHOLD,
+                                                                KW_MINCOMPACTIONTHRESHOLD,
+                                                                CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD));
+        }
+        
+        // Validate memtable settings
+        Integer memMins = getPropertyInt(KW_MEMTABLEFLUSHINMINS, null);
+        Integer memMb = getPropertyInt(KW_MEMTABLESIZEINMB, null);
+        Double memOps = getPropertyDouble(KW_MEMTABLEOPSINMILLIONS, null);
+        
+        if ((memMins != null) && (memMins <= 0))
+            throw new InvalidRequestException(String.format("%s must be non-negative and greater than zero",
+                                                            KW_MEMTABLEFLUSHINMINS));
+        if ((memMb != null) && (memMb <= 0))
+            throw new InvalidRequestException(String.format("%s must be non-negative and greater than zero",
+                                                            KW_MEMTABLESIZEINMB));
+        if ((memOps != null) && (memOps <=0))
+            throw new InvalidRequestException(String.format("%s must be non-negative and greater than zero",
+                                                            KW_MEMTABLEOPSINMILLIONS));
+    }
+    
+    /** Map a column name to a validator for its value */
+    public void addColumn(Term term, String comparator)
+    {
+        columns.put(term, comparator);
+    }
+    
+    /** Map a keyword to the corresponding value */
+    public void addProperty(String name, String value)
+    {
+        properties.put(name, value);
+    }
+    
+    /** Name of the column family to create */
+    public String getName()
+    {
+        return name;
+    }
+    
+    // Column definitions
+    private Map<ByteBuffer, ColumnDefinition> getColumns() throws InvalidRequestException
+    {
+        Map<ByteBuffer, ColumnDefinition> columnDefs = new HashMap<ByteBuffer, ColumnDefinition>();
+        
+        for (Map.Entry<Term, String> col : columns.entrySet())
+        {
+            try
+            {
+                ByteBuffer columnName = col.getKey().getByteBuffer();
+                String validator = comparators.containsKey(col.getValue()) ? comparators.get(col.getValue()) : col.getValue();
+                columnDefs.put(columnName, new ColumnDefinition(columnName, validator, null, null));
+            }
+            catch (ConfigurationException e)
+            {
+                InvalidRequestException ex = new InvalidRequestException(e.toString());
+                ex.initCause(e);
+                throw ex;
+            }
+        }
+        
+        return columnDefs;
+    }
+    
+    /**
+     * Returns a CFMetaData instance based on the parameters parsed from this
+     * <code>CREATE</code> statement, or defaults where applicable.
+     * 
+     * @param keyspace keyspace to apply this column family to
+     * @return a CFMetaData instance corresponding to the values parsed from this statement
+     * @throws InvalidRequestException on failure to validate parsed parameters
+     */
+    public CFMetaData getCFMetaData(String keyspace) throws InvalidRequestException
+    {
+        validate();
+        
+        try
+        {
+            return new CFMetaData(keyspace,
+                                  name,
+                                  ColumnFamilyType.create("Standard"),
+                                  DatabaseDescriptor.getComparator(comparators.get(properties.get(KW_COMPARATOR))),
+                                  null,
+                                  properties.get(KW_COMMENT),
+                                  getPropertyDouble(KW_ROWCACHESIZE, CFMetaData.DEFAULT_ROW_CACHE_SIZE),
+                                  getPropertyDouble(KW_KEYCACHESIZE, CFMetaData.DEFAULT_KEY_CACHE_SIZE),
+                                  getPropertyDouble(KW_READREPAIRCHANCE, CFMetaData.DEFAULT_READ_REPAIR_CHANCE),
+                                  getPropertyBoolean(KW_REPLICATEONWRITE, false),
+                                  getPropertyInt(KW_GCGRACESECONDS, CFMetaData.DEFAULT_GC_GRACE_SECONDS),
+                                  DatabaseDescriptor.getComparator(comparators.get(properties.get(KW_DEFAULTVALIDATION))),
+                                  getPropertyInt(KW_MINCOMPACTIONTHRESHOLD, CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD),
+                                  getPropertyInt(KW_MAXCOMPACTIONTHRESHOLD, CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD),
+                                  getPropertyInt(KW_ROWCACHESAVEPERIODSECS, CFMetaData.DEFAULT_ROW_CACHE_SAVE_PERIOD_IN_SECONDS),
+                                  getPropertyInt(KW_KEYCACHESAVEPERIODSECS, CFMetaData.DEFAULT_KEY_CACHE_SAVE_PERIOD_IN_SECONDS),
+                                  getPropertyInt(KW_MEMTABLEFLUSHINMINS, CFMetaData.DEFAULT_MEMTABLE_LIFETIME_IN_MINS),
+                                  getPropertyInt(KW_MEMTABLESIZEINMB, CFMetaData.DEFAULT_MEMTABLE_THROUGHPUT_IN_MB),
+                                  getPropertyDouble(KW_MEMTABLEOPSINMILLIONS, CFMetaData.DEFAULT_MEMTABLE_OPERATIONS_IN_MILLIONS),
+                                  getColumns());
+        }
+        catch (ConfigurationException e)
+        {
+            throw new InvalidRequestException(e.toString());
+        }
+    }
+    
+    // Return a property value, typed as a Boolean
+    private Boolean getPropertyBoolean(String key, Boolean defaultValue) throws InvalidRequestException
+    {
+        String value = properties.get(key);
+        return (value == null) ? defaultValue : value.toLowerCase().matches("(1|true|yes)");
+    }
+    
+    // Return a property value, typed as a Double
+    private Double getPropertyDouble(String key, Double defaultValue) throws InvalidRequestException
+    {
+        Double result;
+        String value = properties.get(key);
+        
+        if (value == null)
+            result = defaultValue;
+        else
+        {
+            try
+            {
+                result = Double.parseDouble(value);
+            }
+            catch (NumberFormatException e)
+            {
+                throw new InvalidRequestException(String.format("%s not valid for \"%s\"", value, key));
+            }
+        }
+        return result;
+    }
+    
+    // Return a property value, typed as an Integer
+    private Integer getPropertyInt(String key, Integer defaultValue) throws InvalidRequestException
+    {
+        Integer result;
+        String value = properties.get(key);
+        
+        if (value == null)
+            result = defaultValue;
+        else
+        {
+            try
+            {
+                result = Integer.parseInt(value);
+            }
+            catch (NumberFormatException e)
+            {
+                throw new InvalidRequestException(String.format("%s not valid for \"%s\"", value, key));
+            }
+        }
+        return result;
+    }
+}

Modified: cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java?rev=1070571&r1=1070570&r2=1070571&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java Mon Feb 14 16:59:16 2011
@@ -39,6 +39,7 @@ import org.apache.cassandra.config.Confi
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.QueryPath;
+import org.apache.cassandra.db.migration.AddColumnFamily;
 import org.apache.cassandra.db.migration.AddKeyspace;
 import org.apache.cassandra.db.migration.Migration;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -559,6 +560,29 @@ public class QueryProcessor
                 
                 avroResult.type = CqlResultType.VOID;
                 return avroResult;
+               
+            case CREATE_COLUMNFAMILY:
+                CreateColumnFamilyStatement createCf = (CreateColumnFamilyStatement)statement.statement;
+                
+                try
+                {
+                    applyMigrationOnStage(new AddColumnFamily(createCf.getCFMetaData(keyspace)));
+                }
+                catch (ConfigurationException e)
+                {
+                    InvalidRequestException ex = new InvalidRequestException(e.toString());
+                    ex.initCause(e);
+                    throw ex;
+                }
+                catch (IOException e)
+                {
+                    InvalidRequestException ex = new InvalidRequestException(e.toString());
+                    ex.initCause(e);
+                    throw ex;
+                }
+                
+                avroResult.type = CqlResultType.VOID;
+                return avroResult;
                 
         }
         

Modified: cassandra/trunk/src/java/org/apache/cassandra/cql/StatementType.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/cql/StatementType.java?rev=1070571&r1=1070570&r2=1070571&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cql/StatementType.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cql/StatementType.java Mon Feb 14 16:59:16 2011
@@ -22,5 +22,5 @@ package org.apache.cassandra.cql;
 
 public enum StatementType
 {
-    SELECT, UPDATE, BATCH_UPDATE, USE, TRUNCATE, DELETE, CREATE_KEYSPACE;
+    SELECT, UPDATE, BATCH_UPDATE, USE, TRUNCATE, DELETE, CREATE_KEYSPACE, CREATE_COLUMNFAMILY;
 }

Modified: cassandra/trunk/test/system/test_cql.py
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/system/test_cql.py?rev=1070571&r1=1070570&r2=1070571&view=diff
==============================================================================
--- cassandra/trunk/test/system/test_cql.py (original)
+++ cassandra/trunk/test/system/test_cql.py Mon Feb 14 16:59:16 2011
@@ -259,6 +259,62 @@ class TestCql(ThriftTester):
         strategy_class = "org.apache.cassandra.locator.SimpleStrategy"
         assert ksdef.strategy_class == strategy_class
         assert ksdef.strategy_options['DC1'] == "1"
+        
+    def test_create_column_family(self):
+        "create a new column family"
+        conn = init()
+        conn.execute("""
+            CREATE KEYSPACE CreateCFKeyspace WITH replication_factor = 1
+                AND strategy_class = "SimpleStrategy";
+        """)
+        conn.execute("USE CreateCFKeyspace;")
+        
+        conn.execute("""
+            CREATE COLUMNFAMILY NewCf1 (
+                "username" utf8,
+                "age" int,
+                "birthdate" long,
+                "id" uuid
+            ) WITH comparator = utf8 AND comment = "shiny, new, cf" AND
+                    default_validation = ascii;
+        """)
+        
+        # TODO: temporary (until this can be done with CQL).
+        ksdef = thrift_client.describe_keyspace("CreateCFKeyspace")
+        assert len(ksdef.cf_defs) == 1, \
+            "expected 1 column family total, found %d" % len(ksdef.cf_defs)
+        cfam= ksdef.cf_defs[0]
+        assert len(cfam.column_metadata) == 4, \
+            "expected 4 columns, found %d" % len(cfam.column_metadata)
+        assert cfam.comment == "shiny, new, cf"
+        assert cfam.default_validation_class == "org.apache.cassandra.db.marshal.AsciiType"
+        assert cfam.comparator_type == "org.apache.cassandra.db.marshal.UTF8Type"
+        
+        # No column defs, defaults all-around
+        conn.execute("CREATE COLUMNFAMILY NewCf2")
+        ksdef = thrift_client.describe_keyspace("CreateCFKeyspace")
+        assert len(ksdef.cf_defs) == 2, \
+            "expected 2 column families total, found %d" % len(ksdef.cf_defs)
+        
+        # No column defs
+        conn.execute("CREATE COLUMNFAMILY NewCf3 WITH comparator = long")
+        ksdef = thrift_client.describe_keyspace("CreateCFKeyspace")
+        assert len(ksdef.cf_defs) == 3, \
+            "expected 3 column families total, found %d" % len(ksdef.cf_defs)
+        cfam = [i for i in ksdef.cf_defs if i.name == "NewCf3"][0]
+        assert cfam.comparator_type == "org.apache.cassandra.db.marshal.LongType"
+        
+        # Column defs, defaults otherwise
+        conn.execute("CREATE COLUMNFAMILY NewCf4 (\"a\" int, \"b\" int)")
+        ksdef = thrift_client.describe_keyspace("CreateCFKeyspace")
+        assert len(ksdef.cf_defs) == 4, \
+            "expected 4 column families total, found %d" % len(ksdef.cf_defs)
+        cfam = [i for i in ksdef.cf_defs if i.name == "NewCf4"][0]
+        assert len(cfam.column_metadata) == 2, \
+            "expected 2 columns, found %d" % len(cfam.column_metadata)
+        for coldef in cfam.column_metadata:
+            assert coldef.name in ("a", "b"), "Unknown column name"
+            assert coldef.validation_class.endswith("marshal.IntegerType")
 
     def test_time_uuid(self):
         "store and retrieve time-based (type 1) uuids"