You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by gd...@apache.org on 2011/03/17 15:50:33 UTC

svn commit: r1082510 [1/2] - in /cassandra/trunk: bin/ drivers/java/test/org/apache/cassandra/cql/ src/java/org/apache/cassandra/config/ src/java/org/apache/cassandra/cql/ src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/db/migration/ sr...

Author: gdusbabek
Date: Thu Mar 17 14:50:32 2011
New Revision: 1082510

URL: http://svn.apache.org/viewvc?rev=1082510&view=rev
Log:
sanitize configuration code. patch by Jon Hermes, reviewed by gdusbabek. CASSANDRA-1906

Removed:
    cassandra/trunk/bin/schematool
    cassandra/trunk/bin/schematool.bat
    cassandra/trunk/src/java/org/apache/cassandra/config/RawColumnDefinition.java
    cassandra/trunk/src/java/org/apache/cassandra/config/RawColumnFamily.java
    cassandra/trunk/src/java/org/apache/cassandra/config/RawKeyspace.java
    cassandra/trunk/src/java/org/apache/cassandra/tools/SchemaTool.java
Modified:
    cassandra/trunk/drivers/java/test/org/apache/cassandra/cql/EmbeddedServiceBase.java
    cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java
    cassandra/trunk/src/java/org/apache/cassandra/config/ColumnDefinition.java
    cassandra/trunk/src/java/org/apache/cassandra/config/Config.java
    cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
    cassandra/trunk/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
    cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    cassandra/trunk/src/java/org/apache/cassandra/db/migration/AddColumnFamily.java
    cassandra/trunk/src/java/org/apache/cassandra/db/migration/UpdateColumnFamily.java
    cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
    cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java
    cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java
    cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java
    cassandra/trunk/test/conf/cassandra.yaml
    cassandra/trunk/test/unit/org/apache/cassandra/SchemaLoader.java
    cassandra/trunk/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
    cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
    cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
    cassandra/trunk/test/unit/org/apache/cassandra/db/DefsTest.java

Modified: cassandra/trunk/drivers/java/test/org/apache/cassandra/cql/EmbeddedServiceBase.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/drivers/java/test/org/apache/cassandra/cql/EmbeddedServiceBase.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/drivers/java/test/org/apache/cassandra/cql/EmbeddedServiceBase.java (original)
+++ cassandra/trunk/drivers/java/test/org/apache/cassandra/cql/EmbeddedServiceBase.java Thu Mar 17 14:50:32 2011
@@ -4,6 +4,7 @@ import java.io.IOException;
 import java.net.Socket;
 import java.net.UnknownHostException;
 
+import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -43,7 +44,7 @@ public abstract class EmbeddedServiceBas
      */
     static void loadData() throws ConfigurationException
     {
-        for (KSMetaData table : DatabaseDescriptor.readTablesFromYaml())
+        for (KSMetaData table : SchemaLoader.schemaDefinition())
         {
             for (CFMetaData cfm : table.cfMetaData().values())
             {

Modified: cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/config/CFMetaData.java Thu Mar 17 14:50:32 2011
@@ -46,7 +46,6 @@ import org.apache.cassandra.utils.Pair;
 
 public final class CFMetaData
 {
-
     public final static double DEFAULT_ROW_CACHE_SIZE = 0.0;
     public final static double DEFAULT_KEY_CACHE_SIZE = 200000;
     public final static double DEFAULT_READ_REPAIR_CHANCE = 1.0;
@@ -63,43 +62,16 @@ public final class CFMetaData
     public final static double DEFAULT_MERGE_SHARDS_CHANCE = 0.1;
 
     private static final int MIN_CF_ID = 1000;
-
     private static final AtomicInteger idGen = new AtomicInteger(MIN_CF_ID);
     
     private static final BiMap<Pair<String, String>, Integer> cfIdMap = HashBiMap.create();
     
-    public static final CFMetaData StatusCf = newSystemTable(SystemTable.STATUS_CF, 0, "persistent metadata for the local node", BytesType.instance, null, DEFAULT_SYSTEM_MEMTABLE_THROUGHPUT_IN_MB);
-    public static final CFMetaData HintsCf = newSystemTable(HintedHandOffManager.HINTS_CF, 1, "hinted handoff data", BytesType.instance, BytesType.instance, Math.min(256, Math.max(32, DEFAULT_MEMTABLE_THROUGHPUT_IN_MB / 2)));
-    public static final CFMetaData MigrationsCf = newSystemTable(Migration.MIGRATIONS_CF, 2, "individual schema mutations", TimeUUIDType.instance, null, DEFAULT_SYSTEM_MEMTABLE_THROUGHPUT_IN_MB);
-    public static final CFMetaData SchemaCf = newSystemTable(Migration.SCHEMA_CF, 3, "current state of the schema", UTF8Type.instance, null, DEFAULT_SYSTEM_MEMTABLE_THROUGHPUT_IN_MB);
-    public static final CFMetaData IndexCf = newSystemTable(SystemTable.INDEX_CF, 5, "indexes that have been completed", UTF8Type.instance, null, DEFAULT_SYSTEM_MEMTABLE_THROUGHPUT_IN_MB);
-    public static final CFMetaData NodeIdCf = newSystemTable(SystemTable.NODE_ID_CF, 6, "nodeId and their metadata", TimeUUIDType.instance, null, DEFAULT_SYSTEM_MEMTABLE_THROUGHPUT_IN_MB);
-
-    private static CFMetaData newSystemTable(String cfName, int cfId, String comment, AbstractType comparator, AbstractType subComparator, int memtableThroughPutInMB)
-    {
-        return new CFMetaData(Table.SYSTEM_TABLE,
-                              cfName,
-                              subComparator == null ? ColumnFamilyType.Standard : ColumnFamilyType.Super,
-                              comparator,
-                              subComparator,
-                              comment,
-                              0,
-                              0.01,
-                              0,
-                              false,
-                              0,
-                              BytesType.instance,
-                              DEFAULT_MIN_COMPACTION_THRESHOLD,
-                              DEFAULT_MAX_COMPACTION_THRESHOLD,
-                              DEFAULT_ROW_CACHE_SAVE_PERIOD_IN_SECONDS,
-                              DEFAULT_KEY_CACHE_SAVE_PERIOD_IN_SECONDS,
-                              DEFAULT_MEMTABLE_LIFETIME_IN_MINS,
-                              memtableThroughPutInMB,
-                              sizeMemtableOperations(memtableThroughPutInMB),
-                              0,
-                              cfId,
-                              Collections.<ByteBuffer, ColumnDefinition>emptyMap());
-    }
+    public static final CFMetaData StatusCf = newSystemMetadata(SystemTable.STATUS_CF, 0, "persistent metadata for the local node", BytesType.instance, null, DEFAULT_SYSTEM_MEMTABLE_THROUGHPUT_IN_MB);
+    public static final CFMetaData HintsCf = newSystemMetadata(HintedHandOffManager.HINTS_CF, 1, "hinted handoff data", BytesType.instance, BytesType.instance, Math.min(256, Math.max(32, DEFAULT_MEMTABLE_THROUGHPUT_IN_MB / 2)));
+    public static final CFMetaData MigrationsCf = newSystemMetadata(Migration.MIGRATIONS_CF, 2, "individual schema mutations", TimeUUIDType.instance, null, DEFAULT_SYSTEM_MEMTABLE_THROUGHPUT_IN_MB);
+    public static final CFMetaData SchemaCf = newSystemMetadata(Migration.SCHEMA_CF, 3, "current state of the schema", UTF8Type.instance, null, DEFAULT_SYSTEM_MEMTABLE_THROUGHPUT_IN_MB);
+    public static final CFMetaData IndexCf = newSystemMetadata(SystemTable.INDEX_CF, 5, "indexes that have been completed", UTF8Type.instance, null, DEFAULT_SYSTEM_MEMTABLE_THROUGHPUT_IN_MB);
+    public static final CFMetaData NodeIdCf = newSystemMetadata(SystemTable.NODE_ID_CF, 6, "nodeId and their metadata", TimeUUIDType.instance, null, DEFAULT_SYSTEM_MEMTABLE_THROUGHPUT_IN_MB);
 
     /**
      * @return A calculated memtable throughput size for this machine.
@@ -128,9 +100,9 @@ public final class CFMetaData
     /**
      * @return The id for the given (ksname,cfname) pair, or null if it has been dropped.
      */
-    public static Integer getId(String table, String cfName)
+    public static Integer getId(String ksName, String cfName)
     {
-        return cfIdMap.get(new Pair<String, String>(table, cfName));
+        return cfIdMap.get(new Pair<String, String>(ksName, cfName));
     }
     
     // this gets called after initialization to make sure that id generation happens properly.
@@ -140,9 +112,21 @@ public final class CFMetaData
         idGen.set(cfIdMap.size() == 0 ? MIN_CF_ID : Math.max(Collections.max(cfIdMap.values()) + 1, MIN_CF_ID));
     }
 
+    /** adds this cfm to the map. */
+    public static void map(CFMetaData cfm) throws ConfigurationException
+    {
+        Pair<String, String> key = new Pair<String, String>(cfm.ksName, cfm.cfName);
+        if (cfIdMap.containsKey(key))
+            throw new ConfigurationException("Attempt to assign id to existing column family.");
+        else
+        {
+            cfIdMap.put(key, cfm.cfId);
+        }
+    }
+
     //REQUIRED
     public final Integer cfId;                        // internal id, never exposed to user
-    public final String tableName;                    // name of keyspace
+    public final String ksName;                    // name of keyspace
     public final String cfName;                       // name of this column family
     public final ColumnFamilyType cfType;             // standard, super
     public final AbstractType comparator;             // bytes, long, timeuuid, utf8, etc.
@@ -155,9 +139,9 @@ public final class CFMetaData
     private double readRepairChance;                  // default 1.0 (always), chance [0.0,1.0] of read repair
     private boolean replicateOnWrite;                 // default false
     private int gcGraceSeconds;                       // default 864000 (ten days)
-    private AbstractType defaultValidator;            // default none, use comparator types
-    private Integer minCompactionThreshold;           // default 4
-    private Integer maxCompactionThreshold;           // default 32
+    private AbstractType defaultValidator;            // default BytesType (no-op), use comparator types
+    private int minCompactionThreshold;               // default 4
+    private int maxCompactionThreshold;               // default 32
     private int rowCacheSavePeriodInSeconds;          // default 0 (off)
     private int keyCacheSavePeriodInSeconds;          // default 3600 (1 hour)
     private int memtableFlushAfterMins;               // default 60 
@@ -166,208 +150,144 @@ public final class CFMetaData
     private double mergeShardsChance;                 // default 0.1, chance [0.0, 1.0] of merging old shards during replication
     // NOTE: if you find yourself adding members to this class, make sure you keep the convert methods in lockstep.
 
-    private final Map<ByteBuffer, ColumnDefinition> column_metadata;
+    private Map<ByteBuffer, ColumnDefinition> column_metadata;
 
-    private CFMetaData(String tableName,
-                       String cfName,
-                       ColumnFamilyType cfType,
-                       AbstractType comparator,
-                       AbstractType subcolumnComparator,
-                       String comment,
-                       double rowCacheSize,
-                       double keyCacheSize,
-                       double readRepairChance,
-                       boolean replicateOnWrite,
-                       int gcGraceSeconds,
-                       AbstractType defaultValidator,
-                       int minCompactionThreshold,
-                       int maxCompactionThreshold,
-                       int rowCacheSavePeriodInSeconds,
-                       int keyCacheSavePeriodInSeconds,
-                       int memtableFlushAfterMins,
-                       Integer memtableThroughputInMb,
-                       Double memtableOperationsInMillions,
-                       double mergeShardsChance,
-                       Integer cfId,
-                       Map<ByteBuffer, ColumnDefinition> column_metadata)
-
-    {
-        assert column_metadata != null;
-        this.tableName = tableName;
-        this.cfName = cfName;
-        this.cfType = cfType;
-        this.comparator = comparator;
-        // the default subcolumncomparator is null per thrift spec, but only should be null if cfType == Standard. If
-        // cfType == Super, subcolumnComparator should default to BytesType if not set.
-        this.subcolumnComparator = subcolumnComparator == null && cfType == ColumnFamilyType.Super
-                                   ? BytesType.instance
-                                   : subcolumnComparator;
-        this.comment = comment == null ? "" : comment;
-        this.rowCacheSize = rowCacheSize;
-        this.keyCacheSize = keyCacheSize;
-        this.readRepairChance = readRepairChance;
-        this.replicateOnWrite = replicateOnWrite;
-        this.gcGraceSeconds = gcGraceSeconds;
-        this.defaultValidator = defaultValidator;
-        this.minCompactionThreshold = minCompactionThreshold;
-        this.maxCompactionThreshold = maxCompactionThreshold;
-        this.rowCacheSavePeriodInSeconds = rowCacheSavePeriodInSeconds;
-        this.keyCacheSavePeriodInSeconds = keyCacheSavePeriodInSeconds;
-        this.memtableFlushAfterMins = memtableFlushAfterMins;
-        this.memtableThroughputInMb = memtableThroughputInMb == null
-                                      ? DEFAULT_MEMTABLE_THROUGHPUT_IN_MB
-                                      : memtableThroughputInMb;
-
-        this.memtableOperationsInMillions = memtableOperationsInMillions == null
-                                            ? DEFAULT_MEMTABLE_OPERATIONS_IN_MILLIONS
-                                            : memtableOperationsInMillions;
-        this.mergeShardsChance = mergeShardsChance;
-        this.cfId = cfId;
-        this.column_metadata = new HashMap<ByteBuffer, ColumnDefinition>(column_metadata);
-    }
-    
-    /** adds this cfm to the map. */
-    public static void map(CFMetaData cfm) throws ConfigurationException
-    {
-        Pair<String, String> key = new Pair<String, String>(cfm.tableName, cfm.cfName);
-        if (cfIdMap.containsKey(key))
-            throw new ConfigurationException("Attempt to assign id to existing column family.");
-        else
-        {
-            cfIdMap.put(key, cfm.cfId);
-        }
+    public CFMetaData comment(String prop) {comment = prop; return this;}
+    public CFMetaData rowCacheSize(double prop) {rowCacheSize = prop; return this;}
+    public CFMetaData keyCacheSize(double prop) {keyCacheSize = prop; return this;}
+    public CFMetaData readRepairChance(double prop) {readRepairChance = prop; return this;}
+    public CFMetaData replicateOnWrite(boolean prop) {replicateOnWrite = prop; return this;}
+    public CFMetaData gcGraceSeconds(int prop) {gcGraceSeconds = prop; return this;}
+    public CFMetaData defaultValidator(AbstractType prop) {defaultValidator = prop; return this;}
+    public CFMetaData minCompactionThreshold(int prop) {minCompactionThreshold = prop; return this;}
+    public CFMetaData maxCompactionThreshold(int prop) {maxCompactionThreshold = prop; return this;}
+    public CFMetaData rowCacheSavePeriod(int prop) {rowCacheSavePeriodInSeconds = prop; return this;}
+    public CFMetaData keyCacheSavePeriod(int prop) {keyCacheSavePeriodInSeconds = prop; return this;}
+    public CFMetaData memTime(int prop) {memtableFlushAfterMins = prop; return this;}
+    public CFMetaData memSize(int prop) {memtableThroughputInMb = prop; return this;}
+    public CFMetaData memOps(double prop) {memtableOperationsInMillions = prop; return this;}
+    public CFMetaData mergeShardsChance(double prop) {mergeShardsChance = prop; return this;}
+    public CFMetaData columnMetadata(Map<ByteBuffer,ColumnDefinition> prop) {column_metadata = prop; return this;}
+
+    public CFMetaData(String keyspace, String name, ColumnFamilyType type, AbstractType comp, AbstractType subcc)
+    {
+        // Final fields must be set in constructor
+        ksName = keyspace;
+        cfName = name;
+        cfType = type;
+        comparator = comp;
+        subcolumnComparator = enforceSubccDefault(type, subcc);
+
+        // Default new CFMDs get an id chosen for them
+        cfId = nextId();
+
+        this.init();
+    }
+
+    private CFMetaData(String keyspace, String name, ColumnFamilyType type, AbstractType comp, AbstractType subcc, int id)
+    {
+        // Final fields must be set in constructor
+        ksName = keyspace;
+        cfName = name;
+        cfType = type;
+        comparator = comp;
+        subcolumnComparator = enforceSubccDefault(type, subcc);
+
+        // System cfs have specific ids, and copies of old CFMDs need
+        //  to copy over the old id.
+        cfId = id;
+
+        this.init();
+    }
+
+    private AbstractType enforceSubccDefault(ColumnFamilyType cftype, AbstractType subcc)
+    {
+        return (subcc == null) && (cftype == ColumnFamilyType.Super) ? BytesType.instance : subcc;
+    }
+
+    private void init()
+    {
+        // Set a bunch of defaults
+        rowCacheSize                 = DEFAULT_ROW_CACHE_SIZE;
+        keyCacheSize                 = DEFAULT_KEY_CACHE_SIZE;
+        readRepairChance             = DEFAULT_READ_REPAIR_CHANCE;
+        replicateOnWrite             = DEFAULT_REPLICATE_ON_WRITE;
+        gcGraceSeconds               = DEFAULT_GC_GRACE_SECONDS;
+        minCompactionThreshold       = DEFAULT_MIN_COMPACTION_THRESHOLD;
+        maxCompactionThreshold       = DEFAULT_MAX_COMPACTION_THRESHOLD;
+        memtableFlushAfterMins       = DEFAULT_MEMTABLE_LIFETIME_IN_MINS;
+        memtableThroughputInMb       = DEFAULT_MEMTABLE_THROUGHPUT_IN_MB;
+        memtableOperationsInMillions = DEFAULT_MEMTABLE_OPERATIONS_IN_MILLIONS;
+        mergeShardsChance            = DEFAULT_MERGE_SHARDS_CHANCE;
+
+        // Defaults strange or simple enough to not need a DEFAULT_T for
+        defaultValidator = BytesType.instance;
+        comment = "";
+        column_metadata = new HashMap<ByteBuffer,ColumnDefinition>();
+    }
+
+    private static CFMetaData newSystemMetadata(String cfName, int cfId, String comment, AbstractType comparator, AbstractType subcc, int memtableThroughPutInMB)
+    {
+        ColumnFamilyType type = subcc == null ? ColumnFamilyType.Standard : ColumnFamilyType.Super;
+        CFMetaData newCFMD = new CFMetaData(Table.SYSTEM_TABLE, cfName, type, comparator,  subcc, cfId);
+
+        return newCFMD.comment(comment)
+                      .keyCacheSize(0.01)
+                      .readRepairChance(0)
+                      .gcGraceSeconds(0)
+                      .memSize(memtableThroughPutInMB)
+                      .memOps(sizeMemtableOperations(memtableThroughPutInMB))
+                      .mergeShardsChance(0.0);
     }
 
-    public CFMetaData(String tableName,
-                      String cfName,
-                      ColumnFamilyType cfType,
-                      AbstractType comparator,
-                      AbstractType subcolumnComparator,
-                      String comment,
-                      double rowCacheSize,
-                      double keyCacheSize,
-                      double readRepairChance,
-                      boolean replicateOnWrite,
-                      int gcGraceSeconds,
-                      AbstractType defaultValidator,
-                      int minCompactionThreshold,
-                      int maxCompactionThreshold,
-                      int rowCacheSavePeriodInSeconds,
-                      int keyCacheSavePeriodInSeconds,
-                      int memTime,
-                      Integer memSize,
-                      Double memOps,
-                      double mergeShardsChance,
-                      //This constructor generates the id!
-                      Map<ByteBuffer, ColumnDefinition> column_metadata)
-    {
-        this(tableName,
-             cfName,
-             cfType,
-             comparator,
-             subcolumnComparator,
-             comment,
-             rowCacheSize,
-             keyCacheSize,
-             readRepairChance,
-             replicateOnWrite,
-             gcGraceSeconds,
-             defaultValidator,
-             minCompactionThreshold,
-             maxCompactionThreshold,
-             rowCacheSavePeriodInSeconds,
-             keyCacheSavePeriodInSeconds,
-             memTime,
-             memSize,
-             memOps,
-             mergeShardsChance,
-             nextId(),
-             column_metadata);
-    }
-    
     public static CFMetaData newIndexMetadata(CFMetaData parent, ColumnDefinition info, AbstractType columnComparator)
     {
-        return new CFMetaData(parent.tableName,
-                              indexName(parent.cfName, info),
-                              ColumnFamilyType.Standard,
-                              columnComparator,
-                              null,
-                              "",
-                              0,
-                              0,
-                              0,
-                              false,
-                              parent.gcGraceSeconds,
-                              BytesType.instance,
-                              parent.minCompactionThreshold,
-                              parent.maxCompactionThreshold,
-                              0,
-                              0,
-                              parent.memtableFlushAfterMins,
-                              parent.memtableThroughputInMb,
-                              parent.memtableOperationsInMillions,
-                              0,
-                              Collections.<ByteBuffer, ColumnDefinition>emptyMap());
+        return new CFMetaData(parent.ksName, indexName(parent.cfName, info), ColumnFamilyType.Standard, columnComparator, null)
+                             .keyCacheSize(0.0)
+                             .readRepairChance(0.0)
+                             .gcGraceSeconds(parent.gcGraceSeconds)
+                             .minCompactionThreshold(parent.minCompactionThreshold)
+                             .maxCompactionThreshold(parent.maxCompactionThreshold)
+                             .memTime(parent.memtableFlushAfterMins)
+                             .memSize(parent.memtableThroughputInMb)
+                             .memOps(parent.memtableOperationsInMillions);
     }
 
-    /** clones an existing CFMetaData using the same id. */
+    // Create a new CFMD by changing just the cfName
     public static CFMetaData rename(CFMetaData cfm, String newName)
     {
-        return new CFMetaData(cfm.tableName,
-                              newName,
-                              cfm.cfType,
-                              cfm.comparator,
-                              cfm.subcolumnComparator,
-                              cfm.comment,
-                              cfm.rowCacheSize,
-                              cfm.keyCacheSize,
-                              cfm.readRepairChance,
-                              cfm.replicateOnWrite,
-                              cfm.gcGraceSeconds,
-                              cfm.defaultValidator,
-                              cfm.minCompactionThreshold,
-                              cfm.maxCompactionThreshold,
-                              cfm.rowCacheSavePeriodInSeconds,
-                              cfm.keyCacheSavePeriodInSeconds,
-                              cfm.memtableFlushAfterMins,
-                              cfm.memtableThroughputInMb,
-                              cfm.memtableOperationsInMillions,
-                              cfm.mergeShardsChance,
-                              cfm.cfId,
-                              cfm.column_metadata);
-    }
-    
-    /** clones existing CFMetaData. keeps the id but changes the table name.*/
-    public static CFMetaData renameTable(CFMetaData cfm, String tableName)
-    {
-        return new CFMetaData(tableName,
-                              cfm.cfName,
-                              cfm.cfType,
-                              cfm.comparator,
-                              cfm.subcolumnComparator,
-                              cfm.comment,
-                              cfm.rowCacheSize,
-                              cfm.keyCacheSize,
-                              cfm.readRepairChance,
-                              cfm.replicateOnWrite,
-                              cfm.gcGraceSeconds,
-                              cfm.defaultValidator,
-                              cfm.minCompactionThreshold,
-                              cfm.maxCompactionThreshold,
-                              cfm.rowCacheSavePeriodInSeconds,
-                              cfm.keyCacheSavePeriodInSeconds,
-                              cfm.memtableFlushAfterMins,
-                              cfm.memtableThroughputInMb,
-                              cfm.memtableOperationsInMillions,
-                              cfm.mergeShardsChance,
-                              cfm.cfId,
-                              cfm.column_metadata);
+        return copyOpts(new CFMetaData(cfm.ksName, newName, cfm.cfType, cfm.comparator, cfm.subcolumnComparator, cfm.cfId), cfm);
     }
-    
+
+    // Create a new CFMD by changing just the ksName
+    public static CFMetaData renameTable(CFMetaData cfm, String ksName)
+    {
+        return copyOpts(new CFMetaData(ksName, cfm.cfName, cfm.cfType, cfm.comparator, cfm.subcolumnComparator, cfm.cfId), cfm);
+    }
+
+    private static CFMetaData copyOpts(CFMetaData newCFMD, CFMetaData oldCFMD)
+    {
+        return newCFMD.comment(oldCFMD.comment)
+                      .rowCacheSize(oldCFMD.rowCacheSize)
+                      .keyCacheSize(oldCFMD.keyCacheSize)
+                      .readRepairChance(oldCFMD.readRepairChance)
+                      .replicateOnWrite(oldCFMD.replicateOnWrite)
+                      .gcGraceSeconds(oldCFMD.gcGraceSeconds)
+                      .defaultValidator(oldCFMD.defaultValidator)
+                      .minCompactionThreshold(oldCFMD.minCompactionThreshold)
+                      .maxCompactionThreshold(oldCFMD.maxCompactionThreshold)
+                      .rowCacheSavePeriod(oldCFMD.rowCacheSavePeriodInSeconds)
+                      .keyCacheSavePeriod(oldCFMD.keyCacheSavePeriodInSeconds)
+                      .memTime(oldCFMD.memtableFlushAfterMins)
+                      .memSize(oldCFMD.memtableThroughputInMb)
+                      .memOps(oldCFMD.memtableOperationsInMillions)
+                      .columnMetadata(oldCFMD.column_metadata);
+    }
+
     /** used for evicting cf data out of static tracking collections. */
     public static void purge(CFMetaData cfm)
     {
-        cfIdMap.remove(new Pair<String, String>(cfm.tableName, cfm.cfName));
+        cfIdMap.remove(new Pair<String, String>(cfm.ksName, cfm.cfName));
     }
     
     /** convention for nameing secondary indexes. */
@@ -380,7 +300,7 @@ public final class CFMetaData
     {
         org.apache.cassandra.db.migration.avro.CfDef cf = new org.apache.cassandra.db.migration.avro.CfDef();
         cf.id = cfId;
-        cf.keyspace = new Utf8(tableName);
+        cf.keyspace = new Utf8(ksName);
         cf.name = new Utf8(cfName);
         cf.column_type = new Utf8(cfType.name());
         cf.comparator_type = new Utf8(comparator.getClass().getName());
@@ -433,38 +353,34 @@ public final class CFMetaData
             column_metadata.put(cd.name, cd);
         }
 
-        //isn't AVRO supposed to handle stuff like this?
-        Integer minct = cf.min_compaction_threshold == null ? DEFAULT_MIN_COMPACTION_THRESHOLD : cf.min_compaction_threshold;
-        Integer maxct = cf.max_compaction_threshold == null ? DEFAULT_MAX_COMPACTION_THRESHOLD : cf.max_compaction_threshold;
-        Integer row_cache_save_period_in_seconds = cf.row_cache_save_period_in_seconds == null ? DEFAULT_ROW_CACHE_SAVE_PERIOD_IN_SECONDS : cf.row_cache_save_period_in_seconds;
-        Integer key_cache_save_period_in_seconds = cf.key_cache_save_period_in_seconds == null ? DEFAULT_KEY_CACHE_SAVE_PERIOD_IN_SECONDS : cf.key_cache_save_period_in_seconds;
-        Integer memtable_flush_after_mins = cf.memtable_flush_after_mins == null ? DEFAULT_MEMTABLE_LIFETIME_IN_MINS : cf.memtable_flush_after_mins;
-        Integer memtable_throughput_in_mb = cf.memtable_throughput_in_mb == null ? DEFAULT_MEMTABLE_THROUGHPUT_IN_MB : cf.memtable_throughput_in_mb;
-        Double memtable_operations_in_millions = cf.memtable_operations_in_millions == null ? DEFAULT_MEMTABLE_OPERATIONS_IN_MILLIONS : cf.memtable_operations_in_millions;
-        double merge_shards_chance = cf.merge_shards_chance == null ? DEFAULT_MERGE_SHARDS_CHANCE : cf.merge_shards_chance;
-
-        return new CFMetaData(cf.keyspace.toString(),
-                              cf.name.toString(),
-                              ColumnFamilyType.create(cf.column_type.toString()),
-                              comparator,
-                              subcolumnComparator,
-                              cf.comment.toString(),
-                              cf.row_cache_size,
-                              cf.key_cache_size,
-                              cf.read_repair_chance,
-                              cf.replicate_on_write,
-                              cf.gc_grace_seconds,
-                              validator,
-                              minct,
-                              maxct,
-                              row_cache_save_period_in_seconds,
-                              key_cache_save_period_in_seconds,
-                              memtable_flush_after_mins,
-                              memtable_throughput_in_mb,
-                              memtable_operations_in_millions,
-                              merge_shards_chance,
-                              cf.id,
-                              column_metadata);
+        CFMetaData newCFMD = new CFMetaData(cf.keyspace.toString(),
+                                            cf.name.toString(),
+                                            ColumnFamilyType.create(cf.column_type.toString()),
+                                            comparator,
+                                            subcolumnComparator,
+                                            cf.id);
+
+        // When we pull up an old avro CfDef which doesn't have these arguments,
+        //  it doesn't default them correctly. Without explicit defaulting,
+        //  grandfathered metadata becomes wrong or causes crashes.
+        //  Isn't AVRO supposed to handle stuff like this?
+        if (cf.min_compaction_threshold != null) { newCFMD.minCompactionThreshold(cf.min_compaction_threshold); }
+        if (cf.max_compaction_threshold != null) { newCFMD.maxCompactionThreshold(cf.max_compaction_threshold); }
+        if (cf.row_cache_save_period_in_seconds != null) { newCFMD.rowCacheSavePeriod(cf.row_cache_save_period_in_seconds); }
+        if (cf.key_cache_save_period_in_seconds != null) { newCFMD.keyCacheSavePeriod(cf.key_cache_save_period_in_seconds); }
+        if (cf.memtable_flush_after_mins != null) { newCFMD.memTime(cf.memtable_flush_after_mins); }
+        if (cf.memtable_throughput_in_mb != null) { newCFMD.memSize(cf.memtable_throughput_in_mb); }
+        if (cf.memtable_operations_in_millions != null) { newCFMD.memOps(cf.memtable_operations_in_millions); }
+        if (cf.merge_shards_chance != null) { newCFMD.mergeShardsChance(cf.merge_shards_chance); }
+
+        return newCFMD.comment(cf.comment.toString())
+                      .rowCacheSize(cf.row_cache_size)
+                      .keyCacheSize(cf.key_cache_size)
+                      .readRepairChance(cf.read_repair_chance)
+                      .replicateOnWrite(cf.replicate_on_write)
+                      .gcGraceSeconds(cf.gc_grace_seconds)
+                      .defaultValidator(validator)
+                      .columnMetadata(column_metadata);
     }
     
     public String getComment()
@@ -560,7 +476,7 @@ public final class CFMetaData
 
         CFMetaData rhs = (CFMetaData) obj;
         return new EqualsBuilder()
-            .append(tableName, rhs.tableName)
+            .append(ksName, rhs.ksName)
             .append(cfName, rhs.cfName)
             .append(cfType, rhs.cfType)
             .append(comparator, rhs.comparator)
@@ -587,7 +503,7 @@ public final class CFMetaData
     public int hashCode()
     {
         return new HashCodeBuilder(29, 1597)
-            .append(tableName)
+            .append(ksName)
             .append(cfName)
             .append(cfType)
             .append(comparator)
@@ -629,6 +545,8 @@ public final class CFMetaData
     /** applies implicit defaults to cf definition. useful in updates */
     public static void applyImplicitDefaults(org.apache.cassandra.db.migration.avro.CfDef cf_def)
     {
+        if (cf_def.comment == null)
+            cf_def.comment = "";
         if (cf_def.min_compaction_threshold == null)
             cf_def.min_compaction_threshold = CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD;
         if (cf_def.max_compaction_threshold == null)
@@ -650,6 +568,8 @@ public final class CFMetaData
     /** applies implicit defaults to cf definition. useful in updates */
     public static void applyImplicitDefaults(org.apache.cassandra.thrift.CfDef cf_def) 
     {
+        if (!cf_def.isSetComment())
+            cf_def.setComment("");
         if (!cf_def.isSetMin_compaction_threshold())
             cf_def.setMin_compaction_threshold(CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD);
         if (!cf_def.isSetMax_compaction_threshold())
@@ -674,7 +594,7 @@ public final class CFMetaData
         // validate
         if (!cf_def.id.equals(cfId))
             throw new ConfigurationException("ids do not match.");
-        if (!cf_def.keyspace.toString().equals(tableName))
+        if (!cf_def.keyspace.toString().equals(ksName))
             throw new ConfigurationException("keyspaces do not match.");
         if (!cf_def.name.toString().equals(cfName))
             throw new ConfigurationException("names do not match.");
@@ -738,8 +658,9 @@ public final class CFMetaData
         // add the new ones coming in.
         for (org.apache.cassandra.db.migration.avro.ColumnDef def : toAdd)
         {
+            AbstractType dValidClass = DatabaseDescriptor.getComparator(def.validation_class);
             ColumnDefinition cd = new ColumnDefinition(def.name, 
-                                                       def.validation_class.toString(), 
+                                                       dValidClass,
                                                        def.index_type == null ? null : org.apache.cassandra.thrift.IndexType.valueOf(def.index_type.toString()), 
                                                        def.index_name == null ? null : def.index_name.toString());
             column_metadata.put(cd.name, cd);
@@ -749,7 +670,7 @@ public final class CFMetaData
     // converts CFM to thrift CfDef
     public static org.apache.cassandra.thrift.CfDef convertToThrift(CFMetaData cfm)
     {
-        org.apache.cassandra.thrift.CfDef def = new org.apache.cassandra.thrift.CfDef(cfm.tableName, cfm.cfName);
+        org.apache.cassandra.thrift.CfDef def = new org.apache.cassandra.thrift.CfDef(cfm.ksName, cfm.cfName);
         def.setId(cfm.cfId);
         def.setColumn_type(cfm.cfType.name());
         def.setComparator_type(cfm.comparator.getClass().getName());
@@ -792,7 +713,7 @@ public final class CFMetaData
     {
         org.apache.cassandra.db.migration.avro.CfDef def = new org.apache.cassandra.db.migration.avro.CfDef();
         def.name = cfm.cfName;
-        def.keyspace = cfm.tableName;
+        def.keyspace = cfm.ksName;
         def.id = cfm.cfId;
         def.column_type = cfm.cfType.name();
         def.comparator_type = cfm.comparator.getClass().getName();
@@ -957,7 +878,7 @@ public final class CFMetaData
     {
         return new ToStringBuilder(this)
             .append("cfId", cfId)
-            .append("tableName", tableName)
+            .append("ksName", ksName)
             .append("cfName", cfName)
             .append("cfType", cfType)
             .append("comparator", comparator)

Modified: cassandra/trunk/src/java/org/apache/cassandra/config/ColumnDefinition.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/config/ColumnDefinition.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/config/ColumnDefinition.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/config/ColumnDefinition.java Thu Mar 17 14:50:32 2011
@@ -22,10 +22,7 @@ package org.apache.cassandra.config;
 
 
 import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
+import java.util.*;
 
 import org.apache.avro.util.Utf8;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -42,12 +39,12 @@ public class ColumnDefinition
     private IndexType index_type;
     private String index_name;
 
-    public ColumnDefinition(ByteBuffer name, String validation_class, IndexType index_type, String index_name) throws ConfigurationException
+    public ColumnDefinition(ByteBuffer name, AbstractType validator, IndexType index_type, String index_name)
     {
         this.name = name;
         this.index_type = index_type;
         this.index_name = index_name;
-        this.validator = DatabaseDescriptor.getComparator(validation_class);
+        this.validator = validator;
     }
 
     @Override
@@ -96,7 +93,8 @@ public class ColumnDefinition
         String index_name = cd.index_name == null ? null : cd.index_name.toString();
         try
         {
-            return new ColumnDefinition(cd.name, cd.validation_class.toString(), index_type, index_name);
+            AbstractType validatorType = DatabaseDescriptor.getComparator(cd.validation_class);
+            return new ColumnDefinition(cd.name, validatorType, index_type, index_name);
         }
         catch (ConfigurationException e)
         {
@@ -106,14 +104,16 @@ public class ColumnDefinition
 
     public static ColumnDefinition fromColumnDef(ColumnDef thriftColumnDef) throws ConfigurationException
     {
-        return new ColumnDefinition(ByteBufferUtil.clone(thriftColumnDef.name), thriftColumnDef.validation_class, thriftColumnDef.index_type, thriftColumnDef.index_name);
+        AbstractType validatorType = DatabaseDescriptor.getComparator(thriftColumnDef.validation_class);
+        return new ColumnDefinition(ByteBufferUtil.clone(thriftColumnDef.name), validatorType, thriftColumnDef.index_type, thriftColumnDef.index_name);
     }
     
     public static ColumnDefinition fromColumnDef(org.apache.cassandra.db.migration.avro.ColumnDef avroColumnDef) throws ConfigurationException
     {
         validateIndexType(avroColumnDef);
+        AbstractType validatorType = DatabaseDescriptor.getComparator(avroColumnDef.validation_class);
         return new ColumnDefinition(avroColumnDef.name,
-                avroColumnDef.validation_class.toString(),
+                validatorType,
                 IndexType.valueOf(avroColumnDef.index_type == null ? D_COLDEF_INDEXTYPE : avroColumnDef.index_type.name()),
                 avroColumnDef.index_name == null ? D_COLDEF_INDEXNAME : avroColumnDef.index_name.toString());
     }
@@ -121,13 +121,13 @@ public class ColumnDefinition
     public static Map<ByteBuffer, ColumnDefinition> fromColumnDef(List<ColumnDef> thriftDefs) throws ConfigurationException
     {
         if (thriftDefs == null)
-            return Collections.emptyMap();
+            return new HashMap<ByteBuffer,ColumnDefinition>();
 
         Map<ByteBuffer, ColumnDefinition> cds = new TreeMap<ByteBuffer, ColumnDefinition>();
         for (ColumnDef thriftColumnDef : thriftDefs)
             cds.put(ByteBufferUtil.clone(thriftColumnDef.name), fromColumnDef(thriftColumnDef));
 
-        return Collections.unmodifiableMap(cds);
+        return cds;
     }
     
     public static Map<ByteBuffer, ColumnDefinition> fromColumnDefs(Iterable<org.apache.cassandra.db.migration.avro.ColumnDef> avroDefs) throws ConfigurationException

Modified: cassandra/trunk/src/java/org/apache/cassandra/config/Config.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/config/Config.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/config/Config.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/config/Config.java Thu Mar 17 14:50:32 2011
@@ -107,7 +107,6 @@ public class Config
 
     public Integer index_interval = 128;
 
-    public List<RawKeyspace> keyspaces;
     public Double flush_largest_memtables_at = 1.0;
     public Double reduce_cache_sizes_at = 1.0;
     public double reduce_cache_capacity_to = 0.6;

Modified: cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java Thu Mar 17 14:50:32 2011
@@ -131,17 +131,9 @@ public class DatabaseDescriptor
                 throw new AssertionError(e);
             }
             org.yaml.snakeyaml.constructor.Constructor constructor = new org.yaml.snakeyaml.constructor.Constructor(Config.class);
-            TypeDescription desc = new TypeDescription(Config.class);
-            desc.putListPropertyType("keyspaces", RawKeyspace.class);
-            TypeDescription ksDesc = new TypeDescription(RawKeyspace.class);
-            ksDesc.putListPropertyType("column_families", RawColumnFamily.class);
-            TypeDescription cfDesc = new TypeDescription(RawColumnFamily.class);
-            cfDesc.putListPropertyType("column_metadata", RawColumnDefinition.class);
             TypeDescription seedDesc = new TypeDescription(SeedProviderDef.class);
             seedDesc.putMapPropertyType("parameters", String.class, String.class);
-            constructor.addTypeDescription(desc);
-            constructor.addTypeDescription(ksDesc);
-            constructor.addTypeDescription(cfDesc);
+            constructor.addTypeDescription(seedDesc);
             Yaml yaml = new Yaml(new Loader(constructor));
             conf = (Config)yaml.load(input);
             
@@ -463,10 +455,9 @@ public class DatabaseDescriptor
             }
             
             if (hasExistingTables)
-                logger.info("Found table data in data directories. Consider using JMX to call org.apache.cassandra.service.StorageService.loadSchemaFromYaml().");
+                logger.info("Found table data in data directories. Consider using the CLI to define your schema.");
             else
-                logger.info("Consider using JMX to org.apache.cassandra.service.StorageService.loadSchemaFromYaml() or set up a schema using the system_* calls provided via thrift.");
-            
+                logger.info("To create keyspaces and column families, see 'help create keyspace' in the CLI, or set up a schema using the thrift system_* calls.");
         }
         else
         {
@@ -499,174 +490,10 @@ public class DatabaseDescriptor
                 // set defsVersion so that migrations leading up to emptiness aren't replayed.
                 defsVersion = uuid;
             }
-            
-            // since we loaded definitions from local storage, log a warning if definitions exist in yaml.
-            if (conf.keyspaces != null && conf.keyspaces.size() > 0)
-                logger.warn("Schema definitions were defined both locally and in " + DEFAULT_CONFIGURATION +
-                    ". Definitions in " + DEFAULT_CONFIGURATION + " were ignored.");
-            
         }
         CFMetaData.fixMaxId();
     }
 
-    /** reads xml. doesn't populate any internal structures. */
-    public static Collection<KSMetaData> readTablesFromYaml() throws ConfigurationException
-    {
-        List<KSMetaData> defs = new ArrayList<KSMetaData>();
-        if (conf.keyspaces == null)
-            return defs;
-        
-        /* Read the table related stuff from config */
-        for (RawKeyspace keyspace : conf.keyspaces)
-        {
-            /* parsing out the table name */
-            if (keyspace.name == null)
-            {
-                throw new ConfigurationException("Keyspace name attribute is required");
-            }
-            
-            if (keyspace.name.equalsIgnoreCase(Table.SYSTEM_TABLE))
-            {
-                throw new ConfigurationException("'system' is a reserved table name for Cassandra internals");
-            }
-            
-            /* See which replica placement strategy to use */
-            if (keyspace.replica_placement_strategy == null)
-            {
-                throw new ConfigurationException("Missing replica_placement_strategy directive for " + keyspace.name);
-            }
-            String strategyClassName = KSMetaData.convertOldStrategyName(keyspace.replica_placement_strategy);
-            Class<AbstractReplicationStrategy> strategyClass = FBUtilities.classForName(strategyClassName, "replication-strategy");
-            
-            /* Data replication factor */
-            if (keyspace.replication_factor == null)
-            {
-                throw new ConfigurationException("Missing replication_factor directory for keyspace " + keyspace.name);
-            }
-            
-            int size2 = keyspace.column_families.length;
-            CFMetaData[] cfDefs = new CFMetaData[size2];
-            int j = 0;
-            for (RawColumnFamily cf : keyspace.column_families)
-            {
-                if (cf.name == null)
-                {
-                    throw new ConfigurationException("ColumnFamily name attribute is required");
-                }
-                if (!cf.name.matches(Migration.NAME_VALIDATOR_REGEX))
-                {
-                    throw new ConfigurationException("ColumnFamily name contains invalid characters.");
-                }
-                
-                // Parse out the column comparators and validators
-                AbstractType comparator = getComparator(cf.compare_with);
-                AbstractType subcolumnComparator = null;
-                AbstractType default_validator = getComparator(cf.default_validation_class);
-
-                ColumnFamilyType cfType = cf.column_type == null ? ColumnFamilyType.Standard : cf.column_type;
-                if (cfType == ColumnFamilyType.Super)
-                {
-                    subcolumnComparator = getComparator(cf.compare_subcolumns_with);
-                }
-                else if (cf.compare_subcolumns_with != null)
-                {
-                    throw new ConfigurationException("compare_subcolumns_with is only a valid attribute on super columnfamilies (not regular columnfamily " + cf.name + ")");
-                }
-
-                if (cf.read_repair_chance < 0.0 || cf.read_repair_chance > 1.0)
-                {                        
-                    throw new ConfigurationException("read_repair_chance must be between 0.0 and 1.0 (0% and 100%)");
-                }
-
-                if (conf.dynamic_snitch_badness_threshold < 0.0 || conf.dynamic_snitch_badness_threshold > 1.0)
-                {
-                    throw new ConfigurationException("dynamic_snitch_badness_threshold must be between 0.0 and 1.0 (0% and 100%)");
-                }
-                
-                if (cf.min_compaction_threshold < 0 || cf.max_compaction_threshold < 0)
-                {
-                    throw new ConfigurationException("min/max_compaction_thresholds must be positive integers.");
-                }
-                if ((cf.min_compaction_threshold > cf.max_compaction_threshold) && cf.max_compaction_threshold != 0)
-                {
-                    throw new ConfigurationException("min_compaction_threshold must be smaller than max_compaction_threshold, or either must be 0 (disabled)");
-                }
-
-                if (cf.memtable_throughput_in_mb == null)
-                {
-                    cf.memtable_throughput_in_mb = CFMetaData.sizeMemtableThroughput();
-                    logger.info("memtable_throughput_in_mb not configured for " + cf.name + ", using " + cf.memtable_throughput_in_mb);
-                }
-                if (cf.memtable_operations_in_millions == null)
-                {
-                    cf.memtable_operations_in_millions = CFMetaData.sizeMemtableOperations(cf.memtable_throughput_in_mb);
-                    logger.info("memtable_operations_in_millions not configured for " + cf.name + ", using " + cf.memtable_operations_in_millions);
-                }
-
-                if (cf.memtable_operations_in_millions != null && cf.memtable_operations_in_millions <= 0)
-                {
-                    throw new ConfigurationException("memtable_operations_in_millions must be a positive double");
-                }
-
-                if (cf.merge_shards_chance < 0.0 || cf.merge_shards_chance > 1.0)
-                {
-                    throw new ConfigurationException("merge_shards_chance must be between 0.0 and 1.0 (0% and 100%)");
-                }
-
-                 Map<ByteBuffer, ColumnDefinition> metadata = new TreeMap<ByteBuffer, ColumnDefinition>();
-
-                for (RawColumnDefinition rcd : cf.column_metadata)
-                {
-                    if (rcd.name == null)
-                    {
-                        throw new ConfigurationException("name is required for column definitions.");
-                    }
-                    if (rcd.validator_class == null)
-                    {
-                        throw new ConfigurationException("validator is required for column definitions");
-                    }
-                    
-                    if ((rcd.index_type == null) && (rcd.index_name != null))
-                    {
-                        throw new ConfigurationException("index_name cannot be set if index_type is not also set");
-                    }
-
-                    ByteBuffer columnName = ByteBuffer.wrap(rcd.name.getBytes(Charsets.UTF_8));
-                    metadata.put(columnName, new ColumnDefinition(columnName, rcd.validator_class, rcd.index_type, rcd.index_name));
-                }
-
-                cfDefs[j++] = new CFMetaData(keyspace.name, 
-                                             cf.name, 
-                                             cfType,
-                                             comparator, 
-                                             subcolumnComparator, 
-                                             cf.comment, 
-                                             cf.rows_cached,
-                                             cf.keys_cached, 
-                                             cf.read_repair_chance,
-                                             cf.replicate_on_write,
-                                             cf.gc_grace_seconds,
-                                             default_validator,
-                                             cf.min_compaction_threshold,
-                                             cf.max_compaction_threshold,
-                                             cf.row_cache_save_period_in_seconds,
-                                             cf.key_cache_save_period_in_seconds,
-                                             cf.memtable_flush_after_mins,
-                                             cf.memtable_throughput_in_mb,
-                                             cf.memtable_operations_in_millions,
-                                             cf.merge_shards_chance,
-                                             metadata);
-            }
-            defs.add(new KSMetaData(keyspace.name,
-                                    strategyClass,
-                                    keyspace.strategy_options,
-                                    keyspace.replication_factor,
-                                    cfDefs));
-        }
-
-        return defs;
-    }
-
     public static IAuthenticator getAuthenticator()
     {
         return authenticator;
@@ -983,12 +810,12 @@ public class DatabaseDescriptor
         return dataFileDirectory;
     }
     
-    public static AbstractType getComparator(String tableName, String cfName)
+    public static AbstractType getComparator(String ksName, String cfName)
     {
-        assert tableName != null;
-        CFMetaData cfmd = getCFMetaData(tableName, cfName);
+        assert ksName != null;
+        CFMetaData cfmd = getCFMetaData(ksName, cfName);
         if (cfmd == null)
-            throw new IllegalArgumentException("Unknown ColumnFamily " + cfName + " in keyspace " + tableName);
+            throw new IllegalArgumentException("Unknown ColumnFamily " + cfName + " in keyspace " + ksName);
         return cfmd.comparator;
     }
 

Modified: 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=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java Thu Mar 17 14:50:32 2011
@@ -179,7 +179,8 @@ public class CreateColumnFamilyStatement
             try
             {
                 ByteBuffer columnName = col.getKey().getByteBuffer(comparator);
-                String validator = comparators.containsKey(col.getValue()) ? comparators.get(col.getValue()) : col.getValue();
+                String validatorClassName = comparators.containsKey(col.getValue()) ? comparators.get(col.getValue()) : col.getValue();
+                AbstractType validator = DatabaseDescriptor.getComparator(validatorClassName);
                 columnDefs.put(columnName, new ColumnDefinition(columnName, validator, null, null));
             }
             catch (ConfigurationException e)
@@ -204,39 +205,42 @@ public class CreateColumnFamilyStatement
     public CFMetaData getCFMetaData(String keyspace) throws InvalidRequestException
     {
         validate();
-        
+
+        CFMetaData newCFMD;
         try
         {
             // RPC uses BytesType as the default validator/comparator but BytesType expects hex for string terms, (not convenient).
             AbstractType<?> comparator = DatabaseDescriptor.getComparator(comparators.get(getPropertyString(KW_COMPARATOR, "ascii")));
             String validator = getPropertyString(KW_DEFAULTVALIDATION, "ascii");
-            
-            return new CFMetaData(keyspace,
-                                  name,
-                                  ColumnFamilyType.create("Standard"),
-                                  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(validator)),
-                                  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),
-                                  0,
-                                  getColumns(comparator));
+
+            newCFMD = new CFMetaData(keyspace,
+                                     name,
+                                     ColumnFamilyType.Standard,
+                                     comparator,
+                                     null);
+
+            newCFMD.comment(properties.get(KW_COMMENT))
+                   .rowCacheSize(getPropertyDouble(KW_ROWCACHESIZE, CFMetaData.DEFAULT_ROW_CACHE_SIZE))
+                   .keyCacheSize(getPropertyDouble(KW_KEYCACHESIZE, CFMetaData.DEFAULT_KEY_CACHE_SIZE))
+                   .readRepairChance(getPropertyDouble(KW_READREPAIRCHANCE, CFMetaData.DEFAULT_READ_REPAIR_CHANCE))
+                   .replicateOnWrite(getPropertyBoolean(KW_REPLICATEONWRITE, CFMetaData.DEFAULT_REPLICATE_ON_WRITE))
+                   .gcGraceSeconds(getPropertyInt(KW_GCGRACESECONDS, CFMetaData.DEFAULT_GC_GRACE_SECONDS))
+                   .defaultValidator(DatabaseDescriptor.getComparator(comparators.get(validator)))
+                   .minCompactionThreshold(getPropertyInt(KW_MINCOMPACTIONTHRESHOLD, CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD))
+                   .maxCompactionThreshold(getPropertyInt(KW_MAXCOMPACTIONTHRESHOLD, CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD))
+                   .rowCacheSavePeriod(getPropertyInt(KW_ROWCACHESAVEPERIODSECS, CFMetaData.DEFAULT_ROW_CACHE_SAVE_PERIOD_IN_SECONDS))
+                   .keyCacheSavePeriod(getPropertyInt(KW_KEYCACHESAVEPERIODSECS, CFMetaData.DEFAULT_KEY_CACHE_SAVE_PERIOD_IN_SECONDS))
+                   .memTime(getPropertyInt(KW_MEMTABLEFLUSHINMINS, CFMetaData.DEFAULT_MEMTABLE_LIFETIME_IN_MINS))
+                   .memSize(getPropertyInt(KW_MEMTABLESIZEINMB, CFMetaData.DEFAULT_MEMTABLE_THROUGHPUT_IN_MB))
+                   .memOps(getPropertyDouble(KW_MEMTABLEOPSINMILLIONS, CFMetaData.DEFAULT_MEMTABLE_OPERATIONS_IN_MILLIONS))
+                   .mergeShardsChance(0.0)
+                   .columnMetadata(getColumns(comparator));
         }
         catch (ConfigurationException e)
         {
             throw new InvalidRequestException(e.toString());
         }
+        return newCFMD;
     }
     
     private String getPropertyString(String key, String defaultValue)

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=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/cql/QueryProcessor.java Thu Mar 17 14:50:32 2011
@@ -684,18 +684,10 @@ public class QueryProcessor
                 // No meta-data, create a new column definition from scratch.
                 else
                 {
-                    try
-                    {
-                        columnDef = new ColumnDefinition(columnName,
-                                                         null,
-                                                         org.apache.cassandra.thrift.IndexType.KEYS,
-                                                         createIdx.getIndexName());
-                    }
-                    catch (ConfigurationException e)
-                    {
-                        // This should never happen
-                        throw new RuntimeException("Unexpected error creating ColumnDefinition", e);
-                    }
+                    columnDef = new ColumnDefinition(columnName,
+                                                     null,
+                                                     org.apache.cassandra.thrift.IndexType.KEYS,
+                                                     createIdx.getIndexName());
                 }
                 
                 CfDef cfamilyDef = CFMetaData.convertToAvro(oldCfm);

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Thu Mar 17 14:50:32 2011
@@ -215,7 +215,7 @@ public class ColumnFamilyStore implement
             return;
         }
         indexCfs.unregisterMBean();
-        SystemTable.setIndexRemoved(metadata.tableName, indexCfs.columnFamily);
+        SystemTable.setIndexRemoved(metadata.ksName, indexCfs.columnFamily);
         indexCfs.removeAllSSTables();
     }
 

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/migration/AddColumnFamily.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/migration/AddColumnFamily.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/migration/AddColumnFamily.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/migration/AddColumnFamily.java Thu Mar 17 14:50:32 2011
@@ -41,14 +41,14 @@ public class AddColumnFamily extends Mig
     {
         super(UUIDGen.makeType1UUIDFromHost(FBUtilities.getLocalAddress()), DatabaseDescriptor.getDefsVersion());
         this.cfm = cfm;
-        KSMetaData ksm = DatabaseDescriptor.getTableDefinition(cfm.tableName);
+        KSMetaData ksm = DatabaseDescriptor.getTableDefinition(cfm.ksName);
         
         if (ksm == null)
-            throw new ConfigurationException("No such keyspace: " + cfm.tableName);
+            throw new ConfigurationException("No such keyspace: " + cfm.ksName);
         else if (ksm.cfMetaData().containsKey(cfm.cfName))
             throw new ConfigurationException(String.format("%s already exists in keyspace %s",
                                                            cfm.cfName,
-                                                           cfm.tableName));
+                                                           cfm.ksName));
         else if (!Migration.isLegalName(cfm.cfName))
             throw new ConfigurationException("Invalid column family name: " + cfm.cfName);
         for (Map.Entry<ByteBuffer, ColumnDefinition> entry : cfm.getColumn_metadata().entrySet())
@@ -74,7 +74,7 @@ public class AddColumnFamily extends Mig
     public void applyModels() throws IOException
     {
         // reinitialize the table.
-        KSMetaData ksm = DatabaseDescriptor.getTableDefinition(cfm.tableName);
+        KSMetaData ksm = DatabaseDescriptor.getTableDefinition(cfm.ksName);
         ksm = makeNewKeyspaceDefinition(ksm);
         try
         {
@@ -84,7 +84,7 @@ public class AddColumnFamily extends Mig
         {
             throw new IOException(ex);
         }
-        Table.open(cfm.tableName); // make sure it's init-ed w/ the old definitions first, since we're going to call initCf on the new one manually
+        Table.open(cfm.ksName); // make sure it's init-ed w/ the old definitions first, since we're going to call initCf on the new one manually
         DatabaseDescriptor.setTableDefinition(ksm, newVersion);
         // these definitions could have come from somewhere else.
         CFMetaData.fixMaxId();

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/migration/UpdateColumnFamily.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/migration/UpdateColumnFamily.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/migration/UpdateColumnFamily.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/migration/UpdateColumnFamily.java Thu Mar 17 14:50:32 2011
@@ -1,8 +1,6 @@
 package org.apache.cassandra.db.migration;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Map;
 
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -70,7 +68,7 @@ public class UpdateColumnFamily extends 
     {
         if (clientMode)
             return;
-        ColumnFamilyStore cfs = Table.open(metadata.tableName).getColumnFamilyStore(metadata.cfName);
+        ColumnFamilyStore cfs = Table.open(metadata.ksName).getColumnFamilyStore(metadata.cfName);
         cfs.snapshot(Table.getTimestampedSnapshotName(null));
     }
 
@@ -90,7 +88,7 @@ public class UpdateColumnFamily extends 
 
         if (!clientMode)
         {
-            Table table = Table.open(metadata.tableName);
+            Table table = Table.open(metadata.ksName);
             ColumnFamilyStore oldCfs = table.getColumnFamilyStore(metadata.cfName);
             oldCfs.reload();
         }

Modified: cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java Thu Mar 17 14:50:32 2011
@@ -2100,156 +2100,6 @@ public class StorageService implements I
         setMode("Node is drained", true);
     }
 
-    /**
-     * load schema from yaml. This can only be done on a fresh system.
-     * @throws ConfigurationException
-     * @throws IOException
-     */
-    public void loadSchemaFromYAML() throws ConfigurationException, IOException
-    { 
-        // validate
-        final Collection<KSMetaData> tables = DatabaseDescriptor.readTablesFromYaml();
-        if (tables.isEmpty())
-            return;
-
-        for (KSMetaData table : tables)
-        {
-            if (!table.name.matches(Migration.NAME_VALIDATOR_REGEX))
-                throw new ConfigurationException("Invalid table name: " + table.name);
-            for (CFMetaData cfm : table.cfMetaData().values())
-                if (!Migration.isLegalName(cfm.cfName))
-                    throw new ConfigurationException("Invalid column family name: " + cfm.cfName);
-        }
-        
-        Callable<Migration> call = new Callable<Migration>()
-        {
-            public Migration call() throws Exception
-            {
-                // blow up if there is a schema saved.
-                if (DatabaseDescriptor.getDefsVersion().timestamp() > 0 || Migration.getLastMigrationId() != null)
-                    throw new ConfigurationException("Cannot import schema when one already exists");
-             
-                Migration migration = null;
-                for (KSMetaData table : tables)
-                {
-                    migration = new AddKeyspace(table); 
-                    migration.apply();
-                }
-                return migration;
-            }
-        };
-        Migration migration;
-        try
-        {
-            migration = StageManager.getStage(Stage.MIGRATION).submit(call).get();
-        }
-        catch (InterruptedException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (ExecutionException e)
-        {
-            if (e.getCause() instanceof ConfigurationException)
-                throw (ConfigurationException)e.getCause();
-            else if (e.getCause() instanceof IOException)
-                throw (IOException)e.getCause();
-            else if (e.getCause() instanceof Exception)
-                throw new ConfigurationException(e.getCause().getMessage(), (Exception)e.getCause());
-            else
-                throw new RuntimeException(e);
-        }
-        
-        assert DatabaseDescriptor.getDefsVersion().timestamp() > 0;
-        DefsTable.dumpToStorage(DatabaseDescriptor.getDefsVersion());
-        // flush system and definition tables.
-        Collection<Future> flushers = new ArrayList<Future>();
-        flushers.addAll(Table.open(Table.SYSTEM_TABLE).flush());
-        for (Future f : flushers)
-        {
-            try
-            {
-                f.get();
-            }
-            catch (Exception e)
-            {
-                ConfigurationException ce = new ConfigurationException(e.getMessage());
-                ce.initCause(e);
-                throw ce;
-            }
-        }
-        
-        // we don't want to announce after every Migration.apply(). keep track of the last one and then announce the
-        // current version.
-        if (migration != null)
-            migration.announce();
-        
-    }
-
-    public String exportSchema() throws IOException
-    {
-        List<RawKeyspace> keyspaces = new ArrayList<RawKeyspace>();
-        for (String ksname : DatabaseDescriptor.getNonSystemTables())
-        {
-            KSMetaData ksm = DatabaseDescriptor.getTableDefinition(ksname);
-            RawKeyspace rks = new RawKeyspace();
-            rks.name = ksm.name;
-            rks.replica_placement_strategy = ksm.strategyClass.getName();
-            rks.replication_factor = ksm.replicationFactor;
-            rks.column_families = new RawColumnFamily[ksm.cfMetaData().size()];
-            int i = 0;
-            for (CFMetaData cfm : ksm.cfMetaData().values())
-            {
-                RawColumnFamily rcf = new RawColumnFamily();
-                rcf.name = cfm.cfName;
-                rcf.compare_with = cfm.comparator.getClass().getName();
-                rcf.default_validation_class = cfm.getDefaultValidator().getClass().getName();
-                rcf.compare_subcolumns_with = cfm.subcolumnComparator == null ? null : cfm.subcolumnComparator.getClass().getName();
-                rcf.column_type = cfm.cfType;
-                rcf.comment = cfm.getComment();
-                rcf.keys_cached = cfm.getKeyCacheSize();
-                rcf.read_repair_chance = cfm.getReadRepairChance();
-                rcf.replicate_on_write = cfm.getReplicateOnWrite();
-                rcf.gc_grace_seconds = cfm.getGcGraceSeconds();
-                rcf.rows_cached = cfm.getRowCacheSize();
-                rcf.column_metadata = new RawColumnDefinition[cfm.getColumn_metadata().size()];
-                int j = 0;
-                for (ColumnDefinition cd : cfm.getColumn_metadata().values())
-                {
-                    RawColumnDefinition rcd = new RawColumnDefinition();
-                    rcd.index_name = cd.getIndexName();
-                    rcd.index_type = cd.getIndexType();
-                    rcd.name = ByteBufferUtil.string(cd.name, Charsets.UTF_8);
-                    rcd.validator_class = cd.validator.getClass().getName();
-                    rcf.column_metadata[j++] = rcd;
-                }
-                if (j == 0)
-                    rcf.column_metadata = null;
-                rks.column_families[i++] = rcf;
-            }
-            // whew.
-            keyspaces.add(rks);
-        }
-        
-        DumperOptions options = new DumperOptions();
-        /* Use a block YAML arrangement */
-        options.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK);
-        SkipNullRepresenter representer = new SkipNullRepresenter();
-        /* Use Tag.MAP to avoid the class name being included as global tag */
-        representer.addClassTag(RawColumnFamily.class, Tag.MAP);
-        representer.addClassTag(Keyspaces.class, Tag.MAP);
-        representer.addClassTag(ColumnDefinition.class, Tag.MAP);
-        Dumper dumper = new Dumper(representer, options);
-        Yaml yaml = new Yaml(dumper);
-        Keyspaces ks = new Keyspaces();
-        ks.keyspaces = keyspaces;
-        return yaml.dump(ks);
-    }
-    
-    public class Keyspaces
-    {
-        public List<RawKeyspace> keyspaces;
-    }
-    
     // Never ever do this at home. Used by tests.
     IPartitioner setPartitionerUnsafe(IPartitioner newPartitioner)
     {

Modified: cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/StorageServiceMBean.java Thu Mar 17 14:50:32 2011
@@ -238,18 +238,6 @@ public interface StorageServiceMBean
     public void drain() throws IOException, InterruptedException, ExecutionException;
 
     /**
-     * Introduced in 0.7 to allow nodes to load their existing yaml defined schemas.
-     * @todo: deprecate in 0.7+1, remove in 0.7+2.
-     */ 
-    public void loadSchemaFromYAML() throws ConfigurationException, IOException;
-
-    /**
-     * Introduced in 0.7 to allow schema yaml to be exported.
-     * @todo: deprecate in 0.7+1, remove in 0.7+2.
-     */
-    public String exportSchema() throws IOException;
-
-    /**
      * Truncates (deletes) the given columnFamily from the provided keyspace.
      * Calling truncate results in actual deletion of all data in the cluster
      * under the given columnFamily and it will fail unless all hosts are up.

Modified: cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/thrift/CassandraServer.java Thu Mar 17 14:50:32 2011
@@ -922,27 +922,29 @@ public class CassandraServer implements 
         CFMetaData.validateMinMaxCompactionThresholds(cf_def);
         CFMetaData.validateMemtableSettings(cf_def);
 
-        return new CFMetaData(cf_def.keyspace,
-                              cf_def.name,
-                              cfType,
-                              DatabaseDescriptor.getComparator(cf_def.comparator_type),
-                              cf_def.subcomparator_type == null ? null : DatabaseDescriptor.getComparator(cf_def.subcomparator_type),
-                              cf_def.comment,
-                              cf_def.row_cache_size,
-                              cf_def.key_cache_size,
-                              cf_def.read_repair_chance,
-                              cf_def.replicate_on_write,
-                              cf_def.isSetGc_grace_seconds() ? cf_def.gc_grace_seconds : CFMetaData.DEFAULT_GC_GRACE_SECONDS,
-                              DatabaseDescriptor.getComparator(cf_def.default_validation_class),
-                              cf_def.isSetMin_compaction_threshold() ? cf_def.min_compaction_threshold : CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD,
-                              cf_def.isSetMax_compaction_threshold() ? cf_def.max_compaction_threshold : CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD,
-                              cf_def.isSetRow_cache_save_period_in_seconds() ? cf_def.row_cache_save_period_in_seconds : CFMetaData.DEFAULT_ROW_CACHE_SAVE_PERIOD_IN_SECONDS,
-                              cf_def.isSetKey_cache_save_period_in_seconds() ? cf_def.key_cache_save_period_in_seconds : CFMetaData.DEFAULT_KEY_CACHE_SAVE_PERIOD_IN_SECONDS,
-                              cf_def.isSetMemtable_flush_after_mins() ? cf_def.memtable_flush_after_mins : CFMetaData.DEFAULT_MEMTABLE_LIFETIME_IN_MINS,
-                              cf_def.isSetMemtable_throughput_in_mb() ? cf_def.memtable_throughput_in_mb : CFMetaData.DEFAULT_MEMTABLE_THROUGHPUT_IN_MB,
-                              cf_def.isSetMemtable_operations_in_millions() ? cf_def.memtable_operations_in_millions : CFMetaData.DEFAULT_MEMTABLE_OPERATIONS_IN_MILLIONS,
-                              cf_def.isSetMerge_shards_chance() ? cf_def.merge_shards_chance : CFMetaData.DEFAULT_MERGE_SHARDS_CHANCE,
-                              ColumnDefinition.fromColumnDef(cf_def.column_metadata));
+        CFMetaData newCFMD = new CFMetaData(cf_def.keyspace,
+                                            cf_def.name,
+                                            cfType,
+                                            DatabaseDescriptor.getComparator(cf_def.comparator_type),
+                                            cf_def.subcomparator_type == null ? null : DatabaseDescriptor.getComparator(cf_def.subcomparator_type));
+
+        if (cf_def.isSetGc_grace_seconds()) { newCFMD.gcGraceSeconds(cf_def.gc_grace_seconds); }
+        if (cf_def.isSetMin_compaction_threshold()) { newCFMD.minCompactionThreshold(cf_def.min_compaction_threshold); }
+        if (cf_def.isSetMax_compaction_threshold()) { newCFMD.maxCompactionThreshold(cf_def.max_compaction_threshold); }
+        if (cf_def.isSetRow_cache_save_period_in_seconds()) { newCFMD.rowCacheSavePeriod(cf_def.row_cache_save_period_in_seconds); }
+        if (cf_def.isSetKey_cache_save_period_in_seconds()) { newCFMD.keyCacheSavePeriod(cf_def.key_cache_save_period_in_seconds); }
+        if (cf_def.isSetMemtable_flush_after_mins()) { newCFMD.memTime(cf_def.memtable_flush_after_mins); }
+        if (cf_def.isSetMemtable_throughput_in_mb()) { newCFMD.memSize(cf_def.memtable_throughput_in_mb); }
+        if (cf_def.isSetMemtable_operations_in_millions()) { newCFMD.memOps(cf_def.memtable_operations_in_millions); }
+        if (cf_def.isSetMerge_shards_chance()) { newCFMD.mergeShardsChance(cf_def.merge_shards_chance); }
+
+        return newCFMD.comment(cf_def.comment)
+                      .rowCacheSize(cf_def.row_cache_size)
+                      .keyCacheSize(cf_def.key_cache_size)
+                      .readRepairChance(cf_def.read_repair_chance)
+                      .replicateOnWrite(cf_def.replicate_on_write)
+                      .defaultValidator(DatabaseDescriptor.getComparator(cf_def.default_validation_class))
+                      .columnMetadata(ColumnDefinition.fromColumnDef(cf_def.column_metadata));
     }
 
     public void truncate(String cfname) throws InvalidRequestException, UnavailableException, TException

Modified: cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/tools/NodeProbe.java Thu Mar 17 14:50:32 2011
@@ -491,17 +491,6 @@ public class NodeProbe
         }
     }
     
-    @Deprecated
-    public void loadSchemaFromYAML() throws ConfigurationException, IOException
-    {
-        ssProxy.loadSchemaFromYAML();
-    }
-    
-    public String exportSchemaToYAML() throws IOException
-    {
-        return ssProxy.exportSchema();
-    }
-
     public MessagingServiceMBean getMsProxy()
     {
         try

Modified: cassandra/trunk/test/conf/cassandra.yaml
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/conf/cassandra.yaml?rev=1082510&r1=1082509&r2=1082510&view=diff
==============================================================================
--- cassandra/trunk/test/conf/cassandra.yaml (original)
+++ cassandra/trunk/test/conf/cassandra.yaml Thu Mar 17 14:50:32 2011
@@ -33,222 +33,3 @@ encryption_options:
     truststore: conf/.truststore
     truststore_password: cassandra
 incremental_backups: true
-    
-# Big Fat Note: as you add new colum families, be sure to append only (no insertions please), lest you break the 
-# serialization tests which expect columnfamily ids to be constant over time.
-keyspaces:
-    - name: Keyspace1
-      replica_placement_strategy: org.apache.cassandra.locator.SimpleStrategy
-      replication_factor: 1
-      column_families:
-        - name: Standard1
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Standard2
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Standard3
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Standard4
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: StandardLong1
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: StandardLong2
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: StandardInteger1
-          rows_cached: 0
-          keys_cached: 0
-          compare_with: IntegerType
-          
-        - name: Super1
-          column_type: Super
-          compare_subcolumns_with: LongType
-          keys_cached: 0
-          keys_cached: 0
-
-        - name: Super2
-          column_type: Super
-          compare_subcolumns_with: LongType
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Super3
-          column_type: Super
-          compare_subcolumns_with: LongType
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Super4
-          column_type: Super
-          compare_subcolumns_with: UTF8Type
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Counter1
-          column_type: Standard
-          default_validation_class: CounterColumnType
-
-        - name: SuperCounter1
-          column_type: Super
-          default_validation_class: CounterColumnType
-
-        - name: Super5
-          column_type: Super
-          rows_cached: 0
-          keys_cached: 0
-            
-        - name: Indexed1
-          column_metadata:
-            - name: birthdate
-              validator_class: LongType
-              index_type: KEYS
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Indexed2 
-          column_metadata:
-            - name: birthdate
-              validator_class: LongType
-              # index will be added dynamically
-          rows_cached: 0
-          keys_cached: 0
-        
-        - name: JdbcInteger
-          compare_with: IntegerType
-          default_validation_class: IntegerType
-          
-        - name: JdbcUtf8
-          compare_with: UTF8Type
-          default_validation_class: UTF8Type
-          
-        - name: JdbcLong
-          compare_with: LongType
-          default_validation_class: LongType
-          
-        - name: JdbcBytes
-          compare_with: BytesType
-          default_validation_class: BytesType
-          
-        - name: JdbcAscii
-          compare_with: AsciiType
-          default_validation_class: AsciiType
-
-    - name: Keyspace2
-      replica_placement_strategy: org.apache.cassandra.locator.SimpleStrategy
-      replication_factor: 1
-      column_families:
-        - name: Standard1
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Standard3
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Super3
-          column_type: Super
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Super4
-          column_type: Super
-          compare_subcolumns_with: TimeUUIDType
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Indexed1
-          column_metadata:
-            - name: birthdate
-              validator_class: LongType
-              index_type: KEYS
-          rows_cached: 0
-          keys_cached: 0
-
-    - name: Keyspace3
-      replica_placement_strategy: org.apache.cassandra.locator.SimpleStrategy
-      replication_factor: 5
-      column_families:
-        - name: Standard1
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Indexed1
-          column_metadata:
-            - name: birthdate
-              validator_class: LongType
-              index_type: KEYS
-          rows_cached: 0
-          keys_cached: 0
-
-    - name: Keyspace4
-      replica_placement_strategy: org.apache.cassandra.locator.SimpleStrategy
-      replication_factor: 3
-      column_families:
-        - name: Standard1
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Standard3
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Super3
-          column_type: Super
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Super4
-          column_type: Super
-          compare_subcolumns_with: TimeUUIDType
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Super5
-          column_type: Super
-          compare_with: TimeUUIDType
-          compare_subcolumns_with: BytesType
-          rows_cached: 0
-          keys_cached: 0
-
-    - name: Keyspace5
-      replica_placement_strategy: org.apache.cassandra.locator.SimpleStrategy
-      replication_factor: 2
-      column_families:
-        - name: Standard1
-          rows_cached: 0
-          keys_cached: 0
-
-        - name: Counter1
-          column_type: Standard
-          default_validation_class: CounterColumnType
-          rows_cached: 0
-          keys_cached: 0
-
-    - name: KeyCacheSpace
-      replica_placement_strategy: org.apache.cassandra.locator.SimpleStrategy
-      replication_factor: 1
-      column_families:
-        - name: Standard1
-          keys_cached: 0.5
-
-        - name: Standard2
-          keys_cached: 1.0
-
-    - name: RowCacheSpace
-      replica_placement_strategy: org.apache.cassandra.locator.SimpleStrategy
-      replication_factor: 1
-      column_families:
-        - name: CachedCF
-          rows_cached: 100
-        - name: CFWithoutCache
-          rows_cached: 0