You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2014/12/17 00:08:10 UTC

[1/5] cassandra git commit: Isolate schema serializaton code

Repository: cassandra
Updated Branches:
  refs/heads/trunk 32ac6af2b -> 3e9d345f0


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 999d6b6..a349e64 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -3259,7 +3259,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     private Future<StreamState> streamHints()
     {
         // StreamPlan will not fail if there are zero files to transfer, so flush anyway (need to get any in-memory hints, as well)
-        ColumnFamilyStore hintsCF = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS_TABLE);
+        ColumnFamilyStore hintsCF = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS);
         FBUtilities.waitOnFuture(hintsCF.forceFlush());
 
         // gather all live nodes in the cluster that aren't also leaving
@@ -3292,7 +3292,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                                                           preferred,
                                                           SystemKeyspace.NAME,
                                                           ranges,
-                                                          SystemKeyspace.HINTS_TABLE)
+                                                          SystemKeyspace.HINTS)
                                           .execute();
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/thrift/ThriftConversion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftConversion.java b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
index b04a091..d408767 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftConversion.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
@@ -30,6 +30,7 @@ import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ColumnFamilyType;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.CellNames;
@@ -330,9 +331,9 @@ public class ThriftConversion
         List<Map<String, ByteBuffer>> cols = new ArrayList<>(columnsRes.rows.size());
         for (CqlRow row : columnsRes.rows)
             cols.add(convertThriftCqlRow(row));
-        UntypedResultSet colsRow = UntypedResultSet.create(cols);
+        UntypedResultSet colsRows = UntypedResultSet.create(cols);
 
-        return CFMetaData.fromSchemaNoTriggers(cfRow, colsRow);
+        return LegacySchemaTables.createTableFromTableRowAndColumnRows(cfRow, colsRows);
     }
 
     private static Map<String, ByteBuffer> convertThriftCqlRow(CqlRow row)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/BulkLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java b/src/java/org/apache/cassandra/tools/BulkLoader.java
index 8e7e0e9..a720e12 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -26,12 +26,14 @@ import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
 import org.apache.commons.cli.*;
+
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TProtocol;
 import org.apache.thrift.transport.TTransport;
 
 import org.apache.cassandra.auth.IAuthenticator;
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.dht.Range;
@@ -309,7 +311,7 @@ public class BulkLoader
 
                     String cfQuery = String.format("SELECT * FROM %s.%s WHERE keyspace_name = '%s'",
                                                    SystemKeyspace.NAME,
-                                                   SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE,
+                                                   LegacySchemaTables.COLUMNFAMILIES,
                                                    keyspace);
                     CqlResult cfRes = client.execute_cql3_query(ByteBufferUtil.bytes(cfQuery), Compression.NONE, ConsistencyLevel.ONE);
 
@@ -319,7 +321,7 @@ public class BulkLoader
                         String columnFamily = UTF8Type.instance.getString(row.columns.get(1).bufferForName());
                         String columnsQuery = String.format("SELECT * FROM %s.%s WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
                                                             SystemKeyspace.NAME,
-                                                            SystemKeyspace.SCHEMA_COLUMNS_TABLE,
+                                                            LegacySchemaTables.COLUMNS,
                                                             keyspace,
                                                             columnFamily);
                         CqlResult columnsRes = client.execute_cql3_query(ByteBufferUtil.bytes(columnsQuery), Compression.NONE, ConsistencyLevel.ONE);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/SSTableExport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableExport.java b/src/java/org/apache/cassandra/tools/SSTableExport.java
index fa6b973..64dc2d8 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -26,7 +26,6 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -420,7 +419,7 @@ public class SSTableExport
         String[] excludes = cmd.getOptionValues(EXCLUDEKEY_OPTION);
         String ssTableFileName = new File(cmd.getArgs()[0]).getAbsolutePath();
 
-        DatabaseDescriptor.loadSchemas();
+        Schema.instance.loadFromDisk();
         Descriptor descriptor = Descriptor.fromFilename(ssTableFileName);
 
         // Start by validating keyspace name

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index 6f5dc9d..f51610b 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -502,7 +502,7 @@ public class SSTableImport
             isSorted = true;
         }
 
-        DatabaseDescriptor.loadSchemas();
+        Schema.instance.loadFromDisk();
         if (Schema.instance.getNonSystemKeyspaces().size() < 1)
         {
             String msg = "no non-system keyspaces are defined";

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java b/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
index 3954b91..815620e 100644
--- a/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
+++ b/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
@@ -21,7 +21,6 @@ import java.io.PrintStream;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -62,7 +61,7 @@ public class SSTableLevelResetter
         try
         {
             // load keyspace descriptions.
-            DatabaseDescriptor.loadSchemas();
+            Schema.instance.loadFromDisk();
 
             String keyspaceName = args[1];
             String columnfamily = args[2];

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 94bf6e2..b6e2bf8 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -24,7 +24,6 @@ import java.util.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
@@ -53,7 +52,7 @@ public class StandaloneScrubber
         try
         {
             // load keyspace descriptions.
-            DatabaseDescriptor.loadSchemas();
+            Schema.instance.loadFromDisk();
 
             if (Schema.instance.getCFMetaData(options.keyspaceName, options.cfName) == null)
                 throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
index 66cb361..90ab48e 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
@@ -21,10 +21,10 @@ package org.apache.cassandra.tools;
 import java.io.File;
 import java.util.*;
 
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
@@ -52,7 +52,7 @@ public class StandaloneSplitter
         try
         {
             // load keyspace descriptions.
-            DatabaseDescriptor.loadSchemas();
+            Schema.instance.loadFromDisk();
 
             String ksName = null;
             String cfName = null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
index 90eca5d..6617b96 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -46,7 +46,7 @@ public class StandaloneUpgrader
         try
         {
             // load keyspace descriptions.
-            DatabaseDescriptor.loadSchemas();
+            Schema.instance.loadFromDisk();
 
             if (Schema.instance.getCFMetaData(options.keyspace, options.cf) == null)
                 throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
index 79373ea..5fed5be 100644
--- a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
@@ -24,20 +24,21 @@ import java.util.HashMap;
 import java.util.HashSet;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.compress.*;
 import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.CfDef;
 import org.apache.cassandra.thrift.ColumnDef;
 import org.apache.cassandra.thrift.IndexType;
 import org.apache.cassandra.thrift.ThriftConversion;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -132,6 +133,7 @@ public class CFMetaDataTest
     private void checkInverses(CFMetaData cfm) throws Exception
     {
         DecoratedKey k = StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(cfm.ksName));
+        KSMetaData keyspace = Schema.instance.getKSMetaData(cfm.ksName);
 
         // Test thrift conversion
         CFMetaData before = cfm;
@@ -139,11 +141,10 @@ public class CFMetaDataTest
         assert before.equals(after) : String.format("%n%s%n!=%n%s", before, after);
 
         // Test schema conversion
-        Mutation rm = cfm.toSchema(System.currentTimeMillis());
-        ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE));
-        ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, SystemKeyspace.SCHEMA_COLUMNS_TABLE));
-        UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", new Row(k, serializedCf)).one();
-        CFMetaData newCfm = CFMetaData.fromSchemaNoTriggers(result, ColumnDefinition.resultify(new Row(k, serializedCD)));
+        Mutation rm = LegacySchemaTables.makeCreateTableMutation(keyspace, cfm, FBUtilities.timestampMicros());
+        ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNFAMILIES));
+        ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNS));
+        CFMetaData newCfm = LegacySchemaTables.createTableFromTablePartitionAndColumnsPartition(new Row(k, serializedCf), new Row(k, serializedCD));
         assert cfm.equals(newCfm) : String.format("%n%s%n!=%n%s", cfm, newCfm);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index edfd7b6..8ca7a79 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -56,9 +56,10 @@ public class DatabaseDescriptorTest
     @Test
     public void testKSMetaDataSerialization() throws ConfigurationException
     {
-        for (KSMetaData ksm : Schema.instance.getKeyspaceDefinitions())
+        for (String ks : Schema.instance.getNonSystemKeyspaces())
         {
             // Not testing round-trip on the KsDef via serDe() because maps
+            KSMetaData ksm = Schema.instance.getKSMetaData(ks);
             KSMetaData ksmDupe = ThriftConversion.fromThrift(ThriftConversion.toThrift(ksm));
             assertNotNull(ksmDupe);
             assertEquals(ksm, ksmDupe);
@@ -70,7 +71,7 @@ public class DatabaseDescriptorTest
     public void testTransKsMigration() throws ConfigurationException
     {
         SchemaLoader.cleanupAndLeaveDirs();
-        DatabaseDescriptor.loadSchemas();
+        Schema.instance.loadFromDisk();
         assertEquals(0, Schema.instance.getNonSystemKeyspaces().size());
 
         Gossiper.instance.start((int)(System.currentTimeMillis() / 1000));
@@ -91,7 +92,7 @@ public class DatabaseDescriptorTest
             assertNull(Schema.instance.getKSMetaData("ks0"));
             assertNull(Schema.instance.getKSMetaData("ks1"));
 
-            DatabaseDescriptor.loadSchemas();
+            Schema.instance.loadFromDisk();
 
             assertNotNull(Schema.instance.getKSMetaData("ks0"));
             assertNotNull(Schema.instance.getKSMetaData("ks1"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/config/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DefsTest.java b/test/unit/org/apache/cassandra/config/DefsTest.java
deleted file mode 100644
index 6df67bc..0000000
--- a/test/unit/org/apache/cassandra/config/DefsTest.java
+++ /dev/null
@@ -1,564 +0,0 @@
-/**
- * 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.config;
-
-import java.io.File;
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.db.marshal.TimeUUIDType;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.Component;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableDeletingTask;
-import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
-import org.apache.cassandra.locator.SimpleStrategy;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import static org.apache.cassandra.Util.cellname;
-
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-@RunWith(OrderedJUnit4ClassRunner.class)
-public class DefsTest
-{
-    private static final String KEYSPACE1 = "Keyspace1";
-    private static final String KEYSPACE3 = "Keyspace3";
-    private static final String KEYSPACE6 = "Keyspace6";
-    private static final String EMPTYKEYSPACE = "DefsTestEmptyKeyspace";
-    private static final String CF_STANDARD1 = "Standard1";
-    private static final String CF_STANDARD2 = "Standard2";
-    private static final String CF_INDEXED = "Indexed1";
-
-    @BeforeClass
-    public static void defineSchema() throws ConfigurationException
-    {
-        SchemaLoader.prepareServer();
-        SchemaLoader.startGossiper();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    SimpleStrategy.class,
-                                    KSMetaData.optsWithRF(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
-        SchemaLoader.createKeyspace(KEYSPACE3, true, false,
-                                    SimpleStrategy.class,
-                                    KSMetaData.optsWithRF(5),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
-                                    SchemaLoader.indexCFMD(KEYSPACE3, CF_INDEXED, true));
-        SchemaLoader.createKeyspace(KEYSPACE6,
-                                    SimpleStrategy.class,
-                                    KSMetaData.optsWithRF(1),
-                                    SchemaLoader.indexCFMD(KEYSPACE6, CF_INDEXED, true));
-    }
-
-    @Test
-    public void testCFMetaDataApply() throws ConfigurationException
-    {
-        CFMetaData cfm = new CFMetaData(KEYSPACE1,
-                                        "TestApplyCFM_CF",
-                                        ColumnFamilyType.Standard,
-                                        new SimpleDenseCellNameType(BytesType.instance));
-
-        for (int i = 0; i < 5; i++)
-        {
-            ByteBuffer name = ByteBuffer.wrap(new byte[] { (byte)i });
-            cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, name, BytesType.instance, null).setIndex(Integer.toString(i), IndexType.KEYS, null));
-        }
-
-        cfm.comment("No comment")
-           .readRepairChance(0.5)
-           .gcGraceSeconds(100000)
-           .minCompactionThreshold(500)
-           .maxCompactionThreshold(500);
-
-        // we'll be adding this one later. make sure it's not already there.
-        Assert.assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })));
-
-        CFMetaData cfNew = cfm.copy();
-
-        // add one.
-        ColumnDefinition addIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 5 }), BytesType.instance, null)
-                                                       .setIndex("5", IndexType.KEYS, null);
-        cfNew.addColumnDefinition(addIndexDef);
-
-        // remove one.
-        ColumnDefinition removeIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 0 }), BytesType.instance, null)
-                                                          .setIndex("0", IndexType.KEYS, null);
-        Assert.assertTrue(cfNew.removeColumnDefinition(removeIndexDef));
-
-        cfm.apply(cfNew);
-
-        for (int i = 1; i < cfm.allColumns().size(); i++)
-            Assert.assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 1 })));
-        Assert.assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 0 })));
-        Assert.assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })));
-    }
-
-    @Test
-    public void testInvalidNames()
-    {
-        String[] valid = {"1", "a", "_1", "b_", "__", "1_a"};
-        for (String s : valid)
-            Assert.assertTrue(CFMetaData.isNameValid(s));
-
-        String[] invalid = {"b@t", "dash-y", "", " ", "dot.s", ".hidden"};
-        for (String s : invalid)
-            Assert.assertFalse(CFMetaData.isNameValid(s));
-    }
-
-    @Ignore
-    @Test
-    public void saveAndRestore()
-    {
-        /*
-        // verify dump and reload.
-        UUID first = UUIDGen.makeType1UUIDFromHost(FBUtilities.getBroadcastAddress());
-        DefsTables.dumpToStorage(first);
-        List<KSMetaData> defs = new ArrayList<KSMetaData>(DefsTables.loadFromStorage(first));
-
-        Assert.assertTrue(defs.size() > 0);
-        Assert.assertEquals(defs.size(), Schema.instance.getNonSystemKeyspaces().size());
-        for (KSMetaData loaded : defs)
-        {
-            KSMetaData defined = Schema.instance.getKeyspaceDefinition(loaded.name);
-            Assert.assertTrue(String.format("%s != %s", loaded, defined), defined.equals(loaded));
-        }
-        */
-    }
-
-    @Test
-    public void addNewCfToBogusKeyspace()
-    {
-        CFMetaData newCf = addTestCF("MadeUpKeyspace", "NewCF", "new cf");
-        try
-        {
-            MigrationManager.announceNewColumnFamily(newCf);
-            throw new AssertionError("You shouldn't be able to do anything to a keyspace that doesn't exist.");
-        }
-        catch (ConfigurationException expected)
-        {
-        }
-    }
-
-    @Test
-    public void addNewCfWithNullComment() throws ConfigurationException
-    {
-        final String ks = KEYSPACE1;
-        final String cf = "BrandNewCfWithNull";
-        KSMetaData original = Schema.instance.getKSMetaData(ks);
-
-        CFMetaData newCf = addTestCF(original.name, cf, null);
-
-        Assert.assertFalse(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
-        MigrationManager.announceNewColumnFamily(newCf);
-
-        Assert.assertTrue(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
-        Assert.assertEquals(newCf, Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName));
-    }
-
-    @Test
-    public void addNewCF() throws ConfigurationException
-    {
-        final String ks = KEYSPACE1;
-        final String cf = "BrandNewCf";
-        KSMetaData original = Schema.instance.getKSMetaData(ks);
-
-        CFMetaData newCf = addTestCF(original.name, cf, "A New Table");
-
-        Assert.assertFalse(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
-        MigrationManager.announceNewColumnFamily(newCf);
-
-        Assert.assertTrue(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
-        Assert.assertEquals(newCf, Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName));
-
-        // now read and write to it.
-        CellName col0 = cellname("col0");
-        DecoratedKey dk = Util.dk("key0");
-        Mutation rm = new Mutation(ks, dk.getKey());
-        rm.add(cf, col0, ByteBufferUtil.bytes("value0"), 1L);
-        rm.applyUnsafe();
-        ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
-        Assert.assertNotNull(store);
-        store.forceBlockingFlush();
-
-        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
-        Assert.assertNotNull(cfam.getColumn(col0));
-        Cell col = cfam.getColumn(col0);
-        Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
-    }
-
-    @Test
-    public void dropCf() throws ConfigurationException
-    {
-        DecoratedKey dk = Util.dk("dropCf");
-        // sanity
-        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
-        Assert.assertNotNull(ks);
-        final CFMetaData cfm = ks.cfMetaData().get("Standard1");
-        Assert.assertNotNull(cfm);
-
-        // write some data, force a flush, then verify that files exist on disk.
-        Mutation rm = new Mutation(ks.name, dk.getKey());
-        for (int i = 0; i < 100; i++)
-            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
-        rm.applyUnsafe();
-        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
-        Assert.assertNotNull(store);
-        store.forceBlockingFlush();
-        Assert.assertTrue(store.directories.sstableLister().list().size() > 0);
-
-        MigrationManager.announceColumnFamilyDrop(ks.name, cfm.cfName);
-
-        Assert.assertFalse(Schema.instance.getKSMetaData(ks.name).cfMetaData().containsKey(cfm.cfName));
-
-        // any write should fail.
-        rm = new Mutation(ks.name, dk.getKey());
-        boolean success = true;
-        try
-        {
-            rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
-            rm.applyUnsafe();
-        }
-        catch (Throwable th)
-        {
-            success = false;
-        }
-        Assert.assertFalse("This mutation should have failed since the CF no longer exists.", success);
-
-        // verify that the files are gone.
-        for (File file : store.directories.sstableLister().listFiles())
-        {
-            if (file.getPath().endsWith("Data.db") && !new File(file.getPath().replace("Data.db", "Compacted")).exists())
-                throw new AssertionError("undeleted file " + file);
-        }
-    }
-
-    @Test
-    public void addNewKS() throws ConfigurationException
-    {
-        DecoratedKey dk = Util.dk("key0");
-        CFMetaData newCf = addTestCF("NewKeyspace1", "AddedStandard1", "A new cf for a new ks");
-
-        KSMetaData newKs = KSMetaData.testMetadata(newCf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), newCf);
-
-        MigrationManager.announceNewKeyspace(newKs);
-
-        Assert.assertNotNull(Schema.instance.getKSMetaData(newCf.ksName));
-        Assert.assertEquals(Schema.instance.getKSMetaData(newCf.ksName), newKs);
-
-        // test reads and writes.
-        CellName col0 = cellname("col0");
-        Mutation rm = new Mutation(newCf.ksName, dk.getKey());
-        rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
-        rm.applyUnsafe();
-        ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
-        Assert.assertNotNull(store);
-        store.forceBlockingFlush();
-
-        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
-        Assert.assertNotNull(cfam.getColumn(col0));
-        Cell col = cfam.getColumn(col0);
-        Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
-    }
-
-    @Test
-    public void dropKS() throws ConfigurationException
-    {
-        DecoratedKey dk = Util.dk("dropKs");
-        // sanity
-        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
-        Assert.assertNotNull(ks);
-        final CFMetaData cfm = ks.cfMetaData().get("Standard2");
-        Assert.assertNotNull(cfm);
-
-        // write some data, force a flush, then verify that files exist on disk.
-        Mutation rm = new Mutation(ks.name, dk.getKey());
-        for (int i = 0; i < 100; i++)
-            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
-        rm.applyUnsafe();
-        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
-        Assert.assertNotNull(store);
-        store.forceBlockingFlush();
-        Assert.assertTrue(store.directories.sstableLister().list().size() > 0);
-
-        MigrationManager.announceKeyspaceDrop(ks.name);
-
-        Assert.assertNull(Schema.instance.getKSMetaData(ks.name));
-
-        // write should fail.
-        rm = new Mutation(ks.name, dk.getKey());
-        boolean success = true;
-        try
-        {
-            rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
-            rm.applyUnsafe();
-        }
-        catch (Throwable th)
-        {
-            success = false;
-        }
-        Assert.assertFalse("This mutation should have failed since the CF no longer exists.", success);
-
-        // reads should fail too.
-        boolean threw = false;
-        try
-        {
-            Keyspace.open(ks.name);
-        }
-        catch (Throwable th)
-        {
-            threw = true;
-        }
-        Assert.assertTrue(threw);
-    }
-
-    @Test
-    public void dropKSUnflushed() throws ConfigurationException
-    {
-        DecoratedKey dk = Util.dk("dropKs");
-        // sanity
-        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE3);
-        Assert.assertNotNull(ks);
-        final CFMetaData cfm = ks.cfMetaData().get("Standard1");
-        Assert.assertNotNull(cfm);
-
-        // write some data
-        Mutation rm = new Mutation(ks.name, dk.getKey());
-        for (int i = 0; i < 100; i++)
-            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
-        rm.applyUnsafe();
-
-        MigrationManager.announceKeyspaceDrop(ks.name);
-
-        Assert.assertNull(Schema.instance.getKSMetaData(ks.name));
-    }
-
-    @Test
-    public void createEmptyKsAddNewCf() throws ConfigurationException
-    {
-        Assert.assertNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
-
-        KSMetaData newKs = KSMetaData.testMetadata(EMPTYKEYSPACE, SimpleStrategy.class, KSMetaData.optsWithRF(5));
-
-        MigrationManager.announceNewKeyspace(newKs);
-        Assert.assertNotNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
-
-        CFMetaData newCf = addTestCF(EMPTYKEYSPACE, "AddedLater", "A new CF to add to an empty KS");
-
-        //should not exist until apply
-        Assert.assertFalse(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
-
-        //add the new CF to the empty space
-        MigrationManager.announceNewColumnFamily(newCf);
-
-        Assert.assertTrue(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
-        Assert.assertEquals(Schema.instance.getKSMetaData(newKs.name).cfMetaData().get(newCf.cfName), newCf);
-
-        // now read and write to it.
-        CellName col0 = cellname("col0");
-        DecoratedKey dk = Util.dk("key0");
-        Mutation rm = new Mutation(newKs.name, dk.getKey());
-        rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
-        rm.applyUnsafe();
-        ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
-        Assert.assertNotNull(store);
-        store.forceBlockingFlush();
-
-        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
-        Assert.assertNotNull(cfam.getColumn(col0));
-        Cell col = cfam.getColumn(col0);
-        Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
-    }
-
-    @Test
-    public void testUpdateKeyspace() throws ConfigurationException
-    {
-        // create a keyspace to serve as existing.
-        CFMetaData cf = addTestCF("UpdatedKeyspace", "AddedStandard1", "A new cf for a new ks");
-        KSMetaData oldKs = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), cf);
-
-        MigrationManager.announceNewKeyspace(oldKs);
-
-        Assert.assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
-        Assert.assertEquals(Schema.instance.getKSMetaData(cf.ksName), oldKs);
-
-        // names should match.
-        KSMetaData newBadKs2 = KSMetaData.testMetadata(cf.ksName + "trash", SimpleStrategy.class, KSMetaData.optsWithRF(4));
-        try
-        {
-            MigrationManager.announceKeyspaceUpdate(newBadKs2);
-            throw new AssertionError("Should not have been able to update a KS with an invalid KS name.");
-        }
-        catch (ConfigurationException ex)
-        {
-            // expected.
-        }
-
-        KSMetaData newKs = KSMetaData.testMetadata(cf.ksName, OldNetworkTopologyStrategy.class, KSMetaData.optsWithRF(1));
-        MigrationManager.announceKeyspaceUpdate(newKs);
-
-        KSMetaData newFetchedKs = Schema.instance.getKSMetaData(newKs.name);
-        Assert.assertEquals(newFetchedKs.strategyClass, newKs.strategyClass);
-        Assert.assertFalse(newFetchedKs.strategyClass.equals(oldKs.strategyClass));
-    }
-
-    @Test
-    public void testUpdateColumnFamilyNoIndexes() throws ConfigurationException
-    {
-        // create a keyspace with a cf to update.
-        CFMetaData cf = addTestCF("UpdatedCfKs", "Standard1added", "A new cf that will be updated");
-        KSMetaData ksm = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(1), cf);
-        MigrationManager.announceNewKeyspace(ksm);
-
-        Assert.assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
-        Assert.assertEquals(Schema.instance.getKSMetaData(cf.ksName), ksm);
-        Assert.assertNotNull(Schema.instance.getCFMetaData(cf.ksName, cf.cfName));
-
-        // updating certain fields should fail.
-        CFMetaData newCfm = cf.copy();
-        newCfm.defaultValidator(BytesType.instance);
-        newCfm.minCompactionThreshold(5);
-        newCfm.maxCompactionThreshold(31);
-
-        // test valid operations.
-        newCfm.comment("Modified comment");
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false); // doesn't get set back here.
-
-        newCfm.readRepairChance(0.23);
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
-        newCfm.gcGraceSeconds(12);
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
-        newCfm.defaultValidator(UTF8Type.instance);
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
-        newCfm.minCompactionThreshold(3);
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
-        newCfm.maxCompactionThreshold(33);
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
-        // can't test changing the reconciler because there is only one impl.
-
-        // check the cumulative affect.
-        Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getComment(), newCfm.getComment());
-        Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getReadRepairChance(), newCfm.getReadRepairChance(), 0.0001);
-        Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getGcGraceSeconds(), newCfm.getGcGraceSeconds());
-        Assert.assertEquals(UTF8Type.instance, Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getDefaultValidator());
-
-        // Change cfId
-        newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, cf.comparator);
-        CFMetaData.copyOpts(newCfm, cf);
-        try
-        {
-            cf.apply(newCfm);
-            throw new AssertionError("Should have blown up when you used a different id.");
-        }
-        catch (ConfigurationException expected) {}
-
-        // Change cfName
-        newCfm = new CFMetaData(cf.ksName, cf.cfName + "_renamed", cf.cfType, cf.comparator);
-        CFMetaData.copyOpts(newCfm, cf);
-        try
-        {
-            cf.apply(newCfm);
-            throw new AssertionError("Should have blown up when you used a different name.");
-        }
-        catch (ConfigurationException expected) {}
-
-        // Change ksName
-        newCfm = new CFMetaData(cf.ksName + "_renamed", cf.cfName, cf.cfType, cf.comparator);
-        CFMetaData.copyOpts(newCfm, cf);
-        try
-        {
-            cf.apply(newCfm);
-            throw new AssertionError("Should have blown up when you used a different keyspace.");
-        }
-        catch (ConfigurationException expected) {}
-
-        // Change cf type
-        newCfm = new CFMetaData(cf.ksName, cf.cfName, ColumnFamilyType.Super, cf.comparator);
-        CFMetaData.copyOpts(newCfm, cf);
-        try
-        {
-            cf.apply(newCfm);
-            throw new AssertionError("Should have blwon up when you used a different cf type.");
-        }
-        catch (ConfigurationException expected) {}
-
-        // Change comparator
-        newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, new SimpleDenseCellNameType(TimeUUIDType.instance));
-        CFMetaData.copyOpts(newCfm, cf);
-        try
-        {
-            cf.apply(newCfm);
-            throw new AssertionError("Should have blown up when you used a different comparator.");
-        }
-        catch (ConfigurationException expected) {}
-    }
-
-    @Test
-    public void testDropIndex() throws ConfigurationException
-    {
-        // persist keyspace definition in the system keyspace
-        Schema.instance.getKSMetaData(KEYSPACE6).toSchema(System.currentTimeMillis()).applyUnsafe();
-        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE6).getColumnFamilyStore("Indexed1");
-
-        // insert some data.  save the sstable descriptor so we can make sure it's marked for delete after the drop
-        Mutation rm = new Mutation(KEYSPACE6, ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
-        rm.applyUnsafe();
-        cfs.forceBlockingFlush();
-        ColumnFamilyStore indexedCfs = cfs.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
-        Descriptor desc = indexedCfs.getSSTables().iterator().next().descriptor;
-
-        // drop the index
-        CFMetaData meta = cfs.metadata.copy();
-        ColumnDefinition cdOld = meta.regularColumns().iterator().next();
-        ColumnDefinition cdNew = ColumnDefinition.regularDef(meta, cdOld.name.bytes, cdOld.type, null);
-        meta.addOrReplaceColumnDefinition(cdNew);
-        MigrationManager.announceColumnFamilyUpdate(meta, false);
-
-        // check
-        Assert.assertTrue(cfs.indexManager.getIndexes().isEmpty());
-        SSTableDeletingTask.waitForDeletions();
-        Assert.assertFalse(new File(desc.filenameFor(Component.DATA)).exists());
-    }
-
-    private CFMetaData addTestCF(String ks, String cf, String comment)
-    {
-        CFMetaData newCFMD = new CFMetaData(ks, cf, ColumnFamilyType.Standard, new SimpleDenseCellNameType(UTF8Type.instance));
-        newCFMD.comment(comment)
-               .readRepairChance(0.0);
-
-        return newCFMD;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/KSMetaDataTest.java b/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
index 9093086..3d90bef 100644
--- a/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
@@ -39,11 +39,7 @@ public class KSMetaDataTest
         options.put("key2", "value2");
         options.put("key3", "value3");
 
-        KSMetaData ksMeta = new KSMetaData("test",
-                                            SimpleStrategy.class,
-                                            options,
-                                            true,
-                                            Collections.<CFMetaData>emptyList());
+        KSMetaData ksMeta = new KSMetaData("test", SimpleStrategy.class, options, true);
 
         assertTrue(ksMeta.toString().contains(options.toString()));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 883da3a..4e536fe 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -159,10 +159,10 @@ public abstract class CQLTester
                     schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tableToDrop));
 
                     for (String aggregateName : aggregatesToDrop)
-                        schemaChange(String.format("DROP AGGREGATE IF EXISTS %s", aggregateName));
+                        schemaChange(String.format("DROP AGGREGATE IF EXISTS %s.%s", KEYSPACE, aggregateName));
 
                     for (String functionName : functionsToDrop)
-                        schemaChange(String.format("DROP FUNCTION IF EXISTS %s", functionName));
+                        schemaChange(String.format("DROP FUNCTION IF EXISTS %s.%s", KEYSPACE, functionName));
 
                     for (String typeName : typesToDrop)
                         schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typeName));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
index 608aee8..70d1d0c 100644
--- a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
@@ -101,7 +101,7 @@ public class BatchlogManagerTest
         }
 
         // Flush the batchlog to disk (see CASSANDRA-6822).
-        Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG_TABLE).forceBlockingFlush();
+        Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG).forceBlockingFlush();
 
         assertEquals(1000, BatchlogManager.instance.countAllBatches() - initialAllBatches);
         assertEquals(0, BatchlogManager.instance.getTotalBatchesReplayed() - initialReplayedBatches);
@@ -171,7 +171,7 @@ public class BatchlogManagerTest
         }
 
         // Flush the batchlog to disk (see CASSANDRA-6822).
-        Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG_TABLE).forceFlush();
+        Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG).forceFlush();
 
         // Force batchlog replay and wait for it to complete.
         BatchlogManager.instance.startBatchlogReplay().get();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
index 8b6834c..ce14c37 100644
--- a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
+++ b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
@@ -66,7 +66,7 @@ public class HintedHandOffTest
     {
         // prepare hints column family
         Keyspace systemKeyspace = Keyspace.open("system");
-        ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS_TABLE);
+        ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS);
         hintStore.clearUnsafe();
         hintStore.metadata.gcGraceSeconds(36000); // 10 hours
         hintStore.setCompactionStrategyClass(SizeTieredCompactionStrategy.class.getCanonicalName());
@@ -101,7 +101,7 @@ public class HintedHandOffTest
             HintedHandOffManager.instance.metrics.incrPastWindow(InetAddress.getLocalHost());
         HintedHandOffManager.instance.metrics.log();
 
-        UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS_TABLE);
+        UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS);
         Map<UUID, Integer> returned = rows.one().getMap("hints_dropped", UUIDType.instance, Int32Type.instance);
         assertEquals(Iterators.getLast(returned.values().iterator()).intValue(), 99);
     }
@@ -110,7 +110,7 @@ public class HintedHandOffTest
     public void testTruncateHints() throws Exception
     {
         Keyspace systemKeyspace = Keyspace.open("system");
-        ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS_TABLE);
+        ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS);
         hintStore.clearUnsafe();
 
         // insert 1 hint
@@ -138,7 +138,7 @@ public class HintedHandOffTest
     private int getNoOfHints()
     {
         String req = "SELECT * FROM system.%s";
-        UntypedResultSet resultSet = executeInternal(String.format(req, SystemKeyspace.HINTS_TABLE));
+        UntypedResultSet resultSet = executeInternal(String.format(req, SystemKeyspace.HINTS));
         return resultSet.size();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/schema/LegacySchemaTablesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/LegacySchemaTablesTest.java b/test/unit/org/apache/cassandra/schema/LegacySchemaTablesTest.java
new file mode 100644
index 0000000..8190617
--- /dev/null
+++ b/test/unit/org/apache/cassandra/schema/LegacySchemaTablesTest.java
@@ -0,0 +1,568 @@
+/**
+ * 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.schema;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.Util.cellname;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+@RunWith(OrderedJUnit4ClassRunner.class)
+public class LegacySchemaTablesTest
+{
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String KEYSPACE3 = "Keyspace3";
+    private static final String KEYSPACE6 = "Keyspace6";
+    private static final String EMPTYKEYSPACE = "DefsTestEmptyKeyspace";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String CF_INDEXED = "Indexed1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.startGossiper();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+        SchemaLoader.createKeyspace(KEYSPACE3, true, false,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(5),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.indexCFMD(KEYSPACE3, CF_INDEXED, true));
+        SchemaLoader.createKeyspace(KEYSPACE6,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.indexCFMD(KEYSPACE6, CF_INDEXED, true));
+    }
+
+    @Test
+    public void testCFMetaDataApply() throws ConfigurationException
+    {
+        CFMetaData cfm = new CFMetaData(KEYSPACE1,
+                                        "TestApplyCFM_CF",
+                                        ColumnFamilyType.Standard,
+                                        new SimpleDenseCellNameType(BytesType.instance));
+
+        for (int i = 0; i < 5; i++)
+        {
+            ByteBuffer name = ByteBuffer.wrap(new byte[] { (byte)i });
+            cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, name, BytesType.instance, null).setIndex(Integer.toString(i), IndexType.KEYS, null));
+        }
+
+        cfm.comment("No comment")
+           .readRepairChance(0.5)
+           .gcGraceSeconds(100000)
+           .minCompactionThreshold(500)
+           .maxCompactionThreshold(500);
+
+        // we'll be adding this one later. make sure it's not already there.
+        Assert.assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })));
+
+        CFMetaData cfNew = cfm.copy();
+
+        // add one.
+        ColumnDefinition addIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 5 }), BytesType.instance, null)
+                                                       .setIndex("5", IndexType.KEYS, null);
+        cfNew.addColumnDefinition(addIndexDef);
+
+        // remove one.
+        ColumnDefinition removeIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 0 }), BytesType.instance, null)
+                                                          .setIndex("0", IndexType.KEYS, null);
+        Assert.assertTrue(cfNew.removeColumnDefinition(removeIndexDef));
+
+        cfm.apply(cfNew);
+
+        for (int i = 1; i < cfm.allColumns().size(); i++)
+            Assert.assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 1 })));
+        Assert.assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 0 })));
+        Assert.assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })));
+    }
+
+    @Test
+    public void testInvalidNames()
+    {
+        String[] valid = {"1", "a", "_1", "b_", "__", "1_a"};
+        for (String s : valid)
+            Assert.assertTrue(CFMetaData.isNameValid(s));
+
+        String[] invalid = {"b@t", "dash-y", "", " ", "dot.s", ".hidden"};
+        for (String s : invalid)
+            Assert.assertFalse(CFMetaData.isNameValid(s));
+    }
+
+    @Ignore
+    @Test
+    public void saveAndRestore()
+    {
+        /*
+        // verify dump and reload.
+        UUID first = UUIDGen.makeType1UUIDFromHost(FBUtilities.getBroadcastAddress());
+        DefsTables.dumpToStorage(first);
+        List<KSMetaData> defs = new ArrayList<KSMetaData>(DefsTables.loadFromStorage(first));
+
+        Assert.assertTrue(defs.size() > 0);
+        Assert.assertEquals(defs.size(), Schema.instance.getNonSystemKeyspaces().size());
+        for (KSMetaData loaded : defs)
+        {
+            KSMetaData defined = Schema.instance.getKeyspaceDefinition(loaded.name);
+            Assert.assertTrue(String.format("%s != %s", loaded, defined), defined.equals(loaded));
+        }
+        */
+    }
+
+    @Test
+    public void addNewCfToBogusKeyspace()
+    {
+        CFMetaData newCf = addTestCF("MadeUpKeyspace", "NewCF", "new cf");
+        try
+        {
+            MigrationManager.announceNewColumnFamily(newCf);
+            throw new AssertionError("You shouldn't be able to do anything to a keyspace that doesn't exist.");
+        }
+        catch (ConfigurationException expected)
+        {
+        }
+    }
+
+    @Test
+    public void addNewCfWithNullComment() throws ConfigurationException
+    {
+        final String ks = KEYSPACE1;
+        final String cf = "BrandNewCfWithNull";
+        KSMetaData original = Schema.instance.getKSMetaData(ks);
+
+        CFMetaData newCf = addTestCF(original.name, cf, null);
+
+        Assert.assertFalse(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+        MigrationManager.announceNewColumnFamily(newCf);
+
+        Assert.assertTrue(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+        Assert.assertEquals(newCf, Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName));
+    }
+
+    @Test
+    public void addNewCF() throws ConfigurationException
+    {
+        final String ks = KEYSPACE1;
+        final String cf = "BrandNewCf";
+        KSMetaData original = Schema.instance.getKSMetaData(ks);
+
+        CFMetaData newCf = addTestCF(original.name, cf, "A New Table");
+
+        Assert.assertFalse(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+        MigrationManager.announceNewColumnFamily(newCf);
+
+        Assert.assertTrue(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+        Assert.assertEquals(newCf, Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName));
+
+        // now read and write to it.
+        CellName col0 = cellname("col0");
+        DecoratedKey dk = Util.dk("key0");
+        Mutation rm = new Mutation(ks, dk.getKey());
+        rm.add(cf, col0, ByteBufferUtil.bytes("value0"), 1L);
+        rm.applyUnsafe();
+        ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
+        Assert.assertNotNull(store);
+        store.forceBlockingFlush();
+
+        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+        Assert.assertNotNull(cfam.getColumn(col0));
+        Cell col = cfam.getColumn(col0);
+        Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
+    }
+
+    @Test
+    public void dropCf() throws ConfigurationException
+    {
+        DecoratedKey dk = Util.dk("dropCf");
+        // sanity
+        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
+        Assert.assertNotNull(ks);
+        final CFMetaData cfm = ks.cfMetaData().get("Standard1");
+        Assert.assertNotNull(cfm);
+
+        // write some data, force a flush, then verify that files exist on disk.
+        Mutation rm = new Mutation(ks.name, dk.getKey());
+        for (int i = 0; i < 100; i++)
+            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
+        rm.applyUnsafe();
+        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+        Assert.assertNotNull(store);
+        store.forceBlockingFlush();
+        Assert.assertTrue(store.directories.sstableLister().list().size() > 0);
+
+        MigrationManager.announceColumnFamilyDrop(ks.name, cfm.cfName);
+
+        Assert.assertFalse(Schema.instance.getKSMetaData(ks.name).cfMetaData().containsKey(cfm.cfName));
+
+        // any write should fail.
+        rm = new Mutation(ks.name, dk.getKey());
+        boolean success = true;
+        try
+        {
+            rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
+            rm.applyUnsafe();
+        }
+        catch (Throwable th)
+        {
+            success = false;
+        }
+        Assert.assertFalse("This mutation should have failed since the CF no longer exists.", success);
+
+        // verify that the files are gone.
+        for (File file : store.directories.sstableLister().listFiles())
+        {
+            if (file.getPath().endsWith("Data.db") && !new File(file.getPath().replace("Data.db", "Compacted")).exists())
+                throw new AssertionError("undeleted file " + file);
+        }
+    }
+
+    @Test
+    public void addNewKS() throws ConfigurationException
+    {
+        DecoratedKey dk = Util.dk("key0");
+        CFMetaData newCf = addTestCF("NewKeyspace1", "AddedStandard1", "A new cf for a new ks");
+
+        KSMetaData newKs = KSMetaData.testMetadata(newCf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), newCf);
+
+        MigrationManager.announceNewKeyspace(newKs);
+
+        Assert.assertNotNull(Schema.instance.getKSMetaData(newCf.ksName));
+        Assert.assertEquals(Schema.instance.getKSMetaData(newCf.ksName), newKs);
+
+        // test reads and writes.
+        CellName col0 = cellname("col0");
+        Mutation rm = new Mutation(newCf.ksName, dk.getKey());
+        rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
+        rm.applyUnsafe();
+        ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
+        Assert.assertNotNull(store);
+        store.forceBlockingFlush();
+
+        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+        Assert.assertNotNull(cfam.getColumn(col0));
+        Cell col = cfam.getColumn(col0);
+        Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
+    }
+
+    @Test
+    public void dropKS() throws ConfigurationException
+    {
+        DecoratedKey dk = Util.dk("dropKs");
+        // sanity
+        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
+        Assert.assertNotNull(ks);
+        final CFMetaData cfm = ks.cfMetaData().get("Standard2");
+        Assert.assertNotNull(cfm);
+
+        // write some data, force a flush, then verify that files exist on disk.
+        Mutation rm = new Mutation(ks.name, dk.getKey());
+        for (int i = 0; i < 100; i++)
+            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
+        rm.applyUnsafe();
+        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+        Assert.assertNotNull(store);
+        store.forceBlockingFlush();
+        Assert.assertTrue(store.directories.sstableLister().list().size() > 0);
+
+        MigrationManager.announceKeyspaceDrop(ks.name);
+
+        Assert.assertNull(Schema.instance.getKSMetaData(ks.name));
+
+        // write should fail.
+        rm = new Mutation(ks.name, dk.getKey());
+        boolean success = true;
+        try
+        {
+            rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
+            rm.applyUnsafe();
+        }
+        catch (Throwable th)
+        {
+            success = false;
+        }
+        Assert.assertFalse("This mutation should have failed since the CF no longer exists.", success);
+
+        // reads should fail too.
+        boolean threw = false;
+        try
+        {
+            Keyspace.open(ks.name);
+        }
+        catch (Throwable th)
+        {
+            threw = true;
+        }
+        Assert.assertTrue(threw);
+    }
+
+    @Test
+    public void dropKSUnflushed() throws ConfigurationException
+    {
+        DecoratedKey dk = Util.dk("dropKs");
+        // sanity
+        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE3);
+        Assert.assertNotNull(ks);
+        final CFMetaData cfm = ks.cfMetaData().get("Standard1");
+        Assert.assertNotNull(cfm);
+
+        // write some data
+        Mutation rm = new Mutation(ks.name, dk.getKey());
+        for (int i = 0; i < 100; i++)
+            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
+        rm.applyUnsafe();
+
+        MigrationManager.announceKeyspaceDrop(ks.name);
+
+        Assert.assertNull(Schema.instance.getKSMetaData(ks.name));
+    }
+
+    @Test
+    public void createEmptyKsAddNewCf() throws ConfigurationException
+    {
+        Assert.assertNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
+
+        KSMetaData newKs = KSMetaData.testMetadata(EMPTYKEYSPACE, SimpleStrategy.class, KSMetaData.optsWithRF(5));
+
+        MigrationManager.announceNewKeyspace(newKs);
+        Assert.assertNotNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
+
+        CFMetaData newCf = addTestCF(EMPTYKEYSPACE, "AddedLater", "A new CF to add to an empty KS");
+
+        //should not exist until apply
+        Assert.assertFalse(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
+
+        //add the new CF to the empty space
+        MigrationManager.announceNewColumnFamily(newCf);
+
+        Assert.assertTrue(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
+        Assert.assertEquals(Schema.instance.getKSMetaData(newKs.name).cfMetaData().get(newCf.cfName), newCf);
+
+        // now read and write to it.
+        CellName col0 = cellname("col0");
+        DecoratedKey dk = Util.dk("key0");
+        Mutation rm = new Mutation(newKs.name, dk.getKey());
+        rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
+        rm.applyUnsafe();
+        ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
+        Assert.assertNotNull(store);
+        store.forceBlockingFlush();
+
+        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+        Assert.assertNotNull(cfam.getColumn(col0));
+        Cell col = cfam.getColumn(col0);
+        Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
+    }
+
+    @Test
+    public void testUpdateKeyspace() throws ConfigurationException
+    {
+        // create a keyspace to serve as existing.
+        CFMetaData cf = addTestCF("UpdatedKeyspace", "AddedStandard1", "A new cf for a new ks");
+        KSMetaData oldKs = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), cf);
+
+        MigrationManager.announceNewKeyspace(oldKs);
+
+        Assert.assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
+        Assert.assertEquals(Schema.instance.getKSMetaData(cf.ksName), oldKs);
+
+        // names should match.
+        KSMetaData newBadKs2 = KSMetaData.testMetadata(cf.ksName + "trash", SimpleStrategy.class, KSMetaData.optsWithRF(4));
+        try
+        {
+            MigrationManager.announceKeyspaceUpdate(newBadKs2);
+            throw new AssertionError("Should not have been able to update a KS with an invalid KS name.");
+        }
+        catch (ConfigurationException ex)
+        {
+            // expected.
+        }
+
+        KSMetaData newKs = KSMetaData.testMetadata(cf.ksName, OldNetworkTopologyStrategy.class, KSMetaData.optsWithRF(1));
+        MigrationManager.announceKeyspaceUpdate(newKs);
+
+        KSMetaData newFetchedKs = Schema.instance.getKSMetaData(newKs.name);
+        Assert.assertEquals(newFetchedKs.strategyClass, newKs.strategyClass);
+        Assert.assertFalse(newFetchedKs.strategyClass.equals(oldKs.strategyClass));
+    }
+
+    @Test
+    public void testUpdateColumnFamilyNoIndexes() throws ConfigurationException
+    {
+        // create a keyspace with a cf to update.
+        CFMetaData cf = addTestCF("UpdatedCfKs", "Standard1added", "A new cf that will be updated");
+        KSMetaData ksm = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(1), cf);
+        MigrationManager.announceNewKeyspace(ksm);
+
+        Assert.assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
+        Assert.assertEquals(Schema.instance.getKSMetaData(cf.ksName), ksm);
+        Assert.assertNotNull(Schema.instance.getCFMetaData(cf.ksName, cf.cfName));
+
+        // updating certain fields should fail.
+        CFMetaData newCfm = cf.copy();
+        newCfm.defaultValidator(BytesType.instance);
+        newCfm.minCompactionThreshold(5);
+        newCfm.maxCompactionThreshold(31);
+
+        // test valid operations.
+        newCfm.comment("Modified comment");
+        MigrationManager.announceColumnFamilyUpdate(newCfm, false); // doesn't get set back here.
+
+        newCfm.readRepairChance(0.23);
+        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
+
+        newCfm.gcGraceSeconds(12);
+        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
+
+        newCfm.defaultValidator(UTF8Type.instance);
+        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
+
+        newCfm.minCompactionThreshold(3);
+        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
+
+        newCfm.maxCompactionThreshold(33);
+        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
+
+        // can't test changing the reconciler because there is only one impl.
+
+        // check the cumulative affect.
+        Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getComment(), newCfm.getComment());
+        Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getReadRepairChance(), newCfm.getReadRepairChance(), 0.0001);
+        Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getGcGraceSeconds(), newCfm.getGcGraceSeconds());
+        Assert.assertEquals(UTF8Type.instance, Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getDefaultValidator());
+
+        // Change cfId
+        newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, cf.comparator);
+        CFMetaData.copyOpts(newCfm, cf);
+        try
+        {
+            cf.apply(newCfm);
+            throw new AssertionError("Should have blown up when you used a different id.");
+        }
+        catch (ConfigurationException expected) {}
+
+        // Change cfName
+        newCfm = new CFMetaData(cf.ksName, cf.cfName + "_renamed", cf.cfType, cf.comparator);
+        CFMetaData.copyOpts(newCfm, cf);
+        try
+        {
+            cf.apply(newCfm);
+            throw new AssertionError("Should have blown up when you used a different name.");
+        }
+        catch (ConfigurationException expected) {}
+
+        // Change ksName
+        newCfm = new CFMetaData(cf.ksName + "_renamed", cf.cfName, cf.cfType, cf.comparator);
+        CFMetaData.copyOpts(newCfm, cf);
+        try
+        {
+            cf.apply(newCfm);
+            throw new AssertionError("Should have blown up when you used a different keyspace.");
+        }
+        catch (ConfigurationException expected) {}
+
+        // Change cf type
+        newCfm = new CFMetaData(cf.ksName, cf.cfName, ColumnFamilyType.Super, cf.comparator);
+        CFMetaData.copyOpts(newCfm, cf);
+        try
+        {
+            cf.apply(newCfm);
+            throw new AssertionError("Should have blwon up when you used a different cf type.");
+        }
+        catch (ConfigurationException expected) {}
+
+        // Change comparator
+        newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, new SimpleDenseCellNameType(TimeUUIDType.instance));
+        CFMetaData.copyOpts(newCfm, cf);
+        try
+        {
+            cf.apply(newCfm);
+            throw new AssertionError("Should have blown up when you used a different comparator.");
+        }
+        catch (ConfigurationException expected) {}
+    }
+
+    @Test
+    public void testDropIndex() throws ConfigurationException
+    {
+        // persist keyspace definition in the system keyspace
+        LegacySchemaTables.makeCreateKeyspaceMutation(Schema.instance.getKSMetaData(KEYSPACE6), FBUtilities.timestampMicros()).applyUnsafe();
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE6).getColumnFamilyStore("Indexed1");
+
+        // insert some data.  save the sstable descriptor so we can make sure it's marked for delete after the drop
+        Mutation rm = new Mutation(KEYSPACE6, ByteBufferUtil.bytes("k1"));
+        rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.applyUnsafe();
+        cfs.forceBlockingFlush();
+        ColumnFamilyStore indexedCfs = cfs.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
+        Descriptor desc = indexedCfs.getSSTables().iterator().next().descriptor;
+
+        // drop the index
+        CFMetaData meta = cfs.metadata.copy();
+        ColumnDefinition cdOld = meta.regularColumns().iterator().next();
+        ColumnDefinition cdNew = ColumnDefinition.regularDef(meta, cdOld.name.bytes, cdOld.type, null);
+        meta.addOrReplaceColumnDefinition(cdNew);
+        MigrationManager.announceColumnFamilyUpdate(meta, false);
+
+        // check
+        Assert.assertTrue(cfs.indexManager.getIndexes().isEmpty());
+        SSTableDeletingTask.waitForDeletions();
+        Assert.assertFalse(new File(desc.filenameFor(Component.DATA)).exists());
+    }
+
+    private CFMetaData addTestCF(String ks, String cf, String comment)
+    {
+        CFMetaData newCFMD = new CFMetaData(ks, cf, ColumnFamilyType.Standard, new SimpleDenseCellNameType(UTF8Type.instance));
+        newCFMD.comment(comment)
+               .readRepairChance(0.0);
+
+        return newCFMD;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java b/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
index 1437886..ed0efee 100644
--- a/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
+++ b/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
@@ -59,11 +59,11 @@ public class EmbeddedCassandraServiceTest
     public static void defineSchema() throws Exception
     {
         SchemaLoader.prepareServer();
+        setup();
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     SimpleStrategy.class,
                                     KSMetaData.optsWithRF(1),
                                     SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
-        setup();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index f98751a..d1a4d85 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -37,6 +37,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
@@ -98,7 +99,7 @@ public class StorageServiceServerTest
     public void testColumnFamilySnapshot() throws IOException
     {
         // no need to insert extra data, even an "empty" database will have a little information in the system keyspace
-        StorageService.instance.takeColumnFamilySnapshot(SystemKeyspace.NAME, SystemKeyspace.SCHEMA_KEYSPACES_TABLE, "cf_snapshot");
+        StorageService.instance.takeColumnFamilySnapshot(SystemKeyspace.NAME, LegacySchemaTables.KEYSPACES, "cf_snapshot");
     }
 
     @Test


[4/5] cassandra git commit: Isolate schema serializaton code

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 8b42e51..4672451 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -30,11 +30,7 @@ import com.datastax.driver.core.UserType;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.db.marshal.TypeParser;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -47,9 +43,10 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
     protected static final Logger logger = LoggerFactory.getLogger(UDFunction.class);
 
     protected final List<ColumnIdentifier> argNames;
+
     protected final String language;
     protected final String body;
-    private final boolean deterministic;
+    protected final boolean isDeterministic;
 
     protected final DataType[] argDataTypes;
     protected final DataType returnDataType;
@@ -60,10 +57,10 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
                          AbstractType<?> returnType,
                          String language,
                          String body,
-                         boolean deterministic)
+                         boolean isDeterministic)
     {
         this(name, argNames, argTypes, UDHelper.driverTypes(argTypes), returnType,
-             UDHelper.driverType(returnType), language, body, deterministic);
+             UDHelper.driverType(returnType), language, body, isDeterministic);
     }
 
     protected UDFunction(FunctionName name,
@@ -74,14 +71,14 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
                          DataType returnDataType,
                          String language,
                          String body,
-                         boolean deterministic)
+                         boolean isDeterministic)
     {
         super(name, argTypes, returnType);
         assert new HashSet<>(argNames).size() == argNames.size() : "duplicate argument names";
         this.argNames = argNames;
         this.language = language;
         this.body = body;
-        this.deterministic = deterministic;
+        this.isDeterministic = isDeterministic;
         this.argDataTypes = argDataTypes;
         this.returnDataType = returnDataType;
     }
@@ -92,13 +89,13 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
                                     AbstractType<?> returnType,
                                     String language,
                                     String body,
-                                    boolean deterministic)
+                                    boolean isDeterministic)
     throws InvalidRequestException
     {
         switch (language)
         {
-            case "java": return JavaSourceUDFFactory.buildUDF(name, argNames, argTypes, returnType, body, deterministic);
-            default: return new ScriptBasedUDF(name, argNames, argTypes, returnType, language, body, deterministic);
+            case "java": return JavaSourceUDFFactory.buildUDF(name, argNames, argTypes, returnType, body, isDeterministic);
+            default: return new ScriptBasedUDF(name, argNames, argTypes, returnType, language, body, isDeterministic);
         }
     }
 
@@ -111,24 +108,27 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
      *  2) we return a meaningful error message if the function is executed (something more precise
      *     than saying that the function doesn't exist)
      */
-    private static UDFunction createBrokenFunction(FunctionName name,
-                                                   List<ColumnIdentifier> argNames,
-                                                   List<AbstractType<?>> argTypes,
-                                                   AbstractType<?> returnType,
-                                                   String language,
-                                                   String body,
-                                                   final InvalidRequestException reason)
+    public static UDFunction createBrokenFunction(FunctionName name,
+                                                  List<ColumnIdentifier> argNames,
+                                                  List<AbstractType<?>> argTypes,
+                                                  AbstractType<?> returnType,
+                                                  String language,
+                                                  String body,
+                                                  final InvalidRequestException reason)
     {
         return new UDFunction(name, argNames, argTypes, returnType, language, body, true)
         {
             public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException
             {
-                throw new InvalidRequestException(String.format("Function '%s' exists but hasn't been loaded successfully for the following reason: %s. "
-                                                              + "Please see the server log for more details", this, reason.getMessage()));
+                throw new InvalidRequestException(String.format("Function '%s' exists but hasn't been loaded successfully "
+                                                                + "for the following reason: %s. Please see the server log for details",
+                                                                this,
+                                                                reason.getMessage()));
             }
         };
     }
 
+
     public boolean isAggregate()
     {
         return false;
@@ -136,7 +136,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
 
     public boolean isPure()
     {
-        return deterministic;
+        return isDeterministic;
     }
 
     public boolean isNative()
@@ -144,13 +144,33 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
         return false;
     }
 
+    public List<ColumnIdentifier> argNames()
+    {
+        return argNames;
+    }
+
+    public boolean isDeterministic()
+    {
+        return isDeterministic;
+    }
+
+    public String body()
+    {
+        return body;
+    }
+
+    public String language()
+    {
+        return language;
+    }
+
     /**
      * Used by UDF implementations (both Java code generated by {@link org.apache.cassandra.cql3.functions.JavaSourceUDFFactory}
      * and script executor {@link org.apache.cassandra.cql3.functions.ScriptBasedUDF}) to convert the C*
      * serialized representation to the Java object representation.
      *
      * @param protocolVersion the native protocol version used for serialization
-     * @param argIndex        index of the UDF input argument
+     * @param argIndex index of the UDF input argument
      */
     protected Object compose(int protocolVersion, int argIndex, ByteBuffer value)
     {
@@ -169,117 +189,6 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
         return value == null ? null : returnDataType.serialize(value, ProtocolVersion.fromInt(protocolVersion));
     }
 
-    private static Mutation makeSchemaMutation(FunctionName name)
-    {
-        UTF8Type kv = (UTF8Type)SystemKeyspace.SchemaFunctionsTable.getKeyValidator();
-        return new Mutation(SystemKeyspace.NAME, kv.decompose(name.keyspace));
-    }
-
-    public Mutation toSchemaDrop(long timestamp)
-    {
-        Mutation mutation = makeSchemaMutation(name);
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE);
-
-        Composite prefix = SystemKeyspace.SchemaFunctionsTable.comparator.make(name.name, UDHelper.computeSignature(argTypes));
-        int ldt = (int) (System.currentTimeMillis() / 1000);
-        cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-
-        return mutation;
-    }
-
-    public static Map<Composite, UDFunction> fromSchema(Row row)
-    {
-        UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_FUNCTIONS_TABLE, row);
-        Map<Composite, UDFunction> udfs = new HashMap<>(results.size());
-        for (UntypedResultSet.Row result : results)
-            udfs.put(SystemKeyspace.SchemaFunctionsTable.comparator.make(result.getString("function_name"), result.getBlob("signature")),
-                     fromSchema(result));
-        return udfs;
-    }
-
-    public Mutation toSchemaUpdate(long timestamp)
-    {
-        Mutation mutation = makeSchemaMutation(name);
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE);
-
-        Composite prefix = SystemKeyspace.SchemaFunctionsTable.comparator.make(name.name, UDHelper.computeSignature(argTypes));
-        CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
-        adder.resetCollection("argument_names");
-        adder.resetCollection("argument_types");
-        adder.add("return_type", returnType.toString());
-        adder.add("language", language);
-        adder.add("body", body);
-        adder.add("deterministic", deterministic);
-
-        for (int i = 0; i < argNames.size(); i++)
-        {
-            adder.addListEntry("argument_names", argNames.get(i).bytes);
-            adder.addListEntry("argument_types", argTypes.get(i).toString());
-        }
-
-        return mutation;
-    }
-
-    public static UDFunction fromSchema(UntypedResultSet.Row row)
-    {
-        String ksName = row.getString("keyspace_name");
-        String functionName = row.getString("function_name");
-        FunctionName name = new FunctionName(ksName, functionName);
-
-        List<String> names = row.getList("argument_names", UTF8Type.instance);
-        List<String> types = row.getList("argument_types", UTF8Type.instance);
-
-        List<ColumnIdentifier> argNames;
-        if (names == null)
-            argNames = Collections.emptyList();
-        else
-        {
-            argNames = new ArrayList<>(names.size());
-            for (String arg : names)
-                argNames.add(new ColumnIdentifier(arg, true));
-        }
-
-        List<AbstractType<?>> argTypes;
-        if (types == null)
-            argTypes = Collections.emptyList();
-        else
-        {
-            argTypes = new ArrayList<>(types.size());
-            for (String type : types)
-                argTypes.add(parseType(type));
-        }
-
-        AbstractType<?> returnType = parseType(row.getString("return_type"));
-
-        boolean deterministic = row.getBoolean("deterministic");
-        String language = row.getString("language");
-        String body = row.getString("body");
-
-        try
-        {
-            return create(name, argNames, argTypes, returnType, language, body, deterministic);
-        }
-        catch (InvalidRequestException e)
-        {
-            logger.error(String.format("Cannot load function '%s' from schema: this function won't be available (on this node)", name), e);
-            return createBrokenFunction(name, argNames, argTypes, returnType, language, body, e);
-        }
-    }
-
-    private static AbstractType<?> parseType(String str)
-    {
-        // We only use this when reading the schema where we shouldn't get an error
-        try
-        {
-            return TypeParser.parse(str);
-        }
-        catch (SyntaxException | ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
     @Override
     public boolean equals(Object o)
     {
@@ -287,19 +196,19 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
             return false;
 
         UDFunction that = (UDFunction)o;
-        return Objects.equal(this.name, that.name)
-            && Functions.typeEquals(this.argTypes, that.argTypes)
-            && Functions.typeEquals(this.returnType, that.returnType)
-            && Objects.equal(this.argNames, that.argNames)
-            && Objects.equal(this.language, that.language)
-            && Objects.equal(this.body, that.body)
-            && Objects.equal(this.deterministic, that.deterministic);
+        return Objects.equal(name, that.name)
+            && Objects.equal(argNames, that.argNames)
+            && Functions.typeEquals(argTypes, that.argTypes)
+            && Functions.typeEquals(returnType, that.returnType)
+            && Objects.equal(language, that.language)
+            && Objects.equal(body, that.body)
+            && Objects.equal(isDeterministic, that.isDeterministic);
     }
 
     @Override
     public int hashCode()
     {
-        return Objects.hashCode(name, argTypes, returnType, argNames, language, body, deterministic);
+        return Objects.hashCode(name, argNames, argTypes, returnType, language, body, isDeterministic);
     }
 
     public void userTypeUpdated(String ksName, String typeName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDHelper.java b/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
index 2a17c75..0738cbe 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
@@ -31,12 +31,13 @@ import org.slf4j.LoggerFactory;
 import com.datastax.driver.core.DataType;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
  * Helper class for User Defined Functions + Aggregates.
  */
-final class UDHelper
+public final class UDHelper
 {
     protected static final Logger logger = LoggerFactory.getLogger(UDHelper.class);
 
@@ -112,12 +113,13 @@ final class UDHelper
     // we use a "signature" which is just a SHA-1 of it's argument types (we could replace that by
     // using a "signature" UDT that would be comprised of the function name and argument types,
     // which we could then use as clustering column. But as we haven't yet used UDT in system tables,
-    // We'll left that decision to #6717).
-    protected static ByteBuffer computeSignature(List<AbstractType<?>> argTypes)
+    // We'll leave that decision to #6717).
+    public static ByteBuffer calculateSignature(AbstractFunction fun)
     {
         MessageDigest digest = FBUtilities.newMessageDigest("SHA-1");
-        for (AbstractType<?> type : argTypes)
-            digest.update(type.asCQL3Type().toString().getBytes(StandardCharsets.UTF_8));
+        digest.update(UTF8Type.instance.decompose(fun.name().name));
+        for (AbstractType<?> type : fun.argTypes())
+            digest.update(UTF8Type.instance.decompose(type.asCQL3Type().toString()));
         return ByteBuffer.wrap(digest.digest());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
index 6aea3b1..c8c2474 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@ -151,14 +151,32 @@ public class CreateTableStatement extends SchemaAlteringStatement
             .addAllColumnDefinitions(getColumns(cfmd))
             .isDense(isDense);
 
-        cfmd.addColumnMetadataFromAliases(keyAliases, keyValidator, ColumnDefinition.Kind.PARTITION_KEY);
-        cfmd.addColumnMetadataFromAliases(columnAliases, comparator.asAbstractType(), ColumnDefinition.Kind.CLUSTERING_COLUMN);
+        addColumnMetadataFromAliases(cfmd, keyAliases, keyValidator, ColumnDefinition.Kind.PARTITION_KEY);
+        addColumnMetadataFromAliases(cfmd, columnAliases, comparator.asAbstractType(), ColumnDefinition.Kind.CLUSTERING_COLUMN);
         if (valueAlias != null)
-            cfmd.addColumnMetadataFromAliases(Collections.<ByteBuffer>singletonList(valueAlias), defaultValidator, ColumnDefinition.Kind.COMPACT_VALUE);
+            addColumnMetadataFromAliases(cfmd, Collections.singletonList(valueAlias), defaultValidator, ColumnDefinition.Kind.COMPACT_VALUE);
 
         properties.applyToCFMetadata(cfmd);
     }
 
+    private void addColumnMetadataFromAliases(CFMetaData cfm, List<ByteBuffer> aliases, AbstractType<?> comparator, ColumnDefinition.Kind kind)
+    {
+        if (comparator instanceof CompositeType)
+        {
+            CompositeType ct = (CompositeType)comparator;
+            for (int i = 0; i < aliases.size(); ++i)
+                if (aliases.get(i) != null)
+                    cfm.addOrReplaceColumnDefinition(new ColumnDefinition(cfm, aliases.get(i), ct.types.get(i), i, kind));
+        }
+        else
+        {
+            assert aliases.size() <= 1;
+            if (!aliases.isEmpty() && aliases.get(0) != null)
+                cfm.addOrReplaceColumnDefinition(new ColumnDefinition(cfm, aliases.get(0), comparator, null, kind));
+        }
+    }
+
+
     public static class RawStatement extends CFStatement
     {
         private final Map<ColumnIdentifier, CQL3Type.Raw> definitions = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
index 1fcd63c..e766f65 100644
--- a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
+++ b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
@@ -35,6 +35,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.SearchIterator;
 import org.apache.cassandra.utils.btree.BTree;
@@ -59,7 +60,7 @@ import static org.apache.cassandra.db.index.SecondaryIndexManager.Updater;
  */
 public class AtomicBTreeColumns extends ColumnFamily
 {
-    static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreeColumns(SystemKeyspace.BuiltIndexesTable, null))
+    static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreeColumns(CFMetaData.denseCFMetaData("keyspace", "table", BytesType.instance), null))
             + ObjectSizes.measure(new Holder(null, null));
 
     // Reserved values for wasteTracker field. These values must not be consecutive (see avoidReservedValues)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BatchlogManager.java b/src/java/org/apache/cassandra/db/BatchlogManager.java
index b33e457..e71a62c 100644
--- a/src/java/org/apache/cassandra/db/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/db/BatchlogManager.java
@@ -102,7 +102,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
     public int countAllBatches()
     {
-        String query = String.format("SELECT count(*) FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.BATCHLOG_TABLE);
+        String query = String.format("SELECT count(*) FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.BATCHLOG);
         return (int) executeInternal(query).one().getLong("count");
     }
 
@@ -137,8 +137,8 @@ public class BatchlogManager implements BatchlogManagerMBean
     @VisibleForTesting
     static Mutation getBatchlogMutationFor(Collection<Mutation> mutations, UUID uuid, int version, long now)
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(SystemKeyspace.BatchlogTable);
-        CFRowAdder adder = new CFRowAdder(cf, SystemKeyspace.BatchlogTable.comparator.builder().build(), now);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(SystemKeyspace.Batchlog);
+        CFRowAdder adder = new CFRowAdder(cf, SystemKeyspace.Batchlog.comparator.builder().build(), now);
         adder.add("data", serializeMutations(mutations, version))
              .add("written_at", new Date(now / 1000))
              .add("version", version);
@@ -174,7 +174,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
         UntypedResultSet page = executeInternal(String.format("SELECT id, data, written_at, version FROM %s.%s LIMIT %d",
                                                               SystemKeyspace.NAME,
-                                                              SystemKeyspace.BATCHLOG_TABLE,
+                                                              SystemKeyspace.BATCHLOG,
                                                               PAGE_SIZE));
 
         while (!page.isEmpty())
@@ -186,7 +186,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
             page = executeInternal(String.format("SELECT id, data, written_at, version FROM %s.%s WHERE token(id) > token(?) LIMIT %d",
                                                  SystemKeyspace.NAME,
-                                                 SystemKeyspace.BATCHLOG_TABLE,
+                                                 SystemKeyspace.BATCHLOG,
                                                  PAGE_SIZE),
                                    id);
         }
@@ -199,7 +199,7 @@ public class BatchlogManager implements BatchlogManagerMBean
     private void deleteBatch(UUID id)
     {
         Mutation mutation = new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(id));
-        mutation.delete(SystemKeyspace.BATCHLOG_TABLE, FBUtilities.timestampMicros());
+        mutation.delete(SystemKeyspace.BATCHLOG, FBUtilities.timestampMicros());
         mutation.apply();
     }
 
@@ -447,7 +447,7 @@ public class BatchlogManager implements BatchlogManagerMBean
     // force flush + compaction to reclaim space from the replayed batches
     private void cleanup() throws ExecutionException, InterruptedException
     {
-        ColumnFamilyStore cfs = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG_TABLE);
+        ColumnFamilyStore cfs = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG);
         cfs.forceBlockingFlush();
         Collection<Descriptor> descriptors = new ArrayList<>();
         for (SSTableReader sstr : cfs.getSSTables())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java b/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
index 5cb62ed..d5ede03 100644
--- a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.utils.WrappedRunnable;
 
 /**
@@ -46,7 +47,7 @@ public class DefinitionsUpdateVerbHandler implements IVerbHandler<Collection<Mut
         {
             public void runMayThrow() throws Exception
             {
-                DefsTables.mergeSchema(message.payload);
+                LegacySchemaTables.mergeSchema(message.payload);
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/DefsTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefsTables.java b/src/java/org/apache/cassandra/db/DefsTables.java
deleted file mode 100644
index 82a5dd1..0000000
--- a/src/java/org/apache/cassandra/db/DefsTables.java
+++ /dev/null
@@ -1,622 +0,0 @@
-/*
- * 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.db;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.MapDifference;
-import com.google.common.collect.Maps;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.UTMetaData;
-import org.apache.cassandra.cql3.functions.Functions;
-import org.apache.cassandra.cql3.functions.UDAggregate;
-import org.apache.cassandra.cql3.functions.UDFunction;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.UserType;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-/**
- * SCHEMA_{KEYSPACES, COLUMNFAMILIES, COLUMNS}_CF are used to store Keyspace/ColumnFamily attributes to make schema
- * load/distribution easy, it replaces old mechanism when local migrations where serialized, stored in system.Migrations
- * and used for schema distribution.
- */
-public class DefsTables
-{
-    private static final Logger logger = LoggerFactory.getLogger(DefsTables.class);
-
-    /**
-     * Load keyspace definitions for the system keyspace (system.SCHEMA_KEYSPACES_TABLE)
-     *
-     * @return Collection of found keyspace definitions
-     */
-    public static Collection<KSMetaData> loadFromKeyspace()
-    {
-        List<Row> serializedSchema = SystemKeyspace.serializedSchema(SystemKeyspace.SCHEMA_KEYSPACES_TABLE);
-
-        List<KSMetaData> keyspaces = new ArrayList<>(serializedSchema.size());
-
-        for (Row row : serializedSchema)
-        {
-            if (Schema.invalidSchemaRow(row) || Schema.ignoredSchemaRow(row))
-                continue;
-
-            keyspaces.add(KSMetaData.fromSchema(row, serializedColumnFamilies(row.key), serializedUserTypes(row.key)));
-        }
-
-        return keyspaces;
-    }
-
-    private static Row serializedColumnFamilies(DecoratedKey ksNameKey)
-    {
-        ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE);
-        return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey,
-                                                                                         SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE,
-                                                                                         System.currentTimeMillis())));
-    }
-
-    private static Row serializedUserTypes(DecoratedKey ksNameKey)
-    {
-        ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_USER_TYPES_TABLE);
-        return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey,
-                                                                                         SystemKeyspace.SCHEMA_USER_TYPES_TABLE,
-                                                                                         System.currentTimeMillis())));
-    }
-
-    /**
-     * Merge remote schema in form of mutations with local and mutate ks/cf metadata objects
-     * (which also involves fs operations on add/drop ks/cf)
-     *
-     * @param mutations the schema changes to apply
-     *
-     * @throws ConfigurationException If one of metadata attributes has invalid value
-     * @throws IOException If data was corrupted during transportation or failed to apply fs operations
-     */
-    public static synchronized void mergeSchema(Collection<Mutation> mutations) throws ConfigurationException, IOException
-    {
-        mergeSchemaInternal(mutations, true);
-        Schema.instance.updateVersionAndAnnounce();
-    }
-
-    public static synchronized void mergeSchemaInternal(Collection<Mutation> mutations, boolean doFlush) throws IOException
-    {
-        // compare before/after schemas of the affected keyspaces only
-        Set<String> keyspaces = new HashSet<>(mutations.size());
-        for (Mutation mutation : mutations)
-            keyspaces.add(ByteBufferUtil.string(mutation.key()));
-
-        // current state of the schema
-        Map<DecoratedKey, ColumnFamily> oldKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, keyspaces);
-        Map<DecoratedKey, ColumnFamily> oldColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, keyspaces);
-        Map<DecoratedKey, ColumnFamily> oldTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_TABLE, keyspaces);
-        Map<DecoratedKey, ColumnFamily> oldFunctions = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE, keyspaces);
-        Map<DecoratedKey, ColumnFamily> oldAggregates = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_AGGREGATES_TABLE, keyspaces);
-
-        for (Mutation mutation : mutations)
-            mutation.apply();
-
-        if (doFlush)
-            flushSchemaCFs();
-
-        // with new data applied
-        Map<DecoratedKey, ColumnFamily> newKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, keyspaces);
-        Map<DecoratedKey, ColumnFamily> newColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, keyspaces);
-        Map<DecoratedKey, ColumnFamily> newTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_TABLE, keyspaces);
-        Map<DecoratedKey, ColumnFamily> newFunctions = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE, keyspaces);
-        Map<DecoratedKey, ColumnFamily> newAggregates = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_AGGREGATES_TABLE, keyspaces);
-
-        Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
-        mergeColumnFamilies(oldColumnFamilies, newColumnFamilies);
-        mergeTypes(oldTypes, newTypes);
-        mergeFunctions(oldFunctions, newFunctions);
-        mergeAggregates(oldAggregates, newAggregates);
-
-        // it is safe to drop a keyspace only when all nested ColumnFamilies where deleted
-        for (String keyspaceToDrop : keyspacesToDrop)
-            dropKeyspace(keyspaceToDrop);
-    }
-
-    private static Set<String> mergeKeyspaces(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
-    {
-        List<Row> created = new ArrayList<>();
-        List<String> altered = new ArrayList<>();
-        Set<String> dropped = new HashSet<>();
-
-        /*
-         * - we don't care about entriesOnlyOnLeft() or entriesInCommon(), because only the changes are of interest to us
-         * - of all entriesOnlyOnRight(), we only care about ones that have live columns; it's possible to have a ColumnFamily
-         *   there that only has the top-level deletion, if:
-         *      a) a pushed DROP KEYSPACE change for a keyspace hadn't ever made it to this node in the first place
-         *      b) a pulled dropped keyspace that got dropped before it could find a way to this node
-         * - of entriesDiffering(), we don't care about the scenario where both pre and post-values have zero live columns:
-         *   that means that a keyspace had been recreated and dropped, and the recreated keyspace had never found a way
-         *   to this node
-         */
-        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
-        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
-            if (entry.getValue().hasColumns())
-                created.add(new Row(entry.getKey(), entry.getValue()));
-
-        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
-        {
-            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
-
-            ColumnFamily pre  = entry.getValue().leftValue();
-            ColumnFamily post = entry.getValue().rightValue();
-
-            if (pre.hasColumns() && post.hasColumns())
-                altered.add(keyspaceName);
-            else if (pre.hasColumns())
-                dropped.add(keyspaceName);
-            else if (post.hasColumns()) // a (re)created keyspace
-                created.add(new Row(entry.getKey(), post));
-        }
-
-        for (Row row : created)
-            addKeyspace(KSMetaData.fromSchema(row, Collections.<CFMetaData>emptyList(), new UTMetaData()));
-        for (String name : altered)
-            updateKeyspace(name);
-        return dropped;
-    }
-
-    // see the comments for mergeKeyspaces()
-    private static void mergeColumnFamilies(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
-    {
-        List<CFMetaData> created = new ArrayList<>();
-        List<CFMetaData> altered = new ArrayList<>();
-        List<CFMetaData> dropped = new ArrayList<>();
-
-        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
-        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
-            if (entry.getValue().hasColumns())
-                created.addAll(KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), entry.getValue())).values());
-
-        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
-        {
-            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
-
-            ColumnFamily pre  = entry.getValue().leftValue();
-            ColumnFamily post = entry.getValue().rightValue();
-
-            if (pre.hasColumns() && post.hasColumns())
-            {
-                MapDifference<String, CFMetaData> delta =
-                        Maps.difference(Schema.instance.getKSMetaData(keyspaceName).cfMetaData(),
-                                        KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), post)));
-
-                dropped.addAll(delta.entriesOnlyOnLeft().values());
-                created.addAll(delta.entriesOnlyOnRight().values());
-                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<CFMetaData>, CFMetaData>()
-                {
-                    public CFMetaData apply(MapDifference.ValueDifference<CFMetaData> pair)
-                    {
-                        return pair.rightValue();
-                    }
-                }));
-            }
-            else if (pre.hasColumns())
-            {
-                dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).cfMetaData().values());
-            }
-            else if (post.hasColumns())
-            {
-                created.addAll(KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), post)).values());
-            }
-        }
-
-        for (CFMetaData cfm : created)
-            addColumnFamily(cfm);
-        for (CFMetaData cfm : altered)
-            updateColumnFamily(cfm.ksName, cfm.cfName);
-        for (CFMetaData cfm : dropped)
-            dropColumnFamily(cfm.ksName, cfm.cfName);
-    }
-
-    // see the comments for mergeKeyspaces()
-    private static void mergeTypes(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
-    {
-        List<UserType> created = new ArrayList<>();
-        List<UserType> altered = new ArrayList<>();
-        List<UserType> dropped = new ArrayList<>();
-
-        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
-        // New keyspace with types
-        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
-            if (entry.getValue().hasColumns())
-                created.addAll(UTMetaData.fromSchema(new Row(entry.getKey(), entry.getValue())).values());
-
-        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
-        {
-            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
-
-            ColumnFamily pre  = entry.getValue().leftValue();
-            ColumnFamily post = entry.getValue().rightValue();
-
-            if (pre.hasColumns() && post.hasColumns())
-            {
-                MapDifference<ByteBuffer, UserType> delta =
-                        Maps.difference(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes(),
-                                        UTMetaData.fromSchema(new Row(entry.getKey(), post)));
-
-                dropped.addAll(delta.entriesOnlyOnLeft().values());
-                created.addAll(delta.entriesOnlyOnRight().values());
-                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UserType>, UserType>()
-                {
-                    public UserType apply(MapDifference.ValueDifference<UserType> pair)
-                    {
-                        return pair.rightValue();
-                    }
-                }));
-            }
-            else if (pre.hasColumns())
-            {
-                dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes().values());
-            }
-            else if (post.hasColumns())
-            {
-                created.addAll(UTMetaData.fromSchema(new Row(entry.getKey(), post)).values());
-            }
-        }
-
-        for (UserType type : created)
-            addType(type);
-        for (UserType type : altered)
-            updateType(type);
-        for (UserType type : dropped)
-            dropType(type);
-    }
-
-    // see the comments for mergeKeyspaces()
-    private static void mergeFunctions(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
-    {
-        List<UDFunction> created = new ArrayList<>();
-        List<UDFunction> altered = new ArrayList<>();
-        List<UDFunction> dropped = new ArrayList<>();
-
-        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
-        // New keyspace with functions
-        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
-            if (entry.getValue().hasColumns())
-                created.addAll(UDFunction.fromSchema(new Row(entry.getKey(), entry.getValue())).values());
-
-        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
-        {
-            ColumnFamily pre = entry.getValue().leftValue();
-            ColumnFamily post = entry.getValue().rightValue();
-
-            if (pre.hasColumns() && post.hasColumns())
-            {
-                MapDifference<Composite, UDFunction> delta =
-                        Maps.difference(UDFunction.fromSchema(new Row(entry.getKey(), pre)),
-                                        UDFunction.fromSchema(new Row(entry.getKey(), post)));
-
-                dropped.addAll(delta.entriesOnlyOnLeft().values());
-                created.addAll(delta.entriesOnlyOnRight().values());
-                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UDFunction>, UDFunction>()
-                {
-                    public UDFunction apply(MapDifference.ValueDifference<UDFunction> pair)
-                    {
-                        return pair.rightValue();
-                    }
-                }));
-            }
-            else if (pre.hasColumns())
-            {
-                dropped.addAll(UDFunction.fromSchema(new Row(entry.getKey(), pre)).values());
-            }
-            else if (post.hasColumns())
-            {
-                created.addAll(UDFunction.fromSchema(new Row(entry.getKey(), post)).values());
-            }
-        }
-
-        for (UDFunction udf : created)
-            addFunction(udf);
-        for (UDFunction udf : altered)
-            updateFunction(udf);
-        for (UDFunction udf : dropped)
-            dropFunction(udf);
-    }
-
-    // see the comments for mergeKeyspaces()
-    private static void mergeAggregates(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
-    {
-        List<UDAggregate> created = new ArrayList<>();
-        List<UDAggregate> altered = new ArrayList<>();
-        List<UDAggregate> dropped = new ArrayList<>();
-
-        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
-        // New keyspace with functions
-        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
-            if (entry.getValue().hasColumns())
-                created.addAll(UDAggregate.fromSchema(new Row(entry.getKey(), entry.getValue())).values());
-
-        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
-        {
-            ColumnFamily pre = entry.getValue().leftValue();
-            ColumnFamily post = entry.getValue().rightValue();
-
-            if (pre.hasColumns() && post.hasColumns())
-            {
-                MapDifference<Composite, UDAggregate> delta =
-                        Maps.difference(UDAggregate.fromSchema(new Row(entry.getKey(), pre)),
-                                        UDAggregate.fromSchema(new Row(entry.getKey(), post)));
-
-                dropped.addAll(delta.entriesOnlyOnLeft().values());
-                created.addAll(delta.entriesOnlyOnRight().values());
-                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UDAggregate>, UDAggregate>()
-                {
-                    public UDAggregate apply(MapDifference.ValueDifference<UDAggregate> pair)
-                    {
-                        return pair.rightValue();
-                    }
-                }));
-            }
-            else if (pre.hasColumns())
-            {
-                dropped.addAll(UDAggregate.fromSchema(new Row(entry.getKey(), pre)).values());
-            }
-            else if (post.hasColumns())
-            {
-                created.addAll(UDAggregate.fromSchema(new Row(entry.getKey(), post)).values());
-            }
-        }
-
-        for (UDAggregate udf : created)
-            addAggregate(udf);
-        for (UDAggregate udf : altered)
-            updateAggregate(udf);
-        for (UDAggregate udf : dropped)
-            dropAggregate(udf);
-    }
-
-    private static void addKeyspace(KSMetaData ksm)
-    {
-        assert Schema.instance.getKSMetaData(ksm.name) == null;
-        Schema.instance.load(ksm);
-
-        Keyspace.open(ksm.name);
-        MigrationManager.instance.notifyCreateKeyspace(ksm);
-    }
-
-    private static void addColumnFamily(CFMetaData cfm)
-    {
-        assert Schema.instance.getCFMetaData(cfm.ksName, cfm.cfName) == null;
-        KSMetaData ksm = Schema.instance.getKSMetaData(cfm.ksName);
-        ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(cfm)));
-
-        logger.info("Loading {}", cfm);
-
-        Schema.instance.load(cfm);
-
-        // make sure it's init-ed w/ the old definitions first,
-        // since we're going to call initCf on the new one manually
-        Keyspace.open(cfm.ksName);
-
-        Schema.instance.setKeyspaceDefinition(ksm);
-        Keyspace.open(ksm.name).initCf(cfm.cfId, cfm.cfName, true);
-        MigrationManager.instance.notifyCreateColumnFamily(cfm);
-    }
-
-    private static void addType(UserType ut)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ut.keyspace);
-        assert ksm != null;
-
-        logger.info("Loading {}", ut);
-
-        ksm.userTypes.addType(ut);
-
-        MigrationManager.instance.notifyCreateUserType(ut);
-    }
-
-    private static void addFunction(UDFunction udf)
-    {
-        logger.info("Loading {}", udf);
-
-        Functions.addFunction(udf);
-
-        MigrationManager.instance.notifyCreateFunction(udf);
-    }
-
-    private static void addAggregate(UDAggregate udf)
-    {
-        logger.info("Loading {}", udf);
-
-        Functions.addFunction(udf);
-
-        MigrationManager.instance.notifyCreateAggregate(udf);
-    }
-
-    private static void updateKeyspace(String ksName)
-    {
-        KSMetaData oldKsm = Schema.instance.getKSMetaData(ksName);
-        assert oldKsm != null;
-        KSMetaData newKsm = KSMetaData.cloneWith(oldKsm.reloadAttributes(), oldKsm.cfMetaData().values());
-
-        Schema.instance.setKeyspaceDefinition(newKsm);
-
-        Keyspace.open(ksName).createReplicationStrategy(newKsm);
-        MigrationManager.instance.notifyUpdateKeyspace(newKsm);
-    }
-
-    private static void updateColumnFamily(String ksName, String cfName)
-    {
-        CFMetaData cfm = Schema.instance.getCFMetaData(ksName, cfName);
-        assert cfm != null;
-        cfm.reload();
-
-        Keyspace keyspace = Keyspace.open(cfm.ksName);
-        keyspace.getColumnFamilyStore(cfm.cfName).reload();
-        MigrationManager.instance.notifyUpdateColumnFamily(cfm);
-    }
-
-    private static void updateType(UserType ut)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ut.keyspace);
-        assert ksm != null;
-
-        logger.info("Updating {}", ut);
-
-        ksm.userTypes.addType(ut);
-
-        MigrationManager.instance.notifyUpdateUserType(ut);
-    }
-
-    private static void updateFunction(UDFunction udf)
-    {
-        logger.info("Updating {}", udf);
-
-        Functions.replaceFunction(udf);
-
-        MigrationManager.instance.notifyUpdateFunction(udf);
-    }
-
-    private static void updateAggregate(UDAggregate udf)
-    {
-        logger.info("Updating {}", udf);
-
-        Functions.replaceFunction(udf);
-
-        MigrationManager.instance.notifyUpdateAggregate(udf);
-    }
-
-    private static void dropKeyspace(String ksName)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
-        String snapshotName = Keyspace.getTimestampedSnapshotName(ksName);
-
-        CompactionManager.instance.interruptCompactionFor(ksm.cfMetaData().values(), true);
-
-        Keyspace keyspace = Keyspace.open(ksm.name);
-
-        // remove all cfs from the keyspace instance.
-        List<UUID> droppedCfs = new ArrayList<>();
-        for (CFMetaData cfm : ksm.cfMetaData().values())
-        {
-            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfm.cfName);
-
-            Schema.instance.purge(cfm);
-
-            if (DatabaseDescriptor.isAutoSnapshot())
-                cfs.snapshot(snapshotName);
-            Keyspace.open(ksm.name).dropCf(cfm.cfId);
-
-            droppedCfs.add(cfm.cfId);
-        }
-
-        // remove the keyspace from the static instances.
-        Keyspace.clear(ksm.name);
-        Schema.instance.clearKeyspaceDefinition(ksm);
-
-        keyspace.writeOrder.awaitNewBarrier();
-
-        // force a new segment in the CL
-        CommitLog.instance.forceRecycleAllSegments(droppedCfs);
-
-        MigrationManager.instance.notifyDropKeyspace(ksm);
-    }
-
-    private static void dropColumnFamily(String ksName, String cfName)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
-        assert ksm != null;
-        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(cfName);
-        assert cfs != null;
-
-        // reinitialize the keyspace.
-        CFMetaData cfm = ksm.cfMetaData().get(cfName);
-
-        Schema.instance.purge(cfm);
-        Schema.instance.setKeyspaceDefinition(makeNewKeyspaceDefinition(ksm, cfm));
-
-        CompactionManager.instance.interruptCompactionFor(Arrays.asList(cfm), true);
-
-        if (DatabaseDescriptor.isAutoSnapshot())
-            cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name));
-        Keyspace.open(ksm.name).dropCf(cfm.cfId);
-        MigrationManager.instance.notifyDropColumnFamily(cfm);
-
-        CommitLog.instance.forceRecycleAllSegments(Collections.singleton(cfm.cfId));
-    }
-
-    private static void dropType(UserType ut)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ut.keyspace);
-        assert ksm != null;
-
-        ksm.userTypes.removeType(ut);
-
-        MigrationManager.instance.notifyDropUserType(ut);
-    }
-
-    private static void dropFunction(UDFunction udf)
-    {
-        logger.info("Drop {}", udf);
-
-        // TODO: this is kind of broken as this remove all overloads of the function name
-        Functions.removeFunction(udf.name(), udf.argTypes());
-
-        MigrationManager.instance.notifyDropFunction(udf);
-    }
-
-    private static void dropAggregate(UDAggregate udf)
-    {
-        logger.info("Drop {}", udf);
-
-        // TODO: this is kind of broken as this remove all overloads of the function name
-        Functions.removeFunction(udf.name(), udf.argTypes());
-
-        MigrationManager.instance.notifyDropAggregate(udf);
-    }
-
-    private static KSMetaData makeNewKeyspaceDefinition(KSMetaData ksm, CFMetaData toExclude)
-    {
-        // clone ksm but do not include the new def
-        List<CFMetaData> newCfs = new ArrayList<>(ksm.cfMetaData().values());
-        newCfs.remove(toExclude);
-        assert newCfs.size() == ksm.cfMetaData().size() - 1;
-        return KSMetaData.cloneWith(ksm, newCfs);
-    }
-
-    private static void flushSchemaCFs()
-    {
-        for (String cf : SystemKeyspace.ALL_SCHEMA_TABLES)
-            SystemKeyspace.forceBlockingFlush(cf);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/HintedHandOffManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
index 081e01b..8c4477b 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -115,7 +115,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
                                                                                  new NamedThreadFactory("HintedHandoff", Thread.MIN_PRIORITY),
                                                                                  "internal");
 
-    private final ColumnFamilyStore hintStore = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS_TABLE);
+    private final ColumnFamilyStore hintStore = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS);
 
     /**
      * Returns a mutation representing a Hint to be sent to <code>targetId</code>
@@ -134,9 +134,9 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
 
         UUID hintId = UUIDGen.getTimeUUID();
         // serialize the hint with id and version as a composite column name
-        CellName name = SystemKeyspace.HintsTable.comparator.makeCellName(hintId, MessagingService.current_version);
+        CellName name = SystemKeyspace.Hints.comparator.makeCellName(hintId, MessagingService.current_version);
         ByteBuffer value = ByteBuffer.wrap(FBUtilities.serialize(mutation, Mutation.serializer, MessagingService.current_version));
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Schema.instance.getCFMetaData(SystemKeyspace.NAME, SystemKeyspace.HINTS_TABLE));
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Schema.instance.getCFMetaData(SystemKeyspace.NAME, SystemKeyspace.HINTS));
         cf.addColumn(name, value, now, ttl);
         return new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(targetId), cf);
     }
@@ -182,7 +182,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
     private static void deleteHint(ByteBuffer tokenBytes, CellName columnName, long timestamp)
     {
         Mutation mutation = new Mutation(SystemKeyspace.NAME, tokenBytes);
-        mutation.delete(SystemKeyspace.HINTS_TABLE, columnName, timestamp);
+        mutation.delete(SystemKeyspace.HINTS, columnName, timestamp);
         mutation.applyUnsafe(); // don't bother with commitlog since we're going to flush as soon as we're done with delivery
     }
 
@@ -207,7 +207,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         UUID hostId = StorageService.instance.getTokenMetadata().getHostId(endpoint);
         ByteBuffer hostIdBytes = ByteBuffer.wrap(UUIDGen.decompose(hostId));
         final Mutation mutation = new Mutation(SystemKeyspace.NAME, hostIdBytes);
-        mutation.delete(SystemKeyspace.HINTS_TABLE, System.currentTimeMillis());
+        mutation.delete(SystemKeyspace.HINTS, System.currentTimeMillis());
 
         // execute asynchronously to avoid blocking caller (which may be processing gossip)
         Runnable runnable = new Runnable()
@@ -241,7 +241,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
                 try
                 {
                     logger.info("Truncating all stored hints.");
-                    Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS_TABLE).truncateBlocking();
+                    Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS).truncateBlocking();
                 }
                 catch (Exception e)
                 {
@@ -375,7 +375,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         {
             long now = System.currentTimeMillis();
             QueryFilter filter = QueryFilter.getSliceFilter(epkey,
-                                                            SystemKeyspace.HINTS_TABLE,
+                                                            SystemKeyspace.HINTS,
                                                             startColumn,
                                                             Composites.EMPTY,
                                                             false,
@@ -601,7 +601,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         try
         {
             RangeSliceCommand cmd = new RangeSliceCommand(SystemKeyspace.NAME,
-                                                          SystemKeyspace.HINTS_TABLE,
+                                                          SystemKeyspace.HINTS,
                                                           System.currentTimeMillis(),
                                                           predicate,
                                                           range,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 09fc338..b34d589 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -32,7 +32,6 @@ import java.util.concurrent.Future;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,6 +44,7 @@ import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.pager.QueryPagers;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java
index 4cf441e..2381f26 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -357,7 +357,7 @@ public class Memtable
                         // and BL data is strictly local, so we don't need to preserve tombstones for repair.
                         // If we have a data row + row level tombstone, then writing it is effectively an expensive no-op so we skip it.
                         // See CASSANDRA-4667.
-                        if (cfs.name.equals(SystemKeyspace.BATCHLOG_TABLE) && cfs.keyspace.getName().equals(SystemKeyspace.NAME))
+                        if (cfs.name.equals(SystemKeyspace.BATCHLOG) && cfs.keyspace.getName().equals(SystemKeyspace.NAME))
                             continue;
                     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
index d4503ba..79753c1 100644
--- a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.service.MigrationManager;
 
 /**
@@ -40,7 +41,7 @@ public class MigrationRequestVerbHandler implements IVerbHandler
     {
         logger.debug("Received migration request from {}.", message.from);
         MessageOut<Collection<Mutation>> response = new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE,
-                                                                     SystemKeyspace.serializeSchema(),
+                                                                     LegacySchemaTables.convertSchemaToMutations(),
                                                                      MigrationManager.MigrationsSerializer.instance);
         MessagingService.instance().sendReply(response, id, message.from);
     }


[5/5] cassandra git commit: Isolate schema serializaton code

Posted by al...@apache.org.
Isolate schema serializaton code

patch by Aleksey Yeschenko; reviewed by Tyler Hobbs for CASSANDRA-8261


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

Branch: refs/heads/trunk
Commit: 3e9d345f0078922950157de4fd4c7992512b43b8
Parents: 32ac6af
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Dec 17 01:12:19 2014 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Dec 17 01:34:16 2014 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 .../org/apache/cassandra/config/CFMetaData.java |  331 +---
 .../cassandra/config/ColumnDefinition.java      |  134 +-
 .../cassandra/config/DatabaseDescriptor.java    |   87 +-
 .../org/apache/cassandra/config/KSMetaData.java |  155 +-
 .../org/apache/cassandra/config/Schema.java     |  248 ++-
 .../cassandra/config/TriggerDefinition.java     |   63 -
 .../org/apache/cassandra/config/UTMetaData.java |   91 +-
 .../cassandra/cql3/functions/Functions.java     |   22 +-
 .../cql3/functions/JavaSourceUDFFactory.java    |    5 +-
 .../cassandra/cql3/functions/UDAggregate.java   |  206 +--
 .../cassandra/cql3/functions/UDFunction.java    |  193 +--
 .../cassandra/cql3/functions/UDHelper.java      |   12 +-
 .../cql3/statements/CreateTableStatement.java   |   24 +-
 .../apache/cassandra/db/AtomicBTreeColumns.java |    3 +-
 .../apache/cassandra/db/BatchlogManager.java    |   14 +-
 .../db/DefinitionsUpdateVerbHandler.java        |    3 +-
 .../org/apache/cassandra/db/DefsTables.java     |  622 --------
 .../cassandra/db/HintedHandOffManager.java      |   16 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |    2 +-
 src/java/org/apache/cassandra/db/Memtable.java  |    2 +-
 .../db/MigrationRequestVerbHandler.java         |    3 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |  514 ++----
 .../hadoop/ColumnFamilyRecordReader.java        |   28 +-
 .../cassandra/hadoop/cql3/CqlRecordReader.java  |   28 +-
 .../cassandra/hadoop/cql3/CqlRecordWriter.java  |   23 +-
 .../hadoop/pig/AbstractCassandraStorage.java    |   46 +-
 .../apache/cassandra/hadoop/pig/CqlStorage.java |   26 +-
 .../cassandra/io/sstable/CQLSSTableWriter.java  |   10 +-
 .../cassandra/schema/LegacySchemaTables.java    | 1480 ++++++++++++++++++
 .../cassandra/service/CassandraDaemon.java      |    8 +-
 .../apache/cassandra/service/ClientState.java   |    3 +-
 .../cassandra/service/MigrationManager.java     |  127 +-
 .../apache/cassandra/service/MigrationTask.java |    4 +-
 .../apache/cassandra/service/StorageProxy.java  |    2 +-
 .../cassandra/service/StorageService.java       |    4 +-
 .../cassandra/thrift/ThriftConversion.java      |    5 +-
 .../org/apache/cassandra/tools/BulkLoader.java  |    6 +-
 .../apache/cassandra/tools/SSTableExport.java   |    3 +-
 .../apache/cassandra/tools/SSTableImport.java   |    2 +-
 .../cassandra/tools/SSTableLevelResetter.java   |    3 +-
 .../cassandra/tools/StandaloneScrubber.java     |    3 +-
 .../cassandra/tools/StandaloneSplitter.java     |    4 +-
 .../cassandra/tools/StandaloneUpgrader.java     |    2 +-
 .../apache/cassandra/config/CFMetaDataTest.java |   15 +-
 .../config/DatabaseDescriptorTest.java          |    7 +-
 .../org/apache/cassandra/config/DefsTest.java   |  564 -------
 .../apache/cassandra/config/KSMetaDataTest.java |    6 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |    4 +-
 .../cassandra/db/BatchlogManagerTest.java       |    4 +-
 .../apache/cassandra/db/HintedHandOffTest.java  |    8 +-
 .../schema/LegacySchemaTablesTest.java          |  568 +++++++
 .../service/EmbeddedCassandraServiceTest.java   |    2 +-
 .../service/StorageServiceServerTest.java       |    3 +-
 54 files changed, 2792 insertions(+), 2957 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3571c1e..6f4cdec 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0
+ * Modernize schema tables (CASSANDRA-8261)
  * Support for user-defined aggregation functions (CASSANDRA-8053)
  * Fix NPE in SelectStatement with empty IN values (CASSANDRA-8419)
  * Refactor SelectStatement, return IN results in natural order instead

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index eb78ec7..0730ba7 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -37,7 +37,6 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.statements.CFStatement;
 import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.db.*;
@@ -50,14 +49,12 @@ import org.apache.cassandra.io.compress.CompressionParameters;
 import org.apache.cassandra.io.compress.LZ4Compressor;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
 import org.github.jamm.Unmetered;
 
-import static org.apache.cassandra.utils.FBUtilities.fromJsonMap;
-import static org.apache.cassandra.utils.FBUtilities.json;
-
 /**
  * This class can be tricky to modify. Please read http://wiki.apache.org/cassandra/ConfigurationNotes for how to do so safely.
  */
@@ -221,7 +218,7 @@ public final class CFMetaData
     public volatile CompressionParameters compressionParameters = new CompressionParameters(null);
 
     // attribute setters that return the modified CFMetaData instance
-    public CFMetaData comment(String prop) { comment = Strings.nullToEmpty(prop); return this;}
+    public CFMetaData comment(String prop) {comment = Strings.nullToEmpty(prop); return this;}
     public CFMetaData readRepairChance(double prop) {readRepairChance = prop; return this;}
     public CFMetaData dcLocalReadRepairChance(double prop) {dcLocalReadRepairChance = prop; return this;}
     public CFMetaData gcGraceSeconds(int prop) {gcGraceSeconds = prop; return this;}
@@ -344,8 +341,8 @@ public final class CFMetaData
         // Depends on parent's cache setting, turn on its index CF's cache.
         // Row caching is never enabled; see CASSANDRA-5732
         CachingOptions indexCaching = parent.getCaching().keyCache.isEnabled()
-                             ? CachingOptions.KEYS_ONLY
-                             : CachingOptions.NONE;
+                                    ? CachingOptions.KEYS_ONLY
+                                    : CachingOptions.NONE;
 
         return new CFMetaData(parent.ksName, parent.indexColumnFamilyName(info), ColumnFamilyType.Standard, indexComparator, parent.cfId)
                              .keyValidator(info.type)
@@ -386,7 +383,8 @@ public final class CFMetaData
         return copyOpts(new CFMetaData(ksName, cfName, cfType, comparator, newCfId), this);
     }
 
-    static CFMetaData copyOpts(CFMetaData newCFMD, CFMetaData oldCFMD)
+    @VisibleForTesting
+    public static CFMetaData copyOpts(CFMetaData newCFMD, CFMetaData oldCFMD)
     {
         List<ColumnDefinition> clonedColumns = new ArrayList<>(oldCFMD.allColumns().size());
         for (ColumnDefinition cd : oldCFMD.allColumns())
@@ -449,6 +447,11 @@ public final class CFMetaData
         return cfName.contains(".");
     }
 
+    public Map<ByteBuffer, ColumnDefinition> getColumnMetadata()
+    {
+        return columnMetadata;
+    }
+
     /**
      *
      * @return The name of the parent cf if this is a seconday index
@@ -723,14 +726,9 @@ public final class CFMetaData
 
     public void reload()
     {
-        Row cfDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, ksName, cfName);
-
-        if (cfDefRow.cf == null || !cfDefRow.cf.hasColumns())
-            throw new RuntimeException(String.format("%s not found in the schema definitions keyspace.", ksName + ":" + cfName));
-
         try
         {
-            apply(fromSchema(cfDefRow));
+            apply(LegacySchemaTables.createTableFromName(ksName, cfName));
         }
         catch (ConfigurationException e)
         {
@@ -739,13 +737,12 @@ public final class CFMetaData
     }
 
     /**
-     * Updates CFMetaData in-place to match cf_def
-     *
-     * *Note*: This method left package-private only for DefsTest, don't use directly!
+     * Updates CFMetaData in-place to match cfm
      *
      * @throws ConfigurationException if ks/cf names or cf ids didn't match
      */
-    void apply(CFMetaData cfm) throws ConfigurationException
+    @VisibleForTesting
+    public void apply(CFMetaData cfm) throws ConfigurationException
     {
         logger.debug("applying {} to {}", cfm, this);
 
@@ -1116,89 +1113,6 @@ public final class CFMetaData
                                                            "interval (%d).", maxIndexInterval, minIndexInterval));
     }
 
-    /**
-     * Create schema mutations to update this metadata to provided new state.
-     *
-     * @param newState The new metadata (for the same CF)
-     * @param modificationTimestamp Timestamp to use for mutation
-     * @param fromThrift whether the newState comes from thrift
-     *
-     * @return Difference between attributes in form of schema mutation
-     */
-    public Mutation toSchemaUpdate(CFMetaData newState, long modificationTimestamp, boolean fromThrift)
-    {
-        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(ksName));
-
-        newState.toSchemaNoColumnsNoTriggers(mutation, modificationTimestamp);
-
-        MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(columnMetadata, newState.columnMetadata);
-
-        // columns that are no longer needed
-        for (ColumnDefinition cd : columnDiff.entriesOnlyOnLeft().values())
-        {
-            // Thrift only knows about the REGULAR ColumnDefinition type, so don't consider other type
-            // are being deleted just because they are not here.
-            if (fromThrift && cd.kind != ColumnDefinition.Kind.REGULAR)
-                continue;
-
-            cd.deleteFromSchema(mutation, modificationTimestamp);
-        }
-
-        // newly added columns
-        for (ColumnDefinition cd : columnDiff.entriesOnlyOnRight().values())
-            cd.toSchema(mutation, modificationTimestamp);
-
-        // old columns with updated attributes
-        for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
-        {
-            ColumnDefinition cd = newState.columnMetadata.get(name);
-            cd.toSchema(mutation, modificationTimestamp);
-        }
-
-        MapDifference<String, TriggerDefinition> triggerDiff = Maps.difference(triggers, newState.triggers);
-
-        // dropped triggers
-        for (TriggerDefinition td : triggerDiff.entriesOnlyOnLeft().values())
-            td.deleteFromSchema(mutation, cfName, modificationTimestamp);
-
-        // newly created triggers
-        for (TriggerDefinition td : triggerDiff.entriesOnlyOnRight().values())
-            td.toSchema(mutation, cfName, modificationTimestamp);
-
-        return mutation;
-    }
-
-    /**
-     * Remove all CF attributes from schema
-     *
-     * @param timestamp Timestamp to use
-     *
-     * @return Mutation to use to completely remove cf from schema
-     */
-    public Mutation dropFromSchema(long timestamp)
-    {
-        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(ksName));
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnFamiliesTable);
-        int ldt = (int) (System.currentTimeMillis() / 1000);
-
-        Composite prefix = SystemKeyspace.SchemaColumnFamiliesTable.comparator.make(cfName);
-        cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-
-        for (ColumnDefinition cd : allColumns())
-            cd.deleteFromSchema(mutation, timestamp);
-
-        for (TriggerDefinition td : triggers.values())
-            td.deleteFromSchema(mutation, cfName, timestamp);
-
-        for (String indexName : Keyspace.open(this.ksName).getColumnFamilyStore(this.cfName).getBuiltIndexes())
-        {
-            ColumnFamily indexCf = mutation.addOrGet(SystemKeyspace.BuiltIndexesTable);
-            indexCf.addTombstone(indexCf.getComparator().makeCellName(indexName), ldt, timestamp);
-        }
-
-        return mutation;
-    }
-
     public boolean isPurged()
     {
         return isPurged;
@@ -1209,215 +1123,6 @@ public final class CFMetaData
         isPurged = true;
     }
 
-    public void toSchema(Mutation mutation, long timestamp)
-    {
-        toSchemaNoColumnsNoTriggers(mutation, timestamp);
-
-        for (TriggerDefinition td : triggers.values())
-            td.toSchema(mutation, cfName, timestamp);
-
-        for (ColumnDefinition cd : allColumns())
-            cd.toSchema(mutation, timestamp);
-    }
-
-    private void toSchemaNoColumnsNoTriggers(Mutation mutation, long timestamp)
-    {
-        // For property that can be null (and can be changed), we insert tombstones, to make sure
-        // we don't keep a property the user has removed
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnFamiliesTable);
-        Composite prefix = SystemKeyspace.SchemaColumnFamiliesTable.comparator.make(cfName);
-        CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
-        adder.add("cf_id", cfId);
-        adder.add("type", cfType.toString());
-
-        if (isSuper())
-        {
-            // We need to continue saving the comparator and subcomparator separatly, otherwise
-            // we won't know at deserialization if the subcomparator should be taken into account
-            // TODO: we should implement an on-start migration if we want to get rid of that.
-            adder.add("comparator", comparator.subtype(0).toString());
-            adder.add("subcomparator", comparator.subtype(1).toString());
-        }
-        else
-        {
-            adder.add("comparator", comparator.toString());
-        }
-
-        adder.add("comment", comment);
-        adder.add("read_repair_chance", readRepairChance);
-        adder.add("local_read_repair_chance", dcLocalReadRepairChance);
-        adder.add("gc_grace_seconds", gcGraceSeconds);
-        adder.add("default_validator", defaultValidator.toString());
-        adder.add("key_validator", keyValidator.toString());
-        adder.add("min_compaction_threshold", minCompactionThreshold);
-        adder.add("max_compaction_threshold", maxCompactionThreshold);
-        adder.add("bloom_filter_fp_chance", getBloomFilterFpChance());
-        adder.add("memtable_flush_period_in_ms", memtableFlushPeriod);
-        adder.add("caching", caching.toString());
-        adder.add("default_time_to_live", defaultTimeToLive);
-        adder.add("compaction_strategy_class", compactionStrategyClass.getName());
-        adder.add("compression_parameters", json(compressionParameters.asThriftOptions()));
-        adder.add("compaction_strategy_options", json(compactionStrategyOptions));
-        adder.add("min_index_interval", minIndexInterval);
-        adder.add("max_index_interval", maxIndexInterval);
-        adder.add("speculative_retry", speculativeRetry.toString());
-
-        for (Map.Entry<ColumnIdentifier, Long> entry : droppedColumns.entrySet())
-            adder.addMapEntry("dropped_columns", entry.getKey().toString(), entry.getValue());
-
-        adder.add("is_dense", isDense);
-    }
-
-    @VisibleForTesting
-    public static CFMetaData fromSchemaNoTriggers(UntypedResultSet.Row result, UntypedResultSet serializedColumnDefinitions)
-    {
-        try
-        {
-            String ksName = result.getString("keyspace_name");
-            String cfName = result.getString("columnfamily_name");
-
-            AbstractType<?> rawComparator = TypeParser.parse(result.getString("comparator"));
-            AbstractType<?> subComparator = result.has("subcomparator") ? TypeParser.parse(result.getString("subcomparator")) : null;
-            ColumnFamilyType cfType = ColumnFamilyType.valueOf(result.getString("type"));
-
-            AbstractType<?> fullRawComparator = makeRawAbstractType(rawComparator, subComparator);
-
-            List<ColumnDefinition> columnDefs = ColumnDefinition.fromSchema(serializedColumnDefinitions,
-                                                                            ksName,
-                                                                            cfName,
-                                                                            fullRawComparator,
-                                                                            cfType == ColumnFamilyType.Super);
-
-            boolean isDense = result.has("is_dense")
-                            ? result.getBoolean("is_dense")
-                            : calculateIsDense(fullRawComparator, columnDefs);
-
-            CellNameType comparator = CellNames.fromAbstractType(fullRawComparator, isDense);
-
-            // if we are upgrading, we use id generated from names initially
-            UUID cfId = result.has("cf_id")
-                      ? result.getUUID("cf_id")
-                      : generateLegacyCfId(ksName, cfName);
-
-            CFMetaData cfm = new CFMetaData(ksName, cfName, cfType, comparator, cfId);
-            cfm.isDense(isDense);
-
-            cfm.readRepairChance(result.getDouble("read_repair_chance"));
-            cfm.dcLocalReadRepairChance(result.getDouble("local_read_repair_chance"));
-            cfm.gcGraceSeconds(result.getInt("gc_grace_seconds"));
-            cfm.defaultValidator(TypeParser.parse(result.getString("default_validator")));
-            cfm.keyValidator(TypeParser.parse(result.getString("key_validator")));
-            cfm.minCompactionThreshold(result.getInt("min_compaction_threshold"));
-            cfm.maxCompactionThreshold(result.getInt("max_compaction_threshold"));
-            if (result.has("comment"))
-                cfm.comment(result.getString("comment"));
-            if (result.has("memtable_flush_period_in_ms"))
-                cfm.memtableFlushPeriod(result.getInt("memtable_flush_period_in_ms"));
-            cfm.caching(CachingOptions.fromString(result.getString("caching")));
-            if (result.has("default_time_to_live"))
-                cfm.defaultTimeToLive(result.getInt("default_time_to_live"));
-            if (result.has("speculative_retry"))
-                cfm.speculativeRetry(SpeculativeRetry.fromString(result.getString("speculative_retry")));
-            cfm.compactionStrategyClass(createCompactionStrategy(result.getString("compaction_strategy_class")));
-            cfm.compressionParameters(CompressionParameters.create(fromJsonMap(result.getString("compression_parameters"))));
-            cfm.compactionStrategyOptions(fromJsonMap(result.getString("compaction_strategy_options")));
-
-            if (result.has("min_index_interval"))
-                cfm.minIndexInterval(result.getInt("min_index_interval"));
-
-            if (result.has("max_index_interval"))
-                cfm.maxIndexInterval(result.getInt("max_index_interval"));
-
-            if (result.has("bloom_filter_fp_chance"))
-                cfm.bloomFilterFpChance(result.getDouble("bloom_filter_fp_chance"));
-            else
-                cfm.bloomFilterFpChance(cfm.getBloomFilterFpChance());
-
-            if (result.has("dropped_columns"))
-                cfm.droppedColumns(convertDroppedColumns(result.getMap("dropped_columns", UTF8Type.instance, LongType.instance)));
-
-            for (ColumnDefinition cd : columnDefs)
-                cfm.addOrReplaceColumnDefinition(cd);
-
-            return cfm.rebuild();
-        }
-        catch (SyntaxException | ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void addColumnMetadataFromAliases(List<ByteBuffer> aliases, AbstractType<?> comparator, ColumnDefinition.Kind kind)
-    {
-        if (comparator instanceof CompositeType)
-        {
-            CompositeType ct = (CompositeType)comparator;
-            for (int i = 0; i < aliases.size(); ++i)
-            {
-                if (aliases.get(i) != null)
-                {
-                    addOrReplaceColumnDefinition(new ColumnDefinition(this, aliases.get(i), ct.types.get(i), i, kind));
-                }
-            }
-        }
-        else
-        {
-            assert aliases.size() <= 1;
-            if (!aliases.isEmpty() && aliases.get(0) != null)
-                addOrReplaceColumnDefinition(new ColumnDefinition(this, aliases.get(0), comparator, null, kind));
-        }
-    }
-
-    /**
-     * Deserialize CF metadata from low-level representation
-     *
-     * @return Metadata deserialized from schema
-     */
-    public static CFMetaData fromSchema(UntypedResultSet.Row result)
-    {
-        String ksName = result.getString("keyspace_name");
-        String cfName = result.getString("columnfamily_name");
-
-        Row serializedColumns = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_COLUMNS_TABLE, ksName, cfName);
-        CFMetaData cfm = fromSchemaNoTriggers(result, ColumnDefinition.resultify(serializedColumns));
-
-        Row serializedTriggers = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_TRIGGERS_TABLE, ksName, cfName);
-        addTriggerDefinitionsFromSchema(cfm, serializedTriggers);
-
-        return cfm;
-    }
-
-    private static CFMetaData fromSchema(Row row)
-    {
-        UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", row).one();
-        return fromSchema(result);
-    }
-
-    private static Map<ColumnIdentifier, Long> convertDroppedColumns(Map<String, Long> raw)
-    {
-        Map<ColumnIdentifier, Long> converted = Maps.newHashMap();
-        for (Map.Entry<String, Long> entry : raw.entrySet())
-            converted.put(new ColumnIdentifier(entry.getKey(), true), entry.getValue());
-        return converted;
-    }
-
-    /**
-     * Convert current metadata into schema mutation
-     *
-     * @param timestamp Timestamp to use
-     *
-     * @return Low-level representation of the CF
-     *
-     * @throws ConfigurationException if any of the attributes didn't pass validation
-     */
-    public Mutation toSchema(long timestamp) throws ConfigurationException
-    {
-        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(ksName));
-        toSchema(mutation, timestamp);
-        return mutation;
-    }
-
     // The comparator to validate the definition name.
 
     public AbstractType<?> getColumnDefinitionComparator(ColumnDefinition def)
@@ -1474,12 +1179,6 @@ public final class CFMetaData
         return columnMetadata.remove(def.name.bytes) != null;
     }
 
-    private static void addTriggerDefinitionsFromSchema(CFMetaData cfDef, Row serializedTriggerDefinitions)
-    {
-        for (TriggerDefinition td : TriggerDefinition.fromSchema(serializedTriggerDefinitions))
-            cfDef.triggers.put(td.name, td);
-    }
-
     public void addTriggerDefinition(TriggerDefinition def) throws InvalidRequestException
     {
         if (containsTriggerDefinition(def))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index 354a6f1..1cc7f1d 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -26,25 +26,11 @@ import com.google.common.base.Objects;
 import com.google.common.collect.Lists;
 
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.utils.FBUtilities;
-
-import static org.apache.cassandra.utils.FBUtilities.json;
 
 public class ColumnDefinition extends ColumnSpecification
 {
-    // system.schema_columns column names
-    private static final String COLUMN_NAME = "column_name";
-    private static final String TYPE = "validator";
-    private static final String INDEX_TYPE = "index_type";
-    private static final String INDEX_OPTIONS = "index_options";
-    private static final String INDEX_NAME = "index_name";
-    private static final String COMPONENT_INDEX = "component_index";
-    private static final String KIND = "type";
-
     /*
      * The type of CQL3 column this definition represents.
      * There is 3 main type of CQL3 columns: those parts of the partition key,
@@ -62,20 +48,7 @@ public class ColumnDefinition extends ColumnSpecification
         CLUSTERING_COLUMN,
         REGULAR,
         STATIC,
-        COMPACT_VALUE;
-
-        public String serialize()
-        {
-            // For backward compatibility we need to special case CLUSTERING_COLUMN
-            return this == CLUSTERING_COLUMN ? "clustering_key" : this.toString().toLowerCase();
-        }
-
-        public static Kind deserialize(String value)
-        {
-            if (value.equalsIgnoreCase("clustering_key"))
-                return CLUSTERING_COLUMN;
-            return Enum.valueOf(Kind.class, value.toUpperCase());
-        }
+        COMPACT_VALUE
     }
 
     public final Kind kind;
@@ -266,36 +239,6 @@ public class ColumnDefinition extends ColumnSpecification
         return kind == Kind.REGULAR || kind == Kind.STATIC;
     }
 
-    /**
-     * Drop specified column from the schema using given mutation.
-     *
-     * @param mutation  The schema mutation
-     * @param timestamp The timestamp to use for column modification
-     */
-    public void deleteFromSchema(Mutation mutation, long timestamp)
-    {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnsTable);
-        int ldt = (int) (System.currentTimeMillis() / 1000);
-
-        // Note: we do want to use name.toString(), not name.bytes directly for backward compatibility (For CQL3, this won't make a difference).
-        Composite prefix = SystemKeyspace.SchemaColumnsTable.comparator.make(cfName, name.toString());
-        cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-    }
-
-    public void toSchema(Mutation mutation, long timestamp)
-    {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnsTable);
-        Composite prefix = SystemKeyspace.SchemaColumnsTable.comparator.make(cfName, name.toString());
-        CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
-        adder.add(TYPE, type.toString());
-        adder.add(INDEX_TYPE, indexType == null ? null : indexType.toString());
-        adder.add(INDEX_OPTIONS, json(indexOptions));
-        adder.add(INDEX_NAME, indexName);
-        adder.add(COMPONENT_INDEX, componentIndex);
-        adder.add(KIND, kind.serialize());
-    }
-
     public ColumnDefinition apply(ColumnDefinition def)  throws ConfigurationException
     {
         assert kind == def.kind && Objects.equal(componentIndex, def.componentIndex);
@@ -323,81 +266,6 @@ public class ColumnDefinition extends ColumnSpecification
                                     kind);
     }
 
-    public static UntypedResultSet resultify(Row serializedColumns)
-    {
-        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.SCHEMA_COLUMNS_TABLE);
-        return QueryProcessor.resultify(query, serializedColumns);
-    }
-
-    /**
-     * Deserialize columns from storage-level representation
-     *
-     * @param serializedColumns storage-level partition containing the column definitions
-     * @return the list of processed ColumnDefinitions
-     */
-    public static List<ColumnDefinition> fromSchema(UntypedResultSet serializedColumns, String ksName, String cfName, AbstractType<?> rawComparator, boolean isSuper)
-    {
-        List<ColumnDefinition> cds = new ArrayList<>();
-        for (UntypedResultSet.Row row : serializedColumns)
-        {
-            Kind kind = row.has(KIND)
-                      ? Kind.deserialize(row.getString(KIND))
-                      : Kind.REGULAR;
-
-            Integer componentIndex = null;
-            if (row.has(COMPONENT_INDEX))
-                componentIndex = row.getInt(COMPONENT_INDEX);
-            else if (kind == Kind.CLUSTERING_COLUMN && isSuper)
-                componentIndex = 1; // A ColumnDefinition for super columns applies to the column component
-
-            // Note: we save the column name as string, but we should not assume that it is an UTF8 name, we
-            // we need to use the comparator fromString method
-            AbstractType<?> comparator = getComponentComparator(rawComparator, componentIndex, kind);
-            ColumnIdentifier name = new ColumnIdentifier(comparator.fromString(row.getString(COLUMN_NAME)), comparator);
-
-            AbstractType<?> validator;
-            try
-            {
-                validator = TypeParser.parse(row.getString(TYPE));
-            }
-            catch (RequestValidationException e)
-            {
-                throw new RuntimeException(e);
-            }
-
-            IndexType indexType = null;
-            if (row.has(INDEX_TYPE))
-                indexType = IndexType.valueOf(row.getString(INDEX_TYPE));
-
-            Map<String, String> indexOptions = null;
-            if (row.has(INDEX_OPTIONS))
-                indexOptions = FBUtilities.fromJsonMap(row.getString(INDEX_OPTIONS));
-
-            String indexName = null;
-            if (row.has(INDEX_NAME))
-                indexName = row.getString(INDEX_NAME);
-
-            cds.add(new ColumnDefinition(ksName, cfName, name, validator, indexType, indexOptions, indexName, componentIndex, kind));
-        }
-
-        return cds;
-    }
-
-    public static AbstractType<?> getComponentComparator(AbstractType<?> rawComparator, Integer componentIndex, ColumnDefinition.Kind kind)
-    {
-        switch (kind)
-        {
-            case REGULAR:
-                if (componentIndex == null || (componentIndex == 0 && !(rawComparator instanceof CompositeType)))
-                    return rawComparator;
-
-                return ((CompositeType)rawComparator).types.get(componentIndex);
-            default:
-                // CQL3 column names are UTF8
-                return UTF8Type.instance;
-        }
-    }
-
     public String getIndexName()
     {
         return indexName;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index a0e84f9..f2897ee 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -18,49 +18,29 @@
 package org.apache.cassandra.config;
 
 import java.io.File;
-import java.io.FileFilter;
 import java.io.IOException;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Enumeration;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
+import java.net.*;
+import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.primitives.Longs;
-import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.cassandra.auth.AllowAllAuthenticator;
-import org.apache.cassandra.auth.AllowAllAuthorizer;
-import org.apache.cassandra.auth.AllowAllInternodeAuthenticator;
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.auth.IAuthorizer;
-import org.apache.cassandra.auth.IInternodeAuthenticator;
+
+import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.Config.RequestSchedulerId;
 import org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DefsTables;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.IAllocator;
-import org.apache.cassandra.locator.DynamicEndpointSnitch;
-import org.apache.cassandra.locator.EndpointSnitchInfo;
-import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.locator.SeedProvider;
+import org.apache.cassandra.locator.*;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.scheduler.IRequestScheduler;
 import org.apache.cassandra.scheduler.NoScheduler;
@@ -69,10 +49,7 @@ import org.apache.cassandra.thrift.ThriftServer;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.memory.HeapPool;
-import org.apache.cassandra.utils.memory.NativePool;
-import org.apache.cassandra.utils.memory.MemtablePool;
-import org.apache.cassandra.utils.memory.SlabPool;
+import org.apache.cassandra.utils.memory.*;
 
 public class DatabaseDescriptor
 {
@@ -585,9 +562,6 @@ public class DatabaseDescriptor
             conf.server_encryption_options = conf.encryption_options;
         }
 
-        // hardcoded system keyspace
-        Schema.instance.load(SystemKeyspace.definition());
-
         // load the seeds for node contact points
         if (conf.seed_provider == null)
         {
@@ -620,53 +594,6 @@ public class DatabaseDescriptor
         return conf.dynamic_snitch ? new DynamicEndpointSnitch(snitch) : snitch;
     }
 
-    /** load keyspace (keyspace) definitions, but do not initialize the keyspace instances. */
-    public static void loadSchemas()
-    {
-        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_KEYSPACES_TABLE);
-
-        // if keyspace with definitions is empty try loading the old way
-        if (schemaCFS.estimateKeys() == 0)
-        {
-            logger.info("Couldn't detect any schema definitions in local storage.");
-            // peek around the data directories to see if anything is there.
-            if (hasExistingNoSystemTables())
-                logger.info("Found keyspace data in data directories. Consider using cqlsh to define your schema.");
-            else
-                logger.info("To create keyspaces and column families, see 'help create' in cqlsh.");
-        }
-        else
-        {
-            Schema.instance.load(DefsTables.loadFromKeyspace());
-        }
-
-        Schema.instance.updateVersion();
-    }
-
-    private static boolean hasExistingNoSystemTables()
-    {
-        for (String dataDir : getAllDataFileLocations())
-        {
-            File dataPath = new File(dataDir);
-            if (dataPath.exists() && dataPath.isDirectory())
-            {
-                // see if there are other directories present.
-                int dirCount = dataPath.listFiles(new FileFilter()
-                {
-                    public boolean accept(File pathname)
-                    {
-                        return pathname.isDirectory() && !pathname.getName().equals(SystemKeyspace.NAME);
-                    }
-                }).length;
-
-                if (dirCount > 0)
-                    return true;
-            }
-        }
-
-        return false;
-    }
-
     public static IAuthenticator getAuthenticator()
     {
         return authenticator;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/KSMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/KSMetaData.java b/src/java/org/apache/cassandra/config/KSMetaData.java
index e5576ad..1537aae 100644
--- a/src/java/org/apache/cassandra/config/KSMetaData.java
+++ b/src/java/org/apache/cassandra/config/KSMetaData.java
@@ -21,15 +21,10 @@ import java.util.*;
 
 import com.google.common.base.Objects;
 
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.*;
 import org.apache.cassandra.service.StorageService;
 
-import static org.apache.cassandra.utils.FBUtilities.*;
-
 public final class KSMetaData
 {
     public final String name;
@@ -43,18 +38,26 @@ public final class KSMetaData
     public KSMetaData(String name,
                       Class<? extends AbstractReplicationStrategy> strategyClass,
                       Map<String, String> strategyOptions,
+                      boolean durableWrites)
+    {
+        this(name, strategyClass, strategyOptions, durableWrites, Collections.<CFMetaData>emptyList(), new UTMetaData());
+    }
+
+    public KSMetaData(String name,
+                      Class<? extends AbstractReplicationStrategy> strategyClass,
+                      Map<String, String> strategyOptions,
                       boolean durableWrites,
                       Iterable<CFMetaData> cfDefs)
     {
         this(name, strategyClass, strategyOptions, durableWrites, cfDefs, new UTMetaData());
     }
 
-    KSMetaData(String name,
-               Class<? extends AbstractReplicationStrategy> strategyClass,
-               Map<String, String> strategyOptions,
-               boolean durableWrites,
-               Iterable<CFMetaData> cfDefs,
-               UTMetaData userTypes)
+    private KSMetaData(String name,
+                       Class<? extends AbstractReplicationStrategy> strategyClass,
+                       Map<String, String> strategyOptions,
+                       boolean durableWrites,
+                       Iterable<CFMetaData> cfDefs,
+                       UTMetaData userTypes)
     {
         this.name = name;
         this.strategyClass = strategyClass == null ? NetworkTopologyStrategy.class : strategyClass;
@@ -82,9 +85,27 @@ public final class KSMetaData
         return new KSMetaData(name, strategyClass, options, durablesWrites, cfDefs, new UTMetaData());
     }
 
-    public static KSMetaData cloneWith(KSMetaData ksm, Iterable<CFMetaData> cfDefs)
+    public KSMetaData cloneWithTableRemoved(CFMetaData table)
+    {
+        // clone ksm but do not include the new table
+        List<CFMetaData> newTables = new ArrayList<>(cfMetaData().values());
+        newTables.remove(table);
+        assert newTables.size() == cfMetaData().size() - 1;
+        return cloneWith(newTables, userTypes);
+    }
+
+    public KSMetaData cloneWithTableAdded(CFMetaData table)
+    {
+        // clone ksm but include the new table
+        List<CFMetaData> newTables = new ArrayList<>(cfMetaData().values());
+        newTables.add(table);
+        assert newTables.size() == cfMetaData().size() + 1;
+        return cloneWith(newTables, userTypes);
+    }
+
+    public KSMetaData cloneWith(Iterable<CFMetaData> tables, UTMetaData types)
     {
-        return new KSMetaData(ksm.name, ksm.strategyClass, ksm.strategyOptions, ksm.durableWrites, cfDefs, ksm.userTypes);
+        return new KSMetaData(name, strategyClass, strategyOptions, durableWrites, tables, types);
     }
 
     public static KSMetaData testMetadata(String name, Class<? extends AbstractReplicationStrategy> strategyClass, Map<String, String> strategyOptions, CFMetaData... cfDefs)
@@ -145,11 +166,6 @@ public final class KSMetaData
         return Collections.singletonMap("replication_factor", rf.toString());
     }
 
-    public Mutation toSchemaUpdate(KSMetaData newState, long modificationTimestamp)
-    {
-        return newState.toSchema(modificationTimestamp);
-    }
-
     public KSMetaData validate() throws ConfigurationException
     {
         if (!CFMetaData.isNameValid(name))
@@ -165,107 +181,4 @@ public final class KSMetaData
 
         return this;
     }
-
-    public KSMetaData reloadAttributes()
-    {
-        Row ksDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, name);
-
-        if (ksDefRow.cf == null)
-            throw new RuntimeException(String.format("%s not found in the schema definitions keyspaceName (%s).", name, SystemKeyspace.SCHEMA_KEYSPACES_TABLE));
-
-        return fromSchema(ksDefRow, Collections.<CFMetaData>emptyList(), userTypes);
-    }
-
-    public Mutation dropFromSchema(long timestamp)
-    {
-        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(name));
-
-        mutation.delete(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_COLUMNS_TABLE, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_TRIGGERS_TABLE, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_USER_TYPES_TABLE, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_AGGREGATES_TABLE, timestamp);
-        mutation.delete(SystemKeyspace.BUILT_INDEXES_TABLE, timestamp);
-
-        return mutation;
-    }
-
-    public Mutation toSchema(long timestamp)
-    {
-        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(name));
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaKeyspacesTable);
-        CFRowAdder adder = new CFRowAdder(cf, SystemKeyspace.SchemaKeyspacesTable.comparator.builder().build(), timestamp);
-
-        adder.add("durable_writes", durableWrites);
-        adder.add("strategy_class", strategyClass.getName());
-        adder.add("strategy_options", json(strategyOptions));
-
-        for (CFMetaData cfm : cfMetaData.values())
-            cfm.toSchema(mutation, timestamp);
-
-        userTypes.toSchema(mutation, timestamp);
-        return mutation;
-    }
-
-    /**
-     * Deserialize only Keyspace attributes without nested ColumnFamilies
-     *
-     * @param row Keyspace attributes in serialized form
-     *
-     * @return deserialized keyspace without cf_defs
-     */
-    public static KSMetaData fromSchema(Row row, Iterable<CFMetaData> cfms, UTMetaData userTypes)
-    {
-        UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_keyspaces", row).one();
-        try
-        {
-            return new KSMetaData(result.getString("keyspace_name"),
-                                  AbstractReplicationStrategy.getClass(result.getString("strategy_class")),
-                                  fromJsonMap(result.getString("strategy_options")),
-                                  result.getBoolean("durable_writes"),
-                                  cfms,
-                                  userTypes);
-        }
-        catch (ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Deserialize Keyspace with nested ColumnFamilies
-     *
-     * @param serializedKs Keyspace in serialized form
-     * @param serializedCFs Collection of the serialized ColumnFamilies
-     *
-     * @return deserialized keyspace with cf_defs
-     */
-    public static KSMetaData fromSchema(Row serializedKs, Row serializedCFs, Row serializedUserTypes)
-    {
-        Map<String, CFMetaData> cfs = deserializeColumnFamilies(serializedCFs);
-        UTMetaData userTypes = new UTMetaData(UTMetaData.fromSchema(serializedUserTypes));
-        return fromSchema(serializedKs, cfs.values(), userTypes);
-    }
-
-    /**
-     * Deserialize ColumnFamilies from low-level schema representation, all of them belong to the same keyspace
-     *
-     * @return map containing name of the ColumnFamily and it's metadata for faster lookup
-     */
-    public static Map<String, CFMetaData> deserializeColumnFamilies(Row row)
-    {
-        if (row.cf == null)
-            return Collections.emptyMap();
-
-        UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", row);
-        Map<String, CFMetaData> cfms = new HashMap<>(results.size());
-        for (UntypedResultSet.Row result : results)
-        {
-            CFMetaData cfm = CFMetaData.fromSchema(result);
-            cfms.put(cfm.cfName, cfm);
-        }
-        return cfms;
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index 43cc6b5..21244ab 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.config;
 
-import java.nio.charset.CharacterCodingException;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.*;
@@ -27,13 +26,18 @@ import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.utils.ConcurrentBiMap;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
@@ -78,10 +82,20 @@ public class Schema
     }
 
     /**
-     * Initialize empty schema object
+     * Initialize empty schema object and load the hardcoded system tables
      */
     public Schema()
-    {}
+    {
+        load(SystemKeyspace.definition());
+    }
+
+    /** load keyspace (keyspace) definitions, but do not initialize the keyspace instances. */
+    public Schema loadFromDisk()
+    {
+        load(LegacySchemaTables.readSchemaFromSystemTables());
+        updateVersion();
+        return this;
+    }
 
     /**
      * Load up non-system keyspaces
@@ -350,28 +364,8 @@ public class Schema
      */
     public void updateVersion()
     {
-        try
-        {
-            MessageDigest versionDigest = MessageDigest.getInstance("MD5");
-
-            for (Row row : SystemKeyspace.serializedSchema())
-            {
-                if (invalidSchemaRow(row) || ignoredSchemaRow(row))
-                    continue;
-
-                // we want to digest only live columns
-                ColumnFamilyStore.removeDeletedColumnsOnly(row.cf, Integer.MAX_VALUE, SecondaryIndexManager.nullUpdater);
-                row.cf.purgeTombstones(Integer.MAX_VALUE);
-                row.cf.updateDigest(versionDigest);
-            }
-
-            version = UUID.nameUUIDFromBytes(versionDigest.digest());
-            SystemKeyspace.updateSchemaVersion(version);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
+        version = LegacySchemaTables.calculateSchemaDigest();
+        SystemKeyspace.updateSchemaVersion(version);
     }
 
     /*
@@ -399,20 +393,202 @@ public class Schema
         updateVersionAndAnnounce();
     }
 
-    public static boolean invalidSchemaRow(Row row)
+    public void addKeyspace(KSMetaData ksm)
     {
-        return row.cf == null || (row.cf.isMarkedForDelete() && !row.cf.hasColumns());
+        assert getKSMetaData(ksm.name) == null;
+        load(ksm);
+
+        Keyspace.open(ksm.name);
+        MigrationManager.instance.notifyCreateKeyspace(ksm);
     }
 
-    public static boolean ignoredSchemaRow(Row row)
+    public void updateKeyspace(String ksName)
     {
-        try
-        {
-            return ByteBufferUtil.string(row.key.getKey()).equals(SystemKeyspace.NAME);
-        }
-        catch (CharacterCodingException e)
+        KSMetaData oldKsm = getKSMetaData(ksName);
+        assert oldKsm != null;
+        KSMetaData newKsm = LegacySchemaTables.createKeyspaceFromName(ksName).cloneWith(oldKsm.cfMetaData().values(), oldKsm.userTypes);
+
+        setKeyspaceDefinition(newKsm);
+
+        Keyspace.open(ksName).createReplicationStrategy(newKsm);
+        MigrationManager.instance.notifyUpdateKeyspace(newKsm);
+    }
+
+    public void dropKeyspace(String ksName)
+    {
+        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
+        String snapshotName = Keyspace.getTimestampedSnapshotName(ksName);
+
+        CompactionManager.instance.interruptCompactionFor(ksm.cfMetaData().values(), true);
+
+        Keyspace keyspace = Keyspace.open(ksm.name);
+
+        // remove all cfs from the keyspace instance.
+        List<UUID> droppedCfs = new ArrayList<>();
+        for (CFMetaData cfm : ksm.cfMetaData().values())
         {
-            throw new RuntimeException(e);
+            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfm.cfName);
+
+            purge(cfm);
+
+            if (DatabaseDescriptor.isAutoSnapshot())
+                cfs.snapshot(snapshotName);
+            Keyspace.open(ksm.name).dropCf(cfm.cfId);
+
+            droppedCfs.add(cfm.cfId);
         }
+
+        // remove the keyspace from the static instances.
+        Keyspace.clear(ksm.name);
+        clearKeyspaceDefinition(ksm);
+
+        keyspace.writeOrder.awaitNewBarrier();
+
+        // force a new segment in the CL
+        CommitLog.instance.forceRecycleAllSegments(droppedCfs);
+
+        MigrationManager.instance.notifyDropKeyspace(ksm);
+    }
+
+    public void addTable(CFMetaData cfm)
+    {
+        assert getCFMetaData(cfm.ksName, cfm.cfName) == null;
+        KSMetaData ksm = getKSMetaData(cfm.ksName).cloneWithTableAdded(cfm);
+
+        logger.info("Loading {}", cfm);
+
+        load(cfm);
+
+        // make sure it's init-ed w/ the old definitions first,
+        // since we're going to call initCf on the new one manually
+        Keyspace.open(cfm.ksName);
+
+        setKeyspaceDefinition(ksm);
+        Keyspace.open(ksm.name).initCf(cfm.cfId, cfm.cfName, true);
+        MigrationManager.instance.notifyCreateColumnFamily(cfm);
+    }
+
+    public void updateTable(String ksName, String tableName)
+    {
+        CFMetaData cfm = getCFMetaData(ksName, tableName);
+        assert cfm != null;
+        cfm.reload();
+
+        Keyspace keyspace = Keyspace.open(cfm.ksName);
+        keyspace.getColumnFamilyStore(cfm.cfName).reload();
+        MigrationManager.instance.notifyUpdateColumnFamily(cfm);
+    }
+
+    public void dropTable(String ksName, String tableName)
+    {
+        KSMetaData ksm = getKSMetaData(ksName);
+        assert ksm != null;
+        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(tableName);
+        assert cfs != null;
+
+        // reinitialize the keyspace.
+        CFMetaData cfm = ksm.cfMetaData().get(tableName);
+
+        purge(cfm);
+        setKeyspaceDefinition(ksm.cloneWithTableRemoved(cfm));
+
+        CompactionManager.instance.interruptCompactionFor(Arrays.asList(cfm), true);
+
+        if (DatabaseDescriptor.isAutoSnapshot())
+            cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name));
+        Keyspace.open(ksm.name).dropCf(cfm.cfId);
+        MigrationManager.instance.notifyDropColumnFamily(cfm);
+
+        CommitLog.instance.forceRecycleAllSegments(Collections.singleton(cfm.cfId));
+    }
+
+    public void addType(UserType ut)
+    {
+        KSMetaData ksm = getKSMetaData(ut.keyspace);
+        assert ksm != null;
+
+        logger.info("Loading {}", ut);
+
+        ksm.userTypes.addType(ut);
+
+        MigrationManager.instance.notifyCreateUserType(ut);
+    }
+
+    public void updateType(UserType ut)
+    {
+        KSMetaData ksm = getKSMetaData(ut.keyspace);
+        assert ksm != null;
+
+        logger.info("Updating {}", ut);
+
+        ksm.userTypes.addType(ut);
+
+        MigrationManager.instance.notifyUpdateUserType(ut);
+    }
+
+    public void dropType(UserType ut)
+    {
+        KSMetaData ksm = getKSMetaData(ut.keyspace);
+        assert ksm != null;
+
+        ksm.userTypes.removeType(ut);
+
+        MigrationManager.instance.notifyDropUserType(ut);
+    }
+
+    public void addFunction(UDFunction udf)
+    {
+        logger.info("Loading {}", udf);
+
+        Functions.addFunction(udf);
+
+        MigrationManager.instance.notifyCreateFunction(udf);
+    }
+
+    public void updateFunction(UDFunction udf)
+    {
+        logger.info("Updating {}", udf);
+
+        Functions.replaceFunction(udf);
+
+        MigrationManager.instance.notifyUpdateFunction(udf);
+    }
+
+    public void dropFunction(UDFunction udf)
+    {
+        logger.info("Drop {}", udf);
+
+        // TODO: this is kind of broken as this remove all overloads of the function name
+        Functions.removeFunction(udf.name(), udf.argTypes());
+
+        MigrationManager.instance.notifyDropFunction(udf);
+    }
+
+    public void addAggregate(UDAggregate udf)
+    {
+        logger.info("Loading {}", udf);
+
+        Functions.addFunction(udf);
+
+        MigrationManager.instance.notifyCreateAggregate(udf);
+    }
+
+    public void updateAggregate(UDAggregate udf)
+    {
+        logger.info("Updating {}", udf);
+
+        Functions.replaceFunction(udf);
+
+        MigrationManager.instance.notifyUpdateAggregate(udf);
+    }
+
+    public void dropAggregate(UDAggregate udf)
+    {
+        logger.info("Drop {}", udf);
+
+        // TODO: this is kind of broken as this remove all overloads of the function name
+        Functions.removeFunction(udf.name(), udf.argTypes());
+
+        MigrationManager.instance.notifyDropAggregate(udf);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/TriggerDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/TriggerDefinition.java b/src/java/org/apache/cassandra/config/TriggerDefinition.java
index a395549..6a84379 100644
--- a/src/java/org/apache/cassandra/config/TriggerDefinition.java
+++ b/src/java/org/apache/cassandra/config/TriggerDefinition.java
@@ -18,20 +18,10 @@
  */
 package org.apache.cassandra.config;
 
-import java.util.*;
-
 import com.google.common.base.Objects;
 
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.marshal.UTF8Type;
-
 public class TriggerDefinition
 {
-    public static final String TRIGGER_NAME = "trigger_name";
-    public static final String TRIGGER_OPTIONS = "trigger_options";
     public static final String CLASS = "class";
 
     public final String name;
@@ -51,59 +41,6 @@ public class TriggerDefinition
         return new TriggerDefinition(name, classOption);
     }
 
-    /**
-     * Deserialize triggers from storage-level representation.
-     *
-     * @param serializedTriggers storage-level partition containing the trigger definitions
-     * @return the list of processed TriggerDefinitions
-     */
-    public static List<TriggerDefinition> fromSchema(Row serializedTriggers)
-    {
-        List<TriggerDefinition> triggers = new ArrayList<>();
-        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.SCHEMA_TRIGGERS_TABLE);
-        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, serializedTriggers))
-        {
-            String name = row.getString(TRIGGER_NAME);
-            String classOption = row.getMap(TRIGGER_OPTIONS, UTF8Type.instance, UTF8Type.instance).get(CLASS);
-            triggers.add(new TriggerDefinition(name, classOption));
-        }
-        return triggers;
-    }
-
-    /**
-     * Add specified trigger to the schema using given mutation.
-     *
-     * @param mutation  The schema mutation
-     * @param cfName    The name of the parent ColumnFamily
-     * @param timestamp The timestamp to use for the columns
-     */
-    public void toSchema(Mutation mutation, String cfName, long timestamp)
-    {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_TABLE);
-
-        CFMetaData cfm = SystemKeyspace.SchemaTriggersTable;
-        Composite prefix = cfm.comparator.make(cfName, name);
-        CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
-        adder.addMapEntry(TRIGGER_OPTIONS, CLASS, classOption);
-    }
-
-    /**
-     * Drop specified trigger from the schema using given mutation.
-     *
-     * @param mutation  The schema mutation
-     * @param cfName    The name of the parent ColumnFamily
-     * @param timestamp The timestamp to use for the tombstone
-     */
-    public void deleteFromSchema(Mutation mutation, String cfName, long timestamp)
-    {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_TABLE);
-        int ldt = (int) (System.currentTimeMillis() / 1000);
-
-        Composite prefix = SystemKeyspace.SchemaTriggersTable.comparator.make(cfName, name);
-        cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-    }
-
     @Override
     public boolean equals(Object o)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/UTMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/UTMetaData.java b/src/java/org/apache/cassandra/config/UTMetaData.java
index 46a7a4f..08cedee 100644
--- a/src/java/org/apache/cassandra/config/UTMetaData.java
+++ b/src/java/org/apache/cassandra/config/UTMetaData.java
@@ -20,12 +20,7 @@ package org.apache.cassandra.config;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Defined (and loaded) user types.
@@ -42,91 +37,11 @@ public final class UTMetaData
         this(new HashMap<ByteBuffer, UserType>());
     }
 
-    UTMetaData(Map<ByteBuffer, UserType> types)
+    public UTMetaData(Map<ByteBuffer, UserType> types)
     {
         this.userTypes = types;
     }
 
-    private static UserType fromSchema(UntypedResultSet.Row row)
-    {
-        try
-        {
-            String keyspace = row.getString("keyspace_name");
-            ByteBuffer name = ByteBufferUtil.bytes(row.getString("type_name"));
-            List<String> rawColumns = row.getList("field_names", UTF8Type.instance);
-            List<String> rawTypes = row.getList("field_types", UTF8Type.instance);
-
-            List<ByteBuffer> columns = new ArrayList<>(rawColumns.size());
-            for (String rawColumn : rawColumns)
-                columns.add(ByteBufferUtil.bytes(rawColumn));
-
-            List<AbstractType<?>> types = new ArrayList<>(rawTypes.size());
-            for (String rawType : rawTypes)
-                types.add(TypeParser.parse(rawType));
-
-            return new UserType(keyspace, name, columns, types);
-        }
-        catch (RequestValidationException e)
-        {
-            // If it has been written in the schema, it should be valid
-            throw new AssertionError();
-        }
-    }
-
-    public static Map<ByteBuffer, UserType> fromSchema(Row row)
-    {
-        UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_USER_TYPES_TABLE, row);
-        Map<ByteBuffer, UserType> types = new HashMap<>(results.size());
-        for (UntypedResultSet.Row result : results)
-        {
-            UserType type = fromSchema(result);
-            types.put(type.name, type);
-        }
-        return types;
-    }
-
-    public static Mutation toSchema(UserType newType, long timestamp)
-    {
-        return toSchema(new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(newType.keyspace)), newType, timestamp);
-    }
-
-    public static Mutation toSchema(Mutation mutation, UserType newType, long timestamp)
-    {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_USER_TYPES_TABLE);
-
-        Composite prefix = SystemKeyspace.SchemaUserTypesTable.comparator.make(newType.name);
-        CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
-        adder.resetCollection("field_names");
-        adder.resetCollection("field_types");
-
-        for (int i = 0; i < newType.size(); i++)
-        {
-            adder.addListEntry("field_names", newType.fieldName(i));
-            adder.addListEntry("field_types", newType.fieldType(i).toString());
-        }
-        return mutation;
-    }
-
-    public Mutation toSchema(Mutation mutation, long timestamp)
-    {
-        for (UserType ut : userTypes.values())
-            toSchema(mutation, ut, timestamp);
-        return mutation;
-    }
-
-    public static Mutation dropFromSchema(UserType droppedType, long timestamp)
-    {
-        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(droppedType.keyspace));
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_USER_TYPES_TABLE);
-        int ldt = (int) (System.currentTimeMillis() / 1000);
-
-        Composite prefix = SystemKeyspace.SchemaUserTypesTable.comparator.make(droppedType.name);
-        cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-
-        return mutation;
-    }
-
     public UserType getType(ByteBuffer typeName)
     {
         return userTypes.get(typeName);
@@ -134,11 +49,11 @@ public final class UTMetaData
 
     public Map<ByteBuffer, UserType> getAllTypes()
     {
-        // Copy to avoid concurrent modification while iterating. Not intended to be called on a criticial path anyway
+        // Copy to avoid concurrent modification while iterating. Not intended to be called on a critical path anyway
         return new HashMap<>(userTypes);
     }
 
-    // This is *not* thread safe but is only called in DefsTables that is synchronized.
+    // This is *not* thread safe but is only called in Schema that is synchronized.
     public void addType(UserType type)
     {
         UserType old = userTypes.get(type.name);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/functions/Functions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/Functions.java b/src/java/org/apache/cassandra/cql3/functions/Functions.java
index 7d94e47..b55ebc5 100644
--- a/src/java/org/apache/cassandra/cql3/functions/Functions.java
+++ b/src/java/org/apache/cassandra/cql3/functions/Functions.java
@@ -22,12 +22,9 @@ import java.util.Collection;
 import java.util.List;
 
 import com.google.common.collect.ArrayListMultimap;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.service.IMigrationListener;
@@ -35,16 +32,11 @@ import org.apache.cassandra.service.MigrationManager;
 
 public abstract class Functions
 {
-    private static final Logger logger = LoggerFactory.getLogger(Functions.class);
-
     // We special case the token function because that's the only function whose argument types actually
     // depend on the table on which the function is called. Because it's the sole exception, it's easier
     // to handle it as a special case.
     private static final FunctionName TOKEN_FUNCTION_NAME = FunctionName.nativeFunction("token");
 
-    private static final String SELECT_UD_FUNCTION = "SELECT * FROM " + SystemKeyspace.NAME + '.' + SystemKeyspace.SCHEMA_FUNCTIONS_TABLE;
-    private static final String SELECT_UD_AGGREGATE = "SELECT * FROM " + SystemKeyspace.NAME + '.' + SystemKeyspace.SCHEMA_AGGREGATES_TABLE;
-
     private Functions() {}
 
     private static final ArrayListMultimap<FunctionName, Function> declared = ArrayListMultimap.create();
@@ -96,18 +88,6 @@ public abstract class Functions
         declared.put(fun.name(), fun);
     }
 
-    /**
-     * Loading existing UDFs from the schema.
-     */
-    public static void loadUDFFromSchema()
-    {
-        logger.debug("Loading UDFs");
-        for (UntypedResultSet.Row row : QueryProcessor.executeOnceInternal(SELECT_UD_FUNCTION))
-            addFunction(UDFunction.fromSchema(row));
-        for (UntypedResultSet.Row row : QueryProcessor.executeOnceInternal(SELECT_UD_AGGREGATE))
-            addFunction(UDAggregate.fromSchema(row));
-    }
-
     public static ColumnSpecification makeArgSpec(String receiverKs, String receiverCf, Function fun, int i)
     {
         return new ColumnSpecification(receiverKs,
@@ -270,7 +250,7 @@ public abstract class Functions
         return sb.toString();
     }
 
-    // This is *not* thread safe but is only called in DefsTables that is synchronized.
+    // This is *not* thread safe but is only called in SchemaTables that is synchronized.
     public static void addFunction(AbstractFunction fun)
     {
         // We shouldn't get there unless that function don't exist

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java b/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
index 5b1f5bd..e4e6a55 100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
@@ -76,9 +76,8 @@ public final class JavaSourceUDFFactory
         // It is separated to allow return type and argument type checks during compile time via javassist.
         String codeExecInt = generateExecuteInternalMethod(argNames, body, javaReturnType, javaParamTypes);
 
-        if (logger.isDebugEnabled())
-            logger.debug("Generating java source UDF for {} with following c'tor and functions:\n{}\n{}\n{}",
-                         name, codeCtor, codeExecInt, codeExec);
+        logger.debug("Generating java source UDF for {} with following c'tor and functions:\n{}\n{}\n{}",
+                     name, codeCtor, codeExecInt, codeExec);
 
         try
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
index f259265..e9c33ba 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
@@ -24,12 +24,7 @@ import com.google.common.base.Objects;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.*;
 
 /**
@@ -58,6 +53,45 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
         this.initcond = initcond;
     }
 
+    public static UDAggregate create(FunctionName name,
+                                     List<AbstractType<?>> argTypes,
+                                     AbstractType<?> returnType,
+                                     FunctionName stateFunc,
+                                     FunctionName finalFunc,
+                                     AbstractType<?> stateType,
+                                     ByteBuffer initcond)
+    throws InvalidRequestException
+    {
+        List<AbstractType<?>> stateTypes = new ArrayList<>(argTypes.size() + 1);
+        stateTypes.add(stateType);
+        stateTypes.addAll(argTypes);
+        List<AbstractType<?>> finalTypes = Collections.<AbstractType<?>>singletonList(stateType);
+        return new UDAggregate(name,
+                               argTypes,
+                               returnType,
+                               resolveScalar(name, stateFunc, stateTypes),
+                               finalFunc != null ? resolveScalar(name, finalFunc, finalTypes) : null,
+                               initcond);
+    }
+
+    public static UDAggregate createBroken(FunctionName name,
+                                           List<AbstractType<?>> argTypes,
+                                           AbstractType<?> returnType,
+                                           ByteBuffer initcond,
+                                           final InvalidRequestException reason)
+    {
+        return new UDAggregate(name, argTypes, returnType, null, null, initcond)
+        {
+            public Aggregate newAggregate() throws InvalidRequestException
+            {
+                throw new InvalidRequestException(String.format("Aggregate '%s' exists but hasn't been loaded successfully for the following reason: %s. "
+                                                                + "Please see the server log for more details",
+                                                                this,
+                                                                reason.getMessage()));
+            }
+        };
+    }
+
     public boolean hasReferenceTo(Function function)
     {
         return stateFunction == function || finalFunction == function;
@@ -85,6 +119,26 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
         return false;
     }
 
+    public ScalarFunction stateFunction()
+    {
+        return stateFunction;
+    }
+
+    public ScalarFunction finalFunction()
+    {
+        return finalFunction;
+    }
+
+    public ByteBuffer initialCondition()
+    {
+        return initcond;
+    }
+
+    public AbstractType<?> stateType()
+    {
+        return stateType;
+    }
+
     public Aggregate newAggregate() throws InvalidRequestException
     {
         return new Aggregate()
@@ -128,134 +182,6 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
         return (ScalarFunction) func;
     }
 
-    private static Mutation makeSchemaMutation(FunctionName name)
-    {
-        UTF8Type kv = (UTF8Type)SystemKeyspace.SchemaAggregatesTable.getKeyValidator();
-        return new Mutation(SystemKeyspace.NAME, kv.decompose(name.keyspace));
-    }
-
-    public Mutation toSchemaDrop(long timestamp)
-    {
-        Mutation mutation = makeSchemaMutation(name);
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_AGGREGATES_TABLE);
-
-        Composite prefix = SystemKeyspace.SchemaAggregatesTable.comparator.make(name.name, UDHelper.computeSignature(argTypes));
-        int ldt = (int) (System.currentTimeMillis() / 1000);
-        cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-
-        return mutation;
-    }
-
-    public static Map<Composite, UDAggregate> fromSchema(Row row)
-    {
-        UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_AGGREGATES_TABLE, row);
-        Map<Composite, UDAggregate> udfs = new HashMap<>(results.size());
-        for (UntypedResultSet.Row result : results)
-            udfs.put(SystemKeyspace.SchemaAggregatesTable.comparator.make(result.getString("aggregate_name"), result.getBlob("signature")),
-                     fromSchema(result));
-        return udfs;
-    }
-
-    public Mutation toSchemaUpdate(long timestamp)
-    {
-        Mutation mutation = makeSchemaMutation(name);
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_AGGREGATES_TABLE);
-
-        Composite prefix = SystemKeyspace.SchemaAggregatesTable.comparator.make(name.name, UDHelper.computeSignature(argTypes));
-        CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
-        adder.resetCollection("argument_types");
-        adder.add("return_type", returnType.toString());
-        adder.add("state_func", stateFunction.name().name);
-        if (stateType != null)
-            adder.add("state_type", stateType.toString());
-        if (finalFunction != null)
-            adder.add("final_func", finalFunction.name().name);
-        if (initcond != null)
-            adder.add("initcond", initcond);
-
-        for (AbstractType<?> argType : argTypes)
-            adder.addListEntry("argument_types", argType.toString());
-
-        return mutation;
-    }
-
-    public static UDAggregate fromSchema(UntypedResultSet.Row row)
-    {
-        String ksName = row.getString("keyspace_name");
-        String functionName = row.getString("aggregate_name");
-        FunctionName name = new FunctionName(ksName, functionName);
-
-        List<String> types = row.getList("argument_types", UTF8Type.instance);
-
-        List<AbstractType<?>> argTypes;
-        if (types == null)
-        {
-            argTypes = Collections.emptyList();
-        }
-        else
-        {
-            argTypes = new ArrayList<>(types.size());
-            for (String type : types)
-                argTypes.add(parseType(type));
-        }
-
-        AbstractType<?> returnType = parseType(row.getString("return_type"));
-
-        FunctionName stateFunc = new FunctionName(ksName, row.getString("state_func"));
-        FunctionName finalFunc = row.has("final_func") ? new FunctionName(ksName, row.getString("final_func")) : null;
-        AbstractType<?> stateType = row.has("state_type") ? parseType(row.getString("state_type")) : null;
-        ByteBuffer initcond = row.has("initcond") ? row.getBytes("initcond") : null;
-
-        try
-        {
-            return create(name, argTypes, returnType, stateFunc, finalFunc, stateType, initcond);
-        }
-        catch (InvalidRequestException reason)
-        {
-            return createBroken(name, argTypes, returnType, initcond, reason);
-        }
-    }
-
-    private static UDAggregate createBroken(FunctionName name, List<AbstractType<?>> argTypes, AbstractType<?> returnType,
-                                            ByteBuffer initcond, final InvalidRequestException reason)
-    {
-        return new UDAggregate(name, argTypes, returnType, null, null, initcond) {
-            public Aggregate newAggregate() throws InvalidRequestException
-            {
-                throw new InvalidRequestException(String.format("Aggregate '%s' exists but hasn't been loaded successfully for the following reason: %s. "
-                                                                + "Please see the server log for more details", this, reason.getMessage()));
-            }
-        };
-    }
-
-    private static UDAggregate create(FunctionName name, List<AbstractType<?>> argTypes, AbstractType<?> returnType,
-                                      FunctionName stateFunc, FunctionName finalFunc, AbstractType<?> stateType, ByteBuffer initcond)
-    throws InvalidRequestException
-    {
-        List<AbstractType<?>> stateTypes = new ArrayList<>(argTypes.size() + 1);
-        stateTypes.add(stateType);
-        stateTypes.addAll(argTypes);
-        List<AbstractType<?>> finalTypes = Collections.<AbstractType<?>>singletonList(stateType);
-        return new UDAggregate(name, argTypes, returnType,
-                               resolveScalar(name, stateFunc, stateTypes),
-                               finalFunc != null ? resolveScalar(name, finalFunc, finalTypes) : null,
-                               initcond);
-    }
-
-    private static AbstractType<?> parseType(String str)
-    {
-        // We only use this when reading the schema where we shouldn't get an error
-        try
-        {
-            return TypeParser.parse(str);
-        }
-        catch (SyntaxException | ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
     @Override
     public boolean equals(Object o)
     {
@@ -263,13 +189,13 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
             return false;
 
         UDAggregate that = (UDAggregate) o;
-        return Objects.equal(this.name, that.name)
-               && Functions.typeEquals(this.argTypes, that.argTypes)
-               && Functions.typeEquals(this.returnType, that.returnType)
-               && Objects.equal(this.stateFunction, that.stateFunction)
-               && Objects.equal(this.finalFunction, that.finalFunction)
-               && Objects.equal(this.stateType, that.stateType)
-               && Objects.equal(this.initcond, that.initcond);
+        return Objects.equal(name, that.name)
+            && Functions.typeEquals(argTypes, that.argTypes)
+            && Functions.typeEquals(returnType, that.returnType)
+            && Objects.equal(stateFunction, that.stateFunction)
+            && Objects.equal(finalFunction, that.finalFunction)
+            && Objects.equal(stateType, that.stateType)
+            && Objects.equal(initcond, that.initcond);
     }
 
     @Override


[2/5] cassandra git commit: Isolate schema serializaton code

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/LegacySchemaTables.java b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
new file mode 100644
index 0000000..047698c
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
@@ -0,0 +1,1480 @@
+/*
+ * 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.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.io.compress.CompressionParameters;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
+import static org.apache.cassandra.utils.FBUtilities.fromJsonMap;
+import static org.apache.cassandra.utils.FBUtilities.json;
+
+/** system.schema_* tables used to store keyspace/table/type attributes prior to C* 3.0 */
+public class LegacySchemaTables
+{
+    private static final Logger logger = LoggerFactory.getLogger(LegacySchemaTables.class);
+
+    public static final String KEYSPACES = "schema_keyspaces";
+    public static final String COLUMNFAMILIES = "schema_columnfamilies";
+    public static final String COLUMNS = "schema_columns";
+    public static final String TRIGGERS = "schema_triggers";
+    public static final String USERTYPES = "schema_usertypes";
+    public static final String FUNCTIONS = "schema_functions";
+    public static final String AGGREGATES = "schema_aggregates";
+
+    public static final List<String> ALL = Arrays.asList(KEYSPACES, COLUMNFAMILIES, COLUMNS, TRIGGERS, USERTYPES, FUNCTIONS, AGGREGATES);
+
+    private static final CFMetaData Keyspaces =
+        compile(KEYSPACES,
+                "keyspace definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "durable_writes boolean,"
+                + "strategy_class text,"
+                + "strategy_options text,"
+                + "PRIMARY KEY ((keyspace_name))) "
+                + "WITH COMPACT STORAGE");
+
+    private static final CFMetaData Columnfamilies =
+        compile(COLUMNFAMILIES,
+                "table definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "columnfamily_name text,"
+                + "bloom_filter_fp_chance double,"
+                + "caching text,"
+                + "cf_id uuid," // post-2.1 UUID cfid
+                + "comment text,"
+                + "compaction_strategy_class text,"
+                + "compaction_strategy_options text,"
+                + "comparator text,"
+                + "compression_parameters text,"
+                + "default_time_to_live int,"
+                + "default_validator text,"
+                + "dropped_columns map<text, bigint>,"
+                + "gc_grace_seconds int,"
+                + "is_dense boolean,"
+                + "key_validator text,"
+                + "local_read_repair_chance double,"
+                + "max_compaction_threshold int,"
+                + "max_index_interval int,"
+                + "memtable_flush_period_in_ms int,"
+                + "min_compaction_threshold int,"
+                + "min_index_interval int,"
+                + "read_repair_chance double,"
+                + "speculative_retry text,"
+                + "subcomparator text,"
+                + "type text,"
+                + "PRIMARY KEY ((keyspace_name), columnfamily_name))");
+
+    private static final CFMetaData Columns =
+        compile(COLUMNS,
+                "column definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "columnfamily_name text,"
+                + "column_name text,"
+                + "component_index int,"
+                + "index_name text,"
+                + "index_options text,"
+                + "index_type text,"
+                + "type text,"
+                + "validator text,"
+                + "PRIMARY KEY ((keyspace_name), columnfamily_name, column_name))");
+
+    private static final CFMetaData Triggers =
+        compile(TRIGGERS,
+                "trigger definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "columnfamily_name text,"
+                + "trigger_name text,"
+                + "trigger_options map<text, text>,"
+                + "PRIMARY KEY ((keyspace_name), columnfamily_name, trigger_name))");
+
+    private static final CFMetaData Usertypes =
+        compile(USERTYPES,
+                "user defined type definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "type_name text,"
+                + "field_names list<text>,"
+                + "field_types list<text>,"
+                + "PRIMARY KEY ((keyspace_name), type_name))");
+
+    private static final CFMetaData Functions =
+        compile(FUNCTIONS,
+                "user defined function definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "function_name text,"
+                + "signature blob,"
+                + "argument_names list<text>,"
+                + "argument_types list<text>,"
+                + "body text,"
+                + "is_deterministic boolean,"
+                + "language text,"
+                + "return_type text,"
+                + "PRIMARY KEY ((keyspace_name), function_name, signature))");
+
+    private static final CFMetaData Aggregates =
+        compile(AGGREGATES,
+                "user defined aggregate definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "aggregate_name text,"
+                + "signature blob,"
+                + "argument_types list<text>,"
+                + "final_func text,"
+                + "initcond blob,"
+                + "return_type text,"
+                + "state_func text,"
+                + "state_type text,"
+                + "PRIMARY KEY ((keyspace_name), aggregate_name, signature))");
+
+    public static final List<CFMetaData> All = Arrays.asList(Keyspaces, Columnfamilies, Columns, Triggers, Usertypes, Functions, Aggregates);
+
+    private static CFMetaData compile(String name, String description, String schema)
+    {
+        return CFMetaData.compile(String.format(schema, name), SystemKeyspace.NAME)
+                         .comment(description)
+                         .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(7));
+    }
+
+    /** add entries to system.schema_* for the hardcoded system definitions */
+    public static void saveSystemKeyspaceSchema()
+    {
+        KSMetaData keyspace = Schema.instance.getKSMetaData(SystemKeyspace.NAME);
+        // delete old, possibly obsolete entries in schema tables
+        for (String table : ALL)
+            executeOnceInternal(String.format("DELETE FROM system.%s WHERE keyspace_name = ?", table), keyspace.name);
+        // (+1 to timestamp to make sure we don't get shadowed by the tombstones we just added)
+        makeCreateKeyspaceMutation(keyspace, FBUtilities.timestampMicros() + 1).apply();
+    }
+
+    public static Collection<KSMetaData> readSchemaFromSystemTables()
+    {
+        List<Row> serializedSchema = getSchemaPartitionsForTable(KEYSPACES);
+
+        List<KSMetaData> keyspaces = new ArrayList<>(serializedSchema.size());
+
+        for (Row partition : serializedSchema)
+        {
+            if (isEmptySchemaPartition(partition) || isSystemKeyspaceSchemaPartition(partition))
+                continue;
+
+            keyspaces.add(createKeyspaceFromSchemaPartitions(partition,
+                                                             readSchemaPartitionForKeyspace(COLUMNFAMILIES, partition.key),
+                                                             readSchemaPartitionForKeyspace(USERTYPES, partition.key)));
+
+            // Will be moved away in #6717
+            for (UDFunction function : createFunctionsFromFunctionsPartition(readSchemaPartitionForKeyspace(FUNCTIONS, partition.key)).values())
+                org.apache.cassandra.cql3.functions.Functions.addFunction(function);
+
+            // Will be moved away in #6717
+            for (UDAggregate aggregate : createAggregatesFromAggregatesPartition(readSchemaPartitionForKeyspace(AGGREGATES, partition.key)).values())
+                org.apache.cassandra.cql3.functions.Functions.addFunction(aggregate);
+        }
+
+        return keyspaces;
+    }
+
+    public static void truncateSchemaTables()
+    {
+        for (String table : ALL)
+            getSchemaCFS(table).truncateBlocking();
+    }
+
+    private static void flushSchemaTables()
+    {
+        for (String table : ALL)
+            SystemKeyspace.forceBlockingFlush(table);
+    }
+
+    /**
+     * Read schema from system keyspace and calculate MD5 digest of every row, resulting digest
+     * will be converted into UUID which would act as content-based version of the schema.
+     */
+    public static UUID calculateSchemaDigest()
+    {
+        MessageDigest digest;
+        try
+        {
+            digest = MessageDigest.getInstance("MD5");
+        }
+        catch (NoSuchAlgorithmException e)
+        {
+            throw new RuntimeException(e);
+        }
+
+        for (String table : ALL)
+        {
+            for (Row partition : getSchemaPartitionsForTable(table))
+            {
+                if (isEmptySchemaPartition(partition) || isSystemKeyspaceSchemaPartition(partition))
+                    continue;
+
+                // we want to digest only live columns
+                ColumnFamilyStore.removeDeletedColumnsOnly(partition.cf, Integer.MAX_VALUE, SecondaryIndexManager.nullUpdater);
+                partition.cf.purgeTombstones(Integer.MAX_VALUE);
+                partition.cf.updateDigest(digest);
+            }
+        }
+
+        return UUID.nameUUIDFromBytes(digest.digest());
+    }
+
+    /**
+     * @param schemaTableName The name of the table responsible for part of the schema
+     * @return CFS responsible to hold low-level serialized schema
+     */
+    private static ColumnFamilyStore getSchemaCFS(String schemaTableName)
+    {
+        return Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(schemaTableName);
+    }
+
+    /**
+     * @param schemaTableName The name of the table responsible for part of the schema.
+     * @return low-level schema representation
+     */
+    private static List<Row> getSchemaPartitionsForTable(String schemaTableName)
+    {
+        Token minToken = StorageService.getPartitioner().getMinimumToken();
+        return getSchemaCFS(schemaTableName).getRangeSlice(new Range<RowPosition>(minToken.minKeyBound(), minToken.maxKeyBound()),
+                                                           null,
+                                                           new IdentityQueryFilter(),
+                                                           Integer.MAX_VALUE,
+                                                           System.currentTimeMillis());
+    }
+
+    public static Collection<Mutation> convertSchemaToMutations()
+    {
+        Map<DecoratedKey, Mutation> mutationMap = new HashMap<>();
+
+        for (String table : ALL)
+            convertSchemaToMutations(mutationMap, table);
+
+        return mutationMap.values();
+    }
+
+    private static void convertSchemaToMutations(Map<DecoratedKey, Mutation> mutationMap, String schemaTableName)
+    {
+        for (Row partition : getSchemaPartitionsForTable(schemaTableName))
+        {
+            if (isSystemKeyspaceSchemaPartition(partition))
+                continue;
+
+            Mutation mutation = mutationMap.get(partition.key);
+            if (mutation == null)
+            {
+                mutation = new Mutation(SystemKeyspace.NAME, partition.key.getKey());
+                mutationMap.put(partition.key, mutation);
+            }
+
+            mutation.add(partition.cf);
+        }
+    }
+
+    private static Map<DecoratedKey, ColumnFamily> readSchemaForKeyspaces(String schemaTableName, Set<String> keyspaceNames)
+    {
+        Map<DecoratedKey, ColumnFamily> schema = new HashMap<>();
+
+        for (String keyspaceName : keyspaceNames)
+        {
+            Row schemaEntity = readSchemaPartitionForKeyspace(schemaTableName, keyspaceName);
+            if (schemaEntity.cf != null)
+                schema.put(schemaEntity.key, schemaEntity.cf);
+        }
+
+        return schema;
+    }
+
+    private static ByteBuffer getSchemaKSKey(String ksName)
+    {
+        return AsciiType.instance.fromString(ksName);
+    }
+
+    private static Row readSchemaPartitionForKeyspace(String schemaTableName, String keyspaceName)
+    {
+        DecoratedKey keyspaceKey = StorageService.getPartitioner().decorateKey(getSchemaKSKey(keyspaceName));
+        return readSchemaPartitionForKeyspace(schemaTableName, keyspaceKey);
+    }
+
+    private static Row readSchemaPartitionForKeyspace(String schemaTableName, DecoratedKey keyspaceKey)
+    {
+        QueryFilter filter = QueryFilter.getIdentityFilter(keyspaceKey, schemaTableName, System.currentTimeMillis());
+        return new Row(keyspaceKey, getSchemaCFS(schemaTableName).getColumnFamily(filter));
+    }
+
+    private static Row readSchemaPartitionForTable(String schemaTableName, String keyspaceName, String tableName)
+    {
+        DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(keyspaceName));
+        ColumnFamilyStore store = getSchemaCFS(schemaTableName);
+        Composite prefix = store.getComparator().make(tableName);
+        ColumnFamily cells = store.getColumnFamily(key, prefix, prefix.end(), false, Integer.MAX_VALUE, System.currentTimeMillis());
+        return new Row(key, cells);
+    }
+
+    private static boolean isEmptySchemaPartition(Row partition)
+    {
+        return partition.cf == null || (partition.cf.isMarkedForDelete() && !partition.cf.hasColumns());
+    }
+
+    private static boolean isSystemKeyspaceSchemaPartition(Row partition)
+    {
+        return getSchemaKSKey(SystemKeyspace.NAME).equals(partition.key.getKey());
+    }
+
+    /**
+     * Merge remote schema in form of mutations with local and mutate ks/cf metadata objects
+     * (which also involves fs operations on add/drop ks/cf)
+     *
+     * @param mutations the schema changes to apply
+     *
+     * @throws ConfigurationException If one of metadata attributes has invalid value
+     * @throws IOException If data was corrupted during transportation or failed to apply fs operations
+     */
+    public static synchronized void mergeSchema(Collection<Mutation> mutations) throws ConfigurationException, IOException
+    {
+        mergeSchema(mutations, true);
+        Schema.instance.updateVersionAndAnnounce();
+    }
+
+    public static synchronized void mergeSchema(Collection<Mutation> mutations, boolean doFlush) throws IOException
+    {
+        // compare before/after schemas of the affected keyspaces only
+        Set<String> keyspaces = new HashSet<>(mutations.size());
+        for (Mutation mutation : mutations)
+            keyspaces.add(ByteBufferUtil.string(mutation.key()));
+
+        // current state of the schema
+        Map<DecoratedKey, ColumnFamily> oldKeyspaces = readSchemaForKeyspaces(KEYSPACES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldColumnFamilies = readSchemaForKeyspaces(COLUMNFAMILIES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldTypes = readSchemaForKeyspaces(USERTYPES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldFunctions = readSchemaForKeyspaces(FUNCTIONS, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldAggregates = readSchemaForKeyspaces(AGGREGATES, keyspaces);
+
+        for (Mutation mutation : mutations)
+            mutation.apply();
+
+        if (doFlush)
+            flushSchemaTables();
+
+        // with new data applied
+        Map<DecoratedKey, ColumnFamily> newKeyspaces = readSchemaForKeyspaces(KEYSPACES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newColumnFamilies = readSchemaForKeyspaces(COLUMNFAMILIES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newTypes = readSchemaForKeyspaces(USERTYPES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newFunctions = readSchemaForKeyspaces(FUNCTIONS, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newAggregates = readSchemaForKeyspaces(AGGREGATES, keyspaces);
+
+        Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
+        mergeTables(oldColumnFamilies, newColumnFamilies);
+        mergeTypes(oldTypes, newTypes);
+        mergeFunctions(oldFunctions, newFunctions);
+        mergeAggregates(oldAggregates, newAggregates);
+
+        // it is safe to drop a keyspace only when all nested ColumnFamilies where deleted
+        for (String keyspaceToDrop : keyspacesToDrop)
+            Schema.instance.dropKeyspace(keyspaceToDrop);
+    }
+
+    private static Set<String> mergeKeyspaces(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+    {
+        List<Row> created = new ArrayList<>();
+        List<String> altered = new ArrayList<>();
+        Set<String> dropped = new HashSet<>();
+
+        /*
+         * - we don't care about entriesOnlyOnLeft() or entriesInCommon(), because only the changes are of interest to us
+         * - of all entriesOnlyOnRight(), we only care about ones that have live columns; it's possible to have a ColumnFamily
+         *   there that only has the top-level deletion, if:
+         *      a) a pushed DROP KEYSPACE change for a keyspace hadn't ever made it to this node in the first place
+         *      b) a pulled dropped keyspace that got dropped before it could find a way to this node
+         * - of entriesDiffering(), we don't care about the scenario where both pre and post-values have zero live columns:
+         *   that means that a keyspace had been recreated and dropped, and the recreated keyspace had never found a way
+         *   to this node
+         */
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+            if (entry.getValue().hasColumns())
+                created.add(new Row(entry.getKey(), entry.getValue()));
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+        {
+            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
+
+            ColumnFamily pre  = entry.getValue().leftValue();
+            ColumnFamily post = entry.getValue().rightValue();
+
+            if (pre.hasColumns() && post.hasColumns())
+                altered.add(keyspaceName);
+            else if (pre.hasColumns())
+                dropped.add(keyspaceName);
+            else if (post.hasColumns()) // a (re)created keyspace
+                created.add(new Row(entry.getKey(), post));
+        }
+
+        for (Row row : created)
+            Schema.instance.addKeyspace(createKeyspaceFromSchemaPartition(row));
+        for (String name : altered)
+            Schema.instance.updateKeyspace(name);
+        return dropped;
+    }
+
+    // see the comments for mergeKeyspaces()
+    private static void mergeTables(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+    {
+        List<CFMetaData> created = new ArrayList<>();
+        List<CFMetaData> altered = new ArrayList<>();
+        List<CFMetaData> dropped = new ArrayList<>();
+
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+            if (entry.getValue().hasColumns())
+                created.addAll(createTablesFromTablesPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+        {
+            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
+
+            ColumnFamily pre  = entry.getValue().leftValue();
+            ColumnFamily post = entry.getValue().rightValue();
+
+            if (pre.hasColumns() && post.hasColumns())
+            {
+                MapDifference<String, CFMetaData> delta =
+                    Maps.difference(Schema.instance.getKSMetaData(keyspaceName).cfMetaData(),
+                                    createTablesFromTablesPartition(new Row(entry.getKey(), post)));
+
+                dropped.addAll(delta.entriesOnlyOnLeft().values());
+                created.addAll(delta.entriesOnlyOnRight().values());
+                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<CFMetaData>, CFMetaData>()
+                {
+                    public CFMetaData apply(MapDifference.ValueDifference<CFMetaData> pair)
+                    {
+                        return pair.rightValue();
+                    }
+                }));
+            }
+            else if (pre.hasColumns())
+            {
+                dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).cfMetaData().values());
+            }
+            else if (post.hasColumns())
+            {
+                created.addAll(createTablesFromTablesPartition(new Row(entry.getKey(), post)).values());
+            }
+        }
+
+        for (CFMetaData cfm : created)
+            Schema.instance.addTable(cfm);
+        for (CFMetaData cfm : altered)
+            Schema.instance.updateTable(cfm.ksName, cfm.cfName);
+        for (CFMetaData cfm : dropped)
+            Schema.instance.dropTable(cfm.ksName, cfm.cfName);
+    }
+
+    // see the comments for mergeKeyspaces()
+    private static void mergeTypes(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+    {
+        List<UserType> created = new ArrayList<>();
+        List<UserType> altered = new ArrayList<>();
+        List<UserType> dropped = new ArrayList<>();
+
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+        // New keyspace with types
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+            if (entry.getValue().hasColumns())
+                created.addAll(createTypesFromPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+        {
+            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
+
+            ColumnFamily pre  = entry.getValue().leftValue();
+            ColumnFamily post = entry.getValue().rightValue();
+
+            if (pre.hasColumns() && post.hasColumns())
+            {
+                MapDifference<ByteBuffer, UserType> delta =
+                    Maps.difference(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes(),
+                                    createTypesFromPartition(new Row(entry.getKey(), post)));
+
+                dropped.addAll(delta.entriesOnlyOnLeft().values());
+                created.addAll(delta.entriesOnlyOnRight().values());
+                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UserType>, UserType>()
+                {
+                    public UserType apply(MapDifference.ValueDifference<UserType> pair)
+                    {
+                        return pair.rightValue();
+                    }
+                }));
+            }
+            else if (pre.hasColumns())
+            {
+                dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes().values());
+            }
+            else if (post.hasColumns())
+            {
+                created.addAll(createTypesFromPartition(new Row(entry.getKey(), post)).values());
+            }
+        }
+
+        for (UserType type : created)
+            Schema.instance.addType(type);
+        for (UserType type : altered)
+            Schema.instance.updateType(type);
+        for (UserType type : dropped)
+            Schema.instance.dropType(type);
+    }
+
+    // see the comments for mergeKeyspaces()
+    private static void mergeFunctions(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+    {
+        List<UDFunction> created = new ArrayList<>();
+        List<UDFunction> altered = new ArrayList<>();
+        List<UDFunction> dropped = new ArrayList<>();
+
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+        // New keyspace with functions
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+            if (entry.getValue().hasColumns())
+                created.addAll(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+        {
+            ColumnFamily pre = entry.getValue().leftValue();
+            ColumnFamily post = entry.getValue().rightValue();
+
+            if (pre.hasColumns() && post.hasColumns())
+            {
+                MapDifference<ByteBuffer, UDFunction> delta =
+                    Maps.difference(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), pre)),
+                                    createFunctionsFromFunctionsPartition(new Row(entry.getKey(), post)));
+
+                dropped.addAll(delta.entriesOnlyOnLeft().values());
+                created.addAll(delta.entriesOnlyOnRight().values());
+                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UDFunction>, UDFunction>()
+                {
+                    public UDFunction apply(MapDifference.ValueDifference<UDFunction> pair)
+                    {
+                        return pair.rightValue();
+                    }
+                }));
+            }
+            else if (pre.hasColumns())
+            {
+                dropped.addAll(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), pre)).values());
+            }
+            else if (post.hasColumns())
+            {
+                created.addAll(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), post)).values());
+            }
+        }
+
+        for (UDFunction udf : created)
+            Schema.instance.addFunction(udf);
+        for (UDFunction udf : altered)
+            Schema.instance.updateFunction(udf);
+        for (UDFunction udf : dropped)
+            Schema.instance.dropFunction(udf);
+    }
+
+    // see the comments for mergeKeyspaces()
+    private static void mergeAggregates(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+    {
+        List<UDAggregate> created = new ArrayList<>();
+        List<UDAggregate> altered = new ArrayList<>();
+        List<UDAggregate> dropped = new ArrayList<>();
+
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+        // New keyspace with functions
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+            if (entry.getValue().hasColumns())
+                created.addAll(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+        {
+            ColumnFamily pre = entry.getValue().leftValue();
+            ColumnFamily post = entry.getValue().rightValue();
+
+            if (pre.hasColumns() && post.hasColumns())
+            {
+                MapDifference<ByteBuffer, UDAggregate> delta =
+                    Maps.difference(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), pre)),
+                                    createAggregatesFromAggregatesPartition(new Row(entry.getKey(), post)));
+
+                dropped.addAll(delta.entriesOnlyOnLeft().values());
+                created.addAll(delta.entriesOnlyOnRight().values());
+                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UDAggregate>, UDAggregate>()
+                {
+                    public UDAggregate apply(MapDifference.ValueDifference<UDAggregate> pair)
+                    {
+                        return pair.rightValue();
+                    }
+                }));
+            }
+            else if (pre.hasColumns())
+            {
+                dropped.addAll(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), pre)).values());
+            }
+            else if (post.hasColumns())
+            {
+                created.addAll(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), post)).values());
+            }
+        }
+
+        for (UDAggregate udf : created)
+            Schema.instance.addAggregate(udf);
+        for (UDAggregate udf : altered)
+            Schema.instance.updateAggregate(udf);
+        for (UDAggregate udf : dropped)
+            Schema.instance.dropAggregate(udf);
+    }
+
+    /*
+     * Keyspace metadata serialization/deserialization.
+     */
+
+    public static Mutation makeCreateKeyspaceMutation(KSMetaData keyspace, long timestamp)
+    {
+        return makeCreateKeyspaceMutation(keyspace, timestamp, true);
+    }
+
+    private static Mutation makeCreateKeyspaceMutation(KSMetaData keyspace, long timestamp, boolean withTablesAndTypesAndFunctions)
+    {
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, getSchemaKSKey(keyspace.name));
+        ColumnFamily cells = mutation.addOrGet(Keyspaces);
+        CFRowAdder adder = new CFRowAdder(cells, Keyspaces.comparator.builder().build(), timestamp);
+
+        adder.add("durable_writes", keyspace.durableWrites);
+        adder.add("strategy_class", keyspace.strategyClass.getName());
+        adder.add("strategy_options", json(keyspace.strategyOptions));
+
+        if (withTablesAndTypesAndFunctions)
+        {
+            for (UserType type : keyspace.userTypes.getAllTypes().values())
+                addTypeToSchemaMutation(type, timestamp, mutation);
+
+            for (CFMetaData table : keyspace.cfMetaData().values())
+                addTableToSchemaMutation(table, timestamp, true, mutation);
+        }
+
+        return mutation;
+    }
+
+    public static Mutation makeDropKeyspaceMutation(KSMetaData keyspace, long timestamp)
+    {
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, getSchemaKSKey(keyspace.name));
+        for (String schemaTable : ALL)
+            mutation.delete(schemaTable, timestamp);
+        mutation.delete(SystemKeyspace.BUILT_INDEXES, timestamp);
+        return mutation;
+    }
+
+    private static KSMetaData createKeyspaceFromSchemaPartitions(Row serializedKeyspace, Row serializedTables, Row serializedTypes)
+    {
+        Collection<CFMetaData> tables = createTablesFromTablesPartition(serializedTables).values();
+        UTMetaData types = new UTMetaData(createTypesFromPartition(serializedTypes));
+        return createKeyspaceFromSchemaPartition(serializedKeyspace).cloneWith(tables, types);
+    }
+
+    public static KSMetaData createKeyspaceFromName(String keyspace)
+    {
+        Row partition = readSchemaPartitionForKeyspace(KEYSPACES, keyspace);
+
+        if (isEmptySchemaPartition(partition))
+            throw new RuntimeException(String.format("%s not found in the schema definitions keyspaceName (%s).", keyspace, KEYSPACES));
+
+        return createKeyspaceFromSchemaPartition(partition);
+    }
+
+    /**
+     * Deserialize only Keyspace attributes without nested tables or types
+     *
+     * @param partition Keyspace attributes in serialized form
+     */
+    private static KSMetaData createKeyspaceFromSchemaPartition(Row partition)
+    {
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, KEYSPACES);
+        UntypedResultSet.Row row = QueryProcessor.resultify(query, partition).one();
+        try
+        {
+            return new KSMetaData(row.getString("keyspace_name"),
+                                  AbstractReplicationStrategy.getClass(row.getString("strategy_class")),
+                                  fromJsonMap(row.getString("strategy_options")),
+                                  row.getBoolean("durable_writes"));
+        }
+        catch (ConfigurationException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /*
+     * User type metadata serialization/deserialization.
+     */
+
+    public static Mutation makeCreateTypeMutation(KSMetaData keyspace, UserType type, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+        addTypeToSchemaMutation(type, timestamp, mutation);
+        return mutation;
+    }
+
+    private static void addTypeToSchemaMutation(UserType type, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Usertypes);
+
+        Composite prefix = Usertypes.comparator.make(type.name);
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+        adder.resetCollection("field_names");
+        adder.resetCollection("field_types");
+
+        for (int i = 0; i < type.size(); i++)
+        {
+            adder.addListEntry("field_names", type.fieldName(i));
+            adder.addListEntry("field_types", type.fieldType(i).toString());
+        }
+    }
+
+    public static Mutation dropTypeFromSchemaMutation(KSMetaData keyspace, UserType type, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+        ColumnFamily cells = mutation.addOrGet(Usertypes);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        Composite prefix = Usertypes.comparator.make(type.name);
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+        return mutation;
+    }
+
+    private static Map<ByteBuffer, UserType> createTypesFromPartition(Row partition)
+    {
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, USERTYPES);
+        Map<ByteBuffer, UserType> types = new HashMap<>();
+        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+        {
+            UserType type = createTypeFromRow(row);
+            types.put(type.name, type);
+        }
+        return types;
+    }
+
+    private static UserType createTypeFromRow(UntypedResultSet.Row row)
+    {
+        String keyspace = row.getString("keyspace_name");
+        ByteBuffer name = ByteBufferUtil.bytes(row.getString("type_name"));
+        List<String> rawColumns = row.getList("field_names", UTF8Type.instance);
+        List<String> rawTypes = row.getList("field_types", UTF8Type.instance);
+
+        List<ByteBuffer> columns = new ArrayList<>(rawColumns.size());
+        for (String rawColumn : rawColumns)
+            columns.add(ByteBufferUtil.bytes(rawColumn));
+
+        List<AbstractType<?>> types = new ArrayList<>(rawTypes.size());
+        for (String rawType : rawTypes)
+            types.add(parseType(rawType));
+
+        return new UserType(keyspace, name, columns, types);
+    }
+
+    /*
+     * Table metadata serialization/deserialization.
+     */
+
+    public static Mutation makeCreateTableMutation(KSMetaData keyspace, CFMetaData table, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+        addTableToSchemaMutation(table, timestamp, true, mutation);
+        return mutation;
+    }
+
+    private static void addTableToSchemaMutation(CFMetaData table, long timestamp, boolean withColumnsAndTriggers, Mutation mutation)
+    {
+        // For property that can be null (and can be changed), we insert tombstones, to make sure
+        // we don't keep a property the user has removed
+        ColumnFamily cells = mutation.addOrGet(Columnfamilies);
+        Composite prefix = Columnfamilies.comparator.make(table.cfName);
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+        adder.add("cf_id", table.cfId);
+        adder.add("type", table.cfType.toString());
+
+        if (table.isSuper())
+        {
+            // We need to continue saving the comparator and subcomparator separatly, otherwise
+            // we won't know at deserialization if the subcomparator should be taken into account
+            // TODO: we should implement an on-start migration if we want to get rid of that.
+            adder.add("comparator", table.comparator.subtype(0).toString());
+            adder.add("subcomparator", table.comparator.subtype(1).toString());
+        }
+        else
+        {
+            adder.add("comparator", table.comparator.toString());
+        }
+
+        adder.add("bloom_filter_fp_chance", table.getBloomFilterFpChance());
+        adder.add("caching", table.getCaching().toString());
+        adder.add("comment", table.getComment());
+        adder.add("compaction_strategy_class", table.compactionStrategyClass.getName());
+        adder.add("compaction_strategy_options", json(table.compactionStrategyOptions));
+        adder.add("compression_parameters", json(table.compressionParameters.asThriftOptions()));
+        adder.add("default_time_to_live", table.getDefaultTimeToLive());
+        adder.add("default_validator", table.getDefaultValidator().toString());
+        adder.add("gc_grace_seconds", table.getGcGraceSeconds());
+        adder.add("key_validator", table.getKeyValidator().toString());
+        adder.add("local_read_repair_chance", table.getDcLocalReadRepairChance());
+        adder.add("max_compaction_threshold", table.getMaxCompactionThreshold());
+        adder.add("max_index_interval", table.getMaxIndexInterval());
+        adder.add("memtable_flush_period_in_ms", table.getMemtableFlushPeriod());
+        adder.add("min_compaction_threshold", table.getMinCompactionThreshold());
+        adder.add("min_index_interval", table.getMinIndexInterval());
+        adder.add("read_repair_chance", table.getReadRepairChance());
+        adder.add("speculative_retry", table.getSpeculativeRetry().toString());
+
+        for (Map.Entry<ColumnIdentifier, Long> entry : table.getDroppedColumns().entrySet())
+            adder.addMapEntry("dropped_columns", entry.getKey().toString(), entry.getValue());
+
+        adder.add("is_dense", table.getIsDense());
+
+        if (withColumnsAndTriggers)
+        {
+            for (ColumnDefinition column : table.allColumns())
+                addColumnToSchemaMutation(table, column, timestamp, mutation);
+
+            for (TriggerDefinition trigger : table.getTriggers().values())
+                addTriggerToSchemaMutation(table, trigger, timestamp, mutation);
+        }
+    }
+
+    public static Mutation makeUpdateTableMutation(KSMetaData keyspace,
+                                                   CFMetaData oldTable,
+                                                   CFMetaData newTable,
+                                                   long timestamp,
+                                                   boolean fromThrift)
+    {
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+        addTableToSchemaMutation(newTable, timestamp, false, mutation);
+
+        MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(oldTable.getColumnMetadata(),
+                                                                                 newTable.getColumnMetadata());
+
+        // columns that are no longer needed
+        for (ColumnDefinition column : columnDiff.entriesOnlyOnLeft().values())
+        {
+            // Thrift only knows about the REGULAR ColumnDefinition type, so don't consider other type
+            // are being deleted just because they are not here.
+            if (fromThrift && column.kind != ColumnDefinition.Kind.REGULAR)
+                continue;
+
+            dropColumnFromSchemaMutation(oldTable, column, timestamp, mutation);
+        }
+
+        // newly added columns
+        for (ColumnDefinition column : columnDiff.entriesOnlyOnRight().values())
+            addColumnToSchemaMutation(newTable, column, timestamp, mutation);
+
+        // old columns with updated attributes
+        for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
+            addColumnToSchemaMutation(newTable, newTable.getColumnDefinition(name), timestamp, mutation);
+
+        MapDifference<String, TriggerDefinition> triggerDiff = Maps.difference(oldTable.getTriggers(), newTable.getTriggers());
+
+        // dropped triggers
+        for (TriggerDefinition trigger : triggerDiff.entriesOnlyOnLeft().values())
+            dropTriggerFromSchemaMutation(oldTable, trigger, timestamp, mutation);
+
+        // newly created triggers
+        for (TriggerDefinition trigger : triggerDiff.entriesOnlyOnRight().values())
+            addTriggerToSchemaMutation(newTable, trigger, timestamp, mutation);
+
+        return mutation;
+    }
+
+    public static Mutation makeDropTableMutation(KSMetaData keyspace, CFMetaData table, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+        ColumnFamily cells = mutation.addOrGet(Columnfamilies);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        Composite prefix = Columnfamilies.comparator.make(table.cfName);
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+        for (ColumnDefinition column : table.allColumns())
+            dropColumnFromSchemaMutation(table, column, timestamp, mutation);
+
+        for (TriggerDefinition trigger : table.getTriggers().values())
+            dropTriggerFromSchemaMutation(table, trigger, timestamp, mutation);
+
+        // TODO: get rid of in #6717
+        ColumnFamily indexCells = mutation.addOrGet(SystemKeyspace.BuiltIndexes);
+        for (String indexName : Keyspace.open(keyspace.name).getColumnFamilyStore(table.cfName).getBuiltIndexes())
+            indexCells.addTombstone(indexCells.getComparator().makeCellName(indexName), ldt, timestamp);
+
+        return mutation;
+    }
+
+    public static CFMetaData createTableFromName(String keyspace, String table)
+    {
+        Row partition = readSchemaPartitionForTable(COLUMNFAMILIES, keyspace, table);
+
+        if (isEmptySchemaPartition(partition))
+            throw new RuntimeException(String.format("%s:%s not found in the schema definitions keyspace.", keyspace, table));
+
+        return createTableFromTablePartition(partition);
+    }
+
+    /**
+     * Deserialize tables from low-level schema representation, all of them belong to the same keyspace
+     *
+     * @return map containing name of the table and its metadata for faster lookup
+     */
+    private static Map<String, CFMetaData> createTablesFromTablesPartition(Row partition)
+    {
+        if (partition.cf == null)
+            return Collections.emptyMap();
+
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNFAMILIES);
+        Map<String, CFMetaData> tables = new HashMap<>();
+        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+        {
+            CFMetaData cfm = createTableFromTableRow(row);
+            tables.put(cfm.cfName, cfm);
+        }
+        return tables;
+    }
+
+    public static CFMetaData createTableFromTablePartitionAndColumnsPartition(Row serializedTable, Row serializedColumns)
+    {
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNFAMILIES);
+        return createTableFromTableRowAndColumnsPartition(QueryProcessor.resultify(query, serializedTable).one(), serializedColumns);
+    }
+
+    private static CFMetaData createTableFromTableRowAndColumnsPartition(UntypedResultSet.Row tableRow, Row serializedColumns)
+    {
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNS);
+        return createTableFromTableRowAndColumnRows(tableRow, QueryProcessor.resultify(query, serializedColumns));
+    }
+
+    private static CFMetaData createTableFromTablePartition(Row row)
+    {
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNFAMILIES);
+        return createTableFromTableRow(QueryProcessor.resultify(query, row).one());
+    }
+
+    /**
+     * Deserialize table metadata from low-level representation
+     *
+     * @return Metadata deserialized from schema
+     */
+    private static CFMetaData createTableFromTableRow(UntypedResultSet.Row result)
+    {
+        String ksName = result.getString("keyspace_name");
+        String cfName = result.getString("columnfamily_name");
+
+        Row serializedColumns = readSchemaPartitionForTable(COLUMNS, ksName, cfName);
+        CFMetaData cfm = createTableFromTableRowAndColumnsPartition(result, serializedColumns);
+
+        Row serializedTriggers = readSchemaPartitionForTable(TRIGGERS, ksName, cfName);
+        try
+        {
+            for (TriggerDefinition trigger : createTriggersFromTriggersPartition(serializedTriggers))
+                cfm.addTriggerDefinition(trigger);
+        }
+        catch (InvalidRequestException e)
+        {
+            throw new RuntimeException(e);
+        }
+
+        return cfm;
+    }
+
+    public static CFMetaData createTableFromTableRowAndColumnRows(UntypedResultSet.Row result,
+                                                                  UntypedResultSet serializedColumnDefinitions)
+    {
+        try
+        {
+            String ksName = result.getString("keyspace_name");
+            String cfName = result.getString("columnfamily_name");
+
+            AbstractType<?> rawComparator = TypeParser.parse(result.getString("comparator"));
+            AbstractType<?> subComparator = result.has("subcomparator") ? TypeParser.parse(result.getString("subcomparator")) : null;
+            ColumnFamilyType cfType = ColumnFamilyType.valueOf(result.getString("type"));
+
+            AbstractType<?> fullRawComparator = CFMetaData.makeRawAbstractType(rawComparator, subComparator);
+
+            List<ColumnDefinition> columnDefs = createColumnsFromColumnRows(serializedColumnDefinitions,
+                                                                            ksName,
+                                                                            cfName,
+                                                                            fullRawComparator,
+                                                                            cfType == ColumnFamilyType.Super);
+
+            boolean isDense = result.has("is_dense")
+                            ? result.getBoolean("is_dense")
+                            : CFMetaData.calculateIsDense(fullRawComparator, columnDefs);
+
+            CellNameType comparator = CellNames.fromAbstractType(fullRawComparator, isDense);
+
+            // if we are upgrading, we use id generated from names initially
+            UUID cfId = result.has("cf_id")
+                      ? result.getUUID("cf_id")
+                      : CFMetaData.generateLegacyCfId(ksName, cfName);
+
+            CFMetaData cfm = new CFMetaData(ksName, cfName, cfType, comparator, cfId);
+            cfm.isDense(isDense);
+
+            cfm.readRepairChance(result.getDouble("read_repair_chance"));
+            cfm.dcLocalReadRepairChance(result.getDouble("local_read_repair_chance"));
+            cfm.gcGraceSeconds(result.getInt("gc_grace_seconds"));
+            cfm.defaultValidator(TypeParser.parse(result.getString("default_validator")));
+            cfm.keyValidator(TypeParser.parse(result.getString("key_validator")));
+            cfm.minCompactionThreshold(result.getInt("min_compaction_threshold"));
+            cfm.maxCompactionThreshold(result.getInt("max_compaction_threshold"));
+            if (result.has("comment"))
+                cfm.comment(result.getString("comment"));
+            if (result.has("memtable_flush_period_in_ms"))
+                cfm.memtableFlushPeriod(result.getInt("memtable_flush_period_in_ms"));
+            cfm.caching(CachingOptions.fromString(result.getString("caching")));
+            if (result.has("default_time_to_live"))
+                cfm.defaultTimeToLive(result.getInt("default_time_to_live"));
+            if (result.has("speculative_retry"))
+                cfm.speculativeRetry(CFMetaData.SpeculativeRetry.fromString(result.getString("speculative_retry")));
+            cfm.compactionStrategyClass(CFMetaData.createCompactionStrategy(result.getString("compaction_strategy_class")));
+            cfm.compressionParameters(CompressionParameters.create(fromJsonMap(result.getString("compression_parameters"))));
+            cfm.compactionStrategyOptions(fromJsonMap(result.getString("compaction_strategy_options")));
+
+            if (result.has("min_index_interval"))
+                cfm.minIndexInterval(result.getInt("min_index_interval"));
+
+            if (result.has("max_index_interval"))
+                cfm.maxIndexInterval(result.getInt("max_index_interval"));
+
+            if (result.has("bloom_filter_fp_chance"))
+                cfm.bloomFilterFpChance(result.getDouble("bloom_filter_fp_chance"));
+            else
+                cfm.bloomFilterFpChance(cfm.getBloomFilterFpChance());
+
+            if (result.has("dropped_columns"))
+                cfm.droppedColumns(convertDroppedColumns(result.getMap("dropped_columns", UTF8Type.instance, LongType.instance)));
+
+            for (ColumnDefinition cd : columnDefs)
+                cfm.addOrReplaceColumnDefinition(cd);
+
+            return cfm.rebuild();
+        }
+        catch (SyntaxException | ConfigurationException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private static Map<ColumnIdentifier, Long> convertDroppedColumns(Map<String, Long> raw)
+    {
+        Map<ColumnIdentifier, Long> converted = Maps.newHashMap();
+        for (Map.Entry<String, Long> entry : raw.entrySet())
+            converted.put(new ColumnIdentifier(entry.getKey(), true), entry.getValue());
+        return converted;
+    }
+
+    /*
+     * Column metadata serialization/deserialization.
+     */
+
+    private static void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Columns);
+        Composite prefix = Columns.comparator.make(table.cfName, column.name.toString());
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+        adder.add("validator", column.type.toString());
+        adder.add("type", serializeKind(column.kind));
+        adder.add("component_index", column.isOnAllComponents() ? null : column.position());
+        adder.add("index_name", column.getIndexName());
+        adder.add("index_type", column.getIndexType() == null ? null : column.getIndexType().toString());
+        adder.add("index_options", json(column.getIndexOptions()));
+    }
+
+    private static String serializeKind(ColumnDefinition.Kind kind)
+    {
+        // For backward compatibility we need to special case CLUSTERING_COLUMN
+        return kind == ColumnDefinition.Kind.CLUSTERING_COLUMN ? "clustering_key" : kind.toString().toLowerCase();
+    }
+
+    private static ColumnDefinition.Kind deserializeKind(String kind)
+    {
+        if (kind.equalsIgnoreCase("clustering_key"))
+            return ColumnDefinition.Kind.CLUSTERING_COLUMN;
+        return Enum.valueOf(ColumnDefinition.Kind.class, kind.toUpperCase());
+    }
+
+    private static void dropColumnFromSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Columns);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        // Note: we do want to use name.toString(), not name.bytes directly for backward compatibility (For CQL3, this won't make a difference).
+        Composite prefix = Columns.comparator.make(table.cfName, column.name.toString());
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+    }
+
+    private static List<ColumnDefinition> createColumnsFromColumnRows(UntypedResultSet rows,
+                                                                      String keyspace,
+                                                                      String table,
+                                                                      AbstractType<?> rawComparator,
+                                                                      boolean isSuper)
+    {
+        List<ColumnDefinition> columns = new ArrayList<>();
+        for (UntypedResultSet.Row row : rows)
+            columns.add(createColumnFromColumnRow(row, keyspace, table, rawComparator, isSuper));
+        return columns;
+    }
+
+    private static ColumnDefinition createColumnFromColumnRow(UntypedResultSet.Row row,
+                                                              String keyspace,
+                                                              String table,
+                                                              AbstractType<?> rawComparator,
+                                                              boolean isSuper)
+    {
+        ColumnDefinition.Kind kind = deserializeKind(row.getString("type"));
+
+        Integer componentIndex = null;
+        if (row.has("component_index"))
+            componentIndex = row.getInt("component_index");
+        else if (kind == ColumnDefinition.Kind.CLUSTERING_COLUMN && isSuper)
+            componentIndex = 1; // A ColumnDefinition for super columns applies to the column component
+
+        // Note: we save the column name as string, but we should not assume that it is an UTF8 name, we
+        // we need to use the comparator fromString method
+        AbstractType<?> comparator = kind == ColumnDefinition.Kind.REGULAR
+                                   ? getComponentComparator(rawComparator, componentIndex)
+                                   : UTF8Type.instance;
+        ColumnIdentifier name = new ColumnIdentifier(comparator.fromString(row.getString("column_name")), comparator);
+
+        AbstractType<?> validator = parseType(row.getString("validator"));
+
+        IndexType indexType = null;
+        if (row.has("index_type"))
+            indexType = IndexType.valueOf(row.getString("index_type"));
+
+        Map<String, String> indexOptions = null;
+        if (row.has("index_options"))
+            indexOptions = fromJsonMap(row.getString("index_options"));
+
+        String indexName = null;
+        if (row.has("index_name"))
+            indexName = row.getString("index_name");
+
+        return new ColumnDefinition(keyspace, table, name, validator, indexType, indexOptions, indexName, componentIndex, kind);
+    }
+
+    private static AbstractType<?> getComponentComparator(AbstractType<?> rawComparator, Integer componentIndex)
+    {
+        return (componentIndex == null || (componentIndex == 0 && !(rawComparator instanceof CompositeType)))
+               ? rawComparator
+               : ((CompositeType)rawComparator).types.get(componentIndex);
+    }
+
+    /*
+     * Trigger metadata serialization/deserialization.
+     */
+
+    private static void addTriggerToSchemaMutation(CFMetaData table, TriggerDefinition trigger, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Triggers);
+        Composite prefix = Triggers.comparator.make(table.cfName, trigger.name);
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+        adder.addMapEntry("trigger_options", "class", trigger.classOption);
+    }
+
+    private static void dropTriggerFromSchemaMutation(CFMetaData table, TriggerDefinition trigger, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Triggers);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        Composite prefix = Triggers.comparator.make(table.cfName, trigger.name);
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+    }
+
+    /**
+     * Deserialize triggers from storage-level representation.
+     *
+     * @param partition storage-level partition containing the trigger definitions
+     * @return the list of processed TriggerDefinitions
+     */
+    private static List<TriggerDefinition> createTriggersFromTriggersPartition(Row partition)
+    {
+        List<TriggerDefinition> triggers = new ArrayList<>();
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, TRIGGERS);
+        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+        {
+            String name = row.getString("trigger_name");
+            String classOption = row.getMap("trigger_options", UTF8Type.instance, UTF8Type.instance).get("class");
+            triggers.add(new TriggerDefinition(name, classOption));
+        }
+        return triggers;
+    }
+
+    /*
+     * UDF metadata serialization/deserialization.
+     */
+
+    public static Mutation makeCreateFunctionMutation(KSMetaData keyspace, UDFunction function, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+        addFunctionToSchemaMutation(function, timestamp, mutation);
+        return mutation;
+    }
+
+    private static void addFunctionToSchemaMutation(UDFunction function, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Functions);
+        Composite prefix = Functions.comparator.make(function.name().name, UDHelper.calculateSignature(function));
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+        adder.resetCollection("argument_names");
+        adder.resetCollection("argument_types");
+
+        for (int i = 0; i < function.argNames().size(); i++)
+        {
+            adder.addListEntry("argument_names", function.argNames().get(i).bytes);
+            adder.addListEntry("argument_types", function.argTypes().get(i).toString());
+        }
+
+        adder.add("body", function.body());
+        adder.add("is_deterministic", function.isDeterministic());
+        adder.add("language", function.language());
+        adder.add("return_type", function.returnType().toString());
+    }
+
+    public static Mutation makeDropFunctionMutation(KSMetaData keyspace, UDFunction function, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+        ColumnFamily cells = mutation.addOrGet(Functions);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        Composite prefix = Functions.comparator.make(function.name().name, UDHelper.calculateSignature(function));
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+        return mutation;
+    }
+
+    private static Map<ByteBuffer, UDFunction> createFunctionsFromFunctionsPartition(Row partition)
+    {
+        Map<ByteBuffer, UDFunction> functions = new HashMap<>();
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, FUNCTIONS);
+        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+        {
+            UDFunction function = createFunctionFromFunctionRow(row);
+            functions.put(UDHelper.calculateSignature(function), function);
+        }
+        return functions;
+    }
+
+    private static UDFunction createFunctionFromFunctionRow(UntypedResultSet.Row row)
+    {
+        String ksName = row.getString("keyspace_name");
+        String functionName = row.getString("function_name");
+        FunctionName name = new FunctionName(ksName, functionName);
+
+        List<ColumnIdentifier> argNames = new ArrayList<>();
+        if (row.has("argument_names"))
+            for (String arg : row.getList("argument_names", UTF8Type.instance))
+                argNames.add(new ColumnIdentifier(arg, true));
+
+        List<AbstractType<?>> argTypes = new ArrayList<>();
+        if (row.has("argument_types"))
+            for (String type : row.getList("argument_types", UTF8Type.instance))
+                argTypes.add(parseType(type));
+
+        AbstractType<?> returnType = parseType(row.getString("return_type"));
+
+        boolean isDeterministic = row.getBoolean("is_deterministic");
+        String language = row.getString("language");
+        String body = row.getString("body");
+
+        try
+        {
+            return UDFunction.create(name, argNames, argTypes, returnType, language, body, isDeterministic);
+        }
+        catch (InvalidRequestException e)
+        {
+            logger.error(String.format("Cannot load function '%s' from schema: this function won't be available (on this node)", name), e);
+            return UDFunction.createBrokenFunction(name, argNames, argTypes, returnType, language, body, e);
+        }
+    }
+
+    /*
+     * Aggregate UDF metadata serialization/deserialization.
+     */
+
+    public static Mutation makeCreateAggregateMutation(KSMetaData keyspace, UDAggregate aggregate, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+        addAggregateToSchemaMutation(aggregate, timestamp, mutation);
+        return mutation;
+    }
+
+    private static void addAggregateToSchemaMutation(UDAggregate aggregate, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Aggregates);
+        Composite prefix = Aggregates.comparator.make(aggregate.name().name, UDHelper.calculateSignature(aggregate));
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+        adder.resetCollection("argument_types");
+        adder.add("return_type", aggregate.returnType().toString());
+        adder.add("state_func", aggregate.stateFunction().name().name);
+        if (aggregate.stateType() != null)
+            adder.add("state_type", aggregate.stateType().toString());
+        if (aggregate.finalFunction() != null)
+            adder.add("final_func", aggregate.finalFunction().name().name);
+        if (aggregate.initialCondition() != null)
+            adder.add("initcond", aggregate.initialCondition());
+
+        for (AbstractType<?> argType : aggregate.argTypes())
+            adder.addListEntry("argument_types", argType.toString());
+    }
+
+    private static Map<ByteBuffer, UDAggregate> createAggregatesFromAggregatesPartition(Row partition)
+    {
+        Map<ByteBuffer, UDAggregate> aggregates = new HashMap<>();
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, AGGREGATES);
+        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+        {
+            UDAggregate aggregate = createAggregateFromAggregateRow(row);
+            aggregates.put(UDHelper.calculateSignature(aggregate), aggregate);
+        }
+        return aggregates;
+    }
+
+    private static UDAggregate createAggregateFromAggregateRow(UntypedResultSet.Row row)
+    {
+        String ksName = row.getString("keyspace_name");
+        String functionName = row.getString("aggregate_name");
+        FunctionName name = new FunctionName(ksName, functionName);
+
+        List<String> types = row.getList("argument_types", UTF8Type.instance);
+
+        List<AbstractType<?>> argTypes;
+        if (types == null)
+        {
+            argTypes = Collections.emptyList();
+        }
+        else
+        {
+            argTypes = new ArrayList<>(types.size());
+            for (String type : types)
+                argTypes.add(parseType(type));
+        }
+
+        AbstractType<?> returnType = parseType(row.getString("return_type"));
+
+        FunctionName stateFunc = new FunctionName(ksName, row.getString("state_func"));
+        FunctionName finalFunc = row.has("final_func") ? new FunctionName(ksName, row.getString("final_func")) : null;
+        AbstractType<?> stateType = row.has("state_type") ? parseType(row.getString("state_type")) : null;
+        ByteBuffer initcond = row.has("initcond") ? row.getBytes("initcond") : null;
+
+        try
+        {
+            return UDAggregate.create(name, argTypes, returnType, stateFunc, finalFunc, stateType, initcond);
+        }
+        catch (InvalidRequestException reason)
+        {
+            return UDAggregate.createBroken(name, argTypes, returnType, initcond, reason);
+        }
+    }
+
+    public static Mutation makeDropAggregateMutation(KSMetaData keyspace, UDAggregate aggregate, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+        ColumnFamily cells = mutation.addOrGet(Aggregates);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        Composite prefix = Aggregates.comparator.make(aggregate.name().name, UDHelper.calculateSignature(aggregate));
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+        return mutation;
+    }
+
+    private static AbstractType<?> parseType(String str)
+    {
+        try
+        {
+            return TypeParser.parse(str);
+        }
+        catch (SyntaxException | ConfigurationException e)
+        {
+            // We only use this when reading the schema where we shouldn't get an error
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 99a2d87..edc91ab 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -34,7 +34,6 @@ import javax.management.StandardMBean;
 
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.Uninterruptibles;
-import org.hyperic.sigar.SigarException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,7 +45,6 @@ import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.functions.Functions;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
@@ -253,10 +251,8 @@ public class CassandraDaemon
             exitOrFail(100, "Fatal exception during initialization", e);
         }
 
-
-        // load keyspace && function descriptions.
-        DatabaseDescriptor.loadSchemas();
-        Functions.loadUDFFromSchema();
+        // load schema from disk
+        Schema.instance.loadFromDisk();
 
         // clean up compaction leftovers
         Map<Pair<String, String>, Map<Integer, UUID>> unfinishedCompactions = SystemKeyspace.getUnfinishedCompactions();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index 9c8d7d8..dcdf838 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryHandler;
 import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -58,7 +59,7 @@ public class ClientState
     {
         // We want these system cfs to be always readable to authenticated users since many tools rely on them
         // (nodetool, cqlsh, bulkloader, etc.)
-        for (String cf : Iterables.concat(Arrays.asList(SystemKeyspace.LOCAL_TABLE, SystemKeyspace.PEERS_TABLE), SystemKeyspace.ALL_SCHEMA_TABLES))
+        for (String cf : Iterables.concat(Arrays.asList(SystemKeyspace.LOCAL, SystemKeyspace.PEERS), LegacySchemaTables.ALL))
             READABLE_SYSTEM_RESOURCES.add(DataResource.columnFamily(SystemKeyspace.NAME, cf));
 
         PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthenticator().protectedResources());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/service/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java b/src/java/org/apache/cassandra/service/MigrationManager.java
index c3fe1fa..fe32559 100644
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ b/src/java/org/apache/cassandra/service/MigrationManager.java
@@ -37,7 +37,6 @@ import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.UTMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.functions.UDAggregate;
 import org.apache.cassandra.cql3.functions.UDFunction;
@@ -50,6 +49,7 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.WrappedRunnable;
 
@@ -63,7 +63,7 @@ public class MigrationManager
 
     public static final int MIGRATION_DELAY_IN_MS = 60000;
 
-    private final List<IMigrationListener> listeners = new CopyOnWriteArrayList<IMigrationListener>();
+    private final List<IMigrationListener> listeners = new CopyOnWriteArrayList<>();
     
     private MigrationManager() {}
 
@@ -182,52 +182,40 @@ public class MigrationManager
             listener.onCreateFunction(udf.name().keyspace, udf.name().name);
     }
 
-    public void notifyUpdateFunction(UDFunction udf)
-    {
-        for (IMigrationListener listener : listeners)
-            listener.onUpdateFunction(udf.name().keyspace, udf.name().name);
-    }
-
-    public void notifyDropFunction(UDFunction udf)
-    {
-        for (IMigrationListener listener : listeners)
-            listener.onDropFunction(udf.name().keyspace, udf.name().name);
-    }
-
     public void notifyCreateAggregate(UDAggregate udf)
     {
         for (IMigrationListener listener : listeners)
             listener.onCreateAggregate(udf.name().keyspace, udf.name().name);
     }
 
-    public void notifyUpdateAggregate(UDAggregate udf)
+    public void notifyUpdateKeyspace(KSMetaData ksm)
     {
         for (IMigrationListener listener : listeners)
-            listener.onUpdateAggregate(udf.name().keyspace, udf.name().name);
+            listener.onUpdateKeyspace(ksm.name);
     }
 
-    public void notifyDropAggregate(UDAggregate udf)
+    public void notifyUpdateColumnFamily(CFMetaData cfm)
     {
         for (IMigrationListener listener : listeners)
-            listener.onDropAggregate(udf.name().keyspace, udf.name().name);
+            listener.onUpdateColumnFamily(cfm.ksName, cfm.cfName);
     }
 
-    public void notifyUpdateKeyspace(KSMetaData ksm)
+    public void notifyUpdateUserType(UserType ut)
     {
         for (IMigrationListener listener : listeners)
-            listener.onUpdateKeyspace(ksm.name);
+            listener.onUpdateUserType(ut.keyspace, ut.getNameAsString());
     }
 
-    public void notifyUpdateColumnFamily(CFMetaData cfm)
+    public void notifyUpdateFunction(UDFunction udf)
     {
         for (IMigrationListener listener : listeners)
-            listener.onUpdateColumnFamily(cfm.ksName, cfm.cfName);
+            listener.onUpdateFunction(udf.name().keyspace, udf.name().name);
     }
 
-    public void notifyUpdateUserType(UserType ut)
+    public void notifyUpdateAggregate(UDAggregate udf)
     {
         for (IMigrationListener listener : listeners)
-            listener.onUpdateUserType(ut.keyspace, ut.getNameAsString());
+            listener.onUpdateAggregate(udf.name().keyspace, udf.name().name);
     }
 
     public void notifyDropKeyspace(KSMetaData ksm)
@@ -248,6 +236,18 @@ public class MigrationManager
             listener.onDropUserType(ut.keyspace, ut.getNameAsString());
     }
 
+    public void notifyDropFunction(UDFunction udf)
+    {
+        for (IMigrationListener listener : listeners)
+            listener.onDropFunction(udf.name().keyspace, udf.name().name);
+    }
+
+    public void notifyDropAggregate(UDAggregate udf)
+    {
+        for (IMigrationListener listener : listeners)
+            listener.onDropAggregate(udf.name().keyspace, udf.name().name);
+    }
+
     public static void announceNewKeyspace(KSMetaData ksm) throws ConfigurationException
     {
         announceNewKeyspace(ksm, false);
@@ -266,7 +266,7 @@ public class MigrationManager
             throw new AlreadyExistsException(ksm.name);
 
         logger.info(String.format("Create new Keyspace: %s", ksm));
-        announce(ksm.toSchema(timestamp), announceLocally);
+        announce(LegacySchemaTables.makeCreateKeyspaceMutation(ksm, timestamp), announceLocally);
     }
 
     public static void announceNewColumnFamily(CFMetaData cfm) throws ConfigurationException
@@ -285,17 +285,27 @@ public class MigrationManager
             throw new AlreadyExistsException(cfm.ksName, cfm.cfName);
 
         logger.info(String.format("Create new table: %s", cfm));
-        announce(addSerializedKeyspace(cfm.toSchema(FBUtilities.timestampMicros()), cfm.ksName), announceLocally);
+        announce(LegacySchemaTables.makeCreateTableMutation(ksm, cfm, FBUtilities.timestampMicros()), announceLocally);
     }
 
-    public static void announceNewType(UserType newType)
+    public static void announceNewType(UserType newType, boolean announceLocally)
     {
-        announceNewType(newType, false);
+        KSMetaData ksm = Schema.instance.getKSMetaData(newType.keyspace);
+        announce(LegacySchemaTables.makeCreateTypeMutation(ksm, newType, FBUtilities.timestampMicros()), announceLocally);
     }
 
-    public static void announceNewType(UserType newType, boolean announceLocally)
+    public static void announceNewFunction(UDFunction udf, boolean announceLocally)
     {
-        announce(addSerializedKeyspace(UTMetaData.toSchema(newType, FBUtilities.timestampMicros()), newType.keyspace), announceLocally);
+        logger.info(String.format("Create scalar function '%s'", udf.name()));
+        KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+        announce(LegacySchemaTables.makeCreateFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceNewAggregate(UDAggregate udf, boolean announceLocally)
+    {
+        logger.info(String.format("Create aggregate function '%s'", udf.name()));
+        KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+        announce(LegacySchemaTables.makeCreateAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceKeyspaceUpdate(KSMetaData ksm) throws ConfigurationException
@@ -312,7 +322,7 @@ public class MigrationManager
             throw new ConfigurationException(String.format("Cannot update non existing keyspace '%s'.", ksm.name));
 
         logger.info(String.format("Update Keyspace '%s' From %s To %s", ksm.name, oldKsm, ksm));
-        announce(oldKsm.toSchemaUpdate(ksm, FBUtilities.timestampMicros()), announceLocally);
+        announce(LegacySchemaTables.makeCreateKeyspaceMutation(ksm, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceColumnFamilyUpdate(CFMetaData cfm, boolean fromThrift) throws ConfigurationException
@@ -327,16 +337,12 @@ public class MigrationManager
         CFMetaData oldCfm = Schema.instance.getCFMetaData(cfm.ksName, cfm.cfName);
         if (oldCfm == null)
             throw new ConfigurationException(String.format("Cannot update non existing table '%s' in keyspace '%s'.", cfm.cfName, cfm.ksName));
+        KSMetaData ksm = Schema.instance.getKSMetaData(cfm.ksName);
 
         oldCfm.validateCompatility(cfm);
 
         logger.info(String.format("Update table '%s/%s' From %s To %s", cfm.ksName, cfm.cfName, oldCfm, cfm));
-        announce(addSerializedKeyspace(oldCfm.toSchemaUpdate(cfm, FBUtilities.timestampMicros(), fromThrift), cfm.ksName), announceLocally);
-    }
-
-    public static void announceTypeUpdate(UserType updatedType)
-    {
-        announceTypeUpdate(updatedType, false);
+        announce(LegacySchemaTables.makeUpdateTableMutation(ksm, oldCfm, cfm, FBUtilities.timestampMicros(), fromThrift), announceLocally);
     }
 
     public static void announceTypeUpdate(UserType updatedType, boolean announceLocally)
@@ -356,7 +362,7 @@ public class MigrationManager
             throw new ConfigurationException(String.format("Cannot drop non existing keyspace '%s'.", ksName));
 
         logger.info(String.format("Drop Keyspace '%s'", oldKsm.name));
-        announce(oldKsm.dropFromSchema(FBUtilities.timestampMicros()), announceLocally);
+        announce(LegacySchemaTables.makeDropKeyspaceMutation(oldKsm, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceColumnFamilyDrop(String ksName, String cfName) throws ConfigurationException
@@ -369,16 +375,10 @@ public class MigrationManager
         CFMetaData oldCfm = Schema.instance.getCFMetaData(ksName, cfName);
         if (oldCfm == null)
             throw new ConfigurationException(String.format("Cannot drop non existing table '%s' in keyspace '%s'.", cfName, ksName));
+        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
 
         logger.info(String.format("Drop table '%s/%s'", oldCfm.ksName, oldCfm.cfName));
-        announce(addSerializedKeyspace(oldCfm.dropFromSchema(FBUtilities.timestampMicros()), ksName), announceLocally);
-    }
-
-    // Include the serialized keyspace for when a target node missed the CREATE KEYSPACE migration (see #5631).
-    private static Mutation addSerializedKeyspace(Mutation migration, String ksName)
-    {
-        migration.add(SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, ksName).cf);
-        return migration;
+        announce(LegacySchemaTables.makeDropTableMutation(ksm, oldCfm, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceTypeDrop(UserType droppedType)
@@ -388,35 +388,22 @@ public class MigrationManager
 
     public static void announceTypeDrop(UserType droppedType, boolean announceLocally)
     {
-        announce(addSerializedKeyspace(UTMetaData.dropFromSchema(droppedType, FBUtilities.timestampMicros()), droppedType.keyspace), announceLocally);
+        KSMetaData ksm = Schema.instance.getKSMetaData(droppedType.keyspace);
+        announce(LegacySchemaTables.dropTypeFromSchemaMutation(ksm, droppedType, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceFunctionDrop(UDFunction udf, boolean announceLocally)
     {
-        Mutation mutation = udf.toSchemaDrop(FBUtilities.timestampMicros());
         logger.info(String.format("Drop scalar function overload '%s' args '%s'", udf.name(), udf.argTypes()));
-        announce(mutation, announceLocally);
+        KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+        announce(LegacySchemaTables.makeDropFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceAggregateDrop(UDAggregate udf, boolean announceLocally)
     {
-        Mutation mutation = udf.toSchemaDrop(FBUtilities.timestampMicros());
         logger.info(String.format("Drop aggregate function overload '%s' args '%s'", udf.name(), udf.argTypes()));
-        announce(mutation, announceLocally);
-    }
-
-    public static void announceNewFunction(UDFunction udf, boolean announceLocally)
-    {
-        Mutation mutation = udf.toSchemaUpdate(FBUtilities.timestampMicros());
-        logger.info(String.format("Create scalar function '%s'", udf.name()));
-        announce(mutation, announceLocally);
-    }
-
-    public static void announceNewAggregate(UDAggregate udf, boolean announceLocally)
-    {
-        Mutation mutation = udf.toSchemaUpdate(FBUtilities.timestampMicros());
-        logger.info(String.format("Create aggregate function '%s'", udf.name()));
-        announce(mutation, announceLocally);
+        KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+        announce(LegacySchemaTables.makeDropAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
     }
 
     /**
@@ -429,7 +416,7 @@ public class MigrationManager
         {
             try
             {
-                DefsTables.mergeSchemaInternal(Collections.singletonList(schema), false);
+                LegacySchemaTables.mergeSchema(Collections.singletonList(schema), false);
             }
             catch (IOException e)
             {
@@ -457,7 +444,7 @@ public class MigrationManager
         {
             protected void runMayThrow() throws IOException, ConfigurationException
             {
-                DefsTables.mergeSchema(schema);
+                LegacySchemaTables.mergeSchema(schema);
             }
         });
 
@@ -497,9 +484,7 @@ public class MigrationManager
 
         logger.debug("Truncating schema tables...");
 
-        // truncate schema tables
-        for (String cf : SystemKeyspace.ALL_SCHEMA_TABLES)
-            SystemKeyspace.schemaCFS(cf).truncateBlocking();
+        LegacySchemaTables.truncateSchemaTables();
 
         logger.debug("Clearing local schema keyspace definitions...");
 
@@ -536,7 +521,7 @@ public class MigrationManager
         public Collection<Mutation> deserialize(DataInput in, int version) throws IOException
         {
             int count = in.readInt();
-            Collection<Mutation> schema = new ArrayList<Mutation>(count);
+            Collection<Mutation> schema = new ArrayList<>(count);
 
             for (int i = 0; i < count; i++)
                 schema.add(Mutation.serializer.deserialize(in, version));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/service/MigrationTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationTask.java b/src/java/org/apache/cassandra/service/MigrationTask.java
index 9fdbff4..2a67e6d 100644
--- a/src/java/org/apache/cassandra/service/MigrationTask.java
+++ b/src/java/org/apache/cassandra/service/MigrationTask.java
@@ -26,7 +26,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.DefsTables;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.net.IAsyncCallback;
 import org.apache.cassandra.net.MessageIn;
@@ -63,7 +63,7 @@ class MigrationTask extends WrappedRunnable
             {
                 try
                 {
-                    DefsTables.mergeSchema(message.payload);
+                    LegacySchemaTables.mergeSchema(message.payload);
                 }
                 catch (IOException e)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 9c57946..d6eb317 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -713,7 +713,7 @@ public class StorageProxy implements StorageProxyMBean
                                                                         null,
                                                                         WriteType.SIMPLE);
         Mutation mutation = new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(uuid));
-        mutation.delete(SystemKeyspace.BATCHLOG_TABLE, FBUtilities.timestampMicros());
+        mutation.delete(SystemKeyspace.BATCHLOG, FBUtilities.timestampMicros());
         MessageOut<Mutation> message = mutation.createMessage();
         for (InetAddress target : endpoints)
         {


[3/5] cassandra git commit: Isolate schema serializaton code

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 3e8b0a2..503dd7f 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -26,27 +26,20 @@ import java.util.concurrent.TimeUnit;
 import javax.management.openmbean.*;
 
 import com.google.common.base.Function;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.SetMultimap;
-import com.google.common.collect.Sets;
+import com.google.common.collect.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.compaction.CompactionHistoryTabularData;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
-import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
@@ -54,6 +47,7 @@ import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.LocalStrategy;
 import org.apache.cassandra.metrics.RestorableMeter;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.service.paxos.PaxosState;
@@ -70,155 +64,21 @@ public final class SystemKeyspace
 
     public static final String NAME = "system";
 
-    public static final String SCHEMA_KEYSPACES_TABLE = "schema_keyspaces";
-    public static final String SCHEMA_COLUMNFAMILIES_TABLE = "schema_columnfamilies";
-    public static final String SCHEMA_COLUMNS_TABLE = "schema_columns";
-    public static final String SCHEMA_TRIGGERS_TABLE = "schema_triggers";
-    public static final String SCHEMA_USER_TYPES_TABLE = "schema_usertypes";
-    public static final String SCHEMA_FUNCTIONS_TABLE = "schema_functions";
-    public static final String SCHEMA_AGGREGATES_TABLE = "schema_aggregates";
-
-    public static final String BUILT_INDEXES_TABLE = "IndexInfo";
-    public static final String HINTS_TABLE = "hints";
-    public static final String BATCHLOG_TABLE = "batchlog";
-    public static final String PAXOS_TABLE = "paxos";
-    public static final String LOCAL_TABLE = "local";
-    public static final String PEERS_TABLE = "peers";
-    public static final String PEER_EVENTS_TABLE = "peer_events";
-    public static final String RANGE_XFERS_TABLE = "range_xfers";
-    public static final String COMPACTION_LOG_TABLE = "compactions_in_progress";
-    public static final String COMPACTION_HISTORY_TABLE = "compaction_history";
-    public static final String SSTABLE_ACTIVITY_TABLE = "sstable_activity";
-
-    public static final List<String> ALL_SCHEMA_TABLES =
-        Arrays.asList(SCHEMA_KEYSPACES_TABLE,
-                      SCHEMA_COLUMNFAMILIES_TABLE,
-                      SCHEMA_COLUMNS_TABLE,
-                      SCHEMA_TRIGGERS_TABLE,
-                      SCHEMA_USER_TYPES_TABLE,
-                      SCHEMA_FUNCTIONS_TABLE,
-                      SCHEMA_AGGREGATES_TABLE);
-
-    private static int WEEK = (int) TimeUnit.DAYS.toSeconds(7);
-
-    public static final CFMetaData SchemaKeyspacesTable =
-        compile(SCHEMA_KEYSPACES_TABLE, "keyspace definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "durable_writes boolean,"
-                + "strategy_class text,"
-                + "strategy_options text,"
-                + "PRIMARY KEY ((keyspace_name))) "
-                + "WITH COMPACT STORAGE")
-                .gcGraceSeconds(WEEK);
-
-    public static final CFMetaData SchemaColumnFamiliesTable =
-        compile(SCHEMA_COLUMNFAMILIES_TABLE, "table definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "columnfamily_name text,"
-                + "bloom_filter_fp_chance double,"
-                + "caching text,"
-                + "cf_id uuid," // post-2.1 UUID cfid
-                + "comment text,"
-                + "compaction_strategy_class text,"
-                + "compaction_strategy_options text,"
-                + "comparator text,"
-                + "compression_parameters text,"
-                + "default_time_to_live int,"
-                + "default_validator text,"
-                + "dropped_columns map<text, bigint>,"
-                + "gc_grace_seconds int,"
-                + "is_dense boolean,"
-                + "key_validator text,"
-                + "local_read_repair_chance double,"
-                + "max_compaction_threshold int,"
-                + "max_index_interval int,"
-                + "memtable_flush_period_in_ms int,"
-                + "min_compaction_threshold int,"
-                + "min_index_interval int,"
-                + "read_repair_chance double,"
-                + "speculative_retry text,"
-                + "subcomparator text,"
-                + "type text,"
-                + "PRIMARY KEY ((keyspace_name), columnfamily_name))")
-                .gcGraceSeconds(WEEK);
-
-    public static final CFMetaData SchemaColumnsTable =
-        compile(SCHEMA_COLUMNS_TABLE, "column definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "columnfamily_name text,"
-                + "column_name text,"
-                + "component_index int,"
-                + "index_name text,"
-                + "index_options text,"
-                + "index_type text,"
-                + "type text,"
-                + "validator text,"
-                + "PRIMARY KEY ((keyspace_name), columnfamily_name, column_name))")
-                .gcGraceSeconds(WEEK);
-
-    public static final CFMetaData SchemaTriggersTable =
-        compile(SCHEMA_TRIGGERS_TABLE, "trigger definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "columnfamily_name text,"
-                + "trigger_name text,"
-                + "trigger_options map<text, text>,"
-                + "PRIMARY KEY ((keyspace_name), columnfamily_name, trigger_name))")
-                .gcGraceSeconds(WEEK);
-
-    public static final CFMetaData SchemaUserTypesTable =
-        compile(SCHEMA_USER_TYPES_TABLE, "user defined type definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "type_name text,"
-                + "field_names list<text>,"
-                + "field_types list<text>,"
-                + "PRIMARY KEY ((keyspace_name), type_name))")
-                .gcGraceSeconds(WEEK);
-
-    public static final CFMetaData SchemaFunctionsTable =
-        compile(SCHEMA_FUNCTIONS_TABLE, "user defined function definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "function_name text,"
-                + "signature blob,"
-                + "argument_names list<text>,"
-                + "argument_types list<text>,"
-                + "body text,"
-                + "deterministic boolean,"
-                + "language text,"
-                + "return_type text,"
-                + "PRIMARY KEY ((keyspace_name), function_name, signature))")
-                .gcGraceSeconds(WEEK);
-
-    public static final CFMetaData SchemaAggregatesTable =
-        compile(SCHEMA_AGGREGATES_TABLE, "user defined aggregate definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "aggregate_name text,"
-                + "signature blob,"
-                + "argument_types list<text>,"
-                + "return_type text,"
-                + "state_func text,"
-                + "state_type text,"
-                + "final_func text,"
-                + "initcond blob,"
-                + "PRIMARY KEY ((keyspace_name), aggregate_name, signature))")
-                .gcGraceSeconds(WEEK);
-
-    public static final CFMetaData BuiltIndexesTable =
-        compile(BUILT_INDEXES_TABLE, "built column indexes",
-                "CREATE TABLE \"%s\" ("
-                + "table_name text,"
-                + "index_name text,"
-                + "PRIMARY KEY ((table_name), index_name)) "
-                + "WITH COMPACT STORAGE");
-
-    public static final CFMetaData HintsTable =
-        compile(HINTS_TABLE, "hints awaiting delivery",
+    public static final String HINTS = "hints";
+    public static final String BATCHLOG = "batchlog";
+    public static final String PAXOS = "paxos";
+    public static final String BUILT_INDEXES = "IndexInfo";
+    public static final String LOCAL = "local";
+    public static final String PEERS = "peers";
+    public static final String PEER_EVENTS = "peer_events";
+    public static final String RANGE_XFERS = "range_xfers";
+    public static final String COMPACTIONS_IN_PROGRESS = "compactions_in_progress";
+    public static final String COMPACTION_HISTORY = "compaction_history";
+    public static final String SSTABLE_ACTIVITY = "sstable_activity";
+
+    public static final CFMetaData Hints =
+        compile(HINTS,
+                "hints awaiting delivery",
                 "CREATE TABLE %s ("
                 + "target_id uuid,"
                 + "hint_id timeuuid,"
@@ -229,8 +89,9 @@ public final class SystemKeyspace
                 .compactionStrategyOptions(Collections.singletonMap("enabled", "false"))
                 .gcGraceSeconds(0);
 
-    public static final CFMetaData BatchlogTable =
-        compile(BATCHLOG_TABLE, "batches awaiting replay",
+    public static final CFMetaData Batchlog =
+        compile(BATCHLOG,
+                "batches awaiting replay",
                 "CREATE TABLE %s ("
                 + "id uuid,"
                 + "data blob,"
@@ -240,8 +101,9 @@ public final class SystemKeyspace
                 .compactionStrategyOptions(Collections.singletonMap("min_threshold", "2"))
                 .gcGraceSeconds(0);
 
-    private static final CFMetaData PaxosTable =
-        compile(PAXOS_TABLE, "in-progress paxos proposals",
+    private static final CFMetaData Paxos =
+        compile(PAXOS,
+                "in-progress paxos proposals",
                 "CREATE TABLE %s ("
                 + "row_key blob,"
                 + "cf_id UUID,"
@@ -253,8 +115,19 @@ public final class SystemKeyspace
                 + "PRIMARY KEY ((row_key), cf_id))")
                 .compactionStrategyClass(LeveledCompactionStrategy.class);
 
-    private static final CFMetaData LocalTable =
-        compile(LOCAL_TABLE, "information about the local node",
+    // TODO: make private
+    public static final CFMetaData BuiltIndexes =
+        compile(BUILT_INDEXES,
+                "built column indexes",
+                "CREATE TABLE \"%s\" ("
+                + "table_name text,"
+                + "index_name text,"
+                + "PRIMARY KEY ((table_name), index_name)) "
+                + "WITH COMPACT STORAGE");
+
+    private static final CFMetaData Local =
+        compile(LOCAL,
+                "information about the local node",
                 "CREATE TABLE %s ("
                 + "key text,"
                 + "bootstrapped text,"
@@ -273,8 +146,9 @@ public final class SystemKeyspace
                 + "truncated_at map<uuid, blob>,"
                 + "PRIMARY KEY ((key)))");
 
-    private static final CFMetaData PeersTable =
-        compile(PEERS_TABLE, "information about known peers in the cluster",
+    private static final CFMetaData Peers =
+        compile(PEERS,
+                "information about known peers in the cluster",
                 "CREATE TABLE %s ("
                 + "peer inet,"
                 + "data_center text,"
@@ -287,22 +161,25 @@ public final class SystemKeyspace
                 + "tokens set<varchar>,"
                 + "PRIMARY KEY ((peer)))");
 
-    private static final CFMetaData PeerEventsTable =
-        compile(PEER_EVENTS_TABLE, "events related to peers",
+    private static final CFMetaData PeerEvents =
+        compile(PEER_EVENTS,
+                "events related to peers",
                 "CREATE TABLE %s ("
                 + "peer inet,"
                 + "hints_dropped map<uuid, int>,"
                 + "PRIMARY KEY ((peer)))");
 
-    private static final CFMetaData RangeXfersTable =
-        compile(RANGE_XFERS_TABLE, "ranges requested for transfer",
+    private static final CFMetaData RangeXfers =
+        compile(RANGE_XFERS,
+                "ranges requested for transfer",
                 "CREATE TABLE %s ("
                 + "token_bytes blob,"
                 + "requested_at timestamp,"
                 + "PRIMARY KEY ((token_bytes)))");
 
-    private static final CFMetaData CompactionLogTable =
-        compile(COMPACTION_LOG_TABLE, "unfinished compactions",
+    private static final CFMetaData CompactionsInProgress =
+        compile(COMPACTIONS_IN_PROGRESS,
+                "unfinished compactions",
                 "CREATE TABLE %s ("
                 + "id uuid,"
                 + "columnfamily_name text,"
@@ -310,8 +187,9 @@ public final class SystemKeyspace
                 + "keyspace_name text,"
                 + "PRIMARY KEY ((id)))");
 
-    private static final CFMetaData CompactionHistoryTable =
-        compile(COMPACTION_HISTORY_TABLE, "week-long compaction history",
+    private static final CFMetaData CompactionHistory =
+        compile(COMPACTION_HISTORY,
+                "week-long compaction history",
                 "CREATE TABLE %s ("
                 + "id uuid,"
                 + "bytes_in bigint,"
@@ -321,10 +199,11 @@ public final class SystemKeyspace
                 + "keyspace_name text,"
                 + "rows_merged map<int, bigint>,"
                 + "PRIMARY KEY ((id)))")
-                .defaultTimeToLive(WEEK);
+                .defaultTimeToLive((int) TimeUnit.DAYS.toSeconds(7));
 
-    private static final CFMetaData SSTableActivityTable =
-        compile(SSTABLE_ACTIVITY_TABLE, "historic sstable read rates",
+    private static final CFMetaData SSTableActivity =
+        compile(SSTABLE_ACTIVITY,
+                "historic sstable read rates",
                 "CREATE TABLE %s ("
                 + "keyspace_name text,"
                 + "columnfamily_name text,"
@@ -333,37 +212,29 @@ public final class SystemKeyspace
                 + "rate_15m double,"
                 + "PRIMARY KEY ((keyspace_name, columnfamily_name, generation)))");
 
-    private static CFMetaData compile(String table, String comment, String cql)
+    private static CFMetaData compile(String name, String description, String schema)
     {
-        return CFMetaData.compile(String.format(cql, table), NAME).comment(comment);
+        return CFMetaData.compile(String.format(schema, name), NAME).comment(description);
     }
 
     public static KSMetaData definition()
     {
-        List<CFMetaData> tables =
-            Arrays.asList(SchemaKeyspacesTable,
-                          SchemaColumnFamiliesTable,
-                          SchemaColumnsTable,
-                          SchemaTriggersTable,
-                          SchemaUserTypesTable,
-                          SchemaFunctionsTable,
-                          SchemaAggregatesTable,
-                          BuiltIndexesTable,
-                          HintsTable,
-                          BatchlogTable,
-                          PaxosTable,
-                          LocalTable,
-                          PeersTable,
-                          PeerEventsTable,
-                          RangeXfersTable,
-                          CompactionLogTable,
-                          CompactionHistoryTable,
-                          SSTableActivityTable);
+        Iterable<CFMetaData> tables =
+            Iterables.concat(LegacySchemaTables.All,
+                             Arrays.asList(BuiltIndexes,
+                                           Hints,
+                                           Batchlog,
+                                           Paxos,
+                                           Local,
+                                           Peers,
+                                           PeerEvents,
+                                           RangeXfers,
+                                           CompactionsInProgress,
+                                           CompactionHistory,
+                                           SSTableActivity));
         return new KSMetaData(NAME, LocalStrategy.class, Collections.<String, String>emptyMap(), true, tables);
     }
 
-    private static final String LOCAL_KEY = "local";
-
     private static volatile Map<UUID, Pair<ReplayPosition, Long>> truncationRecords;
 
     public enum BootstrapState
@@ -381,24 +252,15 @@ public final class SystemKeyspace
     public static void finishStartup()
     {
         setupVersion();
-
-        // add entries to system schema columnfamilies for the hardcoded system definitions
-        KSMetaData ksmd = Schema.instance.getKSMetaData(NAME);
-
-        // delete old, possibly obsolete entries in schema tables
-        for (String table : ALL_SCHEMA_TABLES)
-            executeOnceInternal(String.format("DELETE FROM system.%s WHERE keyspace_name = ?", table), ksmd.name);
-
-        // (+1 to timestamp to make sure we don't get shadowed by the tombstones we just added)
-        ksmd.toSchema(FBUtilities.timestampMicros() + 1).apply();
+        LegacySchemaTables.saveSystemKeyspaceSchema();
     }
 
     private static void setupVersion()
     {
         String req = "INSERT INTO system.%s (key, release_version, cql_version, thrift_version, native_protocol_version, data_center, rack, partitioner) VALUES (?, ?, ?, ?, ?, ?, ?, ?)";
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        executeOnceInternal(String.format(req, LOCAL_TABLE),
-                            LOCAL_KEY,
+        executeOnceInternal(String.format(req, LOCAL),
+                            LOCAL,
                             FBUtilities.getReleaseVersionString(),
                             QueryProcessor.CQL_VERSION.toString(),
                             cassandraConstants.VERSION,
@@ -429,8 +291,8 @@ public final class SystemKeyspace
             }
         });
         String req = "INSERT INTO system.%s (id, keyspace_name, columnfamily_name, inputs) VALUES (?, ?, ?, ?)";
-        executeInternal(String.format(req, COMPACTION_LOG_TABLE), compactionId, cfs.keyspace.getName(), cfs.name, Sets.newHashSet(generations));
-        forceBlockingFlush(COMPACTION_LOG_TABLE);
+        executeInternal(String.format(req, COMPACTIONS_IN_PROGRESS), compactionId, cfs.keyspace.getName(), cfs.name, Sets.newHashSet(generations));
+        forceBlockingFlush(COMPACTIONS_IN_PROGRESS);
         return compactionId;
     }
 
@@ -443,8 +305,8 @@ public final class SystemKeyspace
     {
         assert taskId != null;
 
-        executeInternal(String.format("DELETE FROM system.%s WHERE id = ?", COMPACTION_LOG_TABLE), taskId);
-        forceBlockingFlush(COMPACTION_LOG_TABLE);
+        executeInternal(String.format("DELETE FROM system.%s WHERE id = ?", COMPACTIONS_IN_PROGRESS), taskId);
+        forceBlockingFlush(COMPACTIONS_IN_PROGRESS);
     }
 
     /**
@@ -454,7 +316,7 @@ public final class SystemKeyspace
     public static Map<Pair<String, String>, Map<Integer, UUID>> getUnfinishedCompactions()
     {
         String req = "SELECT * FROM system.%s";
-        UntypedResultSet resultSet = executeInternal(String.format(req, COMPACTION_LOG_TABLE));
+        UntypedResultSet resultSet = executeInternal(String.format(req, COMPACTIONS_IN_PROGRESS));
 
         Map<Pair<String, String>, Map<Integer, UUID>> unfinishedCompactions = new HashMap<>();
         for (UntypedResultSet.Row row : resultSet)
@@ -479,7 +341,7 @@ public final class SystemKeyspace
 
     public static void discardCompactionsInProgress()
     {
-        ColumnFamilyStore compactionLog = Keyspace.open(NAME).getColumnFamilyStore(COMPACTION_LOG_TABLE);
+        ColumnFamilyStore compactionLog = Keyspace.open(NAME).getColumnFamilyStore(COMPACTIONS_IN_PROGRESS);
         compactionLog.truncateBlocking();
     }
 
@@ -491,24 +353,24 @@ public final class SystemKeyspace
                                                Map<Integer, Long> rowsMerged)
     {
         // don't write anything when the history table itself is compacted, since that would in turn cause new compactions
-        if (ksname.equals("system") && cfname.equals(COMPACTION_HISTORY_TABLE))
+        if (ksname.equals("system") && cfname.equals(COMPACTION_HISTORY))
             return;
         String req = "INSERT INTO system.%s (id, keyspace_name, columnfamily_name, compacted_at, bytes_in, bytes_out, rows_merged) VALUES (?, ?, ?, ?, ?, ?, ?)";
-        executeInternal(String.format(req, COMPACTION_HISTORY_TABLE), UUIDGen.getTimeUUID(), ksname, cfname, ByteBufferUtil.bytes(compactedAt), bytesIn, bytesOut, rowsMerged);
+        executeInternal(String.format(req, COMPACTION_HISTORY), UUIDGen.getTimeUUID(), ksname, cfname, ByteBufferUtil.bytes(compactedAt), bytesIn, bytesOut, rowsMerged);
     }
 
     public static TabularData getCompactionHistory() throws OpenDataException
     {
-        UntypedResultSet queryResultSet = executeInternal(String.format("SELECT * from system.%s", COMPACTION_HISTORY_TABLE));
+        UntypedResultSet queryResultSet = executeInternal(String.format("SELECT * from system.%s", COMPACTION_HISTORY));
         return CompactionHistoryTabularData.from(queryResultSet);
     }
 
     public static synchronized void saveTruncationRecord(ColumnFamilyStore cfs, long truncatedAt, ReplayPosition position)
     {
         String req = "UPDATE system.%s SET truncated_at = truncated_at + ? WHERE key = '%s'";
-        executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), truncationAsMapEntry(cfs, truncatedAt, position));
+        executeInternal(String.format(req, LOCAL, LOCAL), truncationAsMapEntry(cfs, truncatedAt, position));
         truncationRecords = null;
-        forceBlockingFlush(LOCAL_TABLE);
+        forceBlockingFlush(LOCAL);
     }
 
     /**
@@ -517,9 +379,9 @@ public final class SystemKeyspace
     public static synchronized void removeTruncationRecord(UUID cfId)
     {
         String req = "DELETE truncated_at[?] from system.%s WHERE key = '%s'";
-        executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), cfId);
+        executeInternal(String.format(req, LOCAL, LOCAL), cfId);
         truncationRecords = null;
-        forceBlockingFlush(LOCAL_TABLE);
+        forceBlockingFlush(LOCAL);
     }
 
     private static Map<UUID, ByteBuffer> truncationAsMapEntry(ColumnFamilyStore cfs, long truncatedAt, ReplayPosition position)
@@ -558,7 +420,7 @@ public final class SystemKeyspace
 
     private static Map<UUID, Pair<ReplayPosition, Long>> readTruncationRecords()
     {
-        UntypedResultSet rows = executeInternal(String.format("SELECT truncated_at FROM system.%s WHERE key = '%s'", LOCAL_TABLE, LOCAL_KEY));
+        UntypedResultSet rows = executeInternal(String.format("SELECT truncated_at FROM system.%s WHERE key = '%s'", LOCAL, LOCAL));
 
         Map<UUID, Pair<ReplayPosition, Long>> records = new HashMap<>();
 
@@ -597,14 +459,14 @@ public final class SystemKeyspace
         }
 
         String req = "INSERT INTO system.%s (peer, tokens) VALUES (?, ?)";
-        executeInternal(String.format(req, PEERS_TABLE), ep, tokensAsSet(tokens));
+        executeInternal(String.format(req, PEERS), ep, tokensAsSet(tokens));
     }
 
     public static synchronized void updatePreferredIP(InetAddress ep, InetAddress preferred_ip)
     {
         String req = "INSERT INTO system.%s (peer, preferred_ip) VALUES (?, ?)";
-        executeInternal(String.format(req, PEERS_TABLE), ep, preferred_ip);
-        forceBlockingFlush(PEERS_TABLE);
+        executeInternal(String.format(req, PEERS), ep, preferred_ip);
+        forceBlockingFlush(PEERS);
     }
 
     public static synchronized void updatePeerInfo(InetAddress ep, String columnName, Object value)
@@ -613,20 +475,20 @@ public final class SystemKeyspace
             return;
 
         String req = "INSERT INTO system.%s (peer, %s) VALUES (?, ?)";
-        executeInternal(String.format(req, PEERS_TABLE, columnName), ep, value);
+        executeInternal(String.format(req, PEERS, columnName), ep, value);
     }
 
     public static synchronized void updateHintsDropped(InetAddress ep, UUID timePeriod, int value)
     {
         // with 30 day TTL
         String req = "UPDATE system.%s USING TTL 2592000 SET hints_dropped[ ? ] = ? WHERE peer = ?";
-        executeInternal(String.format(req, PEER_EVENTS_TABLE), timePeriod, value, ep);
+        executeInternal(String.format(req, PEER_EVENTS), timePeriod, value, ep);
     }
 
     public static synchronized void updateSchemaVersion(UUID version)
     {
         String req = "INSERT INTO system.%s (key, schema_version) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), version);
+        executeInternal(String.format(req, LOCAL, LOCAL), version);
     }
 
     private static Set<String> tokensAsSet(Collection<Token> tokens)
@@ -653,7 +515,7 @@ public final class SystemKeyspace
     public static synchronized void removeEndpoint(InetAddress ep)
     {
         String req = "DELETE FROM system.%s WHERE peer = ?";
-        executeInternal(String.format(req, PEERS_TABLE), ep);
+        executeInternal(String.format(req, PEERS), ep);
     }
 
     /**
@@ -663,8 +525,8 @@ public final class SystemKeyspace
     {
         assert !tokens.isEmpty() : "removeEndpoint should be used instead";
         String req = "INSERT INTO system.%s (key, tokens) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), tokensAsSet(tokens));
-        forceBlockingFlush(LOCAL_TABLE);
+        executeInternal(String.format(req, LOCAL, LOCAL), tokensAsSet(tokens));
+        forceBlockingFlush(LOCAL);
     }
 
     /**
@@ -696,7 +558,7 @@ public final class SystemKeyspace
     public static SetMultimap<InetAddress, Token> loadTokens()
     {
         SetMultimap<InetAddress, Token> tokenMap = HashMultimap.create();
-        for (UntypedResultSet.Row row : executeInternal("SELECT peer, tokens FROM system." + PEERS_TABLE))
+        for (UntypedResultSet.Row row : executeInternal("SELECT peer, tokens FROM system." + PEERS))
         {
             InetAddress peer = row.getInetAddress("peer");
             if (row.has("tokens"))
@@ -713,7 +575,7 @@ public final class SystemKeyspace
     public static Map<InetAddress, UUID> loadHostIds()
     {
         Map<InetAddress, UUID> hostIdMap = new HashMap<>();
-        for (UntypedResultSet.Row row : executeInternal("SELECT peer, host_id FROM system." + PEERS_TABLE))
+        for (UntypedResultSet.Row row : executeInternal("SELECT peer, host_id FROM system." + PEERS))
         {
             InetAddress peer = row.getInetAddress("peer");
             if (row.has("host_id"))
@@ -733,7 +595,7 @@ public final class SystemKeyspace
     public static InetAddress getPreferredIP(InetAddress ep)
     {
         String req = "SELECT preferred_ip FROM system.%s WHERE peer=?";
-        UntypedResultSet result = executeInternal(String.format(req, PEERS_TABLE), ep);
+        UntypedResultSet result = executeInternal(String.format(req, PEERS), ep);
         if (!result.isEmpty() && result.one().has("preferred_ip"))
             return result.one().getInetAddress("preferred_ip");
         return ep;
@@ -745,7 +607,7 @@ public final class SystemKeyspace
     public static Map<InetAddress, Map<String,String>> loadDcRackInfo()
     {
         Map<InetAddress, Map<String, String>> result = new HashMap<>();
-        for (UntypedResultSet.Row row : executeInternal("SELECT peer, data_center, rack from system." + PEERS_TABLE))
+        for (UntypedResultSet.Row row : executeInternal("SELECT peer, data_center, rack from system." + PEERS))
         {
             InetAddress peer = row.getInetAddress("peer");
             if (row.has("data_center") && row.has("rack"))
@@ -780,10 +642,10 @@ public final class SystemKeyspace
             ex.initCause(err);
             throw ex;
         }
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(LOCAL_TABLE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(LOCAL);
 
         String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY));
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
 
         if (result.isEmpty() || !result.one().has("cluster_name"))
         {
@@ -793,7 +655,7 @@ public final class SystemKeyspace
 
             // no system files.  this is a new node.
             req = "INSERT INTO system.%s (key, cluster_name) VALUES ('%s', ?)";
-            executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), DatabaseDescriptor.getClusterName());
+            executeInternal(String.format(req, LOCAL, LOCAL), DatabaseDescriptor.getClusterName());
             return;
         }
 
@@ -805,7 +667,7 @@ public final class SystemKeyspace
     public static Collection<Token> getSavedTokens()
     {
         String req = "SELECT tokens FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY));
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
         return result.isEmpty() || !result.one().has("tokens")
              ? Collections.<Token>emptyList()
              : deserializeTokens(result.one().getSet("tokens", UTF8Type.instance));
@@ -814,7 +676,7 @@ public final class SystemKeyspace
     public static int incrementAndGetGeneration()
     {
         String req = "SELECT gossip_generation FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY));
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
 
         int generation;
         if (result.isEmpty() || !result.one().has("gossip_generation"))
@@ -842,8 +704,8 @@ public final class SystemKeyspace
         }
 
         req = "INSERT INTO system.%s (key, gossip_generation) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), generation);
-        forceBlockingFlush(LOCAL_TABLE);
+        executeInternal(String.format(req, LOCAL, LOCAL), generation);
+        forceBlockingFlush(LOCAL);
 
         return generation;
     }
@@ -851,7 +713,7 @@ public final class SystemKeyspace
     public static BootstrapState getBootstrapState()
     {
         String req = "SELECT bootstrapped FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY));
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
 
         if (result.isEmpty() || !result.one().has("bootstrapped"))
             return BootstrapState.NEEDS_BOOTSTRAP;
@@ -872,15 +734,15 @@ public final class SystemKeyspace
     public static void setBootstrapState(BootstrapState state)
     {
         String req = "INSERT INTO system.%s (key, bootstrapped) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), state.name());
-        forceBlockingFlush(LOCAL_TABLE);
+        executeInternal(String.format(req, LOCAL, LOCAL), state.name());
+        forceBlockingFlush(LOCAL);
     }
 
     public static boolean isIndexBuilt(String keyspaceName, String indexName)
     {
-        ColumnFamilyStore cfs = Keyspace.open(NAME).getColumnFamilyStore(BUILT_INDEXES_TABLE);
+        ColumnFamilyStore cfs = Keyspace.open(NAME).getColumnFamilyStore(BUILT_INDEXES);
         QueryFilter filter = QueryFilter.getNamesFilter(decorate(ByteBufferUtil.bytes(keyspaceName)),
-                                                        BUILT_INDEXES_TABLE,
+                                                        BUILT_INDEXES,
                                                         FBUtilities.singleton(cfs.getComparator().makeCellName(indexName), cfs.getComparator()),
                                                         System.currentTimeMillis());
         return ColumnFamilyStore.removeDeleted(cfs.getColumnFamily(filter), Integer.MAX_VALUE) != null;
@@ -888,7 +750,7 @@ public final class SystemKeyspace
 
     public static void setIndexBuilt(String keyspaceName, String indexName)
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(NAME, BUILT_INDEXES_TABLE);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(NAME, BUILT_INDEXES);
         cf.addColumn(new BufferCell(cf.getComparator().makeCellName(indexName), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()));
         new Mutation(NAME, ByteBufferUtil.bytes(keyspaceName), cf).apply();
     }
@@ -896,7 +758,7 @@ public final class SystemKeyspace
     public static void setIndexRemoved(String keyspaceName, String indexName)
     {
         Mutation mutation = new Mutation(NAME, ByteBufferUtil.bytes(keyspaceName));
-        mutation.delete(BUILT_INDEXES_TABLE, BuiltIndexesTable.comparator.makeCellName(indexName), FBUtilities.timestampMicros());
+        mutation.delete(BUILT_INDEXES, BuiltIndexes.comparator.makeCellName(indexName), FBUtilities.timestampMicros());
         mutation.apply();
     }
 
@@ -907,7 +769,7 @@ public final class SystemKeyspace
     public static UUID getLocalHostId()
     {
         String req = "SELECT host_id FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY));
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
 
         // Look up the Host UUID (return it if found)
         if (!result.isEmpty() && result.one().has("host_id"))
@@ -925,144 +787,14 @@ public final class SystemKeyspace
     public static UUID setLocalHostId(UUID hostId)
     {
         String req = "INSERT INTO system.%s (key, host_id) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), hostId);
+        executeInternal(String.format(req, LOCAL, LOCAL), hostId);
         return hostId;
     }
 
-    /**
-     * @param cfName The name of the ColumnFamily responsible for part of the schema (keyspace, ColumnFamily, columns)
-     * @return CFS responsible to hold low-level serialized schema
-     */
-    public static ColumnFamilyStore schemaCFS(String cfName)
-    {
-        return Keyspace.open(NAME).getColumnFamilyStore(cfName);
-    }
-
-    public static List<Row> serializedSchema()
-    {
-        List<Row> schema = new ArrayList<>();
-
-        for (String cf : ALL_SCHEMA_TABLES)
-            schema.addAll(serializedSchema(cf));
-
-        return schema;
-    }
-
-    /**
-     * @param schemaCfName The name of the ColumnFamily responsible for part of the schema (keyspace, ColumnFamily, columns)
-     * @return low-level schema representation (each row represents individual Keyspace or ColumnFamily)
-     */
-    public static List<Row> serializedSchema(String schemaCfName)
-    {
-        Token minToken = StorageService.getPartitioner().getMinimumToken();
-
-        return schemaCFS(schemaCfName).getRangeSlice(new Range<RowPosition>(minToken.minKeyBound(), minToken.maxKeyBound()),
-                                                     null,
-                                                     new IdentityQueryFilter(),
-                                                     Integer.MAX_VALUE,
-                                                     System.currentTimeMillis());
-    }
-
-    public static Collection<Mutation> serializeSchema()
-    {
-        Map<DecoratedKey, Mutation> mutationMap = new HashMap<>();
-
-        for (String cf : ALL_SCHEMA_TABLES)
-            serializeSchema(mutationMap, cf);
-
-        return mutationMap.values();
-    }
-
-    private static void serializeSchema(Map<DecoratedKey, Mutation> mutationMap, String schemaCfName)
-    {
-        for (Row schemaRow : serializedSchema(schemaCfName))
-        {
-            if (Schema.ignoredSchemaRow(schemaRow))
-                continue;
-
-            Mutation mutation = mutationMap.get(schemaRow.key);
-            if (mutation == null)
-            {
-                mutation = new Mutation(NAME, schemaRow.key.getKey());
-                mutationMap.put(schemaRow.key, mutation);
-            }
-
-            mutation.add(schemaRow.cf);
-        }
-    }
-
-    public static Map<DecoratedKey, ColumnFamily> getSchema(String cfName)
-    {
-        Map<DecoratedKey, ColumnFamily> schema = new HashMap<>();
-
-        for (Row schemaEntity : SystemKeyspace.serializedSchema(cfName))
-            schema.put(schemaEntity.key, schemaEntity.cf);
-
-        return schema;
-    }
-
-    public static Map<DecoratedKey, ColumnFamily> getSchema(String schemaCfName, Set<String> keyspaces)
-    {
-        Map<DecoratedKey, ColumnFamily> schema = new HashMap<>();
-
-        for (String keyspace : keyspaces)
-        {
-            Row schemaEntity = readSchemaRow(schemaCfName, keyspace);
-            if (schemaEntity.cf != null)
-                schema.put(schemaEntity.key, schemaEntity.cf);
-        }
-
-        return schema;
-    }
-
-    public static ByteBuffer getSchemaKSKey(String ksName)
-    {
-        return AsciiType.instance.fromString(ksName);
-    }
-
-    /**
-     * Fetches a subset of schema (table data, columns metadata or triggers) for the keyspace.
-     *
-     * @param schemaCfName the schema table to get the data from (schema_keyspaces, schema_columnfamilies, schema_columns or schema_triggers)
-     * @param ksName the keyspace of the tables we are interested in
-     * @return a Row containing the schema data of a particular type for the keyspace
-     */
-    public static Row readSchemaRow(String schemaCfName, String ksName)
-    {
-        DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
-
-        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(schemaCfName);
-        ColumnFamily result = schemaCFS.getColumnFamily(QueryFilter.getIdentityFilter(key, schemaCfName, System.currentTimeMillis()));
-
-        return new Row(key, result);
-    }
-
-    /**
-     * Fetches a subset of schema (table data, columns metadata or triggers) for the keyspace+table pair.
-     *
-     * @param schemaCfName the schema table to get the data from (schema_columnfamilies, schema_columns or schema_triggers)
-     * @param ksName the keyspace of the table we are interested in
-     * @param cfName the table we are interested in
-     * @return a Row containing the schema data of a particular type for the table
-     */
-    public static Row readSchemaRow(String schemaCfName, String ksName, String cfName)
-    {
-        DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
-        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(schemaCfName);
-        Composite prefix = schemaCFS.getComparator().make(cfName);
-        ColumnFamily cf = schemaCFS.getColumnFamily(key,
-                                                    prefix,
-                                                    prefix.end(),
-                                                    false,
-                                                    Integer.MAX_VALUE,
-                                                    System.currentTimeMillis());
-        return new Row(key, cf);
-    }
-
     public static PaxosState loadPaxosState(ByteBuffer key, CFMetaData metadata)
     {
         String req = "SELECT * FROM system.%s WHERE row_key = ? AND cf_id = ?";
-        UntypedResultSet results = executeInternal(String.format(req, PAXOS_TABLE), key, metadata.cfId);
+        UntypedResultSet results = executeInternal(String.format(req, PAXOS), key, metadata.cfId);
         if (results.isEmpty())
             return new PaxosState(key, metadata);
         UntypedResultSet.Row row = results.one();
@@ -1083,7 +815,7 @@ public final class SystemKeyspace
     public static void savePaxosPromise(Commit promise)
     {
         String req = "UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET in_progress_ballot = ? WHERE row_key = ? AND cf_id = ?";
-        executeInternal(String.format(req, PAXOS_TABLE),
+        executeInternal(String.format(req, PAXOS),
                         UUIDGen.microsTimestamp(promise.ballot),
                         paxosTtl(promise.update.metadata),
                         promise.ballot,
@@ -1093,7 +825,7 @@ public final class SystemKeyspace
 
     public static void savePaxosProposal(Commit proposal)
     {
-        executeInternal(String.format("UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = ?, proposal = ? WHERE row_key = ? AND cf_id = ?", PAXOS_TABLE),
+        executeInternal(String.format("UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = ?, proposal = ? WHERE row_key = ? AND cf_id = ?", PAXOS),
                         UUIDGen.microsTimestamp(proposal.ballot),
                         paxosTtl(proposal.update.metadata),
                         proposal.ballot,
@@ -1113,7 +845,7 @@ public final class SystemKeyspace
         // We always erase the last proposal (with the commit timestamp to no erase more recent proposal in case the commit is old)
         // even though that's really just an optimization  since SP.beginAndRepairPaxos will exclude accepted proposal older than the mrc.
         String cql = "UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = null, proposal = null, most_recent_commit_at = ?, most_recent_commit = ? WHERE row_key = ? AND cf_id = ?";
-        executeInternal(String.format(cql, PAXOS_TABLE),
+        executeInternal(String.format(cql, PAXOS),
                         UUIDGen.microsTimestamp(commit.ballot),
                         paxosTtl(commit.update.metadata),
                         commit.ballot,
@@ -1132,7 +864,7 @@ public final class SystemKeyspace
     public static RestorableMeter getSSTableReadMeter(String keyspace, String table, int generation)
     {
         String cql = "SELECT * FROM system.%s WHERE keyspace_name=? and columnfamily_name=? and generation=?";
-        UntypedResultSet results = executeInternal(String.format(cql, SSTABLE_ACTIVITY_TABLE), keyspace, table, generation);
+        UntypedResultSet results = executeInternal(String.format(cql, SSTABLE_ACTIVITY), keyspace, table, generation);
 
         if (results.isEmpty())
             return new RestorableMeter();
@@ -1150,7 +882,7 @@ public final class SystemKeyspace
     {
         // Store values with a one-day TTL to handle corner cases where cleanup might not occur
         String cql = "INSERT INTO system.%s (keyspace_name, columnfamily_name, generation, rate_15m, rate_120m) VALUES (?, ?, ?, ?, ?) USING TTL 864000";
-        executeInternal(String.format(cql, SSTABLE_ACTIVITY_TABLE),
+        executeInternal(String.format(cql, SSTABLE_ACTIVITY),
                         keyspace,
                         table,
                         generation,
@@ -1164,6 +896,6 @@ public final class SystemKeyspace
     public static void clearSSTableReadMeter(String keyspace, String table, int generation)
     {
         String cql = "DELETE FROM system.%s WHERE keyspace_name=? AND columnfamily_name=? and generation=?";
-        executeInternal(String.format(cql, SSTABLE_ACTIVITY_TABLE), keyspace, table, generation);
+        executeInternal(String.format(cql, SSTABLE_ACTIVITY), keyspace, table, generation);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index 0b52904..8be9a18 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -24,17 +24,19 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import com.google.common.collect.*;
-import org.apache.cassandra.db.BufferCell;
-import org.apache.cassandra.db.Cell;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.BufferCell;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.db.marshal.TypeParser;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -228,17 +230,15 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
             {
                 partitioner = FBUtilities.newPartitioner(client.describe_partitioner());           
                 // get CF meta data
-                String query = "SELECT comparator," +
-                               "       subcomparator," +
-                               "       type " +
-                               "FROM system.schema_columnfamilies " +
-                               "WHERE keyspace_name = '%s' " +
-                               "  AND columnfamily_name = '%s' ";
-
-                CqlResult result = client.execute_cql3_query(
-                                        ByteBufferUtil.bytes(String.format(query, keyspace, cfName)),
-                                        Compression.NONE,
-                                        ConsistencyLevel.ONE);
+                String query = String.format("SELECT comparator, subcomparator, type " +
+                                             "FROM %s.%s " +
+                                             "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+                                             SystemKeyspace.NAME,
+                                             LegacySchemaTables.COLUMNFAMILIES,
+                                             keyspace,
+                                             cfName);
+
+                CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
 
                 Iterator<CqlRow> iteraRow = result.rows.iterator();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
index 21e30e2..ffaaea9 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
@@ -22,17 +22,7 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
+import java.util.*;
 
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
@@ -40,10 +30,7 @@ import com.google.common.base.Splitter;
 import com.google.common.collect.AbstractIterator;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
-
 import org.apache.commons.lang3.StringUtils;
-
-import org.apache.cassandra.hadoop.HadoopCompat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -55,6 +42,8 @@ import com.datastax.driver.core.Row;
 import com.datastax.driver.core.Session;
 import com.datastax.driver.core.TupleValue;
 import com.datastax.driver.core.UDTValue;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.hadoop.ColumnFamilySplit;
@@ -601,8 +590,15 @@ public class CqlRecordReader extends RecordReader<Long, Row>
 
     private void fetchKeys()
     {
-        String query = "SELECT column_name, component_index, type FROM system.schema_columns WHERE keyspace_name='" +
-                       keyspace + "' and columnfamily_name='" + cfName + "'";
+        String query = String.format("SELECT column_name, component_index, type " +
+                                     "FROM %s.%s " +
+                                     "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+                                     SystemKeyspace.NAME,
+                                     LegacySchemaTables.COLUMNS,
+                                     keyspace,
+                                     cfName);
+
+        // get CF meta data
         List<Row> rows = session.execute(query).all();
         if (rows.isEmpty())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
index 311359a..0956ba5 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
@@ -25,6 +25,9 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.db.marshal.LongType;
@@ -297,10 +300,6 @@ class CqlRecordWriter extends AbstractColumnFamilyRecordWriter<Map<String, ByteB
                 {
                     result = client.prepare_cql3_query(ByteBufferUtil.bytes(cql), Compression.NONE);
                 }
-                catch (InvalidRequestException e)
-                {
-                    throw new RuntimeException("failed to prepare cql query " + cql, e);
-                }
                 catch (TException e)
                 {
                     throw new RuntimeException("failed to prepare cql query " + cql, e);
@@ -331,18 +330,20 @@ class CqlRecordWriter extends AbstractColumnFamilyRecordWriter<Map<String, ByteB
         return partitionKey;
     }
 
+    // FIXME
     /** retrieve the key validator from system.schema_columnfamilies table */
     private void retrievePartitionKeyValidator(Cassandra.Client client) throws Exception
     {
         String keyspace = ConfigHelper.getOutputKeyspace(conf);
         String cfName = ConfigHelper.getOutputColumnFamily(conf);
-        String query = "SELECT key_validator," +
-        		       "       key_aliases," +
-        		       "       column_aliases " +
-                       "FROM system.schema_columnfamilies " +
-                       "WHERE keyspace_name='%s' and columnfamily_name='%s'";
-        String formatted = String.format(query, keyspace, cfName);
-        CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(formatted), Compression.NONE, ConsistencyLevel.ONE);
+        String query = String.format("SELECT key_validator, key_aliases, column_aliases " +
+                                     "FROM %s.%s " +
+                                     "WHERE keyspace_name = '%s' and columnfamily_name = '%s'",
+                                     SystemKeyspace.NAME,
+                                     LegacySchemaTables.COLUMNFAMILIES,
+                                     keyspace,
+                                     cfName);
+        CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
 
         Column rawKeyValidator = result.rows.get(0).columns.get(0);
         String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
index dc37252..04d207f 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
@@ -26,6 +26,8 @@ import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
 import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.auth.IAuthenticator;
@@ -585,20 +587,15 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
                    IOException
     {
         // get CF meta data
-        String query = "SELECT type," +
-                       "       comparator," +
-                       "       subcomparator," +
-                       "       default_validator," +
-                       "       key_validator," +
-                       "       key_aliases " +
-                       "FROM system.schema_columnfamilies " +
-                       "WHERE keyspace_name = '%s' " +
-                       "  AND columnfamily_name = '%s' ";
-
-        CqlResult result = client.execute_cql3_query(
-                                ByteBufferUtil.bytes(String.format(query, keyspace, column_family)),
-                                Compression.NONE,
-                                ConsistencyLevel.ONE);
+        String query = String.format("SELECT type, comparator, subcomparator, default_validator, key_validator, key_aliases " +
+                                     "FROM %s.%s " +
+                                     "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+                                     SystemKeyspace.NAME,
+                                     LegacySchemaTables.COLUMNFAMILIES,
+                                     keyspace,
+                                     column_family);
+
+        CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
 
         if (result == null || result.rows == null || result.rows.isEmpty())
             return null;
@@ -657,18 +654,15 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
             ConfigurationException,
             NotFoundException
     {
-        String query = "SELECT column_name, " +
-                       "       validator, " +
-                       "       index_type, " +
-                       "       type " +
-                       "FROM system.schema_columns " +
-                       "WHERE keyspace_name = '%s' " +
-                       "  AND columnfamily_name = '%s'";
-
-        CqlResult result = client.execute_cql3_query(
-                                   ByteBufferUtil.bytes(String.format(query, keyspace, column_family)),
-                                   Compression.NONE,
-                                   ConsistencyLevel.ONE);
+        String query = String.format("SELECT column_name, validator, index_type, type " +
+                                     "FROM %s.%s " +
+                                     "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+                                     SystemKeyspace.NAME,
+                                     LegacySchemaTables.COLUMNS,
+                                     keyspace,
+                                     column_family);
+
+        CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
 
         List<CqlRow> rows = result.rows;
         List<ColumnDef> columnDefs = new ArrayList<ColumnDef>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
index 6cd5c66..fca1d43 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
@@ -25,6 +25,8 @@ import java.util.*;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.BufferCell;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.marshal.*;
@@ -482,21 +484,15 @@ public class CqlStorage extends AbstractCassandraStorage
     protected List<ColumnDef> getKeysMeta(Cassandra.Client client)
             throws Exception
     {
-        String query = "SELECT key_aliases, " +
-                "       column_aliases, " +
-                "       key_validator, " +
-                "       comparator, " +
-                "       keyspace_name, " +
-                "       value_alias, " +
-                "       default_validator " +
-                "FROM system.schema_columnfamilies " +
-                "WHERE keyspace_name = '%s'" +
-                "  AND columnfamily_name = '%s' ";
-
-        CqlResult result = client.execute_cql3_query(
-                ByteBufferUtil.bytes(String.format(query, keyspace, column_family)),
-                Compression.NONE,
-                ConsistencyLevel.ONE);
+        String query = String.format("SELECT key_aliases, column_aliases, key_validator, comparator, keyspace_name, value_alias, default_validator " +
+                                     "FROM %s.%s " +
+                                     "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+                                     SystemKeyspace.NAME,
+                                     LegacySchemaTables.COLUMNFAMILIES,
+                                     keyspace,
+                                     column_family);
+
+        CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
 
         if (result == null || result.rows == null || result.rows.isEmpty())
             return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 43cd2c0..ec590f3 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -349,16 +349,16 @@ public class CQLSSTableWriter implements Closeable
                     if (ksm == null)
                     {
                         ksm = KSMetaData.newKeyspace(this.schema.ksName,
-                                AbstractReplicationStrategy.getClass("org.apache.cassandra.locator.SimpleStrategy"),
-                                ImmutableMap.of("replication_factor", "1"),
-                                true,
-                                Collections.singleton(this.schema));
+                                                     AbstractReplicationStrategy.getClass("org.apache.cassandra.locator.SimpleStrategy"),
+                                                     ImmutableMap.of("replication_factor", "1"),
+                                                     true,
+                                                     Collections.singleton(this.schema));
                         Schema.instance.load(ksm);
                     }
                     else if (Schema.instance.getCFMetaData(this.schema.ksName, this.schema.cfName) == null)
                     {
                         Schema.instance.load(this.schema);
-                        ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(this.schema)));
+                        ksm = ksm.cloneWithTableAdded(this.schema);
                         Schema.instance.setKeyspaceDefinition(ksm);
                         Keyspace.open(ksm.name).initCf(this.schema.cfId, this.schema.cfName, false);
                     }