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 2017/01/27 22:18:07 UTC

[01/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Repository: cassandra
Updated Branches:
  refs/heads/trunk 3580f6c05 -> af3fe39dc


http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/triggers/TriggersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/triggers/TriggersTest.java b/test/unit/org/apache/cassandra/triggers/TriggersTest.java
index 37638c9..52cebc9 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggersTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggersTest.java
@@ -17,17 +17,15 @@
  */
 package org.apache.cassandra.triggers;
 
-import java.net.InetAddress;
 import java.util.Collection;
 import java.util.Collections;
 
-import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.*;
@@ -39,7 +37,6 @@ import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
-import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 import static org.apache.cassandra.utils.ByteBufferUtil.toInt;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -230,7 +227,7 @@ public class TriggersTest
         public Collection<Mutation> augment(Partition partition)
         {
 
-            RowUpdateBuilder update = new RowUpdateBuilder(Schema.instance.getCFMetaData(ksName, otherCf), FBUtilities.timestampMicros(), partition.partitionKey().getKey());
+            RowUpdateBuilder update = new RowUpdateBuilder(Schema.instance.getTableMetadata(ksName, otherCf), FBUtilities.timestampMicros(), partition.partitionKey().getKey());
             update.add("v2", 999);
 
             return Collections.singletonList(update.build());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
index 41a0d6f..89fd5f9 100644
--- a/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
+++ b/tools/stress/src/org/apache/cassandra/io/sstable/StressCQLSSTableWriter.java
@@ -24,12 +24,15 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.stream.Collectors;
 
+import org.apache.commons.lang3.ArrayUtils;
+
 import com.datastax.driver.core.ProtocolVersion;
 import com.datastax.driver.core.TypeCodec;
 import org.antlr.runtime.RecognitionException;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.CQLFragmentParser;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.CqlParser;
@@ -51,6 +54,7 @@ import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.schema.Types;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -246,7 +250,7 @@ public class StressCQLSSTableWriter implements Closeable
         long now = System.currentTimeMillis() * 1000;
         // Note that we asks indexes to not validate values (the last 'false' arg below) because that triggers a 'Keyspace.open'
         // and that forces a lot of initialization that we don't want.
-        UpdateParameters params = new UpdateParameters(insert.cfm,
+        UpdateParameters params = new UpdateParameters(insert.metadata(),
                                                        insert.updatedColumns(),
                                                        options,
                                                        insert.getTimestamp(now, options),
@@ -307,7 +311,7 @@ public class StressCQLSSTableWriter implements Closeable
      */
     public com.datastax.driver.core.UserType getUDType(String dataType)
     {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(insert.keyspace());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(insert.keyspace());
         UserType userType = ksm.types.getNullable(ByteBufferUtil.bytes(dataType));
         return (com.datastax.driver.core.UserType) UDHelper.driverType(userType);
     }
@@ -579,13 +583,13 @@ public class StressCQLSSTableWriter implements Closeable
 
         private static void createTypes(String keyspace, List<CreateTypeStatement> typeStatements)
         {
-            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
+            KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
             Types.RawBuilder builder = Types.rawBuilder(keyspace);
             for (CreateTypeStatement st : typeStatements)
                 st.addToRawBuilder(builder);
 
             ksm = ksm.withSwapped(builder.build());
-            Schema.instance.setKeyspaceMetadata(ksm);
+            Schema.instance.load(ksm);
         }
 
         public static ColumnFamilyStore createOfflineTable(String schema, List<File> directoryList)
@@ -601,38 +605,40 @@ public class StressCQLSSTableWriter implements Closeable
         {
             String keyspace = schemaStatement.keyspace();
 
-            if (Schema.instance.getKSMetaData(keyspace) == null)
+            if (Schema.instance.getKeyspaceMetadata(keyspace) == null)
                 Schema.instance.load(KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1)));
 
             createTypes(keyspace, typeStatements);
 
-            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
+            KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
 
-            CFMetaData cfMetaData = ksm.tables.getNullable(schemaStatement.columnFamily());
-            assert cfMetaData == null;
+            assert ksm.tables.getNullable(schemaStatement.columnFamily()) == null;
 
             CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(ksm.types).statement;
             statement.validate(ClientState.forInternalCalls());
 
-            //Build metatdata with a portable cfId
-            cfMetaData = statement.metadataBuilder()
-                                  .withId(CFMetaData.generateLegacyCfId(keyspace, statement.columnFamily()))
-                                  .build()
-                                  .params(statement.params());
+            //Build metadata with a portable tableId
+            TableMetadata tableMetadata = statement.builder()
+                                                   .id(deterministicId(statement.keyspace(), statement.columnFamily()))
+                                                   .build();
 
             Keyspace.setInitialized();
-            Directories directories = new Directories(cfMetaData, directoryList.stream().map(Directories.DataDirectory::new).collect(Collectors.toList()));
+            Directories directories = new Directories(tableMetadata, directoryList.stream().map(Directories.DataDirectory::new).collect(Collectors.toList()));
 
             Keyspace ks = Keyspace.openWithoutSSTables(keyspace);
-            ColumnFamilyStore cfs =  ColumnFamilyStore.createColumnFamilyStore(ks, cfMetaData.cfName, cfMetaData, directories, false, false, true);
+            ColumnFamilyStore cfs =  ColumnFamilyStore.createColumnFamilyStore(ks, tableMetadata.name, TableMetadataRef.forOfflineTools(tableMetadata), directories, false, false, true);
 
             ks.initCfCustom(cfs);
-            Schema.instance.load(cfs.metadata);
-            Schema.instance.setKeyspaceMetadata(ksm.withSwapped(ksm.tables.with(cfs.metadata)));
+            Schema.instance.load(ksm.withSwapped(ksm.tables.with(cfs.metadata())));
 
             return cfs;
         }
 
+        private static TableId deterministicId(String keyspace, String table)
+        {
+            return TableId.fromUUID(UUID.nameUUIDFromBytes(ArrayUtils.addAll(keyspace.getBytes(), table.getBytes())));
+        }
+
         /**
          * Prepares insert statement for writing data to SSTable
          *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressProfile.java b/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
index e15e0ba..3662632 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
@@ -35,16 +35,16 @@ import com.google.common.base.Function;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import com.datastax.driver.core.*;
+import com.datastax.driver.core.TableMetadata;
 import com.datastax.driver.core.exceptions.AlreadyExistsException;
 import org.antlr.runtime.RecognitionException;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.CQLFragmentParser;
 import org.apache.cassandra.cql3.CqlParser;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.stress.generate.*;
 import org.apache.cassandra.stress.generate.values.*;
 import org.apache.cassandra.stress.operations.userdefined.TokenRangeQuery;
@@ -300,7 +300,7 @@ public class StressProfile implements Serializable
                     throw new RuntimeException("Unable to find table " + keyspaceName + "." + tableName);
 
                 //Fill in missing column configs
-                for (ColumnMetadata col : metadata.getColumns())
+                for (com.datastax.driver.core.ColumnMetadata col : metadata.getColumns())
                 {
                     if (columnConfigs.containsKey(col.getName()))
                         continue;
@@ -393,31 +393,31 @@ public class StressProfile implements Serializable
 
     public PartitionGenerator getOfflineGenerator()
     {
-        CFMetaData cfMetaData = CFMetaData.compile(tableCql, keyspaceName);
+        org.apache.cassandra.schema.TableMetadata metadata = CreateTableStatement.parse(tableCql, keyspaceName).build();
 
         //Add missing column configs
-        Iterator<ColumnDefinition> it = cfMetaData.allColumnsInSelectOrder();
+        Iterator<ColumnMetadata> it = metadata.allColumnsInSelectOrder();
         while (it.hasNext())
         {
-            ColumnDefinition c = it.next();
+            ColumnMetadata c = it.next();
             if (!columnConfigs.containsKey(c.name.toString()))
                 columnConfigs.put(c.name.toString(), new GeneratorConfig(seedStr + c.name.toString(), null, null, null));
         }
 
-        List<Generator> partitionColumns = cfMetaData.partitionKeyColumns().stream()
-                                                     .map(c -> new ColumnInfo(c.name.toString(), c.type.asCQL3Type().toString(), "", columnConfigs.get(c.name.toString())))
-                                                     .map(c -> c.getGenerator())
-                                                     .collect(Collectors.toList());
+        List<Generator> partitionColumns = metadata.partitionKeyColumns().stream()
+                                                   .map(c -> new ColumnInfo(c.name.toString(), c.type.asCQL3Type().toString(), "", columnConfigs.get(c.name.toString())))
+                                                   .map(c -> c.getGenerator())
+                                                   .collect(Collectors.toList());
 
-        List<Generator> clusteringColumns = cfMetaData.clusteringColumns().stream()
-                                                             .map(c -> new ColumnInfo(c.name.toString(), c.type.asCQL3Type().toString(), "", columnConfigs.get(c.name.toString())))
-                                                             .map(c -> c.getGenerator())
-                                                             .collect(Collectors.toList());
+        List<Generator> clusteringColumns = metadata.clusteringColumns().stream()
+                                                    .map(c -> new ColumnInfo(c.name.toString(), c.type.asCQL3Type().toString(), "", columnConfigs.get(c.name.toString())))
+                                                    .map(c -> c.getGenerator())
+                                                    .collect(Collectors.toList());
 
-        List<Generator> regularColumns = com.google.common.collect.Lists.newArrayList(cfMetaData.partitionColumns().selectOrderIterator()).stream()
-                                                                                                             .map(c -> new ColumnInfo(c.name.toString(), c.type.asCQL3Type().toString(), "", columnConfigs.get(c.name.toString())))
-                                                                                                             .map(c -> c.getGenerator())
-                                                                                                             .collect(Collectors.toList());
+        List<Generator> regularColumns = com.google.common.collect.Lists.newArrayList(metadata.regularAndStaticColumns().selectOrderIterator()).stream()
+                                                                        .map(c -> new ColumnInfo(c.name.toString(), c.type.asCQL3Type().toString(), "", columnConfigs.get(c.name.toString())))
+                                                                        .map(c -> c.getGenerator())
+                                                                        .collect(Collectors.toList());
 
         return new PartitionGenerator(partitionColumns, clusteringColumns, regularColumns, PartitionGenerator.Order.ARBITRARY);
     }
@@ -434,14 +434,14 @@ public class StressProfile implements Serializable
     {
         assert tableCql != null;
 
-        CFMetaData cfMetaData = CFMetaData.compile(tableCql, keyspaceName);
+        org.apache.cassandra.schema.TableMetadata metadata = CreateTableStatement.parse(tableCql, keyspaceName).build();
 
-        List<ColumnDefinition> allColumns = com.google.common.collect.Lists.newArrayList(cfMetaData.allColumnsInSelectOrder());
+        List<ColumnMetadata> allColumns = com.google.common.collect.Lists.newArrayList(metadata.allColumnsInSelectOrder());
 
         StringBuilder sb = new StringBuilder();
         sb.append("INSERT INTO ").append(quoteIdentifier(keyspaceName)).append(".").append(quoteIdentifier(tableName)).append(" (");
         StringBuilder value = new StringBuilder();
-        for (ColumnDefinition c : allColumns)
+        for (ColumnMetadata c : allColumns)
         {
             sb.append(quoteIdentifier(c.name.toString())).append(", ");
             value.append("?, ");
@@ -481,8 +481,8 @@ public class StressProfile implements Serializable
                 {
                     maybeLoadSchemaInfo(settings);
 
-                    Set<ColumnMetadata> keyColumns = com.google.common.collect.Sets.newHashSet(tableMetaData.getPrimaryKey());
-                    Set<ColumnMetadata> allColumns = com.google.common.collect.Sets.newHashSet(tableMetaData.getColumns());
+                    Set<com.datastax.driver.core.ColumnMetadata> keyColumns = com.google.common.collect.Sets.newHashSet(tableMetaData.getPrimaryKey());
+                    Set<com.datastax.driver.core.ColumnMetadata> allColumns = com.google.common.collect.Sets.newHashSet(tableMetaData.getColumns());
                     boolean isKeyOnlyTable = (keyColumns.size() == allColumns.size());
                     //With compact storage
                     if (!isKeyOnlyTable && (keyColumns.size() == (allColumns.size() - 1)))
@@ -490,7 +490,7 @@ public class StressProfile implements Serializable
                         com.google.common.collect.Sets.SetView diff = com.google.common.collect.Sets.difference(allColumns, keyColumns);
                         for (Object obj : diff)
                         {
-                            ColumnMetadata col = (ColumnMetadata)obj;
+                            com.datastax.driver.core.ColumnMetadata col = (com.datastax.driver.core.ColumnMetadata)obj;
                             isKeyOnlyTable = col.getName().isEmpty();
                             break;
                         }
@@ -507,7 +507,7 @@ public class StressProfile implements Serializable
 
                         boolean firstCol = true;
                         boolean firstPred = true;
-                        for (ColumnMetadata c : tableMetaData.getColumns()) {
+                        for (com.datastax.driver.core.ColumnMetadata c : tableMetaData.getColumns()) {
 
                             if (keyColumns.contains(c)) {
                                 if (firstPred)
@@ -545,7 +545,7 @@ public class StressProfile implements Serializable
                     {
                         sb.append("INSERT INTO ").append(quoteIdentifier(tableName)).append(" (");
                         StringBuilder value = new StringBuilder();
-                        for (ColumnMetadata c : tableMetaData.getPrimaryKey())
+                        for (com.datastax.driver.core.ColumnMetadata c : tableMetaData.getPrimaryKey())
                         {
                             sb.append(quoteIdentifier(c.getName())).append(", ");
                             value.append("?, ");
@@ -655,17 +655,17 @@ public class StressProfile implements Serializable
 
         private GeneratorFactory()
         {
-            Set<ColumnMetadata> keyColumns = com.google.common.collect.Sets.newHashSet(tableMetaData.getPrimaryKey());
+            Set<com.datastax.driver.core.ColumnMetadata> keyColumns = com.google.common.collect.Sets.newHashSet(tableMetaData.getPrimaryKey());
 
-            for (ColumnMetadata metadata : tableMetaData.getPartitionKey())
+            for (com.datastax.driver.core.ColumnMetadata metadata : tableMetaData.getPartitionKey())
                 partitionKeys.add(new ColumnInfo(metadata.getName(), metadata.getType().getName().toString(),
                                                  metadata.getType().isCollection() ? metadata.getType().getTypeArguments().get(0).getName().toString() : "",
                                                  columnConfigs.get(metadata.getName())));
-            for (ColumnMetadata metadata : tableMetaData.getClusteringColumns())
+            for (com.datastax.driver.core.ColumnMetadata metadata : tableMetaData.getClusteringColumns())
                 clusteringColumns.add(new ColumnInfo(metadata.getName(), metadata.getType().getName().toString(),
                                                      metadata.getType().isCollection() ? metadata.getType().getTypeArguments().get(0).getName().toString() : "",
                                                      columnConfigs.get(metadata.getName())));
-            for (ColumnMetadata metadata : tableMetaData.getColumns())
+            for (com.datastax.driver.core.ColumnMetadata metadata : tableMetaData.getColumns())
                 if (!keyColumns.contains(metadata))
                     valueColumns.add(new ColumnInfo(metadata.getName(), metadata.getType().getName().toString(),
                                                     metadata.getType().isCollection() ? metadata.getType().getTypeArguments().get(0).getName().toString() : "",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/tools/stress/src/org/apache/cassandra/stress/settings/OptionCompaction.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionCompaction.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionCompaction.java
index 11d5403..c90a14c 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/OptionCompaction.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionCompaction.java
@@ -23,8 +23,8 @@ import java.util.Map;
 
 import com.google.common.base.Function;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.CompactionParams;
 
 /**
  * For specifying replication options
@@ -67,8 +67,9 @@ class OptionCompaction extends OptionMulti
         {
             try
             {
-                CFMetaData.createCompactionStrategy(name);
-            } catch (ConfigurationException e)
+                CompactionParams.classFromName(name);
+            }
+            catch (ConfigurationException e)
             {
                 throw new IllegalArgumentException("Invalid compaction strategy: " + name);
             }


[25/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index d917884..0716d47 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -37,6 +37,9 @@ import org.apache.cassandra.db.commitlog.CommitLog.Configuration;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.CLibrary;
 import org.apache.cassandra.utils.IntegerInterval;
 import org.apache.cassandra.utils.concurrent.OpOrder;
@@ -100,10 +103,10 @@ public abstract class CommitLogSegment
     private final WaitQueue syncComplete = new WaitQueue();
 
     // a map of Cf->dirty interval in this segment; if interval is not covered by the clean set, the log contains unflushed data
-    private final NonBlockingHashMap<UUID, IntegerInterval> cfDirty = new NonBlockingHashMap<>(1024);
+    private final NonBlockingHashMap<TableId, IntegerInterval> tableDirty = new NonBlockingHashMap<>(1024);
 
     // a map of Cf->clean intervals; separate map from above to permit marking Cfs clean whilst the log is still in use
-    private final ConcurrentHashMap<UUID, IntegerInterval.Set> cfClean = new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<TableId, IntegerInterval.Set> tableClean = new ConcurrentHashMap<>();
 
     public final long id;
 
@@ -475,27 +478,27 @@ public abstract class CommitLogSegment
     void markDirty(Mutation mutation, int allocatedPosition)
     {
         for (PartitionUpdate update : mutation.getPartitionUpdates())
-            coverInMap(cfDirty, update.metadata().cfId, allocatedPosition);
+            coverInMap(tableDirty, update.metadata().id, allocatedPosition);
     }
 
     /**
-     * Marks the ColumnFamily specified by cfId as clean for this log segment. If the
+     * Marks the ColumnFamily specified by id as clean for this log segment. If the
      * given context argument is contained in this file, it will only mark the CF as
      * clean if no newer writes have taken place.
      *
-     * @param cfId           the column family ID that is now clean
+     * @param tableId        the table that is now clean
      * @param startPosition  the start of the range that is clean
      * @param endPosition    the end of the range that is clean
      */
-    public synchronized void markClean(UUID cfId, CommitLogPosition startPosition, CommitLogPosition endPosition)
+    public synchronized void markClean(TableId tableId, CommitLogPosition startPosition, CommitLogPosition endPosition)
     {
         if (startPosition.segmentId > id || endPosition.segmentId < id)
             return;
-        if (!cfDirty.containsKey(cfId))
+        if (!tableDirty.containsKey(tableId))
             return;
         int start = startPosition.segmentId == id ? startPosition.position : 0;
         int end = endPosition.segmentId == id ? endPosition.position : Integer.MAX_VALUE;
-        cfClean.computeIfAbsent(cfId, k -> new IntegerInterval.Set()).add(start, end);
+        tableClean.computeIfAbsent(tableId, k -> new IntegerInterval.Set()).add(start, end);
         removeCleanFromDirty();
     }
 
@@ -505,16 +508,16 @@ public abstract class CommitLogSegment
         if (isStillAllocating())
             return;
 
-        Iterator<Map.Entry<UUID, IntegerInterval.Set>> iter = cfClean.entrySet().iterator();
+        Iterator<Map.Entry<TableId, IntegerInterval.Set>> iter = tableClean.entrySet().iterator();
         while (iter.hasNext())
         {
-            Map.Entry<UUID, IntegerInterval.Set> clean = iter.next();
-            UUID cfId = clean.getKey();
+            Map.Entry<TableId, IntegerInterval.Set> clean = iter.next();
+            TableId tableId = clean.getKey();
             IntegerInterval.Set cleanSet = clean.getValue();
-            IntegerInterval dirtyInterval = cfDirty.get(cfId);
+            IntegerInterval dirtyInterval = tableDirty.get(tableId);
             if (dirtyInterval != null && cleanSet.covers(dirtyInterval))
             {
-                cfDirty.remove(cfId);
+                tableDirty.remove(tableId);
                 iter.remove();
             }
         }
@@ -523,17 +526,17 @@ public abstract class CommitLogSegment
     /**
      * @return a collection of dirty CFIDs for this segment file.
      */
-    public synchronized Collection<UUID> getDirtyCFIDs()
+    public synchronized Collection<TableId> getDirtyTableIds()
     {
-        if (cfClean.isEmpty() || cfDirty.isEmpty())
-            return cfDirty.keySet();
+        if (tableClean.isEmpty() || tableDirty.isEmpty())
+            return tableDirty.keySet();
 
-        List<UUID> r = new ArrayList<>(cfDirty.size());
-        for (Map.Entry<UUID, IntegerInterval> dirty : cfDirty.entrySet())
+        List<TableId> r = new ArrayList<>(tableDirty.size());
+        for (Map.Entry<TableId, IntegerInterval> dirty : tableDirty.entrySet())
         {
-            UUID cfId = dirty.getKey();
+            TableId tableId = dirty.getKey();
             IntegerInterval dirtyInterval = dirty.getValue();
-            IntegerInterval.Set cleanSet = cfClean.get(cfId);
+            IntegerInterval.Set cleanSet = tableClean.get(tableId);
             if (cleanSet == null || !cleanSet.covers(dirtyInterval))
                 r.add(dirty.getKey());
         }
@@ -546,12 +549,12 @@ public abstract class CommitLogSegment
     public synchronized boolean isUnused()
     {
         // if room to allocate, we're still in use as the active allocatingFrom,
-        // so we don't want to race with updates to cfClean with removeCleanFromDirty
+        // so we don't want to race with updates to tableClean with removeCleanFromDirty
         if (isStillAllocating())
             return false;
 
         removeCleanFromDirty();
-        return cfDirty.isEmpty();
+        return tableDirty.isEmpty();
     }
 
     /**
@@ -569,12 +572,12 @@ public abstract class CommitLogSegment
     public String dirtyString()
     {
         StringBuilder sb = new StringBuilder();
-        for (UUID cfId : getDirtyCFIDs())
+        for (TableId tableId : getDirtyTableIds())
         {
-            CFMetaData m = Schema.instance.getCFMetaData(cfId);
-            sb.append(m == null ? "<deleted>" : m.cfName).append(" (").append(cfId)
-              .append(", dirty: ").append(cfDirty.get(cfId))
-              .append(", clean: ").append(cfClean.get(cfId))
+            TableMetadata m = Schema.instance.getTableMetadata(tableId);
+            sb.append(m == null ? "<deleted>" : m.name).append(" (").append(tableId)
+              .append(", dirty: ").append(tableDirty.get(tableId))
+              .append(", clean: ").append(tableClean.get(tableId))
               .append("), ");
         }
         return sb.toString();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java b/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java
index 344fa58..d3235bc 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java
@@ -22,22 +22,22 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 
 /** Implements serializable to allow structured info to be returned via JMX. */
 public final class CompactionInfo implements Serializable
 {
     private static final long serialVersionUID = 3695381572726744816L;
-    private final CFMetaData cfm;
+    private final TableMetadata metadata;
     private final OperationType tasktype;
     private final long completed;
     private final long total;
     private final String unit;
     private final UUID compactionId;
 
-    public CompactionInfo(CFMetaData cfm, OperationType tasktype, long bytesComplete, long totalBytes, UUID compactionId)
+    public CompactionInfo(TableMetadata metadata, OperationType tasktype, long bytesComplete, long totalBytes, UUID compactionId)
     {
-        this(cfm, tasktype, bytesComplete, totalBytes, "bytes", compactionId);
+        this(metadata, tasktype, bytesComplete, totalBytes, "bytes", compactionId);
     }
 
     public CompactionInfo(OperationType tasktype, long completed, long total, String unit, UUID compactionId)
@@ -45,12 +45,12 @@ public final class CompactionInfo implements Serializable
         this(null, tasktype, completed, total, unit, compactionId);
     }
 
-    public CompactionInfo(CFMetaData cfm, OperationType tasktype, long completed, long total, String unit, UUID compactionId)
+    public CompactionInfo(TableMetadata metadata, OperationType tasktype, long completed, long total, String unit, UUID compactionId)
     {
         this.tasktype = tasktype;
         this.completed = completed;
         this.total = total;
-        this.cfm = cfm;
+        this.metadata = metadata;
         this.unit = unit;
         this.compactionId = compactionId;
     }
@@ -58,27 +58,27 @@ public final class CompactionInfo implements Serializable
     /** @return A copy of this CompactionInfo with updated progress. */
     public CompactionInfo forProgress(long complete, long total)
     {
-        return new CompactionInfo(cfm, tasktype, complete, total, unit, compactionId);
+        return new CompactionInfo(metadata, tasktype, complete, total, unit, compactionId);
     }
 
     public UUID getId()
     {
-        return cfm != null ? cfm.cfId : null;
+        return metadata != null ? metadata.id.asUUID() : null;
     }
 
     public String getKeyspace()
     {
-        return cfm != null ? cfm.ksName : null;
+        return metadata != null ? metadata.keyspace : null;
     }
 
     public String getColumnFamily()
     {
-        return cfm != null ? cfm.cfName : null;
+        return metadata != null ? metadata.name : null;
     }
 
-    public CFMetaData getCFMetaData()
+    public TableMetadata getTableMetadata()
     {
-        return cfm;
+        return metadata;
     }
 
     public long getCompleted()
@@ -105,7 +105,7 @@ public final class CompactionInfo implements Serializable
     {
         StringBuilder buff = new StringBuilder();
         buff.append(getTaskType());
-        if (cfm != null)
+        if (metadata != null)
         {
             buff.append('@').append(getId()).append('(');
             buff.append(getKeyspace()).append(", ").append(getColumnFamily()).append(", ");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
index 19f4801..8e94fd9 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
@@ -22,7 +22,7 @@ import java.util.function.Predicate;
 
 import com.google.common.collect.Ordering;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.partitions.PurgeFunction;
@@ -101,20 +101,20 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
             metrics.beginCompaction(this);
 
         UnfilteredPartitionIterator merged = scanners.isEmpty()
-                                             ? EmptyIterators.unfilteredPartition(controller.cfs.metadata)
-                                             : UnfilteredPartitionIterators.merge(scanners, nowInSec, listener());
+                                           ? EmptyIterators.unfilteredPartition(controller.cfs.metadata())
+                                           : UnfilteredPartitionIterators.merge(scanners, nowInSec, listener());
         merged = Transformation.apply(merged, new GarbageSkipper(controller, nowInSec));
         this.compacted = Transformation.apply(merged, new Purger(controller, nowInSec));
     }
 
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
-        return controller.cfs.metadata;
+        return controller.cfs.metadata();
     }
 
     public CompactionInfo getCompactionInfo()
     {
-        return new CompactionInfo(controller.cfs.metadata,
+        return new CompactionInfo(controller.cfs.metadata(),
                                   type,
                                   bytesRead,
                                   totalBytes,
@@ -167,7 +167,7 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
                         regulars = regulars.mergeTo(iter.columns().regulars);
                     }
                 }
-                final PartitionColumns partitionColumns = new PartitionColumns(statics, regulars);
+                final RegularAndStaticColumns regularAndStaticColumns = new RegularAndStaticColumns(statics, regulars);
 
                 // If we have a 2ndary index, we must update it with deleted/shadowed cells.
                 // we can reuse a single CleanupTransaction for the duration of a partition.
@@ -181,7 +181,7 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
                 // TODO: this should probably be done asynchronously and batched.
                 final CompactionTransaction indexTransaction =
                     controller.cfs.indexManager.newCompactionTransaction(partitionKey,
-                                                                         partitionColumns,
+                                                                         regularAndStaticColumns,
                                                                          versions.size(),
                                                                          nowInSec);
 
@@ -320,7 +320,7 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
         final Row staticRow;
         final ColumnFilter cf;
         final int nowInSec;
-        final CFMetaData metadata;
+        final TableMetadata metadata;
         final boolean cellLevelGC;
 
         DeletionTime tombOpenDeletionTime = DeletionTime.LIVE;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 9c74f62..91263a7 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -31,7 +31,6 @@ import javax.management.openmbean.TabularData;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
-import com.google.common.primitives.Ints;
 import com.google.common.util.concurrent.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -41,9 +40,9 @@ import org.apache.cassandra.cache.AutoSavingCache;
 import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.CompactionInfo.Holder;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
@@ -794,7 +793,7 @@ public class CompactionManager implements CompactionManagerMBean
         {
             // extract keyspace and columnfamily name from filename
             Descriptor desc = Descriptor.fromFilename(filename.trim());
-            if (Schema.instance.getCFMetaData(desc) == null)
+            if (Schema.instance.getTableMetadataRef(desc) == null)
             {
                 logger.warn("Schema does not exist for file {}. Skipping.", filename);
                 continue;
@@ -820,7 +819,7 @@ public class CompactionManager implements CompactionManagerMBean
         {
             // extract keyspace and columnfamily name from filename
             Descriptor desc = Descriptor.fromFilename(filename.trim());
-            if (Schema.instance.getCFMetaData(desc) == null)
+            if (Schema.instance.getTableMetadataRef(desc) == null)
             {
                 logger.warn("Schema does not exist for file {}. Skipping.", filename);
                 continue;
@@ -1072,7 +1071,7 @@ public class CompactionManager implements CompactionManagerMBean
 
         long totalkeysWritten = 0;
 
-        long expectedBloomFilterSize = Math.max(cfs.metadata.params.minIndexInterval,
+        long expectedBloomFilterSize = Math.max(cfs.metadata().params.minIndexInterval,
                                                SSTableReader.getApproximateKeyCount(txn.originals()));
         if (logger.isTraceEnabled())
             logger.trace("Expected bloom filter size : {}", expectedBloomFilterSize);
@@ -1241,16 +1240,13 @@ public class CompactionManager implements CompactionManagerMBean
                                              LifecycleTransaction txn)
     {
         FileUtils.createDirectory(compactionFileLocation);
-        SerializationHeader header = sstable.header;
-        if (header == null)
-            header = SerializationHeader.make(sstable.metadata, Collections.singleton(sstable));
 
         return SSTableWriter.create(cfs.metadata,
                                     cfs.newSSTableDescriptor(compactionFileLocation),
                                     expectedBloomFilterSize,
                                     repairedAt,
                                     sstable.getSSTableLevel(),
-                                    header,
+                                    sstable.header,
                                     cfs.indexManager.listIndexes(),
                                     txn);
     }
@@ -1282,8 +1278,8 @@ public class CompactionManager implements CompactionManagerMBean
                                     (long) expectedBloomFilterSize,
                                     repairedAt,
                                     cfs.metadata,
-                                    new MetadataCollector(sstables, cfs.metadata.comparator, minLevel),
-                                    SerializationHeader.make(cfs.metadata, sstables),
+                                    new MetadataCollector(sstables, cfs.metadata().comparator, minLevel),
+                                    SerializationHeader.make(cfs.metadata(), sstables),
                                     cfs.indexManager.listIndexes(),
                                     txn);
     }
@@ -1435,7 +1431,7 @@ public class CompactionManager implements CompactionManagerMBean
             return null;
         Set<SSTableReader> sstablesToValidate = new HashSet<>();
         if (prs.isGlobal)
-            prs.markSSTablesRepairing(cfs.metadata.cfId, validator.desc.parentSessionId);
+            prs.markSSTablesRepairing(cfs.metadata.id, validator.desc.parentSessionId);
         // note that we always grab all existing sstables for this - if we were to just grab the ones that
         // were marked as repairing, we would miss any ranges that were compacted away and this would cause us to overstream
         try (ColumnFamilyStore.RefViewFragment sstableCandidates = cfs.selectAndReference(View.select(SSTableSet.CANONICAL, (s) -> !prs.isIncremental || !s.isRepaired())))
@@ -1522,7 +1518,7 @@ public class CompactionManager implements CompactionManagerMBean
              CompactionController controller = new CompactionController(cfs, sstableAsSet, getDefaultGcBefore(cfs, nowInSec));
              CompactionIterator ci = new CompactionIterator(OperationType.ANTICOMPACTION, scanners.scanners, controller, nowInSec, UUIDGen.getTimeUUID(), metrics))
         {
-            int expectedBloomFilterSize = Math.max(cfs.metadata.params.minIndexInterval, (int)(SSTableReader.getApproximateKeyCount(sstableAsSet)));
+            int expectedBloomFilterSize = Math.max(cfs.metadata().params.minIndexInterval, (int)(SSTableReader.getApproximateKeyCount(sstableAsSet)));
 
             repairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, repairedAt, sstableAsSet, anticompactionGroup));
             unRepairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, sstableAsSet, anticompactionGroup));
@@ -1990,7 +1986,7 @@ public class CompactionManager implements CompactionManagerMBean
      * @param interruptValidation true if validation operations for repair should also be interrupted
      *
      */
-    public void interruptCompactionFor(Iterable<CFMetaData> columnFamilies, boolean interruptValidation)
+    public void interruptCompactionFor(Iterable<TableMetadata> columnFamilies, boolean interruptValidation)
     {
         assert columnFamilies != null;
 
@@ -2001,16 +1997,16 @@ public class CompactionManager implements CompactionManagerMBean
             if ((info.getTaskType() == OperationType.VALIDATION) && !interruptValidation)
                 continue;
 
-            if (Iterables.contains(columnFamilies, info.getCFMetaData()))
+            if (Iterables.contains(columnFamilies, info.getTableMetadata()))
                 compactionHolder.stop(); // signal compaction to stop
         }
     }
 
     public void interruptCompactionForCFs(Iterable<ColumnFamilyStore> cfss, boolean interruptValidation)
     {
-        List<CFMetaData> metadata = new ArrayList<>();
+        List<TableMetadata> metadata = new ArrayList<>();
         for (ColumnFamilyStore cfs : cfss)
-            metadata.add(cfs.metadata);
+            metadata.add(cfs.metadata());
 
         interruptCompactionFor(metadata, interruptValidation);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index 5679338..71b160a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -25,13 +25,14 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.index.Index;
 import com.google.common.primitives.Ints;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Memtable;
@@ -88,8 +89,8 @@ public class CompactionStrategyManager implements INotificationConsumer
         logger.trace("{} subscribed to the data tracker.", this);
         this.cfs = cfs;
         this.compactionLogger = new CompactionLogger(cfs, this);
-        reload(cfs.metadata);
-        params = cfs.metadata.params.compaction;
+        reload(cfs.metadata());
+        params = cfs.metadata().params.compaction;
         locations = getDirectories().getWriteableLocations();
         enabled = params.isEnabled();
     }
@@ -108,7 +109,7 @@ public class CompactionStrategyManager implements INotificationConsumer
             if (!isEnabled())
                 return null;
 
-            maybeReload(cfs.metadata);
+            maybeReload(cfs.metadata());
             List<AbstractCompactionStrategy> strategies = new ArrayList<>();
 
             strategies.addAll(repaired);
@@ -270,7 +271,7 @@ public class CompactionStrategyManager implements INotificationConsumer
         }
     }
 
-    public void maybeReload(CFMetaData metadata)
+    public void maybeReload(TableMetadata metadata)
     {
         // compare the old schema configuration to the new one, ignore any locally set changes.
         if (metadata.params.compaction.equals(schemaCompactionParams) &&
@@ -294,7 +295,7 @@ public class CompactionStrategyManager implements INotificationConsumer
      * Called after changing configuration and at startup.
      * @param metadata
      */
-    private void reload(CFMetaData metadata)
+    private void reload(TableMetadata metadata)
     {
         boolean disabledWithJMX = !enabled && shouldBeEnabled();
         if (!metadata.params.compaction.equals(schemaCompactionParams))
@@ -544,7 +545,7 @@ public class CompactionStrategyManager implements INotificationConsumer
 
     public void handleNotification(INotification notification, Object sender)
     {
-        maybeReload(cfs.metadata);
+        maybeReload(cfs.metadata());
         if (notification instanceof SSTableAddedNotification)
         {
             handleFlushNotification(((SSTableAddedNotification) notification).added);
@@ -691,7 +692,7 @@ public class CompactionStrategyManager implements INotificationConsumer
 
     public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes)
     {
-        maybeReload(cfs.metadata);
+        maybeReload(cfs.metadata());
         validateForCompaction(txn.originals(), cfs, getDirectories());
         return getCompactionStrategyFor(txn.originals().iterator().next()).getCompactionTask(txn, gcBefore, maxSSTableBytes);
     }
@@ -713,7 +714,7 @@ public class CompactionStrategyManager implements INotificationConsumer
 
     public Collection<AbstractCompactionTask> getMaximalTasks(final int gcBefore, final boolean splitOutput)
     {
-        maybeReload(cfs.metadata);
+        maybeReload(cfs.metadata());
         // runWithCompactionsDisabled cancels active compactions and disables them, then we are able
         // to make the repaired/unrepaired strategies mark their own sstables as compacting. Once the
         // sstables are marked the compactions are re-enabled
@@ -761,7 +762,7 @@ public class CompactionStrategyManager implements INotificationConsumer
      */
     public List<AbstractCompactionTask> getUserDefinedTasks(Collection<SSTableReader> sstables, int gcBefore)
     {
-        maybeReload(cfs.metadata);
+        maybeReload(cfs.metadata());
         List<AbstractCompactionTask> ret = new ArrayList<>();
         readLock.lock();
         try
@@ -882,14 +883,14 @@ public class CompactionStrategyManager implements INotificationConsumer
             locations = cfs.getDirectories().getWriteableLocations();
             for (int i = 0; i < locations.length; i++)
             {
-                repaired.add(CFMetaData.createCompactionStrategyInstance(cfs, params));
-                unrepaired.add(CFMetaData.createCompactionStrategyInstance(cfs, params));
+                repaired.add(cfs.createCompactionStrategyInstance(params));
+                unrepaired.add(cfs.createCompactionStrategyInstance(params));
             }
         }
         else
         {
-            repaired.add(CFMetaData.createCompactionStrategyInstance(cfs, params));
-            unrepaired.add(CFMetaData.createCompactionStrategyInstance(cfs, params));
+            repaired.add(cfs.createCompactionStrategyInstance(params));
+            unrepaired.add(cfs.createCompactionStrategyInstance(params));
         }
         this.params = params;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
index f29e6cb..b3395d0 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
@@ -28,7 +28,7 @@ import com.google.common.primitives.Doubles;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
@@ -299,7 +299,7 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
                     if (!intersecting.isEmpty())
                     {
                         @SuppressWarnings("resource") // The ScannerList will be in charge of closing (and we close properly on errors)
-                        ISSTableScanner scanner = new LeveledScanner(intersecting, ranges);
+                        ISSTableScanner scanner = new LeveledScanner(cfs.metadata(), intersecting, ranges);
                         scanners.add(scanner);
                     }
                 }
@@ -343,6 +343,7 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
     // same level (e.g. non overlapping) - see #4142
     private static class LeveledScanner extends AbstractIterator<UnfilteredRowIterator> implements ISSTableScanner
     {
+        private final TableMetadata metadata;
         private final Collection<Range<Token>> ranges;
         private final List<SSTableReader> sstables;
         private final Iterator<SSTableReader> sstableIterator;
@@ -353,8 +354,9 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
         private long positionOffset;
         private long totalBytesScanned = 0;
 
-        public LeveledScanner(Collection<SSTableReader> sstables, Collection<Range<Token>> ranges)
+        public LeveledScanner(TableMetadata metadata, Collection<SSTableReader> sstables, Collection<Range<Token>> ranges)
         {
+            this.metadata = metadata;
             this.ranges = ranges;
 
             // add only sstables that intersect our range, and estimate how much data that involves
@@ -402,9 +404,9 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
             return filtered;
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
-            return sstables.get(0).metadata; // The ctor checks we have at least one sstable
+            return metadata;
         }
 
         protected UnfilteredRowIterator computeNext()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 0007e30..e8eee9a 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -24,7 +24,7 @@ import java.util.*;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.rows.*;
@@ -92,7 +92,7 @@ public class Scrubber implements Closeable
         this.sstable = transaction.onlyOne();
         this.outputHandler = outputHandler;
         this.skipCorrupted = skipCorrupted;
-        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata,
+        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(cfs.metadata(),
                                                                                                         sstable.descriptor.version,
                                                                                                         sstable.header);
 
@@ -100,7 +100,7 @@ public class Scrubber implements Closeable
 
         int locIndex = CompactionStrategyManager.getCompactionStrategyIndex(cfs, cfs.getDirectories(), sstable);
         this.destination = cfs.getDirectories().getLocationForDisk(cfs.getDirectories().getWriteableLocations()[locIndex]);
-        this.isCommutative = cfs.metadata.isCounter();
+        this.isCommutative = cfs.metadata().isCounter();
 
         boolean hasIndexFile = (new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX))).exists();
         this.isIndex = cfs.isIndex();
@@ -111,7 +111,7 @@ public class Scrubber implements Closeable
         }
         this.checkData = checkData && !this.isIndex; //LocalByPartitionerType does not support validation
         this.expectedBloomFilterSize = Math.max(
-            cfs.metadata.params.minIndexInterval,
+            cfs.metadata().params.minIndexInterval,
             hasIndexFile ? SSTableReader.getApproximateKeyCount(toScrub) : 0);
 
         // loop through each row, deserializing to check for damage.
@@ -306,7 +306,7 @@ public class Scrubber implements Closeable
         // that one row is out of order, it will stop returning them. The remaining rows will be sorted and added
         // to the outOfOrder set that will be later written to a new SSTable.
         OrderCheckerIterator sstableIterator = new OrderCheckerIterator(new RowMergingSSTableIterator(SSTableIdentityIterator.create(sstable, dataFile, key)),
-                                                                        cfs.metadata.comparator);
+                                                                        cfs.metadata().comparator);
 
         try (UnfilteredRowIterator iterator = withValidation(sstableIterator, dataFile.getPath()))
         {
@@ -438,7 +438,7 @@ public class Scrubber implements Closeable
         {
             try
             {
-                return new CompactionInfo(sstable.metadata,
+                return new CompactionInfo(sstable.metadata(),
                                           OperationType.SCRUB,
                                           dataFile.getFilePointer(),
                                           dataFile.length(),
@@ -545,7 +545,7 @@ public class Scrubber implements Closeable
             this.comparator = comparator;
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
             return iterator.metadata();
         }
@@ -555,7 +555,7 @@ public class Scrubber implements Closeable
             return iterator.isReverseOrder();
         }
 
-        public PartitionColumns columns()
+        public RegularAndStaticColumns columns()
         {
             return iterator.columns();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/Upgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Upgrader.java b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
index aedb208..34ec1dd 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -61,7 +61,7 @@ public class Upgrader
         this.controller = new UpgradeController(cfs);
 
         this.strategyManager = cfs.getCompactionStrategyManager();
-        long estimatedTotalKeys = Math.max(cfs.metadata.params.minIndexInterval, SSTableReader.getApproximateKeyCount(Arrays.asList(this.sstable)));
+        long estimatedTotalKeys = Math.max(cfs.metadata().params.minIndexInterval, SSTableReader.getApproximateKeyCount(Arrays.asList(this.sstable)));
         long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(Arrays.asList(this.sstable)) / strategyManager.getMaxSSTableBytes());
         this.estimatedRows = (long) Math.ceil((double) estimatedTotalKeys / estimatedSSTables);
     }
@@ -75,7 +75,7 @@ public class Upgrader
                                     repairedAt,
                                     cfs.metadata,
                                     sstableMetadataCollector,
-                                    SerializationHeader.make(cfs.metadata, Sets.newHashSet(sstable)),
+                                    SerializationHeader.make(cfs.metadata(), Sets.newHashSet(sstable)),
                                     cfs.indexManager.listIndexes(),
                                     transaction);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/Verifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Verifier.java b/src/java/org/apache/cassandra/db/compaction/Verifier.java
index a52dd82..467d50d 100644
--- a/src/java/org/apache/cassandra/db/compaction/Verifier.java
+++ b/src/java/org/apache/cassandra/db/compaction/Verifier.java
@@ -73,7 +73,7 @@ public class Verifier implements Closeable
         this.cfs = cfs;
         this.sstable = sstable;
         this.outputHandler = outputHandler;
-        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata, sstable.descriptor.version, sstable.header);
+        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(cfs.metadata(), sstable.descriptor.version, sstable.header);
 
         this.controller = new VerifyController(cfs);
 
@@ -260,7 +260,7 @@ public class Verifier implements Closeable
         {
             try
             {
-                return new CompactionInfo(sstable.metadata,
+                return new CompactionInfo(sstable.metadata(),
                                           OperationType.VERIFY,
                                           dataFile.getFilePointer(),
                                           dataFile.length(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
index d279321..6f2586b 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
@@ -73,8 +73,8 @@ public class DefaultCompactionWriter extends CompactionAwareWriter
                                                     estimatedTotalKeys,
                                                     minRepairedAt,
                                                     cfs.metadata,
-                                                    new MetadataCollector(txn.originals(), cfs.metadata.comparator, sstableLevel),
-                                                    SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
+                                                    new MetadataCollector(txn.originals(), cfs.metadata().comparator, sstableLevel),
+                                                    SerializationHeader.make(cfs.metadata(), nonExpiredSSTables),
                                                     cfs.indexManager.listIndexes(),
                                                     txn);
         sstableWriter.switchWriter(writer);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
index a3d8c98..5eac658 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
@@ -109,8 +109,8 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
                 keysPerSSTable,
                 minRepairedAt,
                 cfs.metadata,
-                new MetadataCollector(txn.originals(), cfs.metadata.comparator, currentLevel),
-                SerializationHeader.make(cfs.metadata, txn.originals()),
+                new MetadataCollector(txn.originals(), cfs.metadata().comparator, currentLevel),
+                SerializationHeader.make(cfs.metadata(), txn.originals()),
                 cfs.indexManager.listIndexes(),
                 txn));
         partitionsWritten = 0;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
index 7acb870..862e68b 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
@@ -112,8 +112,8 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
                                                     estimatedTotalKeys / estimatedSSTables,
                                                     minRepairedAt,
                                                     cfs.metadata,
-                                                    new MetadataCollector(allSSTables, cfs.metadata.comparator, level),
-                                                    SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
+                                                    new MetadataCollector(allSSTables, cfs.metadata().comparator, level),
+                                                    SerializationHeader.make(cfs.metadata(), nonExpiredSSTables),
                                                     cfs.indexManager.listIndexes(),
                                                     txn);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
index a01672e..79f9d1a 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -108,8 +108,8 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
                                                     currentPartitionsToWrite,
                                                     minRepairedAt,
                                                     cfs.metadata,
-                                                    new MetadataCollector(allSSTables, cfs.metadata.comparator, 0),
-                                                    SerializationHeader.make(cfs.metadata, nonExpiredSSTables),
+                                                    new MetadataCollector(allSSTables, cfs.metadata().comparator, 0),
+                                                    SerializationHeader.make(cfs.metadata(), nonExpiredSSTables),
                                                     cfs.indexManager.listIndexes(),
                                                     txn);
         logger.trace("Switching writer, currentPartitionsToWrite = {}", currentPartitionsToWrite);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/filter/AbstractClusteringIndexFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/AbstractClusteringIndexFilter.java b/src/java/org/apache/cassandra/db/filter/AbstractClusteringIndexFilter.java
index 51e9d8e..c28117c 100644
--- a/src/java/org/apache/cassandra/db/filter/AbstractClusteringIndexFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/AbstractClusteringIndexFilter.java
@@ -19,8 +19,8 @@ package org.apache.cassandra.db.filter;
 
 import java.io.IOException;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.ReversedType;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -45,13 +45,13 @@ public abstract class AbstractClusteringIndexFilter implements ClusteringIndexFi
     protected abstract void serializeInternal(DataOutputPlus out, int version) throws IOException;
     protected abstract long serializedSizeInternal(int version);
 
-    protected void appendOrderByToCQLString(CFMetaData metadata, StringBuilder sb)
+    protected void appendOrderByToCQLString(TableMetadata metadata, StringBuilder sb)
     {
         if (reversed)
         {
             sb.append(" ORDER BY (");
             int i = 0;
-            for (ColumnDefinition column : metadata.clusteringColumns())
+            for (ColumnMetadata column : metadata.clusteringColumns())
                 sb.append(i++ == 0 ? "" : ", ").append(column.name).append(column.type instanceof ReversedType ? " ASC" : " DESC");
             sb.append(')');
         }
@@ -69,7 +69,7 @@ public abstract class AbstractClusteringIndexFilter implements ClusteringIndexFi
             filter.serializeInternal(out, version);
         }
 
-        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
+        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException
         {
             Kind kind = Kind.values()[in.readUnsignedByte()];
             boolean reversed = in.readBoolean();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
index f184035..cdb61c9 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexFilter.java
@@ -19,14 +19,14 @@ package org.apache.cassandra.db.filter;
 
 import java.io.IOException;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.CachedPartition;
 import org.apache.cassandra.db.partitions.Partition;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
 
 /**
  * A filter that selects a subset of the rows of a given partition by using the "clustering index".
@@ -54,7 +54,7 @@ public interface ClusteringIndexFilter
 
     static interface InternalDeserializer
     {
-        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, CFMetaData metadata, boolean reversed) throws IOException;
+        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, TableMetadata metadata, boolean reversed) throws IOException;
     }
 
     /**
@@ -114,10 +114,10 @@ public interface ClusteringIndexFilter
     /**
      * Returns an iterator that only returns the rows of the provided iterator that this filter selects.
      * <p>
-     * This method is the "dumb" counterpart to {@link #getSlices(CFMetaData)} in that it has no way to quickly get
+     * This method is the "dumb" counterpart to {@link #getSlices(TableMetadata)} in that it has no way to quickly get
      * to what is actually selected, so it simply iterate over it all and filters out what shouldn't be returned. This should
      * be avoided in general.
-     * Another difference with {@link #getSlices(CFMetaData)} is that this method also filter the queried
+     * Another difference with {@link #getSlices(TableMetadata)} is that this method also filter the queried
      * columns in the returned result, while the former assumes that the provided iterator has already done it.
      *
      * @param columnFilter the columns to include in the rows of the result iterator.
@@ -127,7 +127,7 @@ public interface ClusteringIndexFilter
      */
     public UnfilteredRowIterator filterNotIndexed(ColumnFilter columnFilter, UnfilteredRowIterator iterator);
 
-    public Slices getSlices(CFMetaData metadata);
+    public Slices getSlices(TableMetadata metadata);
 
     /**
      * Given a partition, returns a row iterator for the rows of this partition that are selected by this filter.
@@ -150,13 +150,13 @@ public interface ClusteringIndexFilter
 
     public Kind kind();
 
-    public String toString(CFMetaData metadata);
-    public String toCQLString(CFMetaData metadata);
+    public String toString(TableMetadata metadata);
+    public String toCQLString(TableMetadata metadata);
 
     public interface Serializer
     {
         public void serialize(ClusteringIndexFilter filter, DataOutputPlus out, int version) throws IOException;
-        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException;
+        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException;
         public long serializedSize(ClusteringIndexFilter filter, int version);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
index 6a010d9..2f7c13a 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
@@ -21,15 +21,15 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.transform.Transformation;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.SearchIterator;
 import org.apache.cassandra.utils.btree.BTreeSet;
 
@@ -126,7 +126,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
         return Transformation.apply(iterator, new FilterNotIndexed());
     }
 
-    public Slices getSlices(CFMetaData metadata)
+    public Slices getSlices(TableMetadata metadata)
     {
         Slices.Builder builder = new Slices.Builder(metadata.comparator, clusteringsInQueryOrder.size());
         for (Clustering clustering : clusteringsInQueryOrder)
@@ -138,12 +138,12 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
     {
         final SearchIterator<Clustering, Row> searcher = partition.searchIterator(columnFilter, reversed);
         return new AbstractUnfilteredRowIterator(partition.metadata(),
-                                        partition.partitionKey(),
-                                        partition.partitionLevelDeletion(),
-                                        columnFilter.fetchedColumns(),
-                                        searcher.next(Clustering.STATIC_CLUSTERING),
-                                        reversed,
-                                        partition.stats())
+                                                 partition.partitionKey(),
+                                                 partition.partitionLevelDeletion(),
+                                                 columnFilter.fetchedColumns(),
+                                                 searcher.next(Clustering.STATIC_CLUSTERING),
+                                                 reversed,
+                                                 partition.stats())
         {
             private final Iterator<Clustering> clusteringIter = clusteringsInQueryOrder.iterator();
 
@@ -162,7 +162,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
 
     public boolean shouldInclude(SSTableReader sstable)
     {
-        ClusteringComparator comparator = sstable.metadata.comparator;
+        ClusteringComparator comparator = sstable.metadata().comparator;
         List<ByteBuffer> minClusteringValues = sstable.getSSTableMetadata().minClusteringValues;
         List<ByteBuffer> maxClusteringValues = sstable.getSSTableMetadata().maxClusteringValues;
 
@@ -175,7 +175,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
         return false;
     }
 
-    public String toString(CFMetaData metadata)
+    public String toString(TableMetadata metadata)
     {
         StringBuilder sb = new StringBuilder();
         sb.append("names(");
@@ -187,13 +187,13 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
         return sb.append(')').toString();
     }
 
-    public String toCQLString(CFMetaData metadata)
+    public String toCQLString(TableMetadata metadata)
     {
         if (metadata.clusteringColumns().isEmpty() || clusterings.size() <= 1)
             return "";
 
         StringBuilder sb = new StringBuilder();
-        sb.append('(').append(ColumnDefinition.toCQLString(metadata.clusteringColumns())).append(')');
+        sb.append('(').append(ColumnMetadata.toCQLString(metadata.clusteringColumns())).append(')');
         sb.append(clusterings.size() == 1 ? " = " : " IN (");
         int i = 0;
         for (Clustering clustering : clusterings)
@@ -228,7 +228,7 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
 
     private static class NamesDeserializer implements InternalDeserializer
     {
-        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, CFMetaData metadata, boolean reversed) throws IOException
+        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, TableMetadata metadata, boolean reversed) throws IOException
         {
             ClusteringComparator comparator = metadata.comparator;
             BTreeSet.Builder<Clustering> clusterings = BTreeSet.builder(comparator);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
index 02a44d7..9490adf 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexSliceFilter.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.List;
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.CachedPartition;
@@ -109,7 +109,7 @@ public class ClusteringIndexSliceFilter extends AbstractClusteringIndexFilter
         return Transformation.apply(iterator, new FilterNotIndexed());
     }
 
-    public Slices getSlices(CFMetaData metadata)
+    public Slices getSlices(TableMetadata metadata)
     {
         return slices;
     }
@@ -130,12 +130,12 @@ public class ClusteringIndexSliceFilter extends AbstractClusteringIndexFilter
         return slices.intersects(minClusteringValues, maxClusteringValues);
     }
 
-    public String toString(CFMetaData metadata)
+    public String toString(TableMetadata metadata)
     {
         return String.format("slice(slices=%s, reversed=%b)", slices, reversed);
     }
 
-    public String toCQLString(CFMetaData metadata)
+    public String toCQLString(TableMetadata metadata)
     {
         StringBuilder sb = new StringBuilder();
 
@@ -164,7 +164,7 @@ public class ClusteringIndexSliceFilter extends AbstractClusteringIndexFilter
 
     private static class SliceDeserializer implements InternalDeserializer
     {
-        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, CFMetaData metadata, boolean reversed) throws IOException
+        public ClusteringIndexFilter deserialize(DataInputPlus in, int version, TableMetadata metadata, boolean reversed) throws IOException
         {
             Slices slices = Slices.serializer.deserialize(in, version, metadata);
             return new ClusteringIndexSliceFilter(slices, reversed);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
index b3ae505..6f6fc08 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
@@ -24,14 +24,14 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.SortedSetMultimap;
 import com.google.common.collect.TreeMultimap;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.CellPath;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 
 /**
  * Represents which (non-PK) columns (and optionally which sub-part of a column for complex columns) are selected
@@ -68,15 +68,15 @@ public class ColumnFilter
     // null. If false, then _fetched_ == _queried_ and we only store _queried_.
     private final boolean fetchAllRegulars;
 
-    private final CFMetaData metadata; // can be null if !fetchAllRegulars
+    private final TableMetadata metadata; // can be null if !isFetchAll
 
-    private final PartitionColumns queried; // can be null if fetchAllRegulars, to represent a wildcard query (all
+    private final RegularAndStaticColumns queried; // can be null if fetchAllRegulars, to represent a wildcard query (all
                                             // static and regular columns are both _fetched_ and _queried_).
     private final SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections; // can be null
 
     private ColumnFilter(boolean fetchAllRegulars,
-                         CFMetaData metadata,
-                         PartitionColumns queried,
+                         TableMetadata metadata,
+                         RegularAndStaticColumns queried,
                          SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections)
     {
         assert !fetchAllRegulars || metadata != null;
@@ -90,7 +90,7 @@ public class ColumnFilter
     /**
      * A filter that includes all columns for the provided table.
      */
-    public static ColumnFilter all(CFMetaData metadata)
+    public static ColumnFilter all(TableMetadata metadata)
     {
         return new ColumnFilter(true, metadata, null, null);
     }
@@ -102,7 +102,7 @@ public class ColumnFilter
      * preserve CQL semantic (see class javadoc). This is ok for some internal queries however (and
      * for #6588 if/when we implement it).
      */
-    public static ColumnFilter selection(PartitionColumns columns)
+    public static ColumnFilter selection(RegularAndStaticColumns columns)
     {
         return new ColumnFilter(false, null, columns, null);
     }
@@ -111,7 +111,7 @@ public class ColumnFilter
      * A filter that fetches all columns for the provided table, but returns
      * only the queried ones.
      */
-    public static ColumnFilter selection(CFMetaData metadata, PartitionColumns queried)
+    public static ColumnFilter selection(TableMetadata metadata, RegularAndStaticColumns queried)
     {
         return new ColumnFilter(true, metadata, queried, null);
     }
@@ -121,17 +121,17 @@ public class ColumnFilter
      *
      * @return the columns to fetch for this filter.
      */
-    public PartitionColumns fetchedColumns()
+    public RegularAndStaticColumns fetchedColumns()
     {
         if (!fetchAllRegulars)
             return queried;
 
         // We always fetch all regulars, but only fetch the statics in queried. Unless queried == null, in which
         // case it's a wildcard and we fetch everything.
-        PartitionColumns all = metadata.partitionColumns();
+        RegularAndStaticColumns all = metadata.regularAndStaticColumns();
         return queried == null || all.statics.isEmpty()
              ? all
-             : new PartitionColumns(queried.statics, all.regulars);
+             : new RegularAndStaticColumns(queried.statics, all.regulars);
     }
 
     /**
@@ -139,10 +139,10 @@ public class ColumnFilter
      * <p>
      * Note that this is in general not all the columns that are fetched internally (see {@link #fetchedColumns}).
      */
-    public PartitionColumns queriedColumns()
+    public RegularAndStaticColumns queriedColumns()
     {
         assert queried != null || fetchAllRegulars;
-        return queried == null ? metadata.partitionColumns() : queried;
+        return queried == null ? metadata.regularAndStaticColumns() : queried;
     }
 
     /**
@@ -175,7 +175,7 @@ public class ColumnFilter
     /**
      * Whether the provided column is fetched by this filter.
      */
-    public boolean fetches(ColumnDefinition column)
+    public boolean fetches(ColumnMetadata column)
     {
         // For statics, it is included only if it's part of _queried_, or if _queried_ is null (wildcard query).
         if (column.isStatic())
@@ -193,7 +193,7 @@ public class ColumnFilter
      * columns that this class made before using this method. If unsure, you probably want
      * to use the {@link #fetches} method.
      */
-    public boolean fetchedColumnIsQueried(ColumnDefinition column)
+    public boolean fetchedColumnIsQueried(ColumnMetadata column)
     {
         return !fetchAllRegulars || queried == null || queried.contains(column);
     }
@@ -206,7 +206,7 @@ public class ColumnFilter
      * columns that this class made before using this method. If unsure, you probably want
      * to use the {@link #fetches} method.
      */
-    public boolean fetchedCellIsQueried(ColumnDefinition column, CellPath path)
+    public boolean fetchedCellIsQueried(ColumnMetadata column, CellPath path)
     {
         assert path != null;
         if (!fetchAllRegulars || subSelections == null)
@@ -232,7 +232,7 @@ public class ColumnFilter
      * @return the created tester or {@code null} if all the cells from the provided column
      * are queried.
      */
-    public Tester newTester(ColumnDefinition column)
+    public Tester newTester(ColumnMetadata column)
     {
         if (subSelections == null || !column.isComplex())
             return null;
@@ -248,7 +248,7 @@ public class ColumnFilter
      * Returns a {@code ColumnFilter}} builder that fetches all regular columns (and queries the columns
      * added to the builder, or everything if no column is added).
      */
-    public static Builder allRegularColumnsBuilder(CFMetaData metadata)
+    public static Builder allRegularColumnsBuilder(TableMetadata metadata)
     {
         return new Builder(metadata);
     }
@@ -315,32 +315,32 @@ public class ColumnFilter
      *
      * Note that for a allColumnsBuilder, if no queried columns are added, this is interpreted as querying
      * all columns, not querying none (but if you know you want to query all columns, prefer
-     * {@link ColumnFilter#all(CFMetaData)}. For selectionBuilder, adding no queried columns means no column will be
+     * {@link ColumnFilter#all(TableMetadata)}. For selectionBuilder, adding no queried columns means no column will be
      * fetched (so the builder will return {@code PartitionColumns.NONE}).
      */
     public static class Builder
     {
-        private final CFMetaData metadata; // null if we don't fetch all columns
-        private PartitionColumns.Builder queriedBuilder;
+        private final TableMetadata metadata; // null if we don't fetch all columns
+        private RegularAndStaticColumns.Builder queriedBuilder;
         private List<ColumnSubselection> subSelections;
 
-        private Builder(CFMetaData metadata)
+        private Builder(TableMetadata metadata)
         {
             this.metadata = metadata;
         }
 
-        public Builder add(ColumnDefinition c)
+        public Builder add(ColumnMetadata c)
         {
             if (queriedBuilder == null)
-                queriedBuilder = PartitionColumns.builder();
+                queriedBuilder = RegularAndStaticColumns.builder();
             queriedBuilder.add(c);
             return this;
         }
 
-        public Builder addAll(Iterable<ColumnDefinition> columns)
+        public Builder addAll(Iterable<ColumnMetadata> columns)
         {
             if (queriedBuilder == null)
-                queriedBuilder = PartitionColumns.builder();
+                queriedBuilder = RegularAndStaticColumns.builder();
             queriedBuilder.addAll(columns);
             return this;
         }
@@ -354,12 +354,12 @@ public class ColumnFilter
             return this;
         }
 
-        public Builder slice(ColumnDefinition c, CellPath from, CellPath to)
+        public Builder slice(ColumnMetadata c, CellPath from, CellPath to)
         {
             return addSubSelection(ColumnSubselection.slice(c, from, to));
         }
 
-        public Builder select(ColumnDefinition c, CellPath elt)
+        public Builder select(ColumnMetadata c, CellPath elt)
         {
             return addSubSelection(ColumnSubselection.element(c, elt));
         }
@@ -368,11 +368,11 @@ public class ColumnFilter
         {
             boolean isFetchAll = metadata != null;
 
-            PartitionColumns queried = queriedBuilder == null ? null : queriedBuilder.build();
+            RegularAndStaticColumns queried = queriedBuilder == null ? null : queriedBuilder.build();
             // It's only ok to have queried == null in ColumnFilter if isFetchAll. So deal with the case of a selectionBuilder
             // with nothing selected (we can at least happen on some backward compatible queries - CASSANDRA-10471).
             if (!isFetchAll && queried == null)
-                queried = PartitionColumns.NONE;
+                queried = RegularAndStaticColumns.NONE;
 
             SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = null;
             if (subSelections != null)
@@ -395,7 +395,7 @@ public class ColumnFilter
         if (queried.isEmpty())
             return "";
 
-        Iterator<ColumnDefinition> defs = queried.selectOrderIterator();
+        Iterator<ColumnMetadata> defs = queried.selectOrderIterator();
         if (!defs.hasNext())
             return "<none>";
 
@@ -409,7 +409,7 @@ public class ColumnFilter
         return sb.toString();
     }
 
-    private void appendColumnDef(StringBuilder sb, ColumnDefinition column)
+    private void appendColumnDef(StringBuilder sb, ColumnMetadata column)
     {
         if (subSelections == null)
         {
@@ -454,12 +454,12 @@ public class ColumnFilter
             // queried some columns that are actually only fetched, but it's fine during upgrade).
             // More concretely, we replace our filter by a non-fetch-all one that queries every columns that our
             // current filter fetches.
-            Columns allRegulars = selection.metadata.partitionColumns().regulars;
-            Set<ColumnDefinition> queriedStatic = new HashSet<>();
-            Iterables.addAll(queriedStatic, Iterables.filter(selection.queried, ColumnDefinition::isStatic));
+            Columns allRegulars = selection.metadata.regularColumns();
+            Set<ColumnMetadata> queriedStatic = new HashSet<>();
+            Iterables.addAll(queriedStatic, Iterables.filter(selection.queried, ColumnMetadata::isStatic));
             return new ColumnFilter(false,
                                     null,
-                                    new PartitionColumns(Columns.from(queriedStatic), allRegulars),
+                                    new RegularAndStaticColumns(Columns.from(queriedStatic), allRegulars),
                                     selection.subSelections);
         }
 
@@ -483,19 +483,19 @@ public class ColumnFilter
             }
         }
 
-        public ColumnFilter deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
+        public ColumnFilter deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException
         {
             int header = in.readUnsignedByte();
             boolean isFetchAll = (header & FETCH_ALL_MASK) != 0;
             boolean hasQueried = (header & HAS_QUERIED_MASK) != 0;
             boolean hasSubSelections = (header & HAS_SUB_SELECTIONS_MASK) != 0;
 
-            PartitionColumns queried = null;
+            RegularAndStaticColumns queried = null;
             if (hasQueried)
             {
                 Columns statics = Columns.serializer.deserialize(in, metadata);
                 Columns regulars = Columns.serializer.deserialize(in, metadata);
-                queried = new PartitionColumns(statics, regulars);
+                queried = new RegularAndStaticColumns(statics, regulars);
             }
 
             SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections = null;
@@ -517,7 +517,7 @@ public class ColumnFilter
             // statics even though we only care about _fetching_ them all, but that's a minor inefficiency, so fine
             // during upgrade.
             if (version <= MessagingService.VERSION_30 && isFetchAll && queried != null)
-                queried = new PartitionColumns(metadata.partitionColumns().statics, queried.regulars);
+                queried = new RegularAndStaticColumns(metadata.staticColumns(), queried.regulars);
 
             return new ColumnFilter(isFetchAll, isFetchAll ? metadata : null, queried, subSelections);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java b/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
index b762fa5..ddc7b1c 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnSubselection.java
@@ -21,15 +21,15 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Comparator;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
@@ -44,28 +44,28 @@ public abstract class ColumnSubselection implements Comparable<ColumnSubselectio
 
     private enum Kind { SLICE, ELEMENT }
 
-    protected final ColumnDefinition column;
+    protected final ColumnMetadata column;
 
-    protected ColumnSubselection(ColumnDefinition column)
+    protected ColumnSubselection(ColumnMetadata column)
     {
         this.column = column;
     }
 
-    public static ColumnSubselection slice(ColumnDefinition column, CellPath from, CellPath to)
+    public static ColumnSubselection slice(ColumnMetadata column, CellPath from, CellPath to)
     {
         assert column.isComplex() && column.type instanceof CollectionType;
         assert from.size() <= 1 && to.size() <= 1;
         return new Slice(column, from, to);
     }
 
-    public static ColumnSubselection element(ColumnDefinition column, CellPath elt)
+    public static ColumnSubselection element(ColumnMetadata column, CellPath elt)
     {
         assert column.isComplex() && column.type instanceof CollectionType;
         assert elt.size() == 1;
         return new Element(column, elt);
     }
 
-    public ColumnDefinition column()
+    public ColumnMetadata column()
     {
         return column;
     }
@@ -91,7 +91,7 @@ public abstract class ColumnSubselection implements Comparable<ColumnSubselectio
         private final CellPath from;
         private final CellPath to;
 
-        private Slice(ColumnDefinition column, CellPath from, CellPath to)
+        private Slice(ColumnMetadata column, CellPath from, CellPath to)
         {
             super(column);
             this.from = from;
@@ -132,7 +132,7 @@ public abstract class ColumnSubselection implements Comparable<ColumnSubselectio
     {
         private final CellPath element;
 
-        private Element(ColumnDefinition column, CellPath elt)
+        private Element(ColumnMetadata column, CellPath elt)
         {
             super(column);
             this.element = elt;
@@ -166,7 +166,7 @@ public abstract class ColumnSubselection implements Comparable<ColumnSubselectio
     {
         public void serialize(ColumnSubselection subSel, DataOutputPlus out, int version) throws IOException
         {
-            ColumnDefinition column = subSel.column();
+            ColumnMetadata column = subSel.column();
             ByteBufferUtil.writeWithShortLength(column.name.bytes, out);
             out.writeByte(subSel.kind().ordinal());
             switch (subSel.kind())
@@ -185,16 +185,16 @@ public abstract class ColumnSubselection implements Comparable<ColumnSubselectio
             }
         }
 
-        public ColumnSubselection deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
+        public ColumnSubselection deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException
         {
             ByteBuffer name = ByteBufferUtil.readWithShortLength(in);
-            ColumnDefinition column = metadata.getColumnDefinition(name);
+            ColumnMetadata column = metadata.getColumn(name);
             if (column == null)
             {
                 // If we don't find the definition, it could be we have data for a dropped column, and we shouldn't
-                // fail deserialization because of that. So we grab a "fake" ColumnDefinition that ensure proper
+                // fail deserialization because of that. So we grab a "fake" ColumnMetadata that ensure proper
                 // deserialization. The column will be ignore later on anyway.
-                column = metadata.getDroppedColumnDefinition(name);
+                column = metadata.getDroppedColumn(name);
                 if (column == null)
                     throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
             }
@@ -217,7 +217,7 @@ public abstract class ColumnSubselection implements Comparable<ColumnSubselectio
         {
             long size = 0;
 
-            ColumnDefinition column = subSel.column();
+            ColumnMetadata column = subSel.column();
             size += TypeSizes.sizeofWithShortLength(column.name.bytes);
             size += 1; // kind
             switch (subSel.kind())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/filter/DataLimits.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/DataLimits.java b/src/java/org/apache/cassandra/db/filter/DataLimits.java
index 442b5f8..410845c 100644
--- a/src/java/org/apache/cassandra/db/filter/DataLimits.java
+++ b/src/java/org/apache/cassandra/db/filter/DataLimits.java
@@ -412,7 +412,7 @@ public abstract class DataLimits
         {
             // TODO: we should start storing stats on the number of rows (instead of the number of cells, which
             // is what getMeanColumns returns)
-            float rowsPerPartition = ((float) cfs.getMeanColumns()) / cfs.metadata.partitionColumns().regulars.size();
+            float rowsPerPartition = ((float) cfs.getMeanColumns()) / cfs.metadata().regularColumns().size();
             return rowsPerPartition * (cfs.estimateKeys());
         }
 


[10/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
index 975eb8e..30fbb0d 100644
--- a/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
+++ b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
@@ -26,9 +26,9 @@ import java.util.List;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.marshal.*;
@@ -45,7 +45,7 @@ import static org.junit.Assert.assertTrue;
 
 public class SelectionColumnMappingTest extends CQLTester
 {
-    private static final ColumnDefinition NULL_DEF = null;
+    private static final ColumnMetadata NULL_DEF = null;
     String tableName;
     String typeName;
     UserType userType;
@@ -71,7 +71,7 @@ public class SelectionColumnMappingTest extends CQLTester
                                 " v1 int," +
                                 " v2 ascii," +
                                 " v3 frozen<" + typeName + ">)");
-        userType = Schema.instance.getKSMetaData(KEYSPACE).types.get(ByteBufferUtil.bytes(typeName)).get().freeze();
+        userType = Schema.instance.getKeyspaceMetadata(KEYSPACE).types.get(ByteBufferUtil.bytes(typeName)).get().freeze();
         functionName = createFunction(KEYSPACE, "int, ascii",
                                       "CREATE FUNCTION %s (i int, a ascii) " +
                                       "CALLED ON NULL INPUT " +
@@ -130,7 +130,7 @@ public class SelectionColumnMappingTest extends CQLTester
     private void testSimpleTypes() throws Throwable
     {
         // simple column identifiers without aliases are represented in
-        // ResultSet.Metadata by the underlying ColumnDefinition
+        // ResultSet.Metadata by the underlying ColumnMetadata
         ColumnSpecification kSpec = columnSpecification("k", Int32Type.instance);
         ColumnSpecification v1Spec = columnSpecification("v1", Int32Type.instance);
         ColumnSpecification v2Spec = columnSpecification("v2", AsciiType.instance);
@@ -144,12 +144,12 @@ public class SelectionColumnMappingTest extends CQLTester
 
     private void testWildcard() throws Throwable
     {
-        // Wildcard select represents each column in the table with a ColumnDefinition
+        // Wildcard select represents each column in the table with a ColumnMetadata
         // in the ResultSet metadata
-        ColumnDefinition kSpec = columnDefinition("k");
-        ColumnDefinition v1Spec = columnDefinition("v1");
-        ColumnDefinition v2Spec = columnDefinition("v2");
-        ColumnDefinition v3Spec = columnDefinition("v3");
+        ColumnMetadata kSpec = columnDefinition("k");
+        ColumnMetadata v1Spec = columnDefinition("v1");
+        ColumnMetadata v2Spec = columnDefinition("v2");
+        ColumnMetadata v3Spec = columnDefinition("v3");
         SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
                                                                 .addMapping(kSpec, columnDefinition("k"))
                                                                 .addMapping(v1Spec, columnDefinition("v1"))
@@ -162,7 +162,7 @@ public class SelectionColumnMappingTest extends CQLTester
     private void testSimpleTypesWithAliases() throws Throwable
     {
         // simple column identifiers with aliases are represented in ResultSet.Metadata
-        // by a ColumnSpecification based on the underlying ColumnDefinition
+        // by a ColumnSpecification based on the underlying ColumnMetadata
         ColumnSpecification kSpec = columnSpecification("k_alias", Int32Type.instance);
         ColumnSpecification v1Spec = columnSpecification("v1_alias", Int32Type.instance);
         ColumnSpecification v2Spec = columnSpecification("v2_alias", AsciiType.instance);
@@ -406,7 +406,7 @@ public class SelectionColumnMappingTest extends CQLTester
     private void testMultipleUnaliasedSelectionOfSameColumn() throws Throwable
     {
         // simple column identifiers without aliases are represented in
-        // ResultSet.Metadata by the underlying ColumnDefinition
+        // ResultSet.Metadata by the underlying ColumnMetadata
         SelectionColumns expected = SelectionColumnMapping.newMapping()
                                                           .addMapping(columnSpecification("v1", Int32Type.instance),
                                                                       columnDefinition("v1"))
@@ -430,7 +430,7 @@ public class SelectionColumnMappingTest extends CQLTester
     {
         ColumnSpecification listSpec = columnSpecification("(list<int>)[]", ListType.getInstance(Int32Type.instance, false));
         SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(listSpec, (ColumnDefinition) null);
+                                                                .addMapping(listSpec, (ColumnMetadata) null);
 
         verify(expected, "SELECT (list<int>)[] FROM %s");
     }
@@ -449,7 +449,7 @@ public class SelectionColumnMappingTest extends CQLTester
     {
         ColumnSpecification setSpec = columnSpecification("(set<int>){}", SetType.getInstance(Int32Type.instance, false));
         SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(setSpec, (ColumnDefinition) null);
+                                                                .addMapping(setSpec, (ColumnMetadata) null);
 
         verify(expected, "SELECT (set<int>){} FROM %s");
     }
@@ -467,7 +467,7 @@ public class SelectionColumnMappingTest extends CQLTester
     {
         ColumnSpecification mapSpec = columnSpecification("(map<text, int>){}", MapType.getInstance(UTF8Type.instance, Int32Type.instance, false));
         SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(mapSpec, (ColumnDefinition) null);
+                                                                .addMapping(mapSpec, (ColumnMetadata) null);
 
         verify(expected, "SELECT (map<text, int>){} FROM %s");
     }
@@ -557,7 +557,7 @@ public class SelectionColumnMappingTest extends CQLTester
                                        " LANGUAGE javascript" +
                                        " AS 'a*a'");
 
-        ColumnDefinition v1 = columnDefinition("v1");
+        ColumnMetadata v1 = columnDefinition("v1");
         SelectionColumns expected = SelectionColumnMapping.newMapping()
                                                           .addMapping(columnSpecification(aFunc + "(v1)",
                                                                                           Int32Type.instance),
@@ -600,18 +600,17 @@ public class SelectionColumnMappingTest extends CQLTester
         assertEquals(expected, select.getSelection().getColumnMapping());
     }
 
-    private Iterable<ColumnDefinition> columnDefinitions(String...names)
+    private Iterable<ColumnMetadata> columnDefinitions(String...names)
     {
-        List<ColumnDefinition> defs = new ArrayList<>();
+        List<ColumnMetadata> defs = new ArrayList<>();
         for (String n : names)
             defs.add(columnDefinition(n));
         return defs;
     }
 
-    private ColumnDefinition columnDefinition(String name)
+    private ColumnMetadata columnDefinition(String name)
     {
-        return Schema.instance.getCFMetaData(KEYSPACE, tableName)
-                              .getColumnDefinition(new ColumnIdentifier(name, true));
+        return Schema.instance.getTableMetadata(KEYSPACE, tableName).getColumn(new ColumnIdentifier(name, true));
 
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
index 15a93a1..f098126 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -25,8 +25,8 @@ import java.util.concurrent.CountDownLatch;
 import org.apache.commons.lang3.StringUtils;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -111,9 +111,9 @@ public class SecondaryIndexTest extends CQLTester
 
         // IF NOT EXISTS should apply in cases where the new index differs from an existing one in name only
         String otherIndexName = "index_" + System.nanoTime();
-        assertEquals(1, getCurrentColumnFamilyStore().metadata.getIndexes().size());
+        assertEquals(1, getCurrentColumnFamilyStore().metadata().indexes.size());
         createIndex("CREATE INDEX IF NOT EXISTS " + otherIndexName + " ON %s(b)");
-        assertEquals(1, getCurrentColumnFamilyStore().metadata.getIndexes().size());
+        assertEquals(1, getCurrentColumnFamilyStore().metadata().indexes.size());
         assertInvalidMessage(String.format("Index %s is a duplicate of existing index %s",
                                            removeQuotes(otherIndexName.toLowerCase(Locale.US)),
                                            removeQuotes(indexName.toLowerCase(Locale.US))),
@@ -845,11 +845,11 @@ public class SecondaryIndexTest extends CQLTester
         createIndex(String.format("CREATE CUSTOM INDEX c_idx_2 ON %%s(c) USING '%s' WITH OPTIONS = {'foo':'b'}", indexClassName));
 
         ColumnFamilyStore cfs = getCurrentColumnFamilyStore();
-        CFMetaData cfm = cfs.metadata;
-        StubIndex index1 = (StubIndex)cfs.indexManager.getIndex(cfm.getIndexes()
+        TableMetadata cfm = cfs.metadata();
+        StubIndex index1 = (StubIndex)cfs.indexManager.getIndex(cfm.indexes
                                                                    .get("c_idx_1")
                                                                    .orElseThrow(throwAssert("index not found")));
-        StubIndex index2 = (StubIndex)cfs.indexManager.getIndex(cfm.getIndexes()
+        StubIndex index2 = (StubIndex)cfs.indexManager.getIndex(cfm.indexes
                                                                    .get("c_idx_2")
                                                                    .orElseThrow(throwAssert("index not found")));
         Object[] row1a = row(0, 0, 0);
@@ -887,8 +887,8 @@ public class SecondaryIndexTest extends CQLTester
         createIndex(String.format("CREATE CUSTOM INDEX c_idx ON %%s(c) USING '%s'", indexClassName));
 
         ColumnFamilyStore cfs = getCurrentColumnFamilyStore();
-        CFMetaData cfm = cfs.metadata;
-        StubIndex index1 = (StubIndex) cfs.indexManager.getIndex(cfm.getIndexes()
+        TableMetadata cfm = cfs.metadata();
+        StubIndex index1 = (StubIndex) cfs.indexManager.getIndex(cfm.indexes
                 .get("c_idx")
                 .orElseThrow(throwAssert("index not found")));
 
@@ -943,8 +943,8 @@ public class SecondaryIndexTest extends CQLTester
         createIndex(String.format("CREATE CUSTOM INDEX test_index ON %%s() USING '%s'", StubIndex.class.getName()));
         execute("INSERT INTO %s (k, c, v1, v2) VALUES (0, 0, 0, 0) USING TIMESTAMP 0");
 
-        ColumnDefinition v1 = getCurrentColumnFamilyStore().metadata.getColumnDefinition(new ColumnIdentifier("v1", true));
-        ColumnDefinition v2 = getCurrentColumnFamilyStore().metadata.getColumnDefinition(new ColumnIdentifier("v2", true));
+        ColumnMetadata v1 = getCurrentColumnFamilyStore().metadata().getColumn(new ColumnIdentifier("v1", true));
+        ColumnMetadata v2 = getCurrentColumnFamilyStore().metadata().getColumn(new ColumnIdentifier("v2", true));
 
         StubIndex index = (StubIndex)getCurrentColumnFamilyStore().indexManager.getIndexByName("test_index");
         assertEquals(1, index.rowsInserted.size());
@@ -1336,16 +1336,16 @@ public class SecondaryIndexTest extends CQLTester
                                       ClientState.forInternalCalls());
     }
 
-    private void validateCell(Cell cell, ColumnDefinition def, ByteBuffer val, long timestamp)
+    private void validateCell(Cell cell, ColumnMetadata def, ByteBuffer val, long timestamp)
     {
         assertNotNull(cell);
         assertEquals(0, def.type.compare(cell.value(), val));
         assertEquals(timestamp, cell.timestamp());
     }
 
-    private static void assertColumnValue(int expected, String name, Row row, CFMetaData cfm)
+    private static void assertColumnValue(int expected, String name, Row row, TableMetadata cfm)
     {
-        ColumnDefinition col = cfm.getColumnDefinition(new ColumnIdentifier(name, true));
+        ColumnMetadata col = cfm.getColumn(new ColumnIdentifier(name, true));
         AbstractType<?> type = col.type;
         assertEquals(expected, type.compose(row.getCell(col).value()));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
index 3affe9a..ecff0cc 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
@@ -28,7 +28,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.functions.FunctionName;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
index a7c09b1..187871a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
@@ -33,15 +33,14 @@ import com.datastax.driver.core.Row;
 import com.datastax.driver.core.TupleType;
 import com.datastax.driver.core.TupleValue;
 import com.datastax.driver.core.UDTValue;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.exceptions.FunctionExecutionException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.transport.ProtocolVersion;
-import org.apache.cassandra.transport.Server;
 
 public class UFJavaTest extends CQLTester
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
index 74f6409..7940b92 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@ -28,7 +28,6 @@ import org.junit.Test;
 
 import com.datastax.driver.core.*;
 import com.datastax.driver.core.exceptions.InvalidQueryException;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -38,6 +37,7 @@ import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.*;
 import org.apache.cassandra.transport.ProtocolVersion;
@@ -823,7 +823,7 @@ public class UFTest extends CQLTester
                                       "LANGUAGE JAVA\n" +
                                       "AS 'throw new RuntimeException();';");
 
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(KEYSPACE_PER_TEST);
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(KEYSPACE_PER_TEST);
         UDFunction f = (UDFunction) ksm.functions.get(parseFunctionName(fName)).iterator().next();
 
         UDFunction broken = UDFunction.createBrokenFunction(f.name(),
@@ -834,7 +834,7 @@ public class UFTest extends CQLTester
                                                             "java",
                                                             f.body(),
                                                             new InvalidRequestException("foo bar is broken"));
-        Schema.instance.setKeyspaceMetadata(ksm.withSwapped(ksm.functions.without(f.name(), f.argTypes()).with(broken)));
+        Schema.instance.load(ksm.withSwapped(ksm.functions.without(f.name(), f.argTypes()).with(broken)));
 
         assertInvalidThrowMessage("foo bar is broken", InvalidRequestException.class,
                                   "SELECT key, " + fName + "(dval) FROM %s");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
index 2760ae5..becbdad 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
@@ -135,7 +135,7 @@ public class CrcCheckChanceTest extends CQLTester
             alterTable("ALTER TABLE %s WITH compression = {'sstable_compression': 'LZ4Compressor', 'crc_check_chance': 0.5}");
 
         //We should be able to get the new value by accessing directly the schema metadata
-        Assert.assertEquals(0.5, cfs.metadata.params.crcCheckChance);
+        Assert.assertEquals(0.5, cfs.metadata().params.crcCheckChance);
 
         //but previous JMX-set value will persist until next restart
         Assert.assertEquals(0.01, cfs.getLiveSSTables().iterator().next().getCrcCheckChance());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
index cfedc08..c79c596 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -40,8 +40,8 @@ import ch.qos.logback.classic.LoggerContext;
 import ch.qos.logback.classic.spi.TurboFilterList;
 import ch.qos.logback.classic.turbo.ReconfigureOnChangeFilter;
 import ch.qos.logback.classic.turbo.TurboFilter;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -1282,7 +1282,7 @@ public class AggregationTest extends CQLTester
                                    "SFUNC " + shortFunctionName(fState) + " " +
                                    "STYPE int ");
 
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace());
         UDAggregate f = (UDAggregate) ksm.functions.get(parseFunctionName(a)).iterator().next();
 
         UDAggregate broken = UDAggregate.createBroken(f.name(),
@@ -1291,7 +1291,7 @@ public class AggregationTest extends CQLTester
                                                       null,
                                                       new InvalidRequestException("foo bar is broken"));
 
-        Schema.instance.setKeyspaceMetadata(ksm.withSwapped(ksm.functions.without(f.name(), f.argTypes()).with(broken)));
+        Schema.instance.load(ksm.withSwapped(ksm.functions.without(f.name(), f.argTypes()).with(broken)));
 
         assertInvalidThrowMessage("foo bar is broken", InvalidRequestException.class,
                                   "SELECT " + a + "(val) FROM %s");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index 3f98df1..88c6a3f 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.cql3.validation.operations;
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -183,12 +183,12 @@ public class AlterTest extends CQLTester
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(tableName);
 
         alterTable("ALTER TABLE %s WITH min_index_interval=256 AND max_index_interval=512");
-        assertEquals(256, cfs.metadata.params.minIndexInterval);
-        assertEquals(512, cfs.metadata.params.maxIndexInterval);
+        assertEquals(256, cfs.metadata().params.minIndexInterval);
+        assertEquals(512, cfs.metadata().params.maxIndexInterval);
 
         alterTable("ALTER TABLE %s WITH caching = {}");
-        assertEquals(256, cfs.metadata.params.minIndexInterval);
-        assertEquals(512, cfs.metadata.params.maxIndexInterval);
+        assertEquals(256, cfs.metadata().params.minIndexInterval);
+        assertEquals(512, cfs.metadata().params.maxIndexInterval);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
index b92b0df..6339a46 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -26,10 +26,11 @@ import java.util.UUID;
 
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.Duration;
 import org.apache.cassandra.db.Mutation;
@@ -819,14 +820,14 @@ public class CreateTest extends CQLTester
 
     private void assertTriggerExists(String name)
     {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
-        assertTrue("the trigger does not exist", cfm.getTriggers().get(name).isPresent());
+        TableMetadata metadata = Schema.instance.getTableMetadata(keyspace(), currentTable());
+        assertTrue("the trigger does not exist", metadata.triggers.get(name).isPresent());
     }
 
     private void assertTriggerDoesNotExists(String name)
     {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), currentTable()).copy();
-        assertFalse("the trigger exists", cfm.getTriggers().get(name).isPresent());
+        TableMetadata metadata = Schema.instance.getTableMetadata(keyspace(), currentTable());
+        assertFalse("the trigger exists", metadata.triggers.get(name).isPresent());
     }
 
     public static class TestTrigger implements ITrigger

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java
index f491d24..19aba64 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.cql3.validation.operations;
 
 import java.io.File;
 import java.util.List;
-import java.util.UUID;
 
 import org.junit.Test;
 
@@ -30,6 +29,7 @@ import org.apache.cassandra.exceptions.AlreadyExistsException;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.TableId;
 
 public class DropRecreateAndRestoreTest extends CQLTester
 {
@@ -43,7 +43,7 @@ public class DropRecreateAndRestoreTest extends CQLTester
 
 
         long time = System.currentTimeMillis();
-        UUID id = currentTableMetadata().cfId;
+        TableId id = currentTableMetadata().id;
         assertRows(execute("SELECT * FROM %s"), row(0, 0, 0), row(0, 1, 1));
         Thread.sleep(5);
 
@@ -84,7 +84,7 @@ public class DropRecreateAndRestoreTest extends CQLTester
     public void testCreateWithIdDuplicate() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY(a, b))");
-        UUID id = currentTableMetadata().cfId;
+        TableId id = currentTableMetadata().id;
         execute(String.format("CREATE TABLE %%s (a int, b int, c int, PRIMARY KEY(a, b)) WITH ID = %s", id));
     }
 
@@ -98,7 +98,7 @@ public class DropRecreateAndRestoreTest extends CQLTester
     public void testAlterWithId() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY(a, b))");
-        UUID id = currentTableMetadata().cfId;
+        TableId id = currentTableMetadata().id;
         execute(String.format("ALTER TABLE %%s WITH ID = %s", id));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
index b5f8b6c..7cbe085 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
@@ -23,7 +23,7 @@ import java.util.List;
 
 import org.junit.Test;
 
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.SyntaxException;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/CellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CellTest.java b/test/unit/org/apache/cassandra/db/CellTest.java
index 86fb20e..febfa3c 100644
--- a/test/unit/org/apache/cassandra/db/CellTest.java
+++ b/test/unit/org/apache/cassandra/db/CellTest.java
@@ -27,16 +27,16 @@ import junit.framework.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.FieldIdentifier;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -54,13 +54,14 @@ public class CellTest
     private static final String CF_STANDARD1 = "Standard1";
     private static final String CF_COLLECTION = "Collection1";
 
-    private static final CFMetaData cfm = SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1);
-    private static final CFMetaData cfm2 = CFMetaData.Builder.create(KEYSPACE1, CF_COLLECTION)
-                                                             .addPartitionKey("k", IntegerType.instance)
-                                                             .addClusteringColumn("c", IntegerType.instance)
-                                                             .addRegularColumn("v", IntegerType.instance)
-                                                             .addRegularColumn("m", MapType.getInstance(IntegerType.instance, IntegerType.instance, true))
-                                                             .build();
+    private static final TableMetadata cfm = SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1).build();
+    private static final TableMetadata cfm2 =
+        TableMetadata.builder(KEYSPACE1, CF_COLLECTION)
+                     .addPartitionKeyColumn("k", IntegerType.instance)
+                     .addClusteringColumn("c", IntegerType.instance)
+                     .addRegularColumn("v", IntegerType.instance)
+                     .addRegularColumn("m", MapType.getInstance(IntegerType.instance, IntegerType.instance, true))
+                     .build();
 
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
@@ -69,14 +70,14 @@ public class CellTest
         SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1), cfm, cfm2);
     }
 
-    private static ColumnDefinition fakeColumn(String name, AbstractType<?> type)
+    private static ColumnMetadata fakeColumn(String name, AbstractType<?> type)
     {
-        return new ColumnDefinition("fakeKs",
-                                    "fakeTable",
-                                    ColumnIdentifier.getInterned(name, false),
-                                    type,
-                                    ColumnDefinition.NO_POSITION,
-                                    ColumnDefinition.Kind.REGULAR);
+        return new ColumnMetadata("fakeKs",
+                                  "fakeTable",
+                                  ColumnIdentifier.getInterned(name, false),
+                                  type,
+                                  ColumnMetadata.NO_POSITION,
+                                  ColumnMetadata.Kind.REGULAR);
     }
 
     @Test
@@ -130,7 +131,7 @@ public class CellTest
     @Test
     public void testValidate()
     {
-        ColumnDefinition c;
+        ColumnMetadata c;
 
         // Valid cells
         c = fakeColumn("c", Int32Type.instance);
@@ -173,7 +174,7 @@ public class CellTest
                                     asList(f1, f2),
                                     asList(Int32Type.instance, UTF8Type.instance),
                                     true);
-        ColumnDefinition c;
+        ColumnMetadata c;
 
         // Valid cells
         c = fakeColumn("c", udt);
@@ -207,7 +208,7 @@ public class CellTest
                                     asList(Int32Type.instance, UTF8Type.instance),
                                     false);
 
-        ColumnDefinition c = fakeColumn("c", udt);
+        ColumnMetadata c = fakeColumn("c", udt);
         ByteBuffer val = udt(bb(1), bb("foo"));
 
         // Valid cells
@@ -278,7 +279,7 @@ public class CellTest
     @Test
     public void testComplexCellReconcile()
     {
-        ColumnDefinition m = cfm2.getColumnDefinition(new ColumnIdentifier("m", false));
+        ColumnMetadata m = cfm2.getColumn(new ColumnIdentifier("m", false));
         int now1 = FBUtilities.nowInSeconds();
         long ts1 = now1*1000000;
 
@@ -318,21 +319,21 @@ public class CellTest
         return Cells.reconcile(c2, c1, now) == c2 ? 1 : 0;
     }
 
-    private Cell regular(CFMetaData cfm, String columnName, String value, long timestamp)
+    private Cell regular(TableMetadata cfm, String columnName, String value, long timestamp)
     {
-        ColumnDefinition cdef = cfm.getColumnDefinition(ByteBufferUtil.bytes(columnName));
+        ColumnMetadata cdef = cfm.getColumn(ByteBufferUtil.bytes(columnName));
         return BufferCell.live(cdef, timestamp, ByteBufferUtil.bytes(value));
     }
 
-    private Cell expiring(CFMetaData cfm, String columnName, String value, long timestamp, int localExpirationTime)
+    private Cell expiring(TableMetadata cfm, String columnName, String value, long timestamp, int localExpirationTime)
     {
-        ColumnDefinition cdef = cfm.getColumnDefinition(ByteBufferUtil.bytes(columnName));
+        ColumnMetadata cdef = cfm.getColumn(ByteBufferUtil.bytes(columnName));
         return new BufferCell(cdef, timestamp, 1, localExpirationTime, ByteBufferUtil.bytes(value), null);
     }
 
-    private Cell deleted(CFMetaData cfm, String columnName, int localDeletionTime, long timestamp)
+    private Cell deleted(TableMetadata cfm, String columnName, int localDeletionTime, long timestamp)
     {
-        ColumnDefinition cdef = cfm.getColumnDefinition(ByteBufferUtil.bytes(columnName));
+        ColumnMetadata cdef = cfm.getColumn(ByteBufferUtil.bytes(columnName));
         return BufferCell.tombstone(cdef, timestamp, localDeletionTime);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index abd5a04..f576290 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -35,7 +35,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.RowFilter;
@@ -118,7 +118,7 @@ public class CleanupTest
         fillCF(cfs, "birthdate", LOOPS);
         assertEquals(LOOPS, Util.getAll(Util.cmd(cfs).build()).size());
 
-        ColumnDefinition cdef = cfs.metadata.getColumnDefinition(COLUMN);
+        ColumnMetadata cdef = cfs.metadata().getColumn(COLUMN);
         String indexName = "birthdate_key_index";
         long start = System.nanoTime();
         while (!cfs.getBuiltIndexes().contains(indexName) && System.nanoTime() - start < TimeUnit.SECONDS.toNanos(10))
@@ -279,7 +279,7 @@ public class CleanupTest
         {
             String key = String.valueOf(i);
             // create a row and update the birthdate value, test that the index query fetches the new version
-            new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), ByteBufferUtil.bytes(key))
+            new RowUpdateBuilder(cfs.metadata(), System.currentTimeMillis(), ByteBufferUtil.bytes(key))
                     .clustering(COLUMN)
                     .add(colName, VALUE)
                     .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
index 2f7aaa5..c016f9b 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
@@ -55,7 +55,7 @@ public class ColumnFamilyMetricTest
 
         for (int j = 0; j < 10; j++)
         {
-            new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros(), String.valueOf(j))
+            new RowUpdateBuilder(cfs.metadata(), FBUtilities.timestampMicros(), String.valueOf(j))
                     .clustering("0")
                     .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
@@ -91,7 +91,7 @@ public class ColumnFamilyMetricTest
         // This confirms another test/set up did not overflow the histogram
         store.metric.colUpdateTimeDeltaHistogram.cf.getSnapshot().get999thPercentile();
 
-        new RowUpdateBuilder(store.metadata, 0, "4242")
+        new RowUpdateBuilder(store.metadata(), 0, "4242")
             .clustering("0")
             .add("val", ByteBufferUtil.bytes("0"))
             .build()
@@ -101,7 +101,7 @@ public class ColumnFamilyMetricTest
         store.metric.colUpdateTimeDeltaHistogram.cf.getSnapshot().get999thPercentile();
 
         // smallest time delta that would overflow the histogram if unfiltered
-        new RowUpdateBuilder(store.metadata, 18165375903307L, "4242")
+        new RowUpdateBuilder(store.metadata(), 18165375903307L, "4242")
             .clustering("0")
             .add("val", ByteBufferUtil.bytes("0"))
             .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java
index 0b04bb3..59f9db3 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreCQLHelperTest.java
@@ -31,7 +31,6 @@ import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.cassandra.*;
-import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.statements.*;
 import org.apache.cassandra.db.marshal.*;
@@ -87,25 +86,23 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
                                                     typeB),
                                       true);
 
-        CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
-                                           .addPartitionKey("pk1", IntegerType.instance)
-                                           .addClusteringColumn("ck1", IntegerType.instance)
-                                           .addRegularColumn("reg1", typeC)
-                                           .addRegularColumn("reg2", ListType.getInstance(IntegerType.instance, false))
-                                           .addRegularColumn("reg3", MapType.getInstance(AsciiType.instance, IntegerType.instance, true))
-                                           .build();
+        TableMetadata cfm =
+            TableMetadata.builder(keyspace, table)
+                         .addPartitionKeyColumn("pk1", IntegerType.instance)
+                         .addClusteringColumn("ck1", IntegerType.instance)
+                         .addRegularColumn("reg1", typeC)
+                         .addRegularColumn("reg2", ListType.getInstance(IntegerType.instance, false))
+                         .addRegularColumn("reg3", MapType.getInstance(AsciiType.instance, IntegerType.instance, true))
+                         .build();
 
-        SchemaLoader.createKeyspace(keyspace,
-                                    KeyspaceParams.simple(1),
-                                    Tables.of(cfm),
-                                    Types.of(typeA, typeB, typeC));
+        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), Tables.of(cfm), Types.of(typeA, typeB, typeC));
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
 
-        assertEquals(ImmutableList.of("CREATE TYPE cql_test_keyspace_user_types.a(a1 varint, a2 varint, a3 varint);",
-                                      "CREATE TYPE cql_test_keyspace_user_types.b(b1 a, b2 a, b3 a);",
-                                      "CREATE TYPE cql_test_keyspace_user_types.c(c1 b, c2 b, c3 b);"),
-                     ColumnFamilyStoreCQLHelper.getUserTypesAsCQL(cfs.metadata));
+        assertEquals(ImmutableList.of("CREATE TYPE cql_test_keyspace_user_types.a (a1 varint, a2 varint, a3 varint);",
+                                      "CREATE TYPE cql_test_keyspace_user_types.b (b1 a, b2 a, b3 a);",
+                                      "CREATE TYPE cql_test_keyspace_user_types.c (c1 b, c2 b, c3 b);"),
+                     ColumnFamilyStoreCQLHelper.getUserTypesAsCQL(cfs.metadata()));
     }
 
     @Test
@@ -114,39 +111,36 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         String keyspace = "cql_test_keyspace_dropped_columns";
         String table = "test_table_dropped_columns";
 
-        CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
-                                           .addPartitionKey("pk1", IntegerType.instance)
-                                           .addClusteringColumn("ck1", IntegerType.instance)
-                                           .addRegularColumn("reg1", IntegerType.instance)
-                                           .addRegularColumn("reg2", IntegerType.instance)
-                                           .addRegularColumn("reg3", IntegerType.instance)
-                                           .build();
-
+        TableMetadata.Builder builder =
+            TableMetadata.builder(keyspace, table)
+                         .addPartitionKeyColumn("pk1", IntegerType.instance)
+                         .addClusteringColumn("ck1", IntegerType.instance)
+                         .addRegularColumn("reg1", IntegerType.instance)
+                         .addRegularColumn("reg2", IntegerType.instance)
+                         .addRegularColumn("reg3", IntegerType.instance);
 
-        ColumnDefinition reg1 = cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1"));
-        ColumnDefinition reg2 = cfm.getColumnDefinition(ByteBufferUtil.bytes("reg2"));
-        ColumnDefinition reg3 = cfm.getColumnDefinition(ByteBufferUtil.bytes("reg3"));
+        ColumnMetadata reg1 = builder.getColumn(ByteBufferUtil.bytes("reg1"));
+        ColumnMetadata reg2 = builder.getColumn(ByteBufferUtil.bytes("reg2"));
+        ColumnMetadata reg3 = builder.getColumn(ByteBufferUtil.bytes("reg3"));
 
-        cfm.removeColumnDefinition(reg1);
-        cfm.removeColumnDefinition(reg2);
-        cfm.removeColumnDefinition(reg3);
+        builder.removeRegularOrStaticColumn(reg1.name)
+               .removeRegularOrStaticColumn(reg2.name)
+               .removeRegularOrStaticColumn(reg3.name);
 
-        cfm.recordColumnDrop(reg1, 10000);
-        cfm.recordColumnDrop(reg2, 20000);
-        cfm.recordColumnDrop(reg3, 30000);
+        builder.recordColumnDrop(reg1, 10000)
+               .recordColumnDrop(reg2, 20000)
+               .recordColumnDrop(reg3, 30000);
 
-        SchemaLoader.createKeyspace(keyspace,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), builder);
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
 
         assertEquals(ImmutableList.of("ALTER TABLE cql_test_keyspace_dropped_columns.test_table_dropped_columns DROP reg1 USING TIMESTAMP 10000;",
                                       "ALTER TABLE cql_test_keyspace_dropped_columns.test_table_dropped_columns DROP reg3 USING TIMESTAMP 30000;",
                                       "ALTER TABLE cql_test_keyspace_dropped_columns.test_table_dropped_columns DROP reg2 USING TIMESTAMP 20000;"),
-                     ColumnFamilyStoreCQLHelper.getDroppedColumnsAsCQL(cfs.metadata));
+                     ColumnFamilyStoreCQLHelper.getDroppedColumnsAsCQL(cfs.metadata()));
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS cql_test_keyspace_dropped_columns.test_table_dropped_columns (\n" +
         "\tpk1 varint,\n" +
         "\tck1 varint,\n" +
@@ -162,34 +156,32 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         String keyspace = "cql_test_keyspace_readded_columns";
         String table = "test_table_readded_columns";
 
-        CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
-                                           .addPartitionKey("pk1", IntegerType.instance)
-                                           .addClusteringColumn("ck1", IntegerType.instance)
-                                           .addRegularColumn("reg1", IntegerType.instance)
-                                           .addStaticColumn("reg2", IntegerType.instance)
-                                           .addRegularColumn("reg3", IntegerType.instance)
-                                           .build();
+        TableMetadata.Builder builder =
+            TableMetadata.builder(keyspace, table)
+                         .addPartitionKeyColumn("pk1", IntegerType.instance)
+                         .addClusteringColumn("ck1", IntegerType.instance)
+                         .addRegularColumn("reg1", IntegerType.instance)
+                         .addStaticColumn("reg2", IntegerType.instance)
+                         .addRegularColumn("reg3", IntegerType.instance);
 
-        ColumnDefinition reg1 = cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1"));
-        ColumnDefinition reg2 = cfm.getColumnDefinition(ByteBufferUtil.bytes("reg2"));
+        ColumnMetadata reg1 = builder.getColumn(ByteBufferUtil.bytes("reg1"));
+        ColumnMetadata reg2 = builder.getColumn(ByteBufferUtil.bytes("reg2"));
 
-        cfm.removeColumnDefinition(reg1);
-        cfm.removeColumnDefinition(reg2);
+        builder.removeRegularOrStaticColumn(reg1.name);
+        builder.removeRegularOrStaticColumn(reg2.name);
 
-        cfm.recordColumnDrop(reg1, 10000);
-        cfm.recordColumnDrop(reg2, 20000);
+        builder.recordColumnDrop(reg1, 10000);
+        builder.recordColumnDrop(reg2, 20000);
 
-        cfm.addColumnDefinition(reg1);
-        cfm.addColumnDefinition(reg2);
+        builder.addColumn(reg1);
+        builder.addColumn(reg2);
 
-        SchemaLoader.createKeyspace(keyspace,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), builder);
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
 
         // when re-adding, column is present in CREATE, then in DROP and then in ADD again, to record DROP with a proper timestamp
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS cql_test_keyspace_readded_columns.test_table_readded_columns (\n" +
         "\tpk1 varint,\n" +
         "\tck1 varint,\n" +
@@ -202,7 +194,7 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
                                       "ALTER TABLE cql_test_keyspace_readded_columns.test_table_readded_columns ADD reg1 varint;",
                                       "ALTER TABLE cql_test_keyspace_readded_columns.test_table_readded_columns DROP reg2 USING TIMESTAMP 20000;",
                                       "ALTER TABLE cql_test_keyspace_readded_columns.test_table_readded_columns ADD reg2 varint static;"),
-                     ColumnFamilyStoreCQLHelper.getDroppedColumnsAsCQL(cfs.metadata));
+                     ColumnFamilyStoreCQLHelper.getDroppedColumnsAsCQL(cfs.metadata()));
     }
 
     @Test
@@ -211,24 +203,22 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         String keyspace = "cql_test_keyspace_create_table";
         String table = "test_table_create_table";
 
-        CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
-                                           .addPartitionKey("pk1", IntegerType.instance)
-                                           .addPartitionKey("pk2", AsciiType.instance)
-                                           .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
-                                           .addClusteringColumn("ck2", IntegerType.instance)
-                                           .addStaticColumn("st1", AsciiType.instance)
-                                           .addRegularColumn("reg1", AsciiType.instance)
-                                           .addRegularColumn("reg2", ListType.getInstance(IntegerType.instance, false))
-                                           .addRegularColumn("reg3", MapType.getInstance(AsciiType.instance, IntegerType.instance, true))
-                                           .build();
-
-        SchemaLoader.createKeyspace(keyspace,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        TableMetadata.Builder metadata =
+            TableMetadata.builder(keyspace, table)
+                         .addPartitionKeyColumn("pk1", IntegerType.instance)
+                         .addPartitionKeyColumn("pk2", AsciiType.instance)
+                         .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
+                         .addClusteringColumn("ck2", IntegerType.instance)
+                         .addStaticColumn("st1", AsciiType.instance)
+                         .addRegularColumn("reg1", AsciiType.instance)
+                         .addRegularColumn("reg2", ListType.getInstance(IntegerType.instance, false))
+                         .addRegularColumn("reg3", MapType.getInstance(AsciiType.instance, IntegerType.instance, true));
+
+        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), metadata);
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS cql_test_keyspace_create_table.test_table_create_table (\n" +
         "\tpk1 varint,\n" +
         "\tpk2 ascii,\n" +
@@ -239,7 +229,7 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         "\treg2 frozen<list<varint>>,\n" +
         "\treg3 map<ascii, varint>,\n" +
         "\tPRIMARY KEY ((pk1, pk2), ck1, ck2))\n" +
-        "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+        "\tWITH ID = " + cfs.metadata.id + "\n" +
         "\tAND CLUSTERING ORDER BY (ck1 DESC, ck2 ASC)"));
     }
 
@@ -249,21 +239,20 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         String keyspace = "cql_test_keyspace_compact";
         String table = "test_table_compact";
 
-        CFMetaData cfm = CFMetaData.Builder.createDense(keyspace, table, true, false)
-                                           .addPartitionKey("pk1", IntegerType.instance)
-                                           .addPartitionKey("pk2", AsciiType.instance)
-                                           .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
-                                           .addClusteringColumn("ck2", IntegerType.instance)
-                                           .addRegularColumn("reg", IntegerType.instance)
-                                           .build();
+        TableMetadata.Builder metadata =
+            TableMetadata.builder(keyspace, table)
+                         .isDense(true)
+                         .addPartitionKeyColumn("pk1", IntegerType.instance)
+                         .addPartitionKeyColumn("pk2", AsciiType.instance)
+                         .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
+                         .addClusteringColumn("ck2", IntegerType.instance)
+                         .addRegularColumn("reg", IntegerType.instance);
 
-        SchemaLoader.createKeyspace(keyspace,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), metadata);
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS cql_test_keyspace_compact.test_table_compact (\n" +
         "\tpk1 varint,\n" +
         "\tpk2 ascii,\n" +
@@ -271,7 +260,7 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         "\tck2 varint,\n" +
         "\treg varint,\n" +
         "\tPRIMARY KEY ((pk1, pk2), ck1, ck2))\n" +
-        "\tWITH ID = " + cfm.cfId + "\n" +
+        "\tWITH ID = " + cfs.metadata.id + "\n" +
         "\tAND COMPACT STORAGE\n" +
         "\tAND CLUSTERING ORDER BY (ck1 DESC, ck2 ASC)"));
     }
@@ -282,21 +271,21 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         String keyspace = "cql_test_keyspace_counter";
         String table = "test_table_counter";
 
-        CFMetaData cfm = CFMetaData.Builder.createDense(keyspace, table, true, true)
-                                           .addPartitionKey("pk1", IntegerType.instance)
-                                           .addPartitionKey("pk2", AsciiType.instance)
-                                           .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
-                                           .addClusteringColumn("ck2", IntegerType.instance)
-                                           .addRegularColumn("cnt", CounterColumnType.instance)
-                                           .build();
+        TableMetadata.Builder metadata =
+            TableMetadata.builder(keyspace, table)
+                         .isDense(true)
+                         .isCounter(true)
+                         .addPartitionKeyColumn("pk1", IntegerType.instance)
+                         .addPartitionKeyColumn("pk2", AsciiType.instance)
+                         .addClusteringColumn("ck1", ReversedType.getInstance(IntegerType.instance))
+                         .addClusteringColumn("ck2", IntegerType.instance)
+                         .addRegularColumn("cnt", CounterColumnType.instance);
 
-        SchemaLoader.createKeyspace(keyspace,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), metadata);
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS cql_test_keyspace_counter.test_table_counter (\n" +
         "\tpk1 varint,\n" +
         "\tpk2 ascii,\n" +
@@ -304,7 +293,7 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         "\tck2 varint,\n" +
         "\tcnt counter,\n" +
         "\tPRIMARY KEY ((pk1, pk2), ck1, ck2))\n" +
-        "\tWITH ID = " + cfm.cfId + "\n" +
+        "\tWITH ID = " + cfs.metadata.id + "\n" +
         "\tAND COMPACT STORAGE\n" +
         "\tAND CLUSTERING ORDER BY (ck1 DESC, ck2 ASC)"));
     }
@@ -315,35 +304,32 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         String keyspace = "cql_test_keyspace_options";
         String table = "test_table_options";
 
-        CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
-                                           .addPartitionKey("pk1", IntegerType.instance)
-                                           .addClusteringColumn("cl1", IntegerType.instance)
-                                           .addRegularColumn("reg1", AsciiType.instance)
-                                           .build();
-
-        cfm.recordColumnDrop(cfm.getColumnDefinition(ByteBuffer.wrap("reg1".getBytes())), FBUtilities.timestampMicros());
-        cfm.bloomFilterFpChance(1.0);
-        cfm.comment("comment");
-        cfm.compaction(CompactionParams.lcs(Collections.singletonMap("sstable_size_in_mb", "1")));
-        cfm.compression(CompressionParams.lz4(1 << 16));
-        cfm.dcLocalReadRepairChance(0.2);
-        cfm.crcCheckChance(0.3);
-        cfm.defaultTimeToLive(4);
-        cfm.gcGraceSeconds(5);
-        cfm.minIndexInterval(6);
-        cfm.maxIndexInterval(7);
-        cfm.memtableFlushPeriod(8);
-        cfm.readRepairChance(0.9);
-        cfm.speculativeRetry(SpeculativeRetryParam.always());
-        cfm.extensions(ImmutableMap.of("ext1",
-                                       ByteBuffer.wrap("val1".getBytes())));
-
-        SchemaLoader.createKeyspace(keyspace,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        TableMetadata.Builder builder = TableMetadata.builder(keyspace, table);
+        builder.addPartitionKeyColumn("pk1", IntegerType.instance)
+               .addClusteringColumn("cl1", IntegerType.instance)
+               .addRegularColumn("reg1", AsciiType.instance)
+               .bloomFilterFpChance(1.0)
+               .comment("comment")
+               .compaction(CompactionParams.lcs(Collections.singletonMap("sstable_size_in_mb", "1")))
+               .compression(CompressionParams.lz4(1 << 16))
+               .dcLocalReadRepairChance(0.2)
+               .crcCheckChance(0.3)
+               .defaultTimeToLive(4)
+               .gcGraceSeconds(5)
+               .minIndexInterval(6)
+               .maxIndexInterval(7)
+               .memtableFlushPeriod(8)
+               .readRepairChance(0.9)
+               .speculativeRetry(SpeculativeRetryParam.always())
+               .extensions(ImmutableMap.of("ext1", ByteBuffer.wrap("val1".getBytes())))
+               .recordColumnDrop(ColumnMetadata.regularColumn(keyspace, table, "reg1", AsciiType.instance),
+                                 FBUtilities.timestampMicros());
+
+        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), builder);
+
         ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).endsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).endsWith(
         "AND bloom_filter_fp_chance = 1.0\n" +
         "\tAND dclocal_read_repair_chance = 0.2\n" +
         "\tAND crc_check_chance = 0.3\n" +
@@ -369,52 +355,46 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         String keyspace = "cql_test_keyspace_3";
         String table = "test_table_3";
 
-        CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
-                                           .addPartitionKey("pk1", IntegerType.instance)
-                                           .addClusteringColumn("cl1", IntegerType.instance)
-                                           .addRegularColumn("reg1", AsciiType.instance)
-                                           .build();
-
-        cfm.indexes(cfm.getIndexes()
-                       .with(IndexMetadata.fromIndexTargets(cfm,
-                                                            Collections.singletonList(new IndexTarget(cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1")).name,
-                                                                                                      IndexTarget.Type.VALUES)),
-                                                            "indexName",
-                                                            IndexMetadata.Kind.COMPOSITES,
-                                                            Collections.emptyMap()))
-                       .with(IndexMetadata.fromIndexTargets(cfm,
-                                                            Collections.singletonList(new IndexTarget(cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1")).name,
-                                                                                                      IndexTarget.Type.KEYS)),
-                                                            "indexName2",
-                                                            IndexMetadata.Kind.COMPOSITES,
-                                                            Collections.emptyMap()))
-                       .with(IndexMetadata.fromIndexTargets(cfm,
-                                                            Collections.singletonList(new IndexTarget(cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1")).name,
-                                                                                                      IndexTarget.Type.KEYS_AND_VALUES)),
-                                                            "indexName3",
-                                                            IndexMetadata.Kind.COMPOSITES,
-                                                            Collections.emptyMap()))
-                       .with(IndexMetadata.fromIndexTargets(cfm,
-                                                            Collections.singletonList(new IndexTarget(cfm.getColumnDefinition(ByteBufferUtil.bytes("reg1")).name,
-                                                                                                      IndexTarget.Type.KEYS_AND_VALUES)),
-                                                            "indexName4",
-                                                            IndexMetadata.Kind.CUSTOM,
-                                                            Collections.singletonMap(IndexTarget.CUSTOM_INDEX_OPTION_NAME,
-                                                                                     SASIIndex.class.getName()))
-                       ));
-
-
-        SchemaLoader.createKeyspace(keyspace,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        TableMetadata.Builder builder =
+            TableMetadata.builder(keyspace, table)
+                         .addPartitionKeyColumn("pk1", IntegerType.instance)
+                         .addClusteringColumn("cl1", IntegerType.instance)
+                         .addRegularColumn("reg1", AsciiType.instance);
+
+        ColumnIdentifier reg1 = ColumnIdentifier.getInterned("reg1", true);
+
+        builder.indexes(
+            Indexes.of(IndexMetadata.fromIndexTargets(
+            Collections.singletonList(new IndexTarget(reg1, IndexTarget.Type.VALUES)),
+                                                      "indexName",
+                                                      IndexMetadata.Kind.COMPOSITES,
+                                                      Collections.emptyMap()),
+                       IndexMetadata.fromIndexTargets(
+                       Collections.singletonList(new IndexTarget(reg1, IndexTarget.Type.KEYS)),
+                                                      "indexName2",
+                                                      IndexMetadata.Kind.COMPOSITES,
+                                                      Collections.emptyMap()),
+                       IndexMetadata.fromIndexTargets(
+                       Collections.singletonList(new IndexTarget(reg1, IndexTarget.Type.KEYS_AND_VALUES)),
+                                                      "indexName3",
+                                                      IndexMetadata.Kind.COMPOSITES,
+                                                      Collections.emptyMap()),
+                       IndexMetadata.fromIndexTargets(
+                       Collections.singletonList(new IndexTarget(reg1, IndexTarget.Type.KEYS_AND_VALUES)),
+                                                      "indexName4",
+                                                      IndexMetadata.Kind.CUSTOM,
+                                                      Collections.singletonMap(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName()))));
+
+
+        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), builder);
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
 
-        assertEquals(ImmutableList.of("CREATE INDEX \"indexName\" ON cql_test_keyspace_3.test_table_3 (reg1);",
-                                      "CREATE INDEX \"indexName2\" ON cql_test_keyspace_3.test_table_3 (reg1);",
-                                      "CREATE INDEX \"indexName3\" ON cql_test_keyspace_3.test_table_3 (reg1);",
-                                      "CREATE CUSTOM INDEX \"indexName4\" ON cql_test_keyspace_3.test_table_3 (reg1) USING 'org.apache.cassandra.index.sasi.SASIIndex';"),
-                     ColumnFamilyStoreCQLHelper.getIndexesAsCQL(cfs.metadata));
+        assertEquals(ImmutableList.of("CREATE INDEX \"indexName\" ON cql_test_keyspace_3.test_table_3 (values(reg1));",
+                                      "CREATE INDEX \"indexName2\" ON cql_test_keyspace_3.test_table_3 (keys(reg1));",
+                                      "CREATE INDEX \"indexName3\" ON cql_test_keyspace_3.test_table_3 (entries(reg1));",
+                                      "CREATE CUSTOM INDEX \"indexName4\" ON cql_test_keyspace_3.test_table_3 (entries(reg1)) USING 'org.apache.cassandra.index.sasi.SASIIndex';"),
+                     ColumnFamilyStoreCQLHelper.getIndexesAsCQL(cfs.metadata()));
     }
 
     private final static String SNAPSHOT = "testsnapshot";
@@ -447,10 +427,10 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         cfs.snapshot(SNAPSHOT);
 
         String schema = Files.toString(cfs.getDirectories().getSnapshotSchemaFile(SNAPSHOT), Charset.defaultCharset());
-        assertTrue(schema.contains(String.format("CREATE TYPE %s.%s(a1 varint, a2 varint, a3 varint);", keyspace(), typeA)));
-        assertTrue(schema.contains(String.format("CREATE TYPE %s.%s(a1 varint, a2 varint, a3 varint);", keyspace(), typeA)));
-        assertTrue(schema.contains(String.format("CREATE TYPE %s.%s(b1 frozen<%s>, b2 frozen<%s>, b3 frozen<%s>);", keyspace(), typeB, typeA, typeA, typeA)));
-        assertTrue(schema.contains(String.format("CREATE TYPE %s.%s(c1 frozen<%s>, c2 frozen<%s>, c3 frozen<%s>);", keyspace(), typeC, typeB, typeB, typeB)));
+        assertTrue(schema.contains(String.format("CREATE TYPE %s.%s (a1 varint, a2 varint, a3 varint);", keyspace(), typeA)));
+        assertTrue(schema.contains(String.format("CREATE TYPE %s.%s (a1 varint, a2 varint, a3 varint);", keyspace(), typeA)));
+        assertTrue(schema.contains(String.format("CREATE TYPE %s.%s (b1 frozen<%s>, b2 frozen<%s>, b3 frozen<%s>);", keyspace(), typeB, typeA, typeA, typeA)));
+        assertTrue(schema.contains(String.format("CREATE TYPE %s.%s (c1 frozen<%s>, c2 frozen<%s>, c3 frozen<%s>);", keyspace(), typeC, typeB, typeB, typeB)));
 
         schema = schema.substring(schema.indexOf("CREATE TABLE")); // trim to ensure order
 
@@ -463,7 +443,7 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
                                      "\treg3 int,\n" +
                                      "\treg1 " + typeC + ",\n" +
                                      "\tPRIMARY KEY ((pk1, pk2), ck1, ck2))\n" +
-                                     "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+                                     "\tWITH ID = " + cfs.metadata.id + "\n" +
                                      "\tAND CLUSTERING ORDER BY (ck1 ASC, ck2 DESC)"));
 
         schema = schema.substring(schema.indexOf("ALTER"));
@@ -539,11 +519,11 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
         "\tpk1 varint PRIMARY KEY,\n" +
         "\treg1 int)\n" +
-        "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+        "\tWITH ID = " + cfs.metadata.id + "\n" +
         "\tAND COMPACT STORAGE"));
     }
 
@@ -558,12 +538,12 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
         "\tpk1 varint PRIMARY KEY,\n" +
         "\treg1 int,\n" +
         "\treg2 int)\n" +
-        "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+        "\tWITH ID = " + cfs.metadata.id + "\n" +
         "\tAND COMPACT STORAGE"));
     }
 
@@ -579,12 +559,12 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
         "\tpk1 varint PRIMARY KEY,\n" +
         "\treg1 counter,\n" +
         "\treg2 counter)\n" +
-        "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+        "\tWITH ID = " + cfs.metadata.id + "\n" +
         "\tAND COMPACT STORAGE"));
     }
 
@@ -599,12 +579,12 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
         "\tpk1 varint,\n" +
         "\tck1 int,\n" +
         "\tPRIMARY KEY (pk1, ck1))\n" +
-        "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+        "\tWITH ID = " + cfs.metadata.id + "\n" +
         "\tAND COMPACT STORAGE"));
     }
 
@@ -620,13 +600,13 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
 
         ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(tableName);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS " + keyspace() + "." + tableName + " (\n" +
         "\tpk1 varint,\n" +
         "\tck1 int,\n" +
         "\treg int,\n" +
         "\tPRIMARY KEY (pk1, ck1))\n" +
-        "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+        "\tWITH ID = " + cfs.metadata.id + "\n" +
         "\tAND COMPACT STORAGE"));
     }
 
@@ -647,13 +627,13 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
 
         ColumnFamilyStore cfs = Keyspace.open(DYNAMIC_COMPOSITE).getColumnFamilyStore(DYNAMIC_COMPOSITE);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "CREATE TABLE IF NOT EXISTS " + DYNAMIC_COMPOSITE + "." + DYNAMIC_COMPOSITE + " (\n" +
         "\tkey ascii,\n" +
         "\tcols 'org.apache.cassandra.db.marshal.DynamicCompositeType(a=>org.apache.cassandra.db.marshal.BytesType,b=>org.apache.cassandra.db.marshal.BytesType,c=>org.apache.cassandra.db.marshal.BytesType)',\n" +
         "\tval ascii,\n" +
         "\tPRIMARY KEY (key, cols))\n" +
-        "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+        "\tWITH ID = " + cfs.metadata.id + "\n" +
         "\tAND COMPACT STORAGE"));
     }
 
@@ -663,20 +643,19 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         final String KEYSPACE = "thrift_compact_table_with_supercolumns_test";
         final String TABLE = "test_table_1";
 
-        CFMetaData cfm = CFMetaData.Builder.createSuper(KEYSPACE, TABLE, false)
-                                           .addPartitionKey("pk", BytesType.instance)
-                                           .addClusteringColumn("c1", AsciiType.instance)
-                                           .addClusteringColumn("c2", AsciiType.instance)
-                                           .addRegularColumn("", MapType.getInstance(Int32Type.instance, AsciiType.instance, true))
-                                           .build();
+        TableMetadata.Builder table =
+            TableMetadata.builder(KEYSPACE, TABLE)
+                         .isSuper(true)
+                         .addPartitionKeyColumn("pk", BytesType.instance)
+                         .addClusteringColumn("c1", AsciiType.instance)
+                         .addClusteringColumn("c2", AsciiType.instance)
+                         .addRegularColumn("", MapType.getInstance(Int32Type.instance, AsciiType.instance, true));
 
-        SchemaLoader.createKeyspace(KEYSPACE,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), table);
 
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(TABLE);
 
-        assertTrue(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(cfs.metadata, true).startsWith(
+        assertTrue(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(cfs.metadata(), true).startsWith(
         "/*\n" +
         "Warning: Table " + KEYSPACE + "." + TABLE + " omitted because it has constructs not compatible with CQL (was created via legacy API).\n\n" +
         "Approximate structure, for reference:\n" +
@@ -687,7 +666,7 @@ public class ColumnFamilyStoreCQLHelperTest extends CQLTester
         "\tc2 ascii,\n" +
         "\t\"\" map<int, ascii>,\n" +
         "\tPRIMARY KEY (pk, c1, c2))\n" +
-        "\tWITH ID = " + cfs.metadata.cfId + "\n" +
+        "\tWITH ID = " + cfs.metadata.id + "\n" +
         "\tAND COMPACT STORAGE"));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index a2003d8..e5c5727 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -34,7 +34,6 @@ import static org.junit.Assert.assertTrue;
 
 import com.google.common.collect.Iterators;
 import org.apache.cassandra.*;
-import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.rows.*;
@@ -46,6 +45,7 @@ import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.metrics.ClearableHistogram;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -97,14 +97,14 @@ public class ColumnFamilyStoreTest
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
 
-        new RowUpdateBuilder(cfs.metadata, 0, "key1")
+        new RowUpdateBuilder(cfs.metadata(), 0, "key1")
                 .clustering("Column1")
                 .add("val", "asdf")
                 .build()
                 .applyUnsafe();
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 1, "key1")
+        new RowUpdateBuilder(cfs.metadata(), 1, "key1")
                 .clustering("Column1")
                 .add("val", "asdf")
                 .build()
@@ -123,7 +123,7 @@ public class ColumnFamilyStoreTest
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
 
         List<Mutation> rms = new LinkedList<>();
-        rms.add(new RowUpdateBuilder(cfs.metadata, 0, "key1")
+        rms.add(new RowUpdateBuilder(cfs.metadata(), 0, "key1")
                 .clustering("Column1")
                 .add("val", "asdf")
                 .build());
@@ -142,7 +142,7 @@ public class ColumnFamilyStoreTest
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD2);
 
-        RowUpdateBuilder.deleteRow(cfs.metadata, FBUtilities.timestampMicros(), "key1", "Column1").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), FBUtilities.timestampMicros(), "key1", "Column1").applyUnsafe();
 
         Runnable r = new WrappedRunnable()
         {
@@ -170,22 +170,22 @@ public class ColumnFamilyStoreTest
         ByteBuffer val = ByteBufferUtil.bytes("val1");
 
         // insert
-        ColumnDefinition newCol = ColumnDefinition.regularDef(cfs.metadata, ByteBufferUtil.bytes("val2"), AsciiType.instance);
-        new RowUpdateBuilder(cfs.metadata, 0, "key1").clustering("Column1").add("val", "val1").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "key2").clustering("Column1").add("val", "val1").build().applyUnsafe();
+        ColumnMetadata newCol = ColumnMetadata.regularColumn(cfs.metadata(), ByteBufferUtil.bytes("val2"), AsciiType.instance);
+        new RowUpdateBuilder(cfs.metadata(), 0, "key1").clustering("Column1").add("val", "val1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "key2").clustering("Column1").add("val", "val1").build().applyUnsafe();
         assertRangeCount(cfs, col, val, 2);
 
         // flush.
         cfs.forceBlockingFlush();
 
         // insert, don't flush
-        new RowUpdateBuilder(cfs.metadata, 1, "key3").clustering("Column1").add("val", "val1").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 1, "key4").clustering("Column1").add("val", "val1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, "key3").clustering("Column1").add("val", "val1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, "key4").clustering("Column1").add("val", "val1").build().applyUnsafe();
         assertRangeCount(cfs, col, val, 4);
 
         // delete (from sstable and memtable)
-        RowUpdateBuilder.deleteRow(cfs.metadata, 5, "key1", "Column1").applyUnsafe();
-        RowUpdateBuilder.deleteRow(cfs.metadata, 5, "key3", "Column1").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), 5, "key1", "Column1").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), 5, "key3", "Column1").applyUnsafe();
 
         // verify delete
         assertRangeCount(cfs, col, val, 2);
@@ -197,15 +197,15 @@ public class ColumnFamilyStoreTest
         assertRangeCount(cfs, col, val, 2);
 
         // simulate a 'late' insertion that gets put in after the deletion. should get inserted, but fail on read.
-        new RowUpdateBuilder(cfs.metadata, 2, "key1").clustering("Column1").add("val", "val1").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 2, "key3").clustering("Column1").add("val", "val1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 2, "key1").clustering("Column1").add("val", "val1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 2, "key3").clustering("Column1").add("val", "val1").build().applyUnsafe();
 
         // should still be nothing there because we deleted this row. 2nd breakage, but was undetected because of 1837.
         assertRangeCount(cfs, col, val, 2);
 
         // make sure that new writes are recognized.
-        new RowUpdateBuilder(cfs.metadata, 10, "key5").clustering("Column1").add("val", "val1").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 10, "key6").clustering("Column1").add("val", "val1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 10, "key5").clustering("Column1").add("val", "val1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 10, "key6").clustering("Column1").add("val", "val1").build().applyUnsafe();
         assertRangeCount(cfs, col, val, 4);
 
         // and it remains so after flush. (this wasn't failing before, but it's good to check.)
@@ -257,9 +257,9 @@ public class ColumnFamilyStoreTest
     public void testBackupAfterFlush() throws Throwable
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE2).getColumnFamilyStore(CF_STANDARD1);
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key1")).clustering("Column1").add("val", "asdf").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key1")).clustering("Column1").add("val", "asdf").build().applyUnsafe();
         cfs.forceBlockingFlush();
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key2")).clustering("Column1").add("val", "asdf").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key2")).clustering("Column1").add("val", "asdf").build().applyUnsafe();
         cfs.forceBlockingFlush();
 
         for (int version = 1; version <= 2; ++version)
@@ -319,7 +319,7 @@ public class ColumnFamilyStoreTest
 //        ColumnFamilyStore cfs = Keyspace.open(ks).getColumnFamilyStore(cf);
 //        SSTableDeletingTask.waitForDeletions();
 //
-//        final CFMetaData cfmeta = Schema.instance.getCFMetaData(ks, cf);
+//        final CFMetaData cfmeta = Schema.instance.getTableMetadataRef(ks, cf);
 //        Directories dir = new Directories(cfs.metadata);
 //
 //        // clear old SSTables (probably left by CFS.clearUnsafe() calls in other tests)
@@ -405,10 +405,10 @@ public class ColumnFamilyStoreTest
 
     private void assertRangeCount(ColumnFamilyStore cfs, ByteBuffer col, ByteBuffer val, int count)
     {
-        assertRangeCount(cfs, cfs.metadata.getColumnDefinition(col), val, count);
+        assertRangeCount(cfs, cfs.metadata().getColumn(col), val, count);
     }
 
-    private void assertRangeCount(ColumnFamilyStore cfs, ColumnDefinition col, ByteBuffer val, int count)
+    private void assertRangeCount(ColumnFamilyStore cfs, ColumnMetadata col, ByteBuffer val, int count)
     {
 
         int found = 0;
@@ -431,9 +431,9 @@ public class ColumnFamilyStoreTest
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
 
-        ColumnFamilyStore.scrubDataDirectories(cfs.metadata);
+        ColumnFamilyStore.scrubDataDirectories(cfs.metadata());
 
-        new RowUpdateBuilder(cfs.metadata, 2, "key").clustering("name").add("val", "2").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 2, "key").clustering("name").add("val", "2").build().applyUnsafe();
         cfs.forceBlockingFlush();
 
         // Nuke the metadata and reload that sstable
@@ -447,9 +447,9 @@ public class ColumnFamilyStoreTest
 
         ssTable.selfRef().release();
 
-        ColumnFamilyStore.scrubDataDirectories(cfs.metadata);
+        ColumnFamilyStore.scrubDataDirectories(cfs.metadata());
 
-        List<File> ssTableFiles = new Directories(cfs.metadata).sstableLister(Directories.OnTxnErr.THROW).listFiles();
+        List<File> ssTableFiles = new Directories(cfs.metadata()).sstableLister(Directories.OnTxnErr.THROW).listFiles();
         assertNotNull(ssTableFiles);
         assertEquals(0, ssTableFiles.size());
     }


[27/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java b/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
deleted file mode 100644
index 4270a24..0000000
--- a/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
+++ /dev/null
@@ -1,42 +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.util.UUID;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.Schema;
-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.utils.UUIDSerializer;
-
-public class SchemaCheckVerbHandler implements IVerbHandler
-{
-    private final Logger logger = LoggerFactory.getLogger(SchemaCheckVerbHandler.class);
-
-    public void doVerb(MessageIn message, int id)
-    {
-        logger.trace("Received schema check request.");
-        MessageOut<UUID> response = new MessageOut<UUID>(MessagingService.Verb.INTERNAL_RESPONSE, Schema.instance.getVersion(), UUIDSerializer.serializer);
-        MessagingService.instance().sendReply(response, id, message.from);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/SerializationHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java
index 729d556..1f937f8 100644
--- a/src/java/org/apache/cassandra/db/SerializationHeader.java
+++ b/src/java/org/apache/cassandra/db/SerializationHeader.java
@@ -21,20 +21,20 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.rows.*;
 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.db.marshal.UTF8Type;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.Version;
-import org.apache.cassandra.io.sstable.metadata.MetadataType;
-import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.IMetadataComponentSerializer;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class SerializationHeader
@@ -46,7 +46,7 @@ public class SerializationHeader
     private final AbstractType<?> keyType;
     private final List<AbstractType<?>> clusteringTypes;
 
-    private final PartitionColumns columns;
+    private final RegularAndStaticColumns columns;
     private final EncodingStats stats;
 
     private final Map<ByteBuffer, AbstractType<?>> typeMap;
@@ -54,7 +54,7 @@ public class SerializationHeader
     private SerializationHeader(boolean isForSSTable,
                                 AbstractType<?> keyType,
                                 List<AbstractType<?>> clusteringTypes,
-                                PartitionColumns columns,
+                                RegularAndStaticColumns columns,
                                 EncodingStats stats,
                                 Map<ByteBuffer, AbstractType<?>> typeMap)
     {
@@ -66,12 +66,12 @@ public class SerializationHeader
         this.typeMap = typeMap;
     }
 
-    public static SerializationHeader makeWithoutStats(CFMetaData metadata)
+    public static SerializationHeader makeWithoutStats(TableMetadata metadata)
     {
-        return new SerializationHeader(true, metadata, metadata.partitionColumns(), EncodingStats.NO_STATS);
+        return new SerializationHeader(true, metadata, metadata.regularAndStaticColumns(), EncodingStats.NO_STATS);
     }
 
-    public static SerializationHeader make(CFMetaData metadata, Collection<SSTableReader> sstables)
+    public static SerializationHeader make(TableMetadata metadata, Collection<SSTableReader> sstables)
     {
         // The serialization header has to be computed before the start of compaction (since it's used to write)
         // the result. This means that when compacting multiple sources, we won't have perfectly accurate stats
@@ -84,34 +84,31 @@ public class SerializationHeader
         // our stats merging on the compacted files headers, which as we just said can be somewhat inaccurate,
         // but rather on their stats stored in StatsMetadata that are fully accurate.
         EncodingStats.Collector stats = new EncodingStats.Collector();
-        PartitionColumns.Builder columns = PartitionColumns.builder();
+        RegularAndStaticColumns.Builder columns = RegularAndStaticColumns.builder();
         for (SSTableReader sstable : sstables)
         {
             stats.updateTimestamp(sstable.getMinTimestamp());
             stats.updateLocalDeletionTime(sstable.getMinLocalDeletionTime());
             stats.updateTTL(sstable.getMinTTL());
-            if (sstable.header == null)
-                columns.addAll(metadata.partitionColumns());
-            else
-                columns.addAll(sstable.header.columns());
+            columns.addAll(sstable.header.columns());
         }
         return new SerializationHeader(true, metadata, columns.build(), stats.get());
     }
 
     public SerializationHeader(boolean isForSSTable,
-                               CFMetaData metadata,
-                               PartitionColumns columns,
+                               TableMetadata metadata,
+                               RegularAndStaticColumns columns,
                                EncodingStats stats)
     {
         this(isForSSTable,
-             metadata.getKeyValidator(),
+             metadata.partitionKeyType,
              metadata.comparator.subtypes(),
              columns,
              stats,
              null);
     }
 
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
         return columns;
     }
@@ -146,7 +143,7 @@ public class SerializationHeader
         return isStatic ? columns.statics : columns.regulars;
     }
 
-    public AbstractType<?> getType(ColumnDefinition column)
+    public AbstractType<?> getType(ColumnMetadata column)
     {
         return typeMap == null ? column.type : typeMap.get(column.name.bytes);
     }
@@ -240,9 +237,9 @@ public class SerializationHeader
     {
         Map<ByteBuffer, AbstractType<?>> staticColumns = new LinkedHashMap<>();
         Map<ByteBuffer, AbstractType<?>> regularColumns = new LinkedHashMap<>();
-        for (ColumnDefinition column : columns.statics)
+        for (ColumnMetadata column : columns.statics)
             staticColumns.put(column.name.bytes, column.type);
-        for (ColumnDefinition column : columns.regulars)
+        for (ColumnMetadata column : columns.regulars)
             regularColumns.put(column.name.bytes, column.type);
         return new Component(keyType, clusteringTypes, staticColumns, regularColumns, stats);
     }
@@ -254,7 +251,7 @@ public class SerializationHeader
     }
 
     /**
-     * We need the CFMetadata to properly deserialize a SerializationHeader but it's clunky to pass that to
+     * We need the TableMetadata to properly deserialize a SerializationHeader but it's clunky to pass that to
      * a SSTable component, so we use this temporary object to delay the actual need for the metadata.
      */
     public static class Component extends MetadataComponent
@@ -283,16 +280,16 @@ public class SerializationHeader
             return MetadataType.HEADER;
         }
 
-        public SerializationHeader toHeader(CFMetaData metadata)
+        public SerializationHeader toHeader(TableMetadata metadata)
         {
             Map<ByteBuffer, AbstractType<?>> typeMap = new HashMap<>(staticColumns.size() + regularColumns.size());
             typeMap.putAll(staticColumns);
             typeMap.putAll(regularColumns);
 
-            PartitionColumns.Builder builder = PartitionColumns.builder();
+            RegularAndStaticColumns.Builder builder = RegularAndStaticColumns.builder();
             for (ByteBuffer name : typeMap.keySet())
             {
-                ColumnDefinition column = metadata.getColumnDefinition(name);
+                ColumnMetadata column = metadata.getColumn(name);
                 if (column == null)
                 {
                     // TODO: this imply we don't read data for a column we don't yet know about, which imply this is theoretically
@@ -301,10 +298,10 @@ public class SerializationHeader
                     // improve this.
 
                     // If we don't find the definition, it could be we have data for a dropped column, and we shouldn't
-                    // fail deserialization because of that. So we grab a "fake" ColumnDefinition that ensure proper
+                    // fail deserialization because of that. So we grab a "fake" ColumnMetadata that ensure proper
                     // deserialization. The column will be ignore later on anyway.
                     boolean isStatic = staticColumns.containsKey(name);
-                    column = metadata.getDroppedColumnDefinition(name, isStatic);
+                    column = metadata.getDroppedColumn(name, isStatic);
                     if (column == null)
                         throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
                 }
@@ -386,11 +383,11 @@ public class SerializationHeader
             }
         }
 
-        public SerializationHeader deserializeForMessaging(DataInputPlus in, CFMetaData metadata, ColumnFilter selection, boolean hasStatic) throws IOException
+        public SerializationHeader deserializeForMessaging(DataInputPlus in, TableMetadata metadata, ColumnFilter selection, boolean hasStatic) throws IOException
         {
             EncodingStats stats = EncodingStats.serializer.deserialize(in);
 
-            AbstractType<?> keyType = metadata.getKeyValidator();
+            AbstractType<?> keyType = metadata.partitionKeyType;
             List<AbstractType<?>> clusteringTypes = metadata.comparator.subtypes();
 
             Columns statics, regulars;
@@ -405,7 +402,7 @@ public class SerializationHeader
                 regulars = Columns.serializer.deserializeSubset(selection.fetchedColumns().regulars, in);
             }
 
-            return new SerializationHeader(false, keyType, clusteringTypes, new PartitionColumns(statics, regulars), stats, null);
+            return new SerializationHeader(false, keyType, clusteringTypes, new RegularAndStaticColumns(statics, regulars), stats, null);
         }
 
         public long serializedSizeForMessaging(SerializationHeader header, ColumnFilter selection, boolean hasStatic)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/SimpleBuilders.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SimpleBuilders.java b/src/java/org/apache/cassandra/db/SimpleBuilders.java
index 6e65743..a212834 100644
--- a/src/java/org/apache/cassandra/db/SimpleBuilders.java
+++ b/src/java/org/apache/cassandra/db/SimpleBuilders.java
@@ -20,9 +20,10 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -43,16 +44,16 @@ public abstract class SimpleBuilders
     {
     }
 
-    private static DecoratedKey makePartitonKey(CFMetaData metadata, Object... partitionKey)
+    private static DecoratedKey makePartitonKey(TableMetadata metadata, Object... partitionKey)
     {
         if (partitionKey.length == 1 && partitionKey[0] instanceof DecoratedKey)
             return (DecoratedKey)partitionKey[0];
 
-        ByteBuffer key = CFMetaData.serializePartitionKey(metadata.getKeyValidatorAsClusteringComparator().make(partitionKey));
-        return metadata.decorateKey(key);
+        ByteBuffer key = metadata.partitionKeyAsClusteringComparator().make(partitionKey).serializeAsPartitionKey();
+        return metadata.partitioner.decorateKey(key);
     }
 
-    private static Clustering makeClustering(CFMetaData metadata, Object... clusteringColumns)
+    private static Clustering makeClustering(TableMetadata metadata, Object... clusteringColumns)
     {
         if (clusteringColumns.length == 1 && clusteringColumns[0] instanceof Clustering)
             return (Clustering)clusteringColumns[0];
@@ -61,7 +62,7 @@ public abstract class SimpleBuilders
         {
             // If the table has clustering columns, passing no values is for updating the static values, so check we
             // do have some static columns defined.
-            assert metadata.comparator.size() == 0 || !metadata.partitionColumns().statics.isEmpty();
+            assert metadata.comparator.size() == 0 || !metadata.staticColumns().isEmpty();
             return metadata.comparator.size() == 0 ? Clustering.EMPTY : Clustering.STATIC_CLUSTERING;
         }
         else
@@ -107,7 +108,7 @@ public abstract class SimpleBuilders
         private final String keyspaceName;
         private final DecoratedKey key;
 
-        private final Map<UUID, PartitionUpdateBuilder> updateBuilders = new HashMap<>();
+        private final Map<TableId, PartitionUpdateBuilder> updateBuilders = new HashMap<>();
 
         public MutationBuilder(String keyspaceName, DecoratedKey key)
         {
@@ -115,15 +116,15 @@ public abstract class SimpleBuilders
             this.key = key;
         }
 
-        public PartitionUpdate.SimpleBuilder update(CFMetaData metadata)
+        public PartitionUpdate.SimpleBuilder update(TableMetadata metadata)
         {
-            assert metadata.ksName.equals(keyspaceName);
+            assert metadata.keyspace.equals(keyspaceName);
 
-            PartitionUpdateBuilder builder = updateBuilders.get(metadata.cfId);
+            PartitionUpdateBuilder builder = updateBuilders.get(metadata.id);
             if (builder == null)
             {
                 builder = new PartitionUpdateBuilder(metadata, key);
-                updateBuilders.put(metadata.cfId, builder);
+                updateBuilders.put(metadata.id, builder);
             }
 
             copyParams(builder);
@@ -133,7 +134,7 @@ public abstract class SimpleBuilders
 
         public PartitionUpdate.SimpleBuilder update(String tableName)
         {
-            CFMetaData metadata = Schema.instance.getCFMetaData(keyspaceName, tableName);
+            TableMetadata metadata = Schema.instance.getTableMetadata(keyspaceName, tableName);
             assert metadata != null : "Unknown table " + tableName + " in keyspace " + keyspaceName;
             return update(metadata);
         }
@@ -154,20 +155,20 @@ public abstract class SimpleBuilders
 
     public static class PartitionUpdateBuilder extends AbstractBuilder<PartitionUpdate.SimpleBuilder> implements PartitionUpdate.SimpleBuilder
     {
-        private final CFMetaData metadata;
+        private final TableMetadata metadata;
         private final DecoratedKey key;
         private final Map<Clustering, RowBuilder> rowBuilders = new HashMap<>();
         private List<RTBuilder> rangeBuilders = null; // We use that rarely, so create lazily
 
         private DeletionTime partitionDeletion = DeletionTime.LIVE;
 
-        public PartitionUpdateBuilder(CFMetaData metadata, Object... partitionKeyValues)
+        public PartitionUpdateBuilder(TableMetadata metadata, Object... partitionKeyValues)
         {
             this.metadata = metadata;
             this.key = makePartitonKey(metadata, partitionKeyValues);
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
             return metadata;
         }
@@ -206,7 +207,7 @@ public abstract class SimpleBuilders
         public PartitionUpdate build()
         {
             // Collect all updated columns
-            PartitionColumns.Builder columns = PartitionColumns.builder();
+            RegularAndStaticColumns.Builder columns = RegularAndStaticColumns.builder();
             for (RowBuilder builder : rowBuilders.values())
                 columns.addAll(builder.columns());
 
@@ -296,15 +297,15 @@ public abstract class SimpleBuilders
 
     public static class RowBuilder extends AbstractBuilder<Row.SimpleBuilder> implements Row.SimpleBuilder
     {
-        private final CFMetaData metadata;
+        private final TableMetadata metadata;
 
-        private final Set<ColumnDefinition> columns = new HashSet<>();
+        private final Set<ColumnMetadata> columns = new HashSet<>();
         private final Row.Builder builder;
 
         private boolean initiated;
         private boolean noPrimaryKeyLivenessInfo;
 
-        public RowBuilder(CFMetaData metadata, Object... clusteringColumns)
+        public RowBuilder(TableMetadata metadata, Object... clusteringColumns)
         {
             this.metadata = metadata;
             this.builder = BTreeRow.unsortedBuilder(FBUtilities.nowInSeconds());
@@ -312,7 +313,7 @@ public abstract class SimpleBuilders
             this.builder.newRow(makeClustering(metadata, clusteringColumns));
         }
 
-        Set<ColumnDefinition> columns()
+        Set<ColumnMetadata> columns()
         {
             return columns;
         }
@@ -345,7 +346,7 @@ public abstract class SimpleBuilders
         private Row.SimpleBuilder add(String columnName, Object value, boolean overwriteForCollection)
         {
             maybeInit();
-            ColumnDefinition column = getColumn(columnName);
+            ColumnMetadata column = getColumn(columnName);
 
             if (!overwriteForCollection && !(column.type.isMultiCell() && column.type.isCollection()))
                 throw new IllegalArgumentException("appendAll() can only be called on non-frozen colletions");
@@ -421,16 +422,16 @@ public abstract class SimpleBuilders
             return builder.build();
         }
 
-        private ColumnDefinition getColumn(String columnName)
+        private ColumnMetadata getColumn(String columnName)
         {
-            ColumnDefinition column = metadata.getColumnDefinition(new ColumnIdentifier(columnName, true));
+            ColumnMetadata column = metadata.getColumn(new ColumnIdentifier(columnName, true));
             assert column != null : "Cannot find column " + columnName;
             assert !column.isPrimaryKeyColumn();
             assert !column.isStatic() || builder.clustering() == Clustering.STATIC_CLUSTERING : "Cannot add non-static column to static-row";
             return column;
         }
 
-        private Cell cell(ColumnDefinition column, ByteBuffer value, CellPath path)
+        private Cell cell(ColumnMetadata column, ByteBuffer value, CellPath path)
         {
             if (value == null)
                 return BufferCell.tombstone(column, timestamp, nowInSec, path);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 9781bb9..cb50c43 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -32,11 +32,9 @@ import org.apache.cassandra.cache.RowCacheKey;
 import org.apache.cassandra.cache.RowCacheSentinel;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.lifecycle.*;
 import org.apache.cassandra.db.filter.*;
+import org.apache.cassandra.db.lifecycle.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.transform.Transformation;
@@ -47,7 +45,9 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.metrics.TableMetrics;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.IndexMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.StorageProxy;
@@ -58,7 +58,6 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.SearchIterator;
 import org.apache.cassandra.utils.btree.BTreeSet;
 
-
 /**
  * A read command that selects a (part of a) single partition.
  */
@@ -73,7 +72,7 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     public SinglePartitionReadCommand(boolean isDigest,
                                       int digestVersion,
-                                      CFMetaData metadata,
+                                      TableMetadata metadata,
                                       int nowInSec,
                                       ColumnFilter columnFilter,
                                       RowFilter rowFilter,
@@ -100,7 +99,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      *
      * @return a newly created read command.
      */
-    public static SinglePartitionReadCommand create(CFMetaData metadata,
+    public static SinglePartitionReadCommand create(TableMetadata metadata,
                                                     int nowInSec,
                                                     ColumnFilter columnFilter,
                                                     RowFilter rowFilter,
@@ -122,7 +121,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      *
      * @return a newly created read command. The returned command will use no row filter and have no limits.
      */
-    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, ColumnFilter columnFilter, ClusteringIndexFilter filter)
+    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, ColumnFilter columnFilter, ClusteringIndexFilter filter)
     {
         return create(metadata, nowInSec, columnFilter, RowFilter.NONE, DataLimits.NONE, key, filter);
     }
@@ -136,7 +135,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      *
      * @return a newly created read command that queries all the rows of {@code key}.
      */
-    public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, DecoratedKey key)
+    public static SinglePartitionReadCommand fullPartitionRead(TableMetadata metadata, int nowInSec, DecoratedKey key)
     {
         return SinglePartitionReadCommand.create(metadata, nowInSec, key, Slices.ALL);
     }
@@ -150,9 +149,9 @@ public class SinglePartitionReadCommand extends ReadCommand
      *
      * @return a newly created read command that queries all the rows of {@code key}.
      */
-    public static SinglePartitionReadCommand fullPartitionRead(CFMetaData metadata, int nowInSec, ByteBuffer key)
+    public static SinglePartitionReadCommand fullPartitionRead(TableMetadata metadata, int nowInSec, ByteBuffer key)
     {
-        return SinglePartitionReadCommand.create(metadata, nowInSec, metadata.decorateKey(key), Slices.ALL);
+        return SinglePartitionReadCommand.create(metadata, nowInSec, metadata.partitioner.decorateKey(key), Slices.ALL);
     }
 
     /**
@@ -166,7 +165,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      * @return a newly created read command that queries {@code slice} in {@code key}. The returned query will
      * query every columns for the table (without limit or row filtering) and be in forward order.
      */
-    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Slice slice)
+    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, Slice slice)
     {
         return create(metadata, nowInSec, key, Slices.with(metadata.comparator, slice));
     }
@@ -182,7 +181,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      * @return a newly created read command that queries the {@code slices} in {@code key}. The returned query will
      * query every columns for the table (without limit or row filtering) and be in forward order.
      */
-    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Slices slices)
+    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, Slices slices)
     {
         ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(slices, false);
         return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
@@ -199,9 +198,9 @@ public class SinglePartitionReadCommand extends ReadCommand
      * @return a newly created read command that queries the {@code slices} in {@code key}. The returned query will
      * query every columns for the table (without limit or row filtering) and be in forward order.
      */
-    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, ByteBuffer key, Slices slices)
+    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, ByteBuffer key, Slices slices)
     {
-        return create(metadata, nowInSec, metadata.decorateKey(key), slices);
+        return create(metadata, nowInSec, metadata.partitioner.decorateKey(key), slices);
     }
 
     /**
@@ -215,7 +214,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      * @return a newly created read command that queries the {@code names} in {@code key}. The returned query will
      * query every columns (without limit or row filtering) and be in forward order.
      */
-    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
+    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
     {
         ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names, false);
         return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
@@ -232,7 +231,7 @@ public class SinglePartitionReadCommand extends ReadCommand
      * @return a newly created read command that queries {@code name} in {@code key}. The returned query will
      * query every columns (without limit or row filtering).
      */
-    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Clustering name)
+    public static SinglePartitionReadCommand create(TableMetadata metadata, int nowInSec, DecoratedKey key, Clustering name)
     {
         return create(metadata, nowInSec, key, FBUtilities.singleton(name, metadata.comparator));
     }
@@ -267,7 +266,7 @@ public class SinglePartitionReadCommand extends ReadCommand
         if (!this.partitionKey().equals(key))
             return false;
 
-        return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().getKeyValidator());
+        return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().partitionKeyType);
     }
 
     public boolean selectsClustering(DecoratedKey key, Clustering clustering)
@@ -360,7 +359,7 @@ public class SinglePartitionReadCommand extends ReadCommand
         assert !cfs.isIndex(); // CASSANDRA-5732
         assert cfs.isRowCacheEnabled() : String.format("Row cache is not enabled on table [%s]", cfs.name);
 
-        RowCacheKey key = new RowCacheKey(metadata().ksAndCFName, partitionKey());
+        RowCacheKey key = new RowCacheKey(metadata(), partitionKey());
 
         // Attempt a sentinel-read-cache sequence.  if a write invalidates our sentinel, we'll return our
         // (now potentially obsolete) data, but won't cache it. see CASSANDRA-3862
@@ -601,9 +600,9 @@ public class SinglePartitionReadCommand extends ReadCommand
                                nonIntersectingSSTables, view.sstables.size(), includedDueToTombstones);
 
             if (iterators.isEmpty())
-                return EmptyIterators.unfilteredRow(cfs.metadata, partitionKey(), filter.isReversed());
+                return EmptyIterators.unfilteredRow(cfs.metadata(), partitionKey(), filter.isReversed());
 
-            StorageHook.instance.reportRead(cfs.metadata.cfId, partitionKey());
+            StorageHook.instance.reportRead(cfs.metadata().id, partitionKey());
             return withSSTablesIterated(iterators, cfs.metric);
         }
         catch (RuntimeException | Error e)
@@ -676,7 +675,7 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     private boolean queriesMulticellType()
     {
-        for (ColumnDefinition column : columnFilter().fetchedColumns())
+        for (ColumnMetadata column : columnFilter().fetchedColumns())
         {
             if (column.type.isMultiCell() || column.type.isCounter())
                 return true;
@@ -777,7 +776,7 @@ public class SinglePartitionReadCommand extends ReadCommand
 
         DecoratedKey key = result.partitionKey();
         cfs.metric.samplers.get(TableMetrics.Sampler.READS).addSample(key.getKey(), key.hashCode(), 1);
-        StorageHook.instance.reportRead(cfs.metadata.cfId, partitionKey());
+        StorageHook.instance.reportRead(cfs.metadata.id, partitionKey());
 
         // "hoist up" the requested data into a more recent sstable
         if (sstablesIterated > cfs.getMinimumCompactionThreshold()
@@ -824,7 +823,7 @@ public class SinglePartitionReadCommand extends ReadCommand
 
         SearchIterator<Clustering, Row> searchIter = result.searchIterator(columnFilter(), false);
 
-        PartitionColumns columns = columnFilter().fetchedColumns();
+        RegularAndStaticColumns columns = columnFilter().fetchedColumns();
         NavigableSet<Clustering> clusterings = filter.requestedRows();
 
         // We want to remove rows for which we have values for all requested columns. We have to deal with both static and regular rows.
@@ -879,7 +878,7 @@ public class SinglePartitionReadCommand extends ReadCommand
         if (row.primaryKeyLivenessInfo().isEmpty() || row.primaryKeyLivenessInfo().timestamp() <= sstableTimestamp)
             return false;
 
-        for (ColumnDefinition column : requestedColumns)
+        for (ColumnMetadata column : requestedColumns)
         {
             Cell cell = row.getCell(column);
             if (cell == null || cell.timestamp() <= sstableTimestamp)
@@ -891,13 +890,12 @@ public class SinglePartitionReadCommand extends ReadCommand
     @Override
     public String toString()
     {
-        return String.format("Read(%s.%s columns=%s rowFilter=%s limits=%s key=%s filter=%s, nowInSec=%d)",
-                             metadata().ksName,
-                             metadata().cfName,
+        return String.format("Read(%s columns=%s rowFilter=%s limits=%s key=%s filter=%s, nowInSec=%d)",
+                             metadata().toString(),
                              columnFilter(),
                              rowFilter(),
                              limits(),
-                             metadata().getKeyValidator().getString(partitionKey().getKey()),
+                             metadata().partitionKeyType.getString(partitionKey().getKey()),
                              clusteringIndexFilter.toString(metadata()),
                              nowInSec());
     }
@@ -911,8 +909,8 @@ public class SinglePartitionReadCommand extends ReadCommand
     {
         sb.append(" WHERE ");
 
-        sb.append(ColumnDefinition.toCQLString(metadata().partitionKeyColumns())).append(" = ");
-        DataRange.appendKeyString(sb, metadata().getKeyValidator(), partitionKey().getKey());
+        sb.append(ColumnMetadata.toCQLString(metadata().partitionKeyColumns())).append(" = ");
+        DataRange.appendKeyString(sb, metadata().partitionKeyType, partitionKey().getKey());
 
         // We put the row filter first because the clustering index filter can end by "ORDER BY"
         if (!rowFilter().isEmpty())
@@ -925,13 +923,13 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     protected void serializeSelection(DataOutputPlus out, int version) throws IOException
     {
-        metadata().getKeyValidator().writeValue(partitionKey().getKey(), out);
+        metadata().partitionKeyType.writeValue(partitionKey().getKey(), out);
         ClusteringIndexFilter.serializer.serialize(clusteringIndexFilter(), out, version);
     }
 
     protected long selectionSerializedSize(int version)
     {
-        return metadata().getKeyValidator().writtenLength(partitionKey().getKey())
+        return metadata().partitionKeyType.writtenLength(partitionKey().getKey())
              + ClusteringIndexFilter.serializer.serializedSize(clusteringIndexFilter(), version);
     }
 
@@ -974,7 +972,7 @@ public class SinglePartitionReadCommand extends ReadCommand
             return limits;
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
             return commands.get(0).metadata();
         }
@@ -1046,10 +1044,10 @@ public class SinglePartitionReadCommand extends ReadCommand
 
     private static class Deserializer extends SelectionDeserializer
     {
-        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
+        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, TableMetadata metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
         throws IOException
         {
-            DecoratedKey key = metadata.decorateKey(metadata.getKeyValidator().readValue(in, DatabaseDescriptor.getMaxValueSize()));
+            DecoratedKey key = metadata.partitioner.decorateKey(metadata.partitionKeyType.readValue(in, DatabaseDescriptor.getMaxValueSize()));
             ClusteringIndexFilter filter = ClusteringIndexFilter.serializer.deserialize(in, version, metadata);
             return new SinglePartitionReadCommand(isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, key, filter);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
index 9c199b6..f70e45e 100644
--- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
+++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
@@ -30,8 +30,8 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.service.MigrationListener;
-import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaChangeListener;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
@@ -46,7 +46,7 @@ import org.apache.cassandra.utils.concurrent.Refs;
  *
  * See CASSANDRA-7688.
  */
-public class SizeEstimatesRecorder extends MigrationListener implements Runnable
+public class SizeEstimatesRecorder extends SchemaChangeListener implements Runnable
 {
     private static final Logger logger = LoggerFactory.getLogger(SizeEstimatesRecorder.class);
 
@@ -54,7 +54,7 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable
 
     private SizeEstimatesRecorder()
     {
-        MigrationManager.instance.register(this);
+        Schema.instance.registerListener(this);
     }
 
     public void run()
@@ -81,8 +81,8 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable
                 long passed = System.nanoTime() - start;
                 logger.trace("Spent {} milliseconds on estimating {}.{} size",
                              TimeUnit.NANOSECONDS.toMillis(passed),
-                             table.metadata.ksName,
-                             table.metadata.cfName);
+                             table.metadata.keyspace,
+                             table.metadata.name);
             }
         }
     }
@@ -124,7 +124,7 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable
         }
 
         // atomically update the estimates.
-        SystemKeyspace.updateSizeEstimates(table.metadata.ksName, table.metadata.cfName, estimates);
+        SystemKeyspace.updateSizeEstimates(table.metadata.keyspace, table.metadata.name, estimates);
     }
 
     private long estimatePartitionsCount(Collection<SSTableReader> sstables, Range<Token> range)
@@ -148,7 +148,7 @@ public class SizeEstimatesRecorder extends MigrationListener implements Runnable
     }
 
     @Override
-    public void onDropColumnFamily(String keyspace, String table)
+    public void onDropTable(String keyspace, String table)
     {
         SystemKeyspace.clearSizeEstimates(keyspace, table);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/Slice.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Slice.java b/src/java/org/apache/cassandra/db/Slice.java
index c3da222..681d79c 100644
--- a/src/java/org/apache/cassandra/db/Slice.java
+++ b/src/java/org/apache/cassandra/db/Slice.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -260,11 +259,6 @@ public class Slice
         return true;
     }
 
-    public String toString(CFMetaData metadata)
-    {
-        return toString(metadata.comparator);
-    }
-
     public String toString(ClusteringComparator comparator)
     {
         StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/Slices.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Slices.java b/src/java/org/apache/cassandra/db/Slices.java
index b3fd20a..9900112 100644
--- a/src/java/org/apache/cassandra/db/Slices.java
+++ b/src/java/org/apache/cassandra/db/Slices.java
@@ -23,8 +23,8 @@ import java.util.*;
 
 import com.google.common.collect.Iterators;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -140,7 +140,7 @@ public abstract class Slices implements Iterable<Slice>
      */
     public abstract boolean intersects(List<ByteBuffer> minClusteringValues, List<ByteBuffer> maxClusteringValues);
 
-    public abstract String toCQLString(CFMetaData metadata);
+    public abstract String toCQLString(TableMetadata metadata);
 
     /**
      * Checks if this <code>Slices</code> is empty.
@@ -323,7 +323,7 @@ public abstract class Slices implements Iterable<Slice>
             return size;
         }
 
-        public Slices deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
+        public Slices deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException
         {
             int size = (int)in.readUnsignedVInt();
 
@@ -548,7 +548,7 @@ public abstract class Slices implements Iterable<Slice>
             return sb.append("}").toString();
         }
 
-        public String toCQLString(CFMetaData metadata)
+        public String toCQLString(TableMetadata metadata)
         {
             StringBuilder sb = new StringBuilder();
 
@@ -572,7 +572,7 @@ public abstract class Slices implements Iterable<Slice>
             boolean needAnd = false;
             for (int i = 0; i < clusteringSize; i++)
             {
-                ColumnDefinition column = metadata.clusteringColumns().get(i);
+                ColumnMetadata column = metadata.clusteringColumns().get(i);
                 List<ComponentOfSlice> componentInfo = columnComponents.get(i);
                 if (componentInfo.isEmpty())
                     break;
@@ -634,7 +634,7 @@ public abstract class Slices implements Iterable<Slice>
             return sb.toString();
         }
 
-        // An somewhat adhoc utility class only used by toCQLString
+        // An somewhat adhoc utility class only used by nameAsCQLString
         private static class ComponentOfSlice
         {
             public final boolean startInclusive;
@@ -751,7 +751,7 @@ public abstract class Slices implements Iterable<Slice>
             return "ALL";
         }
 
-        public String toCQLString(CFMetaData metadata)
+        public String toCQLString(TableMetadata metadata)
         {
             return "";
         }
@@ -826,7 +826,7 @@ public abstract class Slices implements Iterable<Slice>
             return "NONE";
         }
 
-        public String toCQLString(CFMetaData metadata)
+        public String toCQLString(TableMetadata metadata)
         {
             return "";
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/StorageHook.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/StorageHook.java b/src/java/org/apache/cassandra/db/StorageHook.java
index c1e7f66..3df8805 100644
--- a/src/java/org/apache/cassandra/db/StorageHook.java
+++ b/src/java/org/apache/cassandra/db/StorageHook.java
@@ -18,22 +18,21 @@
 
 package org.apache.cassandra.db;
 
-import java.util.UUID;
-
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.FBUtilities;
 
 public interface StorageHook
 {
     public static final StorageHook instance = createHook();
 
-    public void reportWrite(UUID cfid, PartitionUpdate partitionUpdate);
-    public void reportRead(UUID cfid, DecoratedKey key);
+    public void reportWrite(TableId tableId, PartitionUpdate partitionUpdate);
+    public void reportRead(TableId tableId, DecoratedKey key);
     public UnfilteredRowIteratorWithLowerBound makeRowIteratorWithLowerBound(ColumnFamilyStore cfs,
                                                                       DecoratedKey partitionKey,
                                                                       SSTableReader sstable,
@@ -57,9 +56,9 @@ public interface StorageHook
         {
             return new StorageHook()
             {
-                public void reportWrite(UUID cfid, PartitionUpdate partitionUpdate) {}
+                public void reportWrite(TableId tableId, PartitionUpdate partitionUpdate) {}
 
-                public void reportRead(UUID cfid, DecoratedKey key) {}
+                public void reportRead(TableId tableId, DecoratedKey key) {}
 
                 public UnfilteredRowIteratorWithLowerBound makeRowIteratorWithLowerBound(ColumnFamilyStore cfs, DecoratedKey partitionKey, SSTableReader sstable, ClusteringIndexFilter filter, ColumnFilter selectedColumns)
                 {


[36/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cache/OHCProvider.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/OHCProvider.java b/src/java/org/apache/cassandra/cache/OHCProvider.java
index 6f75c74..a465bd9 100644
--- a/src/java/org/apache/cassandra/cache/OHCProvider.java
+++ b/src/java/org/apache/cassandra/cache/OHCProvider.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.cache;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Iterator;
+import java.util.UUID;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.TypeSizes;
@@ -28,7 +29,7 @@ import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.io.util.RebufferingInputStream;
-import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.schema.TableId;
 import org.caffinitas.ohc.OHCache;
 import org.caffinitas.ohc.OHCacheBuilder;
 
@@ -129,8 +130,8 @@ public class OHCProvider implements CacheProvider<RowCacheKey, IRowCacheEntry>
             DataOutputBuffer dataOutput = new DataOutputBufferFixed(buf);
             try
             {
-                dataOutput.writeUTF(rowCacheKey.ksAndCFName.left);
-                dataOutput.writeUTF(rowCacheKey.ksAndCFName.right);
+                rowCacheKey.tableId.serialize(dataOutput);
+                dataOutput.writeUTF(rowCacheKey.indexName != null ? rowCacheKey.indexName : "");
             }
             catch (IOException e)
             {
@@ -144,12 +145,14 @@ public class OHCProvider implements CacheProvider<RowCacheKey, IRowCacheEntry>
         {
             @SuppressWarnings("resource")
             DataInputBuffer dataInput = new DataInputBuffer(buf, false);
-            String ksName = null;
-            String cfName = null;
+            TableId tableId = null;
+            String indexName = null;
             try
             {
-                ksName = dataInput.readUTF();
-                cfName = dataInput.readUTF();
+                tableId = TableId.deserialize(dataInput);
+                indexName = dataInput.readUTF();
+                if (indexName.isEmpty())
+                    indexName = null;
             }
             catch (IOException e)
             {
@@ -157,15 +160,15 @@ public class OHCProvider implements CacheProvider<RowCacheKey, IRowCacheEntry>
             }
             byte[] key = new byte[buf.getInt()];
             buf.get(key);
-            return new RowCacheKey(Pair.create(ksName, cfName), key);
+            return new RowCacheKey(tableId, indexName, key);
         }
 
         public int serializedSize(RowCacheKey rowCacheKey)
         {
-            return TypeSizes.sizeof(rowCacheKey.ksAndCFName.left)
-                    + TypeSizes.sizeof(rowCacheKey.ksAndCFName.right)
-                    + 4
-                    + rowCacheKey.key.length;
+            return rowCacheKey.tableId.serializedSize()
+                   + TypeSizes.sizeof(rowCacheKey.indexName != null ? rowCacheKey.indexName : "")
+                   + 4
+                   + rowCacheKey.key.length;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cache/RowCacheKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/RowCacheKey.java b/src/java/org/apache/cassandra/cache/RowCacheKey.java
index e02db42..bbf289a 100644
--- a/src/java/org/apache/cassandra/cache/RowCacheKey.java
+++ b/src/java/org/apache/cassandra/cache/RowCacheKey.java
@@ -19,32 +19,41 @@ package org.apache.cassandra.cache;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.Objects;
+
+import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
-import org.apache.cassandra.utils.Pair;
 
 public final class RowCacheKey extends CacheKey
 {
     public final byte[] key;
 
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new RowCacheKey(null, ByteBufferUtil.EMPTY_BYTE_BUFFER));
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new RowCacheKey(null, null, new byte[0]));
 
-    public RowCacheKey(Pair<String, String> ksAndCFName, byte[] key)
+    public RowCacheKey(TableId tableId, String indexName, byte[] key)
     {
-        super(ksAndCFName);
+        super(tableId, indexName);
         this.key = key;
     }
 
-    public RowCacheKey(Pair<String, String> ksAndCFName, DecoratedKey key)
+    public RowCacheKey(TableMetadata metadata, DecoratedKey key)
     {
-        this(ksAndCFName, key.getKey());
+        super(metadata);
+        this.key = ByteBufferUtil.getArray(key.getKey());
+        assert this.key != null;
     }
 
-    public RowCacheKey(Pair<String, String> ksAndCFName, ByteBuffer key)
+    @VisibleForTesting
+    public RowCacheKey(TableId tableId, String indexName, ByteBuffer key)
     {
-        super(ksAndCFName);
+        super(tableId, indexName);
         this.key = ByteBufferUtil.getArray(key);
         assert this.key != null;
     }
@@ -62,13 +71,16 @@ public final class RowCacheKey extends CacheKey
 
         RowCacheKey that = (RowCacheKey) o;
 
-        return ksAndCFName.equals(that.ksAndCFName) && Arrays.equals(key, that.key);
+        return tableId.equals(that.tableId)
+               && Objects.equals(indexName, that.indexName)
+               && Arrays.equals(key, that.key);
     }
 
     @Override
     public int hashCode()
     {
-        int result = ksAndCFName.hashCode();
+        int result = tableId.hashCode();
+        result = 31 * result + Objects.hashCode(indexName);
         result = 31 * result + (key != null ? Arrays.hashCode(key) : 0);
         return result;
     }
@@ -76,6 +88,7 @@ public final class RowCacheKey extends CacheKey
     @Override
     public String toString()
     {
-        return String.format("RowCacheKey(ksAndCFName:%s, key:%s)", ksAndCFName, Arrays.toString(key));
+        TableMetadataRef tableRef = Schema.instance.getTableMetadataRef(tableId);
+        return String.format("RowCacheKey(%s, %s, key:%s)", tableRef, indexName, Arrays.toString(key));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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
deleted file mode 100644
index c8180f3..0000000
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ /dev/null
@@ -1,1362 +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.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.apache.commons.lang3.ArrayUtils;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.auth.DataResource;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.statements.CFStatement;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.schema.*;
-import org.apache.cassandra.utils.*;
-import org.github.jamm.Unmetered;
-
-/**
- * This class can be tricky to modify. Please read http://wiki.apache.org/cassandra/ConfigurationNotes for how to do so safely.
- */
-@Unmetered
-public final class CFMetaData
-{
-    public enum Flag
-    {
-        SUPER, COUNTER, DENSE, COMPOUND
-    }
-
-    private static final Pattern PATTERN_WORD_CHARS = Pattern.compile("\\w+");
-
-    private static final Logger logger = LoggerFactory.getLogger(CFMetaData.class);
-
-    public static final Serializer serializer = new Serializer();
-
-    //REQUIRED
-    public final UUID cfId;                           // internal id, never exposed to user
-    public final String ksName;                       // name of keyspace
-    public final String cfName;                       // name of this column family
-    public final Pair<String, String> ksAndCFName;
-    public final byte[] ksAndCFBytes;
-
-    private final ImmutableSet<Flag> flags;
-    private final boolean isDense;
-    private final boolean isCompound;
-    private final boolean isSuper;
-    private final boolean isCounter;
-    private final boolean isView;
-    private final boolean isIndex;
-
-    public volatile ClusteringComparator comparator;  // bytes, long, timeuuid, utf8, etc. This is built directly from clusteringColumns
-    public final IPartitioner partitioner;            // partitioner the table uses
-    private volatile AbstractType<?> keyValidator;
-
-    // non-final, for now
-    public volatile TableParams params = TableParams.DEFAULT;
-
-    private volatile Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>();
-    private volatile Triggers triggers = Triggers.none();
-    private volatile Indexes indexes = Indexes.none();
-
-    /*
-     * All CQL3 columns definition are stored in the columnMetadata map.
-     * On top of that, we keep separated collection of each kind of definition, to
-     * 1) allow easy access to each kind and 2) for the partition key and
-     * clustering key ones, those list are ordered by the "component index" of the
-     * elements.
-     */
-    private volatile Map<ByteBuffer, ColumnDefinition> columnMetadata = new HashMap<>();
-    private volatile List<ColumnDefinition> partitionKeyColumns;  // Always of size keyValidator.componentsCount, null padded if necessary
-    private volatile List<ColumnDefinition> clusteringColumns;    // Of size comparator.componentsCount or comparator.componentsCount -1, null padded if necessary
-    private volatile PartitionColumns partitionColumns;           // Always non-PK, non-clustering columns
-
-    // For dense tables, this alias the single non-PK column the table contains (since it can only have one). We keep
-    // that as convenience to access that column more easily (but we could replace calls by partitionColumns().iterator().next()
-    // for those tables in practice).
-    private volatile ColumnDefinition compactValueColumn;
-
-    public final DataResource resource;
-
-    /*
-     * All of these methods will go away once CFMetaData becomes completely immutable.
-     */
-    public CFMetaData params(TableParams params)
-    {
-        this.params = params;
-        return this;
-    }
-
-    public CFMetaData bloomFilterFpChance(double prop)
-    {
-        params = TableParams.builder(params).bloomFilterFpChance(prop).build();
-        return this;
-    }
-
-    public CFMetaData caching(CachingParams prop)
-    {
-        params = TableParams.builder(params).caching(prop).build();
-        return this;
-    }
-
-    public CFMetaData comment(String prop)
-    {
-        params = TableParams.builder(params).comment(prop).build();
-        return this;
-    }
-
-    public CFMetaData compaction(CompactionParams prop)
-    {
-        params = TableParams.builder(params).compaction(prop).build();
-        return this;
-    }
-
-    public CFMetaData compression(CompressionParams prop)
-    {
-        params = TableParams.builder(params).compression(prop).build();
-        return this;
-    }
-
-    public CFMetaData dcLocalReadRepairChance(double prop)
-    {
-        params = TableParams.builder(params).dcLocalReadRepairChance(prop).build();
-        return this;
-    }
-
-    public CFMetaData defaultTimeToLive(int prop)
-    {
-        params = TableParams.builder(params).defaultTimeToLive(prop).build();
-        return this;
-    }
-
-    public CFMetaData gcGraceSeconds(int prop)
-    {
-        params = TableParams.builder(params).gcGraceSeconds(prop).build();
-        return this;
-    }
-
-    public CFMetaData maxIndexInterval(int prop)
-    {
-        params = TableParams.builder(params).maxIndexInterval(prop).build();
-        return this;
-    }
-
-    public CFMetaData memtableFlushPeriod(int prop)
-    {
-        params = TableParams.builder(params).memtableFlushPeriodInMs(prop).build();
-        return this;
-    }
-
-    public CFMetaData minIndexInterval(int prop)
-    {
-        params = TableParams.builder(params).minIndexInterval(prop).build();
-        return this;
-    }
-
-    public CFMetaData readRepairChance(double prop)
-    {
-        params = TableParams.builder(params).readRepairChance(prop).build();
-        return this;
-    }
-
-    public CFMetaData crcCheckChance(double prop)
-    {
-        params = TableParams.builder(params).crcCheckChance(prop).build();
-        return this;
-    }
-
-    public CFMetaData speculativeRetry(SpeculativeRetryParam prop)
-    {
-        params = TableParams.builder(params).speculativeRetry(prop).build();
-        return this;
-    }
-
-    public CFMetaData extensions(Map<String, ByteBuffer> extensions)
-    {
-        params = TableParams.builder(params).extensions(extensions).build();
-        return this;
-    }
-
-    public CFMetaData droppedColumns(Map<ByteBuffer, DroppedColumn> cols)
-    {
-        droppedColumns = cols;
-        return this;
-    }
-
-    public CFMetaData triggers(Triggers prop)
-    {
-        triggers = prop;
-        return this;
-    }
-
-    public CFMetaData indexes(Indexes indexes)
-    {
-        this.indexes = indexes;
-        return this;
-    }
-
-    private CFMetaData(String keyspace,
-                       String name,
-                       UUID cfId,
-                       boolean isSuper,
-                       boolean isCounter,
-                       boolean isDense,
-                       boolean isCompound,
-                       boolean isView,
-                       List<ColumnDefinition> partitionKeyColumns,
-                       List<ColumnDefinition> clusteringColumns,
-                       PartitionColumns partitionColumns,
-                       IPartitioner partitioner)
-    {
-        this.cfId = cfId;
-        this.ksName = keyspace;
-        this.cfName = name;
-        ksAndCFName = Pair.create(keyspace, name);
-        byte[] ksBytes = FBUtilities.toWriteUTFBytes(ksName);
-        byte[] cfBytes = FBUtilities.toWriteUTFBytes(cfName);
-        ksAndCFBytes = Arrays.copyOf(ksBytes, ksBytes.length + cfBytes.length);
-        System.arraycopy(cfBytes, 0, ksAndCFBytes, ksBytes.length, cfBytes.length);
-
-        this.isDense = isDense;
-        this.isCompound = isCompound;
-        this.isSuper = isSuper;
-        this.isCounter = isCounter;
-        this.isView = isView;
-
-        EnumSet<Flag> flags = EnumSet.noneOf(Flag.class);
-        if (isSuper)
-            flags.add(Flag.SUPER);
-        if (isCounter)
-            flags.add(Flag.COUNTER);
-        if (isDense)
-            flags.add(Flag.DENSE);
-        if (isCompound)
-            flags.add(Flag.COMPOUND);
-        this.flags = Sets.immutableEnumSet(flags);
-
-        isIndex = cfName.contains(".");
-
-        assert partitioner != null : "This assertion failure is probably due to accessing Schema.instance " +
-                                     "from client-mode tools - See CASSANDRA-8143.";
-        this.partitioner = partitioner;
-
-        // A compact table should always have a clustering
-        assert isCQLTable() || !clusteringColumns.isEmpty() : String.format("For table %s.%s, isDense=%b, isCompound=%b, clustering=%s", ksName, cfName, isDense, isCompound, clusteringColumns);
-
-        // All tables should have a partition key
-        assert !partitionKeyColumns.isEmpty() : String.format("Have no partition keys for table %s.%s", ksName, cfName);
-
-        this.partitionKeyColumns = partitionKeyColumns;
-        this.clusteringColumns = clusteringColumns;
-        this.partitionColumns = partitionColumns;
-
-        rebuild();
-
-        this.resource = DataResource.table(ksName, cfName);
-    }
-
-    // This rebuild informations that are intrinsically duplicate of the table definition but
-    // are kept because they are often useful in a different format.
-    private void rebuild()
-    {
-        this.comparator = new ClusteringComparator(extractTypes(clusteringColumns));
-
-        Map<ByteBuffer, ColumnDefinition> newColumnMetadata = Maps.newHashMapWithExpectedSize(partitionKeyColumns.size() + clusteringColumns.size() + partitionColumns.size());
-
-        for (ColumnDefinition def : partitionKeyColumns)
-            newColumnMetadata.put(def.name.bytes, def);
-        for (ColumnDefinition def : clusteringColumns)
-            newColumnMetadata.put(def.name.bytes, def);
-        for (ColumnDefinition def : partitionColumns)
-            newColumnMetadata.put(def.name.bytes, def);
-
-        this.columnMetadata = newColumnMetadata;
-
-        List<AbstractType<?>> keyTypes = extractTypes(partitionKeyColumns);
-        this.keyValidator = keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);
-
-        if (isCompactTable())
-            this.compactValueColumn = CompactTables.getCompactValueColumn(partitionColumns, isSuper());
-    }
-
-    public Indexes getIndexes()
-    {
-        return indexes;
-    }
-
-    public static CFMetaData create(String ksName,
-                                    String name,
-                                    UUID cfId,
-                                    boolean isDense,
-                                    boolean isCompound,
-                                    boolean isSuper,
-                                    boolean isCounter,
-                                    boolean isView,
-                                    List<ColumnDefinition> columns,
-                                    IPartitioner partitioner)
-    {
-        List<ColumnDefinition> partitions = new ArrayList<>();
-        List<ColumnDefinition> clusterings = new ArrayList<>();
-        PartitionColumns.Builder builder = PartitionColumns.builder();
-
-        for (ColumnDefinition column : columns)
-        {
-            switch (column.kind)
-            {
-                case PARTITION_KEY:
-                    partitions.add(column);
-                    break;
-                case CLUSTERING:
-                    clusterings.add(column);
-                    break;
-                default:
-                    builder.add(column);
-                    break;
-            }
-        }
-
-        Collections.sort(partitions);
-        Collections.sort(clusterings);
-
-        return new CFMetaData(ksName,
-                              name,
-                              cfId,
-                              isSuper,
-                              isCounter,
-                              isDense,
-                              isCompound,
-                              isView,
-                              partitions,
-                              clusterings,
-                              builder.build(),
-                              partitioner);
-    }
-
-    public static List<AbstractType<?>> extractTypes(Iterable<ColumnDefinition> clusteringColumns)
-    {
-        List<AbstractType<?>> types = new ArrayList<>();
-        for (ColumnDefinition def : clusteringColumns)
-            types.add(def.type);
-        return types;
-    }
-
-    public Set<Flag> flags()
-    {
-        return flags;
-    }
-
-    /**
-     * There is a couple of places in the code where we need a CFMetaData object and don't have one readily available
-     * and know that only the keyspace and name matter. This creates such "fake" metadata. Use only if you know what
-     * you're doing.
-     */
-    public static CFMetaData createFake(String keyspace, String name)
-    {
-        return CFMetaData.Builder.create(keyspace, name).addPartitionKey("key", BytesType.instance).build();
-    }
-
-    public Triggers getTriggers()
-    {
-        return triggers;
-    }
-
-    // Compiles a system metadata
-    public static CFMetaData compile(String cql, String keyspace)
-    {
-        CFStatement parsed = (CFStatement)QueryProcessor.parseStatement(cql);
-        parsed.prepareKeyspace(keyspace);
-        CreateTableStatement statement = (CreateTableStatement) ((CreateTableStatement.RawStatement) parsed).prepare(Types.none()).statement;
-
-        return statement.metadataBuilder()
-                        .withId(generateLegacyCfId(keyspace, statement.columnFamily()))
-                        .build()
-                        .params(statement.params())
-                        .readRepairChance(0.0)
-                        .dcLocalReadRepairChance(0.0)
-                        .gcGraceSeconds(0)
-                        .memtableFlushPeriod((int) TimeUnit.HOURS.toMillis(1));
-    }
-
-    /**
-     * Generates deterministic UUID from keyspace/columnfamily name pair.
-     * This is used to generate the same UUID for {@code C* version < 2.1}
-     *
-     * Since 2.1, this is only used for system columnfamilies and tests.
-     */
-    public static UUID generateLegacyCfId(String ksName, String cfName)
-    {
-        return UUID.nameUUIDFromBytes(ArrayUtils.addAll(ksName.getBytes(), cfName.getBytes()));
-    }
-
-    public CFMetaData reloadIndexMetadataProperties(CFMetaData parent)
-    {
-        TableParams.Builder indexParams = TableParams.builder(parent.params);
-
-        // Depends on parent's cache setting, turn on its index CF's cache.
-        // Row caching is never enabled; see CASSANDRA-5732
-        if (parent.params.caching.cacheKeys())
-            indexParams.caching(CachingParams.CACHE_KEYS);
-        else
-            indexParams.caching(CachingParams.CACHE_NOTHING);
-
-        indexParams.readRepairChance(0.0)
-                   .dcLocalReadRepairChance(0.0)
-                   .gcGraceSeconds(0);
-
-        return params(indexParams.build());
-    }
-
-    public CFMetaData copy()
-    {
-        return copy(cfId);
-    }
-
-    /**
-     * Clones the CFMetaData, but sets a different cfId
-     *
-     * @param newCfId the cfId for the cloned CFMetaData
-     * @return the cloned CFMetaData instance with the new cfId
-     */
-    public CFMetaData copy(UUID newCfId)
-    {
-        return copyOpts(new CFMetaData(ksName,
-                                       cfName,
-                                       newCfId,
-                                       isSuper(),
-                                       isCounter(),
-                                       isDense(),
-                                       isCompound(),
-                                       isView(),
-                                       copy(partitionKeyColumns),
-                                       copy(clusteringColumns),
-                                       copy(partitionColumns),
-                                       partitioner),
-                        this);
-    }
-
-    public CFMetaData copy(IPartitioner partitioner)
-    {
-        return copyOpts(new CFMetaData(ksName,
-                                       cfName,
-                                       cfId,
-                                       isSuper,
-                                       isCounter,
-                                       isDense,
-                                       isCompound,
-                                       isView,
-                                       copy(partitionKeyColumns),
-                                       copy(clusteringColumns),
-                                       copy(partitionColumns),
-                                       partitioner),
-                        this);
-    }
-
-    private static List<ColumnDefinition> copy(List<ColumnDefinition> l)
-    {
-        List<ColumnDefinition> copied = new ArrayList<>(l.size());
-        for (ColumnDefinition cd : l)
-            copied.add(cd.copy());
-        return copied;
-    }
-
-    private static PartitionColumns copy(PartitionColumns columns)
-    {
-        PartitionColumns.Builder newColumns = PartitionColumns.builder();
-        for (ColumnDefinition cd : columns)
-            newColumns.add(cd.copy());
-        return newColumns.build();
-    }
-
-    @VisibleForTesting
-    public static CFMetaData copyOpts(CFMetaData newCFMD, CFMetaData oldCFMD)
-    {
-        return newCFMD.params(oldCFMD.params)
-                      .droppedColumns(new HashMap<>(oldCFMD.droppedColumns))
-                      .triggers(oldCFMD.triggers)
-                      .indexes(oldCFMD.indexes);
-    }
-
-    /**
-     * generate a column family name for an index corresponding to the given column.
-     * This is NOT the same as the index's name! This is only used in sstable filenames and is not exposed to users.
-     *
-     * @param info A definition of the column with index
-     *
-     * @return name of the index ColumnFamily
-     */
-    public String indexColumnFamilyName(IndexMetadata info)
-    {
-        // TODO simplify this when info.index_name is guaranteed to be set
-        return cfName + Directories.SECONDARY_INDEX_NAME_SEPARATOR + info.name;
-    }
-
-    /**
-     * true if this CFS contains secondary index data.
-     */
-    public boolean isIndex()
-    {
-        return isIndex;
-    }
-
-    public DecoratedKey decorateKey(ByteBuffer key)
-    {
-        return partitioner.decorateKey(key);
-    }
-
-    public Map<ByteBuffer, ColumnDefinition> getColumnMetadata()
-    {
-        return columnMetadata;
-    }
-
-    public ReadRepairDecision newReadRepairDecision()
-    {
-        double chance = ThreadLocalRandom.current().nextDouble();
-        if (params.readRepairChance > chance)
-            return ReadRepairDecision.GLOBAL;
-
-        if (params.dcLocalReadRepairChance > chance)
-            return ReadRepairDecision.DC_LOCAL;
-
-        return ReadRepairDecision.NONE;
-    }
-
-    public AbstractType<?> getColumnDefinitionNameComparator(ColumnDefinition.Kind kind)
-    {
-        return (isSuper() && kind == ColumnDefinition.Kind.REGULAR) || (isStaticCompactTable() && kind == ColumnDefinition.Kind.STATIC)
-             ? staticCompactOrSuperTableColumnNameType()
-             : UTF8Type.instance;
-    }
-
-    public AbstractType<?> getKeyValidator()
-    {
-        return keyValidator;
-    }
-
-    public Collection<ColumnDefinition> allColumns()
-    {
-        return columnMetadata.values();
-    }
-
-    // An iterator over all column definitions but that respect the order of a SELECT *.
-    // This also "hide" the clustering/regular columns for a non-CQL3 non-dense table for backward compatibility
-    // sake.
-    public Iterator<ColumnDefinition> allColumnsInSelectOrder()
-    {
-        final boolean isStaticCompactTable = isStaticCompactTable();
-        final boolean noNonPkColumns = isCompactTable() && CompactTables.hasEmptyCompactValue(this);
-        return new AbstractIterator<ColumnDefinition>()
-        {
-            private final Iterator<ColumnDefinition> partitionKeyIter = partitionKeyColumns.iterator();
-            private final Iterator<ColumnDefinition> clusteringIter = isStaticCompactTable ? Collections.<ColumnDefinition>emptyIterator() : clusteringColumns.iterator();
-            private final Iterator<ColumnDefinition> otherColumns = noNonPkColumns
-                                                                  ? Collections.<ColumnDefinition>emptyIterator()
-                                                                  : (isStaticCompactTable
-                                                                     ?  partitionColumns.statics.selectOrderIterator()
-                                                                     :  partitionColumns.selectOrderIterator());
-
-            protected ColumnDefinition computeNext()
-            {
-                if (partitionKeyIter.hasNext())
-                    return partitionKeyIter.next();
-
-                if (clusteringIter.hasNext())
-                    return clusteringIter.next();
-
-                return otherColumns.hasNext() ? otherColumns.next() : endOfData();
-            }
-        };
-    }
-
-    public Iterable<ColumnDefinition> primaryKeyColumns()
-    {
-        return Iterables.concat(partitionKeyColumns, clusteringColumns);
-    }
-
-    public List<ColumnDefinition> partitionKeyColumns()
-    {
-        return partitionKeyColumns;
-    }
-
-    public List<ColumnDefinition> clusteringColumns()
-    {
-        return clusteringColumns;
-    }
-
-    public PartitionColumns partitionColumns()
-    {
-        return partitionColumns;
-    }
-
-    public ColumnDefinition compactValueColumn()
-    {
-        return compactValueColumn;
-    }
-
-    public ClusteringComparator getKeyValidatorAsClusteringComparator()
-    {
-        boolean isCompound = keyValidator instanceof CompositeType;
-        List<AbstractType<?>> types = isCompound
-                                    ? ((CompositeType) keyValidator).types
-                                    : Collections.<AbstractType<?>>singletonList(keyValidator);
-        return new ClusteringComparator(types);
-    }
-
-    public static ByteBuffer serializePartitionKey(ClusteringPrefix keyAsClustering)
-    {
-        // TODO: we should stop using Clustering for partition keys. Maybe we can add
-        // a few methods to DecoratedKey so we don't have to (note that while using a Clustering
-        // allows to use buildBound(), it's actually used for partition keys only when every restriction
-        // is an equal, so we could easily create a specific method for keys for that.
-        if (keyAsClustering.size() == 1)
-            return keyAsClustering.get(0);
-
-        ByteBuffer[] values = new ByteBuffer[keyAsClustering.size()];
-        for (int i = 0; i < keyAsClustering.size(); i++)
-            values[i] = keyAsClustering.get(i);
-        return CompositeType.build(values);
-    }
-
-    public Map<ByteBuffer, DroppedColumn> getDroppedColumns()
-    {
-        return droppedColumns;
-    }
-
-    public ColumnDefinition getDroppedColumnDefinition(ByteBuffer name)
-    {
-        return getDroppedColumnDefinition(name, false);
-    }
-
-    /**
-     * Returns a "fake" ColumnDefinition corresponding to the dropped column {@code name}
-     * of {@code null} if there is no such dropped column.
-     *
-     * @param name - the column name
-     * @param isStatic - whether the column was a static column, if known
-     */
-    public ColumnDefinition getDroppedColumnDefinition(ByteBuffer name, boolean isStatic)
-    {
-        DroppedColumn dropped = droppedColumns.get(name);
-        if (dropped == null)
-            return null;
-
-        // We need the type for deserialization purpose. If we don't have the type however,
-        // it means that it's a dropped column from before 3.0, and in that case using
-        // BytesType is fine for what we'll be using it for, even if that's a hack.
-        AbstractType<?> type = dropped.type == null ? BytesType.instance : dropped.type;
-        return isStatic || dropped.kind == ColumnDefinition.Kind.STATIC
-               ? ColumnDefinition.staticDef(this, name, type)
-               : ColumnDefinition.regularDef(this, name, type);
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o)
-            return true;
-
-        if (!(o instanceof CFMetaData))
-            return false;
-
-        CFMetaData other = (CFMetaData) o;
-
-        return Objects.equal(cfId, other.cfId)
-            && Objects.equal(flags, other.flags)
-            && Objects.equal(ksName, other.ksName)
-            && Objects.equal(cfName, other.cfName)
-            && Objects.equal(params, other.params)
-            && Objects.equal(comparator, other.comparator)
-            && Objects.equal(keyValidator, other.keyValidator)
-            && Objects.equal(columnMetadata, other.columnMetadata)
-            && Objects.equal(droppedColumns, other.droppedColumns)
-            && Objects.equal(triggers, other.triggers)
-            && Objects.equal(indexes, other.indexes);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return new HashCodeBuilder(29, 1597)
-            .append(cfId)
-            .append(ksName)
-            .append(cfName)
-            .append(flags)
-            .append(comparator)
-            .append(params)
-            .append(keyValidator)
-            .append(columnMetadata)
-            .append(droppedColumns)
-            .append(triggers)
-            .append(indexes)
-            .toHashCode();
-    }
-
-    /**
-     * Updates CFMetaData in-place to match cfm
-     *
-     * @return true if any change was made which impacts queries/updates on the table,
-     *         e.g. any columns or indexes were added, removed, or altered; otherwise, false is returned.
-     *         Used to determine whether prepared statements against this table need to be re-prepared.
-     * @throws ConfigurationException if ks/cf names or cf ids didn't match
-     */
-    @VisibleForTesting
-    public boolean apply(CFMetaData cfm) throws ConfigurationException
-    {
-        logger.debug("applying {} to {}", cfm, this);
-
-        validateCompatibility(cfm);
-
-        partitionKeyColumns = cfm.partitionKeyColumns;
-        clusteringColumns = cfm.clusteringColumns;
-
-        boolean changeAffectsStatements = !partitionColumns.equals(cfm.partitionColumns);
-        partitionColumns = cfm.partitionColumns;
-
-        rebuild();
-
-        params = cfm.params;
-
-        if (!cfm.droppedColumns.isEmpty())
-            droppedColumns = cfm.droppedColumns;
-
-        triggers = cfm.triggers;
-
-        changeAffectsStatements |= !indexes.equals(cfm.indexes);
-        indexes = cfm.indexes;
-
-        logger.debug("application result is {}", this);
-
-        return changeAffectsStatements;
-    }
-
-    public void validateCompatibility(CFMetaData cfm) throws ConfigurationException
-    {
-        // validate
-        if (!cfm.ksName.equals(ksName))
-            throw new ConfigurationException(String.format("Keyspace mismatch (found %s; expected %s)",
-                                                           cfm.ksName, ksName));
-        if (!cfm.cfName.equals(cfName))
-            throw new ConfigurationException(String.format("Column family mismatch (found %s; expected %s)",
-                                                           cfm.cfName, cfName));
-        if (!cfm.cfId.equals(cfId))
-            throw new ConfigurationException(String.format("Column family ID mismatch (found %s; expected %s)",
-                                                           cfm.cfId, cfId));
-        if (!cfm.flags.equals(flags))
-            throw new ConfigurationException(String.format("Column family type mismatch (found %s; expected %s)", cfm.flags, flags));
-    }
-
-
-    public static Class<? extends AbstractCompactionStrategy> createCompactionStrategy(String className) throws ConfigurationException
-    {
-        className = className.contains(".") ? className : "org.apache.cassandra.db.compaction." + className;
-        Class<AbstractCompactionStrategy> strategyClass = FBUtilities.classForName(className, "compaction strategy");
-        if (!AbstractCompactionStrategy.class.isAssignableFrom(strategyClass))
-            throw new ConfigurationException(String.format("Specified compaction strategy class (%s) is not derived from AbstractCompactionStrategy", className));
-
-        return strategyClass;
-    }
-
-    public static AbstractCompactionStrategy createCompactionStrategyInstance(ColumnFamilyStore cfs,
-                                                                              CompactionParams compactionParams)
-    {
-        try
-        {
-            Constructor<? extends AbstractCompactionStrategy> constructor =
-                compactionParams.klass().getConstructor(ColumnFamilyStore.class, Map.class);
-            return constructor.newInstance(cfs, compactionParams.options());
-        }
-        catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Returns the ColumnDefinition for {@code name}.
-     */
-    public ColumnDefinition getColumnDefinition(ColumnIdentifier name)
-    {
-        return columnMetadata.get(name.bytes);
-    }
-
-    // In general it is preferable to work with ColumnIdentifier to make it
-    // clear that we are talking about a CQL column, not a cell name, but there
-    // is a few cases where all we have is a ByteBuffer (when dealing with IndexExpression
-    // for instance) so...
-    public ColumnDefinition getColumnDefinition(ByteBuffer name)
-    {
-        return columnMetadata.get(name);
-    }
-
-    public static boolean isNameValid(String name)
-    {
-        return name != null && !name.isEmpty()
-               && name.length() <= SchemaConstants.NAME_LENGTH && PATTERN_WORD_CHARS.matcher(name).matches();
-    }
-
-    public CFMetaData validate() throws ConfigurationException
-    {
-        rebuild();
-
-        if (!isNameValid(ksName))
-            throw new ConfigurationException(String.format("Keyspace name must not be empty, more than %s characters long, or contain non-alphanumeric-underscore characters (got \"%s\")", SchemaConstants.NAME_LENGTH, ksName));
-        if (!isNameValid(cfName))
-            throw new ConfigurationException(String.format("ColumnFamily name must not be empty, more than %s characters long, or contain non-alphanumeric-underscore characters (got \"%s\")", SchemaConstants.NAME_LENGTH, cfName));
-
-        params.validate();
-
-        for (int i = 0; i < comparator.size(); i++)
-        {
-            if (comparator.subtype(i) instanceof CounterColumnType)
-                throw new ConfigurationException("CounterColumnType is not a valid comparator");
-        }
-        if (keyValidator instanceof CounterColumnType)
-            throw new ConfigurationException("CounterColumnType is not a valid key validator");
-
-        // Mixing counter with non counter columns is not supported (#2614)
-        if (isCounter())
-        {
-            for (ColumnDefinition def : partitionColumns())
-                if (!(def.type instanceof CounterColumnType) && !CompactTables.isSuperColumnMapColumn(def))
-                    throw new ConfigurationException("Cannot add a non counter column (" + def.name + ") in a counter column family");
-        }
-        else
-        {
-            for (ColumnDefinition def : allColumns())
-                if (def.type instanceof CounterColumnType)
-                    throw new ConfigurationException("Cannot add a counter column (" + def.name + ") in a non counter column family");
-        }
-
-        if (!indexes.isEmpty() && isSuper())
-            throw new ConfigurationException("Secondary indexes are not supported on super column families");
-
-        // initialize a set of names NOT in the CF under consideration
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(ksName);
-        Set<String> indexNames = ksm == null ? new HashSet<>() : ksm.existingIndexNames(cfName);
-        for (IndexMetadata index : indexes)
-        {
-            // check index names against this CF _and_ globally
-            if (indexNames.contains(index.name))
-                throw new ConfigurationException("Duplicate index name " + index.name);
-            indexNames.add(index.name);
-
-            index.validate(this);
-        }
-
-        return this;
-    }
-
-    /**
-     * The type to use to compare column names in "static compact"
-     * tables or superColum ones.
-     * <p>
-     * This exists because for historical reasons, "static compact" tables as
-     * well as super column ones can have non-UTF8 column names.
-     * <p>
-     * This method should only be called for superColumn tables and "static
-     * compact" ones. For any other table, all column names are UTF8.
-     */
-    public AbstractType<?> staticCompactOrSuperTableColumnNameType()
-    {
-        if (isSuper())
-        {
-            ColumnDefinition def = compactValueColumn();
-            assert def != null && def.type instanceof MapType;
-            return ((MapType)def.type).nameComparator();
-        }
-
-        assert isStaticCompactTable();
-        return clusteringColumns.get(0).type;
-    }
-
-    public CFMetaData addColumnDefinition(ColumnDefinition def) throws ConfigurationException
-    {
-        if (columnMetadata.containsKey(def.name.bytes))
-            throw new ConfigurationException(String.format("Cannot add column %s, a column with the same name already exists", def.name));
-
-        return addOrReplaceColumnDefinition(def);
-    }
-
-    // This method doesn't check if a def of the same name already exist and should only be used when we
-    // know this cannot happen.
-    public CFMetaData addOrReplaceColumnDefinition(ColumnDefinition def)
-    {
-        // Adds the definition and rebuild what is necessary. We could call rebuild() but it's not too hard to
-        // only rebuild the necessary bits.
-        switch (def.kind)
-        {
-            case PARTITION_KEY:
-                partitionKeyColumns.set(def.position(), def);
-                break;
-            case CLUSTERING:
-                clusteringColumns.set(def.position(), def);
-                break;
-            case REGULAR:
-            case STATIC:
-                PartitionColumns.Builder builder = PartitionColumns.builder();
-                for (ColumnDefinition column : partitionColumns)
-                    if (!column.name.equals(def.name))
-                        builder.add(column);
-                builder.add(def);
-                partitionColumns = builder.build();
-                // If dense, we must have modified the compact value since that's the only one we can have.
-                if (isDense())
-                    this.compactValueColumn = def;
-                break;
-        }
-        this.columnMetadata.put(def.name.bytes, def);
-        return this;
-    }
-
-    public boolean removeColumnDefinition(ColumnDefinition def)
-    {
-        assert !def.isPartitionKey();
-        boolean removed = columnMetadata.remove(def.name.bytes) != null;
-        if (removed)
-            partitionColumns = partitionColumns.without(def);
-        return removed;
-    }
-
-    /**
-     * Adds the column definition as a dropped column, recording the drop with the provided timestamp.
-     */
-    public void recordColumnDrop(ColumnDefinition def, long timeMicros)
-    {
-        droppedColumns.put(def.name.bytes, new DroppedColumn(def, timeMicros));
-    }
-
-    public void renameColumn(ColumnIdentifier from, ColumnIdentifier to) throws InvalidRequestException
-    {
-        ColumnDefinition def = getColumnDefinition(from);
-        if (def == null)
-            throw new InvalidRequestException(String.format("Cannot rename unknown column %s in keyspace %s", from, cfName));
-
-        if (getColumnDefinition(to) != null)
-            throw new InvalidRequestException(String.format("Cannot rename column %s to %s in keyspace %s; another column of that name already exist", from, to, cfName));
-
-        if (!def.isPrimaryKeyColumn())
-            throw new InvalidRequestException(String.format("Cannot rename non PRIMARY KEY part %s", from));
-
-        if (!getIndexes().isEmpty())
-        {
-            ColumnFamilyStore store = Keyspace.openAndGetStore(this);
-            Set<IndexMetadata> dependentIndexes = store.indexManager.getDependentIndexes(def);
-            if (!dependentIndexes.isEmpty())
-                throw new InvalidRequestException(String.format("Cannot rename column %s because it has " +
-                                                                "dependent secondary indexes (%s)",
-                                                                from,
-                                                                dependentIndexes.stream()
-                                                                                .map(i -> i.name)
-                                                                                .collect(Collectors.joining(","))));
-        }
-
-        ColumnDefinition newDef = def.withNewName(to);
-        addOrReplaceColumnDefinition(newDef);
-
-        // removeColumnDefinition doesn't work for partition key (expectedly) but renaming one is fine so we still
-        // want to update columnMetadata.
-        if (def.isPartitionKey())
-            columnMetadata.remove(def.name.bytes);
-        else
-            removeColumnDefinition(def);
-    }
-
-    /**
-     * Records a deprecated column for a system table.
-     */
-    public CFMetaData recordDeprecatedSystemColumn(String name, AbstractType<?> type)
-    {
-        // As we play fast and loose with the removal timestamp, make sure this is misued for a non system table.
-        assert SchemaConstants.isSystemKeyspace(ksName);
-        ByteBuffer bb = ByteBufferUtil.bytes(name);
-        recordColumnDrop(ColumnDefinition.regularDef(this, bb, type), Long.MAX_VALUE);
-        return this;
-    }
-
-
-    public boolean isCQLTable()
-    {
-        return !isSuper() && !isDense() && isCompound();
-    }
-
-    public boolean isCompactTable()
-    {
-        return !isCQLTable();
-    }
-
-    public boolean isStaticCompactTable()
-    {
-        return !isSuper() && !isDense() && !isCompound();
-    }
-
-    public boolean hasStaticColumns()
-    {
-        return !partitionColumns.statics.isEmpty();
-    }
-
-    public boolean isSuper()
-    {
-        return isSuper;
-    }
-
-    public boolean isCounter()
-    {
-        return isCounter;
-    }
-
-    // We call dense a CF for which each component of the comparator is a clustering column, i.e. no
-    // component is used to store a regular column names. In other words, non-composite static "thrift"
-    // and CQL3 CF are *not* dense.
-    public boolean isDense()
-    {
-        return isDense;
-    }
-
-    public boolean isCompound()
-    {
-        return isCompound;
-    }
-
-    public boolean isView()
-    {
-        return isView;
-    }
-
-    public static Set<Flag> flagsFromStrings(Set<String> strings)
-    {
-        return strings.stream()
-                      .map(String::toUpperCase)
-                      .map(Flag::valueOf)
-                      .collect(Collectors.toSet());
-    }
-
-    public static Set<String> flagsToStrings(Set<Flag> flags)
-    {
-        return flags.stream()
-                    .map(Flag::toString)
-                    .map(String::toLowerCase)
-                    .collect(Collectors.toSet());
-    }
-
-
-    @Override
-    public String toString()
-    {
-        return new ToStringBuilder(this)
-            .append("cfId", cfId)
-            .append("ksName", ksName)
-            .append("cfName", cfName)
-            .append("flags", flags)
-            .append("params", params)
-            .append("comparator", comparator)
-            .append("partitionColumns", partitionColumns)
-            .append("partitionKeyColumns", partitionKeyColumns)
-            .append("clusteringColumns", clusteringColumns)
-            .append("keyValidator", keyValidator)
-            .append("columnMetadata", columnMetadata.values())
-            .append("droppedColumns", droppedColumns)
-            .append("triggers", triggers)
-            .append("indexes", indexes)
-            .toString();
-    }
-
-    public static class Builder
-    {
-        private final String keyspace;
-        private final String table;
-        private final boolean isDense;
-        private final boolean isCompound;
-        private final boolean isSuper;
-        private final boolean isCounter;
-        private final boolean isView;
-        private Optional<IPartitioner> partitioner;
-
-        private UUID tableId;
-
-        private final List<Pair<ColumnIdentifier, AbstractType>> partitionKeys = new ArrayList<>();
-        private final List<Pair<ColumnIdentifier, AbstractType>> clusteringColumns = new ArrayList<>();
-        private final List<Pair<ColumnIdentifier, AbstractType>> staticColumns = new ArrayList<>();
-        private final List<Pair<ColumnIdentifier, AbstractType>> regularColumns = new ArrayList<>();
-
-        private Builder(String keyspace, String table, boolean isDense, boolean isCompound, boolean isSuper, boolean isCounter, boolean isView)
-        {
-            this.keyspace = keyspace;
-            this.table = table;
-            this.isDense = isDense;
-            this.isCompound = isCompound;
-            this.isSuper = isSuper;
-            this.isCounter = isCounter;
-            this.isView = isView;
-            this.partitioner = Optional.empty();
-        }
-
-        public static Builder create(String keyspace, String table)
-        {
-            return create(keyspace, table, false, true, false);
-        }
-
-        public static Builder create(String keyspace, String table, boolean isDense, boolean isCompound, boolean isCounter)
-        {
-            return create(keyspace, table, isDense, isCompound, false, isCounter);
-        }
-
-        public static Builder create(String keyspace, String table, boolean isDense, boolean isCompound, boolean isSuper, boolean isCounter)
-        {
-            return new Builder(keyspace, table, isDense, isCompound, isSuper, isCounter, false);
-        }
-
-        public static Builder createView(String keyspace, String table)
-        {
-            return new Builder(keyspace, table, false, true, false, false, true);
-        }
-
-        public static Builder createDense(String keyspace, String table, boolean isCompound, boolean isCounter)
-        {
-            return create(keyspace, table, true, isCompound, isCounter);
-        }
-
-        public static Builder createSuper(String keyspace, String table, boolean isCounter)
-        {
-            return create(keyspace, table, false, false, true, isCounter);
-        }
-
-        public Builder withPartitioner(IPartitioner partitioner)
-        {
-            this.partitioner = Optional.ofNullable(partitioner);
-            return this;
-        }
-
-        public Builder withId(UUID tableId)
-        {
-            this.tableId = tableId;
-            return this;
-        }
-
-        public Builder addPartitionKey(String name, AbstractType type)
-        {
-            return addPartitionKey(ColumnIdentifier.getInterned(name, false), type);
-        }
-
-        public Builder addPartitionKey(ColumnIdentifier name, AbstractType type)
-        {
-            this.partitionKeys.add(Pair.create(name, type));
-            return this;
-        }
-
-        public Builder addClusteringColumn(String name, AbstractType type)
-        {
-            return addClusteringColumn(ColumnIdentifier.getInterned(name, false), type);
-        }
-
-        public Builder addClusteringColumn(ColumnIdentifier name, AbstractType type)
-        {
-            this.clusteringColumns.add(Pair.create(name, type));
-            return this;
-        }
-
-        public Builder addRegularColumn(String name, AbstractType type)
-        {
-            return addRegularColumn(ColumnIdentifier.getInterned(name, false), type);
-        }
-
-        public Builder addRegularColumn(ColumnIdentifier name, AbstractType type)
-        {
-            this.regularColumns.add(Pair.create(name, type));
-            return this;
-        }
-
-        public boolean hasRegulars()
-        {
-            return !this.regularColumns.isEmpty();
-        }
-
-        public Builder addStaticColumn(String name, AbstractType type)
-        {
-            return addStaticColumn(ColumnIdentifier.getInterned(name, false), type);
-        }
-
-        public Builder addStaticColumn(ColumnIdentifier name, AbstractType type)
-        {
-            this.staticColumns.add(Pair.create(name, type));
-            return this;
-        }
-
-        public Set<String> usedColumnNames()
-        {
-            Set<String> usedNames = Sets.newHashSetWithExpectedSize(partitionKeys.size() + clusteringColumns.size() + staticColumns.size() + regularColumns.size());
-            for (Pair<ColumnIdentifier, AbstractType> p : partitionKeys)
-                usedNames.add(p.left.toString());
-            for (Pair<ColumnIdentifier, AbstractType> p : clusteringColumns)
-                usedNames.add(p.left.toString());
-            for (Pair<ColumnIdentifier, AbstractType> p : staticColumns)
-                usedNames.add(p.left.toString());
-            for (Pair<ColumnIdentifier, AbstractType> p : regularColumns)
-                usedNames.add(p.left.toString());
-            return usedNames;
-        }
-
-        public CFMetaData build()
-        {
-            if (tableId == null)
-                tableId = UUIDGen.getTimeUUID();
-
-            List<ColumnDefinition> partitions = new ArrayList<>(partitionKeys.size());
-            List<ColumnDefinition> clusterings = new ArrayList<>(clusteringColumns.size());
-            PartitionColumns.Builder builder = PartitionColumns.builder();
-
-            for (int i = 0; i < partitionKeys.size(); i++)
-            {
-                Pair<ColumnIdentifier, AbstractType> p = partitionKeys.get(i);
-                partitions.add(new ColumnDefinition(keyspace, table, p.left, p.right, i, ColumnDefinition.Kind.PARTITION_KEY));
-            }
-
-            for (int i = 0; i < clusteringColumns.size(); i++)
-            {
-                Pair<ColumnIdentifier, AbstractType> p = clusteringColumns.get(i);
-                clusterings.add(new ColumnDefinition(keyspace, table, p.left, p.right, i, ColumnDefinition.Kind.CLUSTERING));
-            }
-
-            for (Pair<ColumnIdentifier, AbstractType> p : regularColumns)
-                builder.add(new ColumnDefinition(keyspace, table, p.left, p.right, ColumnDefinition.NO_POSITION, ColumnDefinition.Kind.REGULAR));
-
-            for (Pair<ColumnIdentifier, AbstractType> p : staticColumns)
-                builder.add(new ColumnDefinition(keyspace, table, p.left, p.right, ColumnDefinition.NO_POSITION, ColumnDefinition.Kind.STATIC));
-
-            return new CFMetaData(keyspace,
-                                  table,
-                                  tableId,
-                                  isSuper,
-                                  isCounter,
-                                  isDense,
-                                  isCompound,
-                                  isView,
-                                  partitions,
-                                  clusterings,
-                                  builder.build(),
-                                  partitioner.orElseGet(DatabaseDescriptor::getPartitioner));
-        }
-    }
-
-    public static class Serializer
-    {
-        public void serialize(CFMetaData metadata, DataOutputPlus out, int version) throws IOException
-        {
-            UUIDSerializer.serializer.serialize(metadata.cfId, out, version);
-        }
-
-        public CFMetaData deserialize(DataInputPlus in, int version) throws IOException
-        {
-            UUID cfId = UUIDSerializer.serializer.deserialize(in, version);
-            CFMetaData metadata = Schema.instance.getCFMetaData(cfId);
-            if (metadata == null)
-            {
-                String message = String.format("Couldn't find table for cfId %s. If a table was just " +
-                        "created, this is likely due to the schema not being fully propagated.  Please wait for schema " +
-                        "agreement on table creation.", cfId);
-                throw new UnknownColumnFamilyException(message, cfId);
-            }
-
-            return metadata;
-        }
-
-        public long serializedSize(CFMetaData metadata, int version)
-        {
-            return UUIDSerializer.serializer.serializedSize(metadata.cfId, version);
-        }
-    }
-
-    public static class DroppedColumn
-    {
-        // we only allow dropping REGULAR columns, from CQL-native tables, so the names are always of UTF8Type
-        public final String name;
-        public final AbstractType<?> type;
-
-        // drop timestamp, in microseconds, yet with millisecond granularity
-        public final long droppedTime;
-
-        public final ColumnDefinition.Kind kind;
-
-        public DroppedColumn(ColumnDefinition def, long droppedTime)
-        {
-            this(def.name.toString(), def.type, droppedTime, def.kind);
-        }
-
-        public DroppedColumn(String name, AbstractType<?> type, long droppedTime, ColumnDefinition.Kind kind)
-        {
-            this.name = name;
-            this.type = type;
-            this.droppedTime = droppedTime;
-            this.kind = kind;
-        }
-
-        @Override
-        public boolean equals(Object o)
-        {
-            if (this == o)
-                return true;
-
-            if (!(o instanceof DroppedColumn))
-                return false;
-
-            DroppedColumn dc = (DroppedColumn) o;
-
-            return name.equals(dc.name) && type.equals(dc.type) && droppedTime == dc.droppedTime;
-        }
-
-        @Override
-        public int hashCode()
-        {
-            return Objects.hashCode(name, type, droppedTime);
-        }
-
-        @Override
-        public String toString()
-        {
-            return MoreObjects.toStringHelper(this)
-                              .add("name", name)
-                              .add("type", type)
-                              .add("droppedTime", droppedTime)
-                              .toString();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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
deleted file mode 100644
index 65cf837..0000000
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ /dev/null
@@ -1,623 +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.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Function;
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import com.google.common.collect.Collections2;
-
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.selection.Selectable;
-import org.apache.cassandra.cql3.selection.Selector;
-import org.apache.cassandra.cql3.selection.SimpleSelector;
-import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class ColumnDefinition extends ColumnSpecification implements Selectable, Comparable<ColumnDefinition>
-{
-    public static final Comparator<Object> asymmetricColumnDataComparator =
-        (a, b) -> ((ColumnData) a).column().compareTo((ColumnDefinition) b);
-
-    public static final int NO_POSITION = -1;
-
-    public enum ClusteringOrder
-    {
-        ASC, DESC, NONE
-    }
-
-    /**
-     * The type of CQL3 column this definition represents.
-     * There is 4 main type of CQL3 columns: those parts of the partition key,
-     * those parts of the clustering columns and amongst the others, regular and
-     * static ones.
-     *
-     * IMPORTANT: this enum is serialized as toString() and deserialized by calling
-     * Kind.valueOf(), so do not override toString() or rename existing values.
-     */
-    public enum Kind
-    {
-        // NOTE: if adding a new type, must modify comparisonOrder
-        PARTITION_KEY,
-        CLUSTERING,
-        REGULAR,
-        STATIC;
-
-        public boolean isPrimaryKeyKind()
-        {
-            return this == PARTITION_KEY || this == CLUSTERING;
-        }
-
-    }
-
-    public final Kind kind;
-
-    /*
-     * If the column is a partition key or clustering column, its position relative to
-     * other columns of the same kind. Otherwise,  NO_POSITION (-1).
-     *
-     * Note that partition key and clustering columns are numbered separately so
-     * the first clustering column is 0.
-     */
-    private final int position;
-
-    private final Comparator<CellPath> cellPathComparator;
-    private final Comparator<Object> asymmetricCellPathComparator;
-    private final Comparator<? super Cell> cellComparator;
-
-    private int hash;
-
-    /**
-     * These objects are compared frequently, so we encode several of their comparison components
-     * into a single long value so that this can be done efficiently
-     */
-    private final long comparisonOrder;
-
-    private static long comparisonOrder(Kind kind, boolean isComplex, long position, ColumnIdentifier name)
-    {
-        assert position >= 0 && position < 1 << 12;
-        return   (((long) kind.ordinal()) << 61)
-               | (isComplex ? 1L << 60 : 0)
-               | (position << 48)
-               | (name.prefixComparison >>> 16);
-    }
-
-    public static ColumnDefinition partitionKeyDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> type, int position)
-    {
-        return new ColumnDefinition(cfm, name, type, position, Kind.PARTITION_KEY);
-    }
-
-    public static ColumnDefinition partitionKeyDef(String ksName, String cfName, String name, AbstractType<?> type, int position)
-    {
-        return new ColumnDefinition(ksName, cfName, ColumnIdentifier.getInterned(name, true), type, position, Kind.PARTITION_KEY);
-    }
-
-    public static ColumnDefinition clusteringDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> type, int position)
-    {
-        return new ColumnDefinition(cfm, name, type, position, Kind.CLUSTERING);
-    }
-
-    public static ColumnDefinition clusteringDef(String ksName, String cfName, String name, AbstractType<?> type, int position)
-    {
-        return new ColumnDefinition(ksName, cfName, ColumnIdentifier.getInterned(name, true),  type, position, Kind.CLUSTERING);
-    }
-
-    public static ColumnDefinition regularDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> type)
-    {
-        return new ColumnDefinition(cfm, name, type, NO_POSITION, Kind.REGULAR);
-    }
-
-    public static ColumnDefinition regularDef(String ksName, String cfName, String name, AbstractType<?> type)
-    {
-        return new ColumnDefinition(ksName, cfName, ColumnIdentifier.getInterned(name, true), type, NO_POSITION, Kind.REGULAR);
-    }
-
-    public static ColumnDefinition staticDef(CFMetaData cfm, ByteBuffer name, AbstractType<?> type)
-    {
-        return new ColumnDefinition(cfm, name, type, NO_POSITION, Kind.STATIC);
-    }
-
-    public ColumnDefinition(CFMetaData cfm, ByteBuffer name, AbstractType<?> type, int position, Kind kind)
-    {
-        this(cfm.ksName,
-             cfm.cfName,
-             ColumnIdentifier.getInterned(name, cfm.getColumnDefinitionNameComparator(kind)),
-             type,
-             position,
-             kind);
-    }
-
-    @VisibleForTesting
-    public ColumnDefinition(String ksName,
-                            String cfName,
-                            ColumnIdentifier name,
-                            AbstractType<?> type,
-                            int position,
-                            Kind kind)
-    {
-        super(ksName, cfName, name, type);
-        assert name != null && type != null && kind != null;
-        assert name.isInterned();
-        assert (position == NO_POSITION) == !kind.isPrimaryKeyKind(); // The position really only make sense for partition and clustering columns (and those must have one),
-                                                                      // so make sure we don't sneak it for something else since it'd breaks equals()
-        this.kind = kind;
-        this.position = position;
-        this.cellPathComparator = makeCellPathComparator(kind, type);
-        this.cellComparator = cellPathComparator == null ? ColumnData.comparator : (a, b) -> cellPathComparator.compare(a.path(), b.path());
-        this.asymmetricCellPathComparator = cellPathComparator == null ? null : (a, b) -> cellPathComparator.compare(((Cell)a).path(), (CellPath) b);
-        this.comparisonOrder = comparisonOrder(kind, isComplex(), Math.max(0, position), name);
-    }
-
-    private static Comparator<CellPath> makeCellPathComparator(Kind kind, AbstractType<?> type)
-    {
-        if (kind.isPrimaryKeyKind() || !type.isMultiCell())
-            return null;
-
-        AbstractType<?> nameComparator = type.isCollection()
-                                       ? ((CollectionType) type).nameComparator()
-                                       : ((UserType) type).nameComparator();
-
-
-        return new Comparator<CellPath>()
-        {
-            public int compare(CellPath path1, CellPath path2)
-            {
-                if (path1.size() == 0 || path2.size() == 0)
-                {
-                    if (path1 == CellPath.BOTTOM)
-                        return path2 == CellPath.BOTTOM ? 0 : -1;
-                    if (path1 == CellPath.TOP)
-                        return path2 == CellPath.TOP ? 0 : 1;
-                    return path2 == CellPath.BOTTOM ? 1 : -1;
-                }
-
-                // This will get more complicated once we have non-frozen UDT and nested collections
-                assert path1.size() == 1 && path2.size() == 1;
-                return nameComparator.compare(path1.get(0), path2.get(0));
-            }
-        };
-    }
-
-    public ColumnDefinition copy()
-    {
-        return new ColumnDefinition(ksName, cfName, name, type, position, kind);
-    }
-
-    public ColumnDefinition withNewName(ColumnIdentifier newName)
-    {
-        return new ColumnDefinition(ksName, cfName, newName, type, position, kind);
-    }
-
-    public ColumnDefinition withNewType(AbstractType<?> newType)
-    {
-        return new ColumnDefinition(ksName, cfName, name, newType, position, kind);
-    }
-
-    public boolean isPartitionKey()
-    {
-        return kind == Kind.PARTITION_KEY;
-    }
-
-    public boolean isClusteringColumn()
-    {
-        return kind == Kind.CLUSTERING;
-    }
-
-    public boolean isStatic()
-    {
-        return kind == Kind.STATIC;
-    }
-
-    public boolean isRegular()
-    {
-        return kind == Kind.REGULAR;
-    }
-
-    public ClusteringOrder clusteringOrder()
-    {
-        if (!isClusteringColumn())
-            return ClusteringOrder.NONE;
-
-        return type.isReversed() ? ClusteringOrder.DESC : ClusteringOrder.ASC;
-    }
-
-    public int position()
-    {
-        return position;
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o)
-            return true;
-
-        if (!(o instanceof ColumnDefinition))
-            return false;
-
-        ColumnDefinition cd = (ColumnDefinition) o;
-
-        return Objects.equal(ksName, cd.ksName)
-            && Objects.equal(cfName, cd.cfName)
-            && Objects.equal(name, cd.name)
-            && Objects.equal(type, cd.type)
-            && Objects.equal(kind, cd.kind)
-            && Objects.equal(position, cd.position);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        // This achieves the same as Objects.hashcode, but avoids the object array allocation
-        // which features significantly in the allocation profile and caches the result.
-        int result = hash;
-        if(result == 0)
-        {
-            result = 31 + (ksName == null ? 0 : ksName.hashCode());
-            result = 31 * result + (cfName == null ? 0 : cfName.hashCode());
-            result = 31 * result + (name == null ? 0 : name.hashCode());
-            result = 31 * result + (type == null ? 0 : type.hashCode());
-            result = 31 * result + (kind == null ? 0 : kind.hashCode());
-            result = 31 * result + position;
-            hash = result;
-        }
-        return result;
-    }
-
-    @Override
-    public String toString()
-    {
-        return name.toString();
-    }
-
-    public String debugString()
-    {
-        return MoreObjects.toStringHelper(this)
-                          .add("name", name)
-                          .add("type", type)
-                          .add("kind", kind)
-                          .add("position", position)
-                          .toString();
-    }
-
-    public boolean isPrimaryKeyColumn()
-    {
-        return kind.isPrimaryKeyKind();
-    }
-
-    /**
-     * Converts the specified column definitions into column identifiers.
-     *
-     * @param definitions the column definitions to convert.
-     * @return the column identifiers corresponding to the specified definitions
-     */
-    public static Collection<ColumnIdentifier> toIdentifiers(Collection<ColumnDefinition> definitions)
-    {
-        return Collections2.transform(definitions, new Function<ColumnDefinition, ColumnIdentifier>()
-        {
-            @Override
-            public ColumnIdentifier apply(ColumnDefinition columnDef)
-            {
-                return columnDef.name;
-            }
-        });
-    }
-
-    public int compareTo(ColumnDefinition other)
-    {
-        if (this == other)
-            return 0;
-
-        if (comparisonOrder != other.comparisonOrder)
-            return Long.compare(comparisonOrder, other.comparisonOrder);
-
-        return this.name.compareTo(other.name);
-    }
-
-    public Comparator<CellPath> cellPathComparator()
-    {
-        return cellPathComparator;
-    }
-
-    public Comparator<Object> asymmetricCellPathComparator()
-    {
-        return asymmetricCellPathComparator;
-    }
-
-    public Comparator<? super Cell> cellComparator()
-    {
-        return cellComparator;
-    }
-
-    public boolean isComplex()
-    {
-        return cellPathComparator != null;
-    }
-
-    public boolean isSimple()
-    {
-        return !isComplex();
-    }
-
-    public CellPath.Serializer cellPathSerializer()
-    {
-        // Collections are our only complex so far, so keep it simple
-        return CollectionType.cellPathSerializer;
-    }
-
-    public void validateCell(Cell cell)
-    {
-        if (cell.isTombstone())
-        {
-            if (cell.value().hasRemaining())
-                throw new MarshalException("A tombstone should not have a value");
-            if (cell.path() != null)
-                validateCellPath(cell.path());
-        }
-        else if(type.isUDT())
-        {
-            // To validate a non-frozen UDT field, both the path and the value
-            // are needed, the path being an index into an array of value types.
-            ((UserType)type).validateCell(cell);
-        }
-        else
-        {
-            type.validateCellValue(cell.value());
-            if (cell.path() != null)
-                validateCellPath(cell.path());
-        }
-    }
-
-    private void validateCellPath(CellPath path)
-    {
-        if (!isComplex())
-            throw new MarshalException("Only complex cells should have a cell path");
-
-        assert type.isMultiCell();
-        if (type.isCollection())
-            ((CollectionType)type).nameComparator().validate(path.get(0));
-        else
-            ((UserType)type).nameComparator().validate(path.get(0));
-    }
-
-    public static String toCQLString(Iterable<ColumnDefinition> defs)
-    {
-        return toCQLString(defs.iterator());
-    }
-
-    public static String toCQLString(Iterator<ColumnDefinition> defs)
-    {
-        if (!defs.hasNext())
-            return "";
-
-        StringBuilder sb = new StringBuilder();
-        sb.append(defs.next().name);
-        while (defs.hasNext())
-            sb.append(", ").append(defs.next().name);
-        return sb.toString();
-    }
-
-    /**
-     * The type of the cell values for cell belonging to this column.
-     *
-     * This is the same than the column type, except for collections where it's the 'valueComparator'
-     * of the collection.
-     */
-    public AbstractType<?> cellValueType()
-    {
-        return type instanceof CollectionType
-             ? ((CollectionType)type).valueComparator()
-             : type;
-    }
-
-    public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames) throws InvalidRequestException
-    {
-        return SimpleSelector.newFactory(this, addAndGetIndex(this, defs));
-    }
-
-    public AbstractType<?> getExactTypeIfKnown(String keyspace)
-    {
-        return type;
-    }
-
-    /**
-     * Because legacy-created tables may have a non-text comparator, we cannot determine the proper 'key' until
-     * we know the comparator. ColumnDefinition.Raw is a placeholder that can be converted to a real ColumnIdentifier
-     * once the comparator is known with prepare(). This should only be used with identifiers that are actual
-     * column names. See CASSANDRA-8178 for more background.
-     */
-    public static abstract class Raw extends Selectable.Raw
-    {
-        /**
-         * Creates a {@code ColumnDefinition.Raw} from an unquoted identifier string.
-         */
-        public static Raw forUnquoted(String text)
-        {
-            return new Literal(text, false);
-        }
-
-        /**
-         * Creates a {@code ColumnDefinition.Raw} from a quoted identifier string.
-         */
-        public static Raw forQuoted(String text)
-        {
-            return new Literal(text, true);
-        }
-
-        /**
-         * Creates a {@code ColumnDefinition.Raw} from a pre-existing {@code ColumnDefinition}
-         * (useful in the rare cases where we already have the column but need
-         * a {@code ColumnDefinition.Raw} for typing purposes).
-         */
-        public static Raw forColumn(ColumnDefinition column)
-        {
-            return new ForColumn(column);
-        }
-
-        /**
-         * Get the identifier corresponding to this raw column, without assuming this is an
-         * existing column (unlike {@link #prepare}).
-         */
-        public abstract ColumnIdentifier getIdentifier(CFMetaData cfm);
-
-        public abstract String rawText();
-
-        @Override
-        public abstract ColumnDefinition prepare(CFMetaData cfm);
-
-        @Override
-        public boolean processesSelection()
-        {
-            return false;
-        }
-
-        @Override
-        public final int hashCode()
-        {
-            return toString().hashCode();
-        }
-
-        @Override
-        public final boolean equals(Object o)
-        {
-            if(!(o instanceof Raw))
-                return false;
-
-            Raw that = (Raw)o;
-            return this.toString().equals(that.toString());
-        }
-
-        private static class Literal extends Raw
-        {
-            private final String text;
-
-            public Literal(String rawText, boolean keepCase)
-            {
-                this.text =  keepCase ? rawText : rawText.toLowerCase(Locale.US);
-            }
-
-            public ColumnIdentifier getIdentifier(CFMetaData cfm)
-            {
-                if (!cfm.isStaticCompactTable())
-                    return ColumnIdentifier.getInterned(text, true);
-
-                AbstractType<?> columnNameType = cfm.staticCompactOrSuperTableColumnNameType();
-                if (columnNameType instanceof UTF8Type)
-                    return ColumnIdentifier.getInterned(text, true);
-
-                // We have a legacy-created table with a non-text comparator. Check if we have a matching column, otherwise assume we should use
-                // columnNameType
-                ByteBuffer bufferName = ByteBufferUtil.bytes(text);
-                for (ColumnDefinition def : cfm.allColumns())
-                {
-                    if (def.name.bytes.equals(bufferName))
-                        return def.name;
-                }
-                return ColumnIdentifier.getInterned(columnNameType, columnNameType.fromString(text), text);
-            }
-
-            public ColumnDefinition prepare(CFMetaData cfm)
-            {
-                if (!cfm.isStaticCompactTable())
-                    return find(cfm);
-
-                AbstractType<?> columnNameType = cfm.staticCompactOrSuperTableColumnNameType();
-                if (columnNameType instanceof UTF8Type)
-                    return find(cfm);
-
-                // We have a legacy-created table with a non-text comparator. Check if we have a match column, otherwise assume we should use
-                // columnNameType
-                ByteBuffer bufferName = ByteBufferUtil.bytes(text);
-                for (ColumnDefinition def : cfm.allColumns())
-                {
-                    if (def.name.bytes.equals(bufferName))
-                        return def;
-                }
-                return find(columnNameType.fromString(text), cfm);
-            }
-
-            private ColumnDefinition find(CFMetaData cfm)
-            {
-                return find(ByteBufferUtil.bytes(text), cfm);
-            }
-
-            private ColumnDefinition find(ByteBuffer id, CFMetaData cfm)
-            {
-                ColumnDefinition def = cfm.getColumnDefinition(id);
-                if (def == null)
-                    throw new InvalidRequestException(String.format("Undefined column name %s", toString()));
-                return def;
-            }
-
-            public String rawText()
-            {
-                return text;
-            }
-
-            @Override
-            public String toString()
-            {
-                return ColumnIdentifier.maybeQuote(text);
-            }
-        }
-
-        // Use internally in the rare case where we need a ColumnDefinition.Raw for type-checking but
-        // actually already have the column itself.
-        private static class ForColumn extends Raw
-        {
-            private final ColumnDefinition column;
-
-            private ForColumn(ColumnDefinition column)
-            {
-                this.column = column;
-            }
-
-            public ColumnIdentifier getIdentifier(CFMetaData cfm)
-            {
-                return column.name;
-            }
-
-            public ColumnDefinition prepare(CFMetaData cfm)
-            {
-                assert cfm.getColumnDefinition(column.name) != null; // Sanity check that we're not doing something crazy
-                return column;
-            }
-
-            public String rawText()
-            {
-                return column.name.toString();
-            }
-
-            @Override
-            public String toString()
-            {
-                return column.name.toCQLString();
-            }
-        }
-    }
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/config/ReadRepairDecision.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ReadRepairDecision.java b/src/java/org/apache/cassandra/config/ReadRepairDecision.java
deleted file mode 100644
index 1b4b648..0000000
--- a/src/java/org/apache/cassandra/config/ReadRepairDecision.java
+++ /dev/null
@@ -1,23 +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;
-
-public enum ReadRepairDecision
-{
-    NONE, GLOBAL, DC_LOCAL;
-}


[17/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/TableMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/TableMetadata.java b/src/java/org/apache/cassandra/schema/TableMetadata.java
new file mode 100644
index 0000000..44b1f8a
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/TableMetadata.java
@@ -0,0 +1,956 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.*;
+import java.util.Objects;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.auth.DataResource;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.github.jamm.Unmetered;
+
+import static java.lang.String.format;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toSet;
+
+import static com.google.common.collect.Iterables.transform;
+import static org.apache.cassandra.schema.IndexMetadata.isNameValid;
+
+@Unmetered
+public final class TableMetadata
+{
+    public enum Flag
+    {
+        SUPER, COUNTER, DENSE, COMPOUND;
+
+        public static Set<Flag> fromStringSet(Set<String> strings)
+        {
+            return strings.stream().map(String::toUpperCase).map(Flag::valueOf).collect(toSet());
+        }
+
+        public static Set<String> toStringSet(Set<Flag> flags)
+        {
+            return flags.stream().map(Flag::toString).map(String::toLowerCase).collect(toSet());
+        }
+    }
+
+    public final String keyspace;
+    public final String name;
+    public final TableId id;
+
+    public final IPartitioner partitioner;
+    public final TableParams params;
+    public final ImmutableSet<Flag> flags;
+
+    private final boolean isView;
+    private final String indexName; // derived from table name
+
+    /*
+     * All CQL3 columns definition are stored in the columns map.
+     * On top of that, we keep separated collection of each kind of definition, to
+     * 1) allow easy access to each kind and
+     * 2) for the partition key and clustering key ones, those list are ordered by the "component index" of the elements.
+     */
+    public final ImmutableMap<ByteBuffer, DroppedColumn> droppedColumns;
+    final ImmutableMap<ByteBuffer, ColumnMetadata> columns;
+
+    private final ImmutableList<ColumnMetadata> partitionKeyColumns;
+    private final ImmutableList<ColumnMetadata> clusteringColumns;
+    private final RegularAndStaticColumns regularAndStaticColumns;
+
+    public final Indexes indexes;
+    public final Triggers triggers;
+
+    // derived automatically from flags and columns
+    public final AbstractType<?> partitionKeyType;
+    public final ClusteringComparator comparator;
+
+    /*
+     * For dense tables, this alias the single non-PK column the table contains (since it can only have one). We keep
+     * that as convenience to access that column more easily (but we could replace calls by regularAndStaticColumns().iterator().next()
+     * for those tables in practice).
+     */
+    public final ColumnMetadata compactValueColumn;
+
+    // performance hacks; TODO see if all are really necessary
+    public final DataResource resource;
+
+    private TableMetadata(Builder builder)
+    {
+        keyspace = builder.keyspace;
+        name = builder.name;
+        id = builder.id;
+
+        partitioner = builder.partitioner;
+        params = builder.params.build();
+        flags = Sets.immutableEnumSet(builder.flags);
+        isView = builder.isView;
+
+        indexName = name.contains(".")
+                  ? name.substring(name.indexOf('.') + 1)
+                  : null;
+
+        droppedColumns = ImmutableMap.copyOf(builder.droppedColumns);
+        Collections.sort(builder.partitionKeyColumns);
+        partitionKeyColumns = ImmutableList.copyOf(builder.partitionKeyColumns);
+        Collections.sort(builder.clusteringColumns);
+        clusteringColumns = ImmutableList.copyOf(builder.clusteringColumns);
+        regularAndStaticColumns = RegularAndStaticColumns.builder().addAll(builder.regularAndStaticColumns).build();
+        columns = ImmutableMap.copyOf(builder.columns);
+
+        indexes = builder.indexes;
+        triggers = builder.triggers;
+
+        partitionKeyType = partitionKeyColumns.size() == 1
+                         ? partitionKeyColumns.get(0).type
+                         : CompositeType.getInstance(transform(partitionKeyColumns, t -> t.type));
+
+        comparator = new ClusteringComparator(transform(clusteringColumns, c -> c.type));
+
+        compactValueColumn = isCompactTable()
+                           ? CompactTables.getCompactValueColumn(regularAndStaticColumns, isSuper())
+                           : null;
+
+        resource = DataResource.table(keyspace, name);
+    }
+
+    public static Builder builder(String keyspace, String table)
+    {
+        return new Builder(keyspace, table);
+    }
+
+    public static Builder builder(String keyspace, String table, TableId id)
+    {
+        return new Builder(keyspace, table, id);
+    }
+
+    public Builder unbuild()
+    {
+        return builder(keyspace, name, id)
+               .partitioner(partitioner)
+               .params(params)
+               .flags(flags)
+               .isView(isView)
+               .addColumns(columns())
+               .droppedColumns(droppedColumns)
+               .indexes(indexes)
+               .triggers(triggers);
+    }
+
+    public boolean isView()
+    {
+        return isView;
+    }
+
+    public boolean isIndex()
+    {
+        return indexName != null;
+    }
+
+    public Optional<String> indexName()
+    {
+        return Optional.ofNullable(indexName);
+    }
+
+    /*
+     *  We call dense a CF for which each component of the comparator is a clustering column, i.e. no
+     * component is used to store a regular column names. In other words, non-composite static "thrift"
+     * and CQL3 CF are *not* dense.
+     */
+    public boolean isDense()
+    {
+        return flags.contains(Flag.DENSE);
+    }
+
+    public boolean isCompound()
+    {
+        return flags.contains(Flag.COMPOUND);
+    }
+
+    public boolean isSuper()
+    {
+        return flags.contains(Flag.SUPER);
+    }
+
+    public boolean isCounter()
+    {
+        return flags.contains(Flag.COUNTER);
+    }
+
+    public boolean isCQLTable()
+    {
+        return !isSuper() && !isDense() && isCompound();
+    }
+
+    public boolean isCompactTable()
+    {
+        return !isCQLTable();
+    }
+
+    public boolean isStaticCompactTable()
+    {
+        return !isSuper() && !isDense() && !isCompound();
+    }
+
+    public ImmutableCollection<ColumnMetadata> columns()
+    {
+        return columns.values();
+    }
+
+    public Iterable<ColumnMetadata> primaryKeyColumns()
+    {
+        return Iterables.concat(partitionKeyColumns, clusteringColumns);
+    }
+
+    public ImmutableList<ColumnMetadata> partitionKeyColumns()
+    {
+        return partitionKeyColumns;
+    }
+
+    public ImmutableList<ColumnMetadata> clusteringColumns()
+    {
+        return clusteringColumns;
+    }
+
+    public RegularAndStaticColumns regularAndStaticColumns()
+    {
+        return regularAndStaticColumns;
+    }
+
+    public Columns regularColumns()
+    {
+        return regularAndStaticColumns.regulars;
+    }
+
+    public Columns staticColumns()
+    {
+        return regularAndStaticColumns.statics;
+    }
+
+    /*
+     * An iterator over all column definitions but that respect the order of a SELECT *.
+     * This also "hide" the clustering/regular columns for a non-CQL3 non-dense table for backward compatibility
+     * sake.
+     */
+    public Iterator<ColumnMetadata> allColumnsInSelectOrder()
+    {
+        final boolean isStaticCompactTable = isStaticCompactTable();
+        final boolean noNonPkColumns = isCompactTable() && CompactTables.hasEmptyCompactValue(this);
+
+        return new AbstractIterator<ColumnMetadata>()
+        {
+            private final Iterator<ColumnMetadata> partitionKeyIter = partitionKeyColumns.iterator();
+            private final Iterator<ColumnMetadata> clusteringIter =
+                isStaticCompactTable ? Collections.emptyIterator() : clusteringColumns.iterator();
+            private final Iterator<ColumnMetadata> otherColumns =
+                noNonPkColumns
+              ? Collections.emptyIterator()
+              : (isStaticCompactTable ? staticColumns().selectOrderIterator()
+                                      : regularAndStaticColumns.selectOrderIterator());
+
+            protected ColumnMetadata computeNext()
+            {
+                if (partitionKeyIter.hasNext())
+                    return partitionKeyIter.next();
+
+                if (clusteringIter.hasNext())
+                    return clusteringIter.next();
+
+                return otherColumns.hasNext() ? otherColumns.next() : endOfData();
+            }
+        };
+    }
+
+    /**
+     * Returns the ColumnMetadata for {@code name}.
+     */
+    public ColumnMetadata getColumn(ColumnIdentifier name)
+    {
+        return columns.get(name.bytes);
+    }
+
+    /*
+     * In general it is preferable to work with ColumnIdentifier to make it
+     * clear that we are talking about a CQL column, not a cell name, but there
+     * is a few cases where all we have is a ByteBuffer (when dealing with IndexExpression
+     * for instance) so...
+     */
+    public ColumnMetadata getColumn(ByteBuffer name)
+    {
+        return columns.get(name);
+    }
+
+    public ColumnMetadata getDroppedColumn(ByteBuffer name)
+    {
+        DroppedColumn dropped = droppedColumns.get(name);
+        return dropped == null ? null : dropped.column;
+    }
+
+    /**
+     * Returns a "fake" ColumnMetadata corresponding to the dropped column {@code name}
+     * of {@code null} if there is no such dropped column.
+     *
+     * @param name - the column name
+     * @param isStatic - whether the column was a static column, if known
+     */
+    public ColumnMetadata getDroppedColumn(ByteBuffer name, boolean isStatic)
+    {
+        DroppedColumn dropped = droppedColumns.get(name);
+        if (dropped == null)
+            return null;
+
+        if (isStatic && !dropped.column.isStatic())
+            return ColumnMetadata.staticColumn(this, name, dropped.column.type);
+
+        return dropped.column;
+    }
+
+    public boolean hasStaticColumns()
+    {
+        return !staticColumns().isEmpty();
+    }
+
+    public void validate()
+    {
+        if (!isNameValid(keyspace))
+            except("Keyspace name must not be empty, more than %s characters long, or contain non-alphanumeric-underscore characters (got \"%s\")", SchemaConstants.NAME_LENGTH, keyspace);
+
+        if (!isNameValid(name))
+            except("Table name must not be empty, more than %s characters long, or contain non-alphanumeric-underscore characters (got \"%s\")", SchemaConstants.NAME_LENGTH, name);
+
+        params.validate();
+
+        if (partitionKeyColumns.stream().anyMatch(c -> c.type.isCounter()))
+            except("PRIMARY KEY columns cannot contain counters");
+
+        // Mixing counter with non counter columns is not supported (#2614)
+        if (isCounter())
+        {
+            for (ColumnMetadata column : regularAndStaticColumns)
+                if (!(column.type.isCounter()) && !CompactTables.isSuperColumnMapColumn(column))
+                    except("Cannot have a non counter column (\"%s\") in a counter table", column.name);
+        }
+        else
+        {
+            for (ColumnMetadata column : regularAndStaticColumns)
+                if (column.type.isCounter())
+                    except("Cannot have a counter column (\"%s\") in a non counter column table", column.name);
+        }
+
+        // All tables should have a partition key
+        if (partitionKeyColumns.isEmpty())
+            except("Missing partition keys for table %s", toString());
+
+        // A compact table should always have a clustering
+        if (isCompactTable() && clusteringColumns.isEmpty())
+            except("For table %s, isDense=%b, isCompound=%b, clustering=%s", toString(), isDense(), isCompound(), clusteringColumns);
+
+        if (!indexes.isEmpty() && isSuper())
+            except("Secondary indexes are not supported on super column families");
+
+        indexes.validate(this);
+    }
+
+    void validateCompatibility(TableMetadata other)
+    {
+        if (isIndex())
+            return;
+
+        if (!other.keyspace.equals(keyspace))
+            except("Keyspace mismatch (found %s; expected %s)", other.keyspace, keyspace);
+
+        if (!other.name.equals(name))
+            except("Table mismatch (found %s; expected %s)", other.name, name);
+
+        if (!other.id.equals(id))
+            except("Table ID mismatch (found %s; expected %s)", other.id, id);
+
+        if (!other.flags.equals(flags))
+            except("Table type mismatch (found %s; expected %s)", other.flags, flags);
+
+        if (other.partitionKeyColumns.size() != partitionKeyColumns.size())
+            except("Partition keys of different length (found %s; expected %s)", other.partitionKeyColumns.size(), partitionKeyColumns.size());
+
+        for (int i = 0; i < partitionKeyColumns.size(); i++)
+            if (!other.partitionKeyColumns.get(i).type.isCompatibleWith(partitionKeyColumns.get(i).type))
+                except("Partition key column mismatch (found %s; expected %s)", other.partitionKeyColumns.get(i).type, partitionKeyColumns.get(i).type);
+
+        if (other.clusteringColumns.size() != clusteringColumns.size())
+            except("Clustering columns of different length (found %s; expected %s)", other.clusteringColumns.size(), clusteringColumns.size());
+
+        for (int i = 0; i < clusteringColumns.size(); i++)
+            if (!other.clusteringColumns.get(i).type.isCompatibleWith(clusteringColumns.get(i).type))
+                except("Clustering column mismatch (found %s; expected %s)", other.clusteringColumns.get(i).type, clusteringColumns.get(i).type);
+
+        for (ColumnMetadata otherColumn : other.regularAndStaticColumns)
+        {
+            ColumnMetadata column = getColumn(otherColumn.name);
+            if (column != null && !otherColumn.type.isCompatibleWith(column.type))
+                except("Column mismatch (found %s; expected %s", otherColumn, column);
+        }
+    }
+
+    public ClusteringComparator partitionKeyAsClusteringComparator()
+    {
+        return new ClusteringComparator(partitionKeyColumns.stream().map(c -> c.type).collect(toList()));
+    }
+
+    /**
+     * The type to use to compare column names in "static compact"
+     * tables or superColum ones.
+     * <p>
+     * This exists because for historical reasons, "static compact" tables as
+     * well as super column ones can have non-UTF8 column names.
+     * <p>
+     * This method should only be called for superColumn tables and "static
+     * compact" ones. For any other table, all column names are UTF8.
+     */
+    public AbstractType<?> staticCompactOrSuperTableColumnNameType()
+    {
+        if (isSuper())
+        {
+            assert compactValueColumn != null && compactValueColumn.type instanceof MapType;
+            return ((MapType) compactValueColumn.type).nameComparator();
+        }
+
+        assert isStaticCompactTable();
+        return clusteringColumns.get(0).type;
+    }
+
+    public AbstractType<?> columnDefinitionNameComparator(ColumnMetadata.Kind kind)
+    {
+        return (isSuper() && kind == ColumnMetadata.Kind.REGULAR) || (isStaticCompactTable() && kind == ColumnMetadata.Kind.STATIC)
+             ? staticCompactOrSuperTableColumnNameType()
+             : UTF8Type.instance;
+    }
+
+    /**
+     * Generate a table name for an index corresponding to the given column.
+     * This is NOT the same as the index's name! This is only used in sstable filenames and is not exposed to users.
+     *
+     * @param info A definition of the column with index
+     *
+     * @return name of the index table
+     */
+    public String indexTableName(IndexMetadata info)
+    {
+        // TODO simplify this when info.index_name is guaranteed to be set
+        return name + Directories.SECONDARY_INDEX_NAME_SEPARATOR + info.name;
+    }
+
+    /**
+     * @return true if the change as made impacts queries/updates on the table,
+     *         e.g. any columns or indexes were added, removed, or altered; otherwise, false is returned.
+     *         Used to determine whether prepared statements against this table need to be re-prepared.
+     */
+    boolean changeAffectsPreparedStatements(TableMetadata updated)
+    {
+        return !partitionKeyColumns.equals(updated.partitionKeyColumns)
+            || !clusteringColumns.equals(updated.clusteringColumns)
+            || !regularAndStaticColumns.equals(updated.regularAndStaticColumns)
+            || !indexes.equals(updated.indexes)
+            || params.defaultTimeToLive != updated.params.defaultTimeToLive
+            || params.gcGraceSeconds != updated.params.gcGraceSeconds;
+    }
+
+    /**
+     * There is a couple of places in the code where we need a TableMetadata object and don't have one readily available
+     * and know that only the keyspace and name matter. This creates such "fake" metadata. Use only if you know what
+     * you're doing.
+     */
+    public static TableMetadata minimal(String keyspace, String name)
+    {
+        return TableMetadata.builder(keyspace, name)
+                            .addPartitionKeyColumn("key", BytesType.instance)
+                            .build();
+    }
+
+    public TableMetadata updateIndexTableMetadata(TableParams baseTableParams)
+    {
+        TableParams.Builder builder =
+            baseTableParams.unbuild()
+                           .readRepairChance(0.0)
+                           .dcLocalReadRepairChance(0.0)
+                           .gcGraceSeconds(0);
+
+        // Depends on parent's cache setting, turn on its index table's cache.
+        // Row caching is never enabled; see CASSANDRA-5732
+        builder.caching(baseTableParams.caching.cacheKeys() ? CachingParams.CACHE_KEYS : CachingParams.CACHE_NOTHING);
+
+        return unbuild().params(builder.build()).build();
+    }
+
+    private void except(String format, Object... args)
+    {
+        throw new ConfigurationException(format(format, args));
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof TableMetadata))
+            return false;
+
+        TableMetadata tm = (TableMetadata) o;
+
+        return keyspace.equals(tm.keyspace)
+            && name.equals(tm.name)
+            && id.equals(tm.id)
+            && partitioner.equals(tm.partitioner)
+            && params.equals(tm.params)
+            && flags.equals(tm.flags)
+            && isView == tm.isView
+            && columns.equals(tm.columns)
+            && droppedColumns.equals(tm.droppedColumns)
+            && indexes.equals(tm.indexes)
+            && triggers.equals(tm.triggers);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(keyspace, name, id, partitioner, params, flags, isView, columns, droppedColumns, indexes, triggers);
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("%s.%s", ColumnIdentifier.maybeQuote(keyspace), ColumnIdentifier.maybeQuote(name));
+    }
+
+    public String toDebugString()
+    {
+        return MoreObjects.toStringHelper(this)
+                          .add("keyspace", keyspace)
+                          .add("table", name)
+                          .add("id", id)
+                          .add("partitioner", partitioner)
+                          .add("params", params)
+                          .add("flags", flags)
+                          .add("isView", isView)
+                          .add("columns", columns())
+                          .add("droppedColumns", droppedColumns.values())
+                          .add("indexes", indexes)
+                          .add("triggers", triggers)
+                          .toString();
+    }
+
+    public static final class Builder
+    {
+        final String keyspace;
+        final String name;
+
+        private TableId id;
+
+        private IPartitioner partitioner;
+        private TableParams.Builder params = TableParams.builder();
+
+        // Setting compound as default as "normal" CQL tables are compound and that's what we want by default
+        private Set<Flag> flags = EnumSet.of(Flag.COMPOUND);
+        private Triggers triggers = Triggers.none();
+        private Indexes indexes = Indexes.none();
+
+        private final Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>();
+        private final Map<ByteBuffer, ColumnMetadata> columns = new HashMap<>();
+        private final List<ColumnMetadata> partitionKeyColumns = new ArrayList<>();
+        private final List<ColumnMetadata> clusteringColumns = new ArrayList<>();
+        private final List<ColumnMetadata> regularAndStaticColumns = new ArrayList<>();
+
+        private boolean isView;
+
+        private Builder(String keyspace, String name, TableId id)
+        {
+            this.keyspace = keyspace;
+            this.name = name;
+            this.id = id;
+        }
+
+        private Builder(String keyspace, String name)
+        {
+            this.keyspace = keyspace;
+            this.name = name;
+        }
+
+        public TableMetadata build()
+        {
+            if (partitioner == null)
+                partitioner = DatabaseDescriptor.getPartitioner();
+
+            if (id == null)
+                id = TableId.generate();
+
+            return new TableMetadata(this);
+        }
+
+        public Builder id(TableId val)
+        {
+            id = val;
+            return this;
+        }
+
+        public Builder partitioner(IPartitioner val)
+        {
+            partitioner = val;
+            return this;
+        }
+
+        public Builder params(TableParams val)
+        {
+            params = val.unbuild();
+            return this;
+        }
+
+        public Builder bloomFilterFpChance(double val)
+        {
+            params.bloomFilterFpChance(val);
+            return this;
+        }
+
+        public Builder caching(CachingParams val)
+        {
+            params.caching(val);
+            return this;
+        }
+
+        public Builder comment(String val)
+        {
+            params.comment(val);
+            return this;
+        }
+
+        public Builder compaction(CompactionParams val)
+        {
+            params.compaction(val);
+            return this;
+        }
+
+        public Builder compression(CompressionParams val)
+        {
+            params.compression(val);
+            return this;
+        }
+
+        public Builder dcLocalReadRepairChance(double val)
+        {
+            params.dcLocalReadRepairChance(val);
+            return this;
+        }
+
+        public Builder defaultTimeToLive(int val)
+        {
+            params.defaultTimeToLive(val);
+            return this;
+        }
+
+        public Builder gcGraceSeconds(int val)
+        {
+            params.gcGraceSeconds(val);
+            return this;
+        }
+
+        public Builder maxIndexInterval(int val)
+        {
+            params.maxIndexInterval(val);
+            return this;
+        }
+
+        public Builder memtableFlushPeriod(int val)
+        {
+            params.memtableFlushPeriodInMs(val);
+            return this;
+        }
+
+        public Builder minIndexInterval(int val)
+        {
+            params.minIndexInterval(val);
+            return this;
+        }
+
+        public Builder readRepairChance(double val)
+        {
+            params.readRepairChance(val);
+            return this;
+        }
+
+        public Builder crcCheckChance(double val)
+        {
+            params.crcCheckChance(val);
+            return this;
+        }
+
+        public Builder speculativeRetry(SpeculativeRetryParam val)
+        {
+            params.speculativeRetry(val);
+            return this;
+        }
+
+        public Builder extensions(Map<String, ByteBuffer> val)
+        {
+            params.extensions(val);
+            return this;
+        }
+
+        public Builder isView(boolean val)
+        {
+            isView = val;
+            return this;
+        }
+
+        public Builder flags(Set<Flag> val)
+        {
+            flags = val;
+            return this;
+        }
+
+        public Builder isSuper(boolean val)
+        {
+            return flag(Flag.SUPER, val);
+        }
+
+        public Builder isCounter(boolean val)
+        {
+            return flag(Flag.COUNTER, val);
+        }
+
+        public Builder isDense(boolean val)
+        {
+            return flag(Flag.DENSE, val);
+        }
+
+        public Builder isCompound(boolean val)
+        {
+            return flag(Flag.COMPOUND, val);
+        }
+
+        private Builder flag(Flag flag, boolean set)
+        {
+            if (set) flags.add(flag); else flags.remove(flag);
+            return this;
+        }
+
+        public Builder triggers(Triggers val)
+        {
+            triggers = val;
+            return this;
+        }
+
+        public Builder indexes(Indexes val)
+        {
+            indexes = val;
+            return this;
+        }
+
+        public Builder addPartitionKeyColumn(String name, AbstractType type)
+        {
+            return addPartitionKeyColumn(ColumnIdentifier.getInterned(name, false), type);
+        }
+
+        public Builder addPartitionKeyColumn(ColumnIdentifier name, AbstractType type)
+        {
+            return addColumn(new ColumnMetadata(keyspace, this.name, name, type, partitionKeyColumns.size(), ColumnMetadata.Kind.PARTITION_KEY));
+        }
+
+        public Builder addClusteringColumn(String name, AbstractType type)
+        {
+            return addClusteringColumn(ColumnIdentifier.getInterned(name, false), type);
+        }
+
+        public Builder addClusteringColumn(ColumnIdentifier name, AbstractType type)
+        {
+            return addColumn(new ColumnMetadata(keyspace, this.name, name, type, clusteringColumns.size(), ColumnMetadata.Kind.CLUSTERING));
+        }
+
+        public Builder addRegularColumn(String name, AbstractType type)
+        {
+            return addRegularColumn(ColumnIdentifier.getInterned(name, false), type);
+        }
+
+        public Builder addRegularColumn(ColumnIdentifier name, AbstractType type)
+        {
+            return addColumn(new ColumnMetadata(keyspace, this.name, name, type, ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.REGULAR));
+        }
+
+        public Builder addStaticColumn(String name, AbstractType type)
+        {
+            return addStaticColumn(ColumnIdentifier.getInterned(name, false), type);
+        }
+
+        public Builder addStaticColumn(ColumnIdentifier name, AbstractType type)
+        {
+            return addColumn(new ColumnMetadata(keyspace, this.name, name, type, ColumnMetadata.NO_POSITION, ColumnMetadata.Kind.STATIC));
+        }
+
+        public Builder addColumn(ColumnMetadata column)
+        {
+            if (columns.containsKey(column.name.bytes))
+                throw new IllegalArgumentException();
+
+            switch (column.kind)
+            {
+                case PARTITION_KEY:
+                    partitionKeyColumns.add(column);
+                    Collections.sort(partitionKeyColumns);
+                    break;
+                case CLUSTERING:
+                    column.type.checkComparable();
+                    clusteringColumns.add(column);
+                    Collections.sort(clusteringColumns);
+                    break;
+                default:
+                    regularAndStaticColumns.add(column);
+            }
+
+            columns.put(column.name.bytes, column);
+
+            return this;
+        }
+
+        public Builder addColumns(Iterable<ColumnMetadata> columns)
+        {
+            columns.forEach(this::addColumn);
+            return this;
+        }
+
+        public Builder droppedColumns(Map<ByteBuffer, DroppedColumn> droppedColumns)
+        {
+            this.droppedColumns.clear();
+            this.droppedColumns.putAll(droppedColumns);
+            return this;
+        }
+
+        /**
+         * Records a deprecated column for a system table.
+         */
+        public Builder recordDeprecatedSystemColumn(String name, AbstractType<?> type)
+        {
+            // As we play fast and loose with the removal timestamp, make sure this is misued for a non system table.
+            assert SchemaConstants.isSystemKeyspace(keyspace);
+            recordColumnDrop(ColumnMetadata.regularColumn(keyspace, this.name, name, type), Long.MAX_VALUE);
+            return this;
+        }
+
+        public Builder recordColumnDrop(ColumnMetadata column, long timeMicros)
+        {
+            droppedColumns.put(column.name.bytes, new DroppedColumn(column, timeMicros));
+            return this;
+        }
+
+        public Iterable<ColumnMetadata> columns()
+        {
+            return columns.values();
+        }
+
+        public Set<String> columnNames()
+        {
+            return columns.values().stream().map(c -> c.name.toString()).collect(toSet());
+        }
+
+        public ColumnMetadata getColumn(ColumnIdentifier identifier)
+        {
+            return columns.get(identifier.bytes);
+        }
+
+        public ColumnMetadata getColumn(ByteBuffer name)
+        {
+            return columns.get(name);
+        }
+
+        public boolean hasRegularColumns()
+        {
+            return regularAndStaticColumns.stream().anyMatch(ColumnMetadata::isRegular);
+        }
+
+        /*
+         * The following methods all assume a Builder with valid set of partition key, clustering, regular and static columns.
+         */
+
+        public Builder removeRegularOrStaticColumn(ColumnIdentifier identifier)
+        {
+            ColumnMetadata column = columns.get(identifier.bytes);
+            if (column == null || column.isPrimaryKeyColumn())
+                throw new IllegalArgumentException();
+
+            columns.remove(identifier.bytes);
+            regularAndStaticColumns.remove(column);
+
+            return this;
+        }
+
+        public Builder renamePrimaryKeyColumn(ColumnIdentifier from, ColumnIdentifier to)
+        {
+            if (columns.containsKey(to.bytes))
+                throw new IllegalArgumentException();
+
+            ColumnMetadata column = columns.get(from.bytes);
+            if (column == null || !column.isPrimaryKeyColumn())
+                throw new IllegalArgumentException();
+
+            ColumnMetadata newColumn = column.withNewName(to);
+            if (column.isPartitionKey())
+                partitionKeyColumns.set(column.position(), newColumn);
+            else
+                clusteringColumns.set(column.position(), newColumn);
+
+            columns.remove(from.bytes);
+            columns.put(to.bytes, newColumn);
+
+            return this;
+        }
+
+        public Builder alterColumnType(ColumnIdentifier name, AbstractType<?> type)
+        {
+            ColumnMetadata column = columns.get(name.bytes);
+            if (column == null)
+                throw new IllegalArgumentException();
+
+            ColumnMetadata newColumn = column.withNewType(type);
+
+            switch (column.kind)
+            {
+                case PARTITION_KEY:
+                    partitionKeyColumns.set(column.position(), newColumn);
+                    break;
+                case CLUSTERING:
+                    clusteringColumns.set(column.position(), newColumn);
+                    break;
+                case REGULAR:
+                case STATIC:
+                    regularAndStaticColumns.remove(column);
+                    regularAndStaticColumns.add(newColumn);
+                    break;
+            }
+
+            columns.put(column.name.bytes, newColumn);
+
+            return this;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/TableMetadataRef.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/TableMetadataRef.java b/src/java/org/apache/cassandra/schema/TableMetadataRef.java
new file mode 100644
index 0000000..5ff9d5b
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/TableMetadataRef.java
@@ -0,0 +1,78 @@
+/*
+ * 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 org.github.jamm.Unmetered;
+
+/**
+ * Encapsulates a volatile reference to an immutable {@link TableMetadata} instance.
+ *
+ * Used in classes that need up-to-date metadata to avoid the cost of looking up {@link Schema} hashmaps.
+ */
+@Unmetered
+public final class TableMetadataRef
+{
+    public final TableId id;
+    public final String keyspace;
+    public final String name;
+
+    private volatile TableMetadata metadata;
+
+    TableMetadataRef(TableMetadata metadata)
+    {
+        this.metadata = metadata;
+
+        id = metadata.id;
+        keyspace = metadata.keyspace;
+        name = metadata.name;
+    }
+
+    /**
+     * Create a new ref to the passed {@link TableMetadata} for use by offline tools only.
+     *
+     * @param metadata {@link TableMetadata} to reference
+     * @return a new TableMetadataRef instance linking to the passed {@link TableMetadata}
+     */
+    public static TableMetadataRef forOfflineTools(TableMetadata metadata)
+    {
+        return new TableMetadataRef(metadata);
+    }
+
+    public TableMetadata get()
+    {
+        return metadata;
+    }
+
+    /**
+     * Update the reference with the most current version of {@link TableMetadata}
+     * <p>
+     * Must only be used by methods in {@link Schema}, *DO NOT* make public
+     * even for testing purposes, it isn't safe.
+     */
+    void set(TableMetadata metadata)
+    {
+        get().validateCompatibility(metadata);
+        this.metadata = metadata;
+    }
+
+    @Override
+    public String toString()
+    {
+        return get().toString();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/TableParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/TableParams.java b/src/java/org/apache/cassandra/schema/TableParams.java
index 02112af..e68048c 100644
--- a/src/java/org/apache/cassandra/schema/TableParams.java
+++ b/src/java/org/apache/cassandra/schema/TableParams.java
@@ -133,6 +133,11 @@ public final class TableParams
                             .cdc(params.cdc);
     }
 
+    public Builder unbuild()
+    {
+        return builder(this);
+    }
+
     public void validate()
     {
         compaction.validate();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/Tables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Tables.java b/src/java/org/apache/cassandra/schema/Tables.java
index 4f728d4..a83c061 100644
--- a/src/java/org/apache/cassandra/schema/Tables.java
+++ b/src/java/org/apache/cassandra/schema/Tables.java
@@ -17,7 +17,9 @@
  */
 package org.apache.cassandra.schema;
 
+import java.util.HashMap;
 import java.util.Iterator;
+import java.util.Map;
 import java.util.Optional;
 
 import javax.annotation.Nullable;
@@ -26,20 +28,22 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.MapDifference;
 import com.google.common.collect.Maps;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.index.internal.CassandraIndex;
 
 import static com.google.common.collect.Iterables.filter;
 
 /**
  * An immutable container for a keyspace's Tables.
  */
-public final class Tables implements Iterable<CFMetaData>
+public final class Tables implements Iterable<TableMetadata>
 {
-    private final ImmutableMap<String, CFMetaData> tables;
+    private final ImmutableMap<String, TableMetadata> tables;
+    private final ImmutableMap<String, TableMetadata> indexTables;
 
     private Tables(Builder builder)
     {
         tables = builder.tables.build();
+        indexTables = builder.indexTables.build();
     }
 
     public static Builder builder()
@@ -52,21 +56,26 @@ public final class Tables implements Iterable<CFMetaData>
         return builder().build();
     }
 
-    public static Tables of(CFMetaData... tables)
+    public static Tables of(TableMetadata... tables)
     {
         return builder().add(tables).build();
     }
 
-    public static Tables of(Iterable<CFMetaData> tables)
+    public static Tables of(Iterable<TableMetadata> tables)
     {
         return builder().add(tables).build();
     }
 
-    public Iterator<CFMetaData> iterator()
+    public Iterator<TableMetadata> iterator()
     {
         return tables.values().iterator();
     }
 
+    ImmutableMap<String, TableMetadata> indexTables()
+    {
+        return indexTables;
+    }
+
     public int size()
     {
         return tables.size();
@@ -76,9 +85,9 @@ public final class Tables implements Iterable<CFMetaData>
      * Get the table with the specified name
      *
      * @param name a non-qualified table name
-     * @return an empty {@link Optional} if the table name is not found; a non-empty optional of {@link CFMetaData} otherwise
+     * @return an empty {@link Optional} if the table name is not found; a non-empty optional of {@link TableMetadataRef} otherwise
      */
-    public Optional<CFMetaData> get(String name)
+    public Optional<TableMetadata> get(String name)
     {
         return Optional.ofNullable(tables.get(name));
     }
@@ -87,39 +96,67 @@ public final class Tables implements Iterable<CFMetaData>
      * Get the table with the specified name
      *
      * @param name a non-qualified table name
-     * @return null if the table name is not found; the found {@link CFMetaData} otherwise
+     * @return null if the table name is not found; the found {@link TableMetadataRef} otherwise
      */
     @Nullable
-    public CFMetaData getNullable(String name)
+    public TableMetadata getNullable(String name)
     {
         return tables.get(name);
     }
 
+    @Nullable
+    public TableMetadata getIndexTableNullable(String name)
+    {
+        return indexTables.get(name);
+    }
+
     /**
      * Create a Tables instance with the provided table added
      */
-    public Tables with(CFMetaData table)
+    public Tables with(TableMetadata table)
     {
-        if (get(table.cfName).isPresent())
-            throw new IllegalStateException(String.format("Table %s already exists", table.cfName));
+        if (get(table.name).isPresent())
+            throw new IllegalStateException(String.format("Table %s already exists", table.name));
 
         return builder().add(this).add(table).build();
     }
 
+    public Tables withSwapped(TableMetadata table)
+    {
+        return without(table.name).with(table);
+    }
+
     /**
      * Creates a Tables instance with the table with the provided name removed
      */
     public Tables without(String name)
     {
-        CFMetaData table =
+        TableMetadata table =
             get(name).orElseThrow(() -> new IllegalStateException(String.format("Table %s doesn't exists", name)));
 
         return builder().add(filter(this, t -> t != table)).build();
     }
 
-    MapDifference<String, CFMetaData> diff(Tables other)
+    MapDifference<TableId, TableMetadata> diff(Tables other)
     {
-        return Maps.difference(tables, other.tables);
+        Map<TableId, TableMetadata> thisTables = new HashMap<>();
+        this.forEach(t -> thisTables.put(t.id, t));
+
+        Map<TableId, TableMetadata> otherTables = new HashMap<>();
+        other.forEach(t -> otherTables.put(t.id, t));
+
+        return Maps.difference(thisTables, otherTables);
+    }
+
+    MapDifference<String, TableMetadata> indexesDiff(Tables other)
+    {
+        Map<String, TableMetadata> thisIndexTables = new HashMap<>();
+        this.indexTables.values().forEach(t -> thisIndexTables.put(t.indexName().get(), t));
+
+        Map<String, TableMetadata> otherIndexTables = new HashMap<>();
+        other.indexTables.values().forEach(t -> otherIndexTables.put(t.indexName().get(), t));
+
+        return Maps.difference(thisIndexTables, otherIndexTables);
     }
 
     @Override
@@ -142,7 +179,8 @@ public final class Tables implements Iterable<CFMetaData>
 
     public static final class Builder
     {
-        final ImmutableMap.Builder<String, CFMetaData> tables = new ImmutableMap.Builder<>();
+        final ImmutableMap.Builder<String, TableMetadata> tables = new ImmutableMap.Builder<>();
+        final ImmutableMap.Builder<String, TableMetadata> indexTables = new ImmutableMap.Builder<>();
 
         private Builder()
         {
@@ -153,20 +191,27 @@ public final class Tables implements Iterable<CFMetaData>
             return new Tables(this);
         }
 
-        public Builder add(CFMetaData table)
+        public Builder add(TableMetadata table)
         {
-            tables.put(table.cfName, table);
+            tables.put(table.name, table);
+
+            table.indexes
+                 .stream()
+                 .filter(i -> !i.isCustom())
+                 .map(i -> CassandraIndex.indexCfsMetadata(table, i))
+                 .forEach(i -> indexTables.put(i.indexName().get(), i));
+
             return this;
         }
 
-        public Builder add(CFMetaData... tables)
+        public Builder add(TableMetadata... tables)
         {
-            for (CFMetaData table : tables)
+            for (TableMetadata table : tables)
                 add(table);
             return this;
         }
 
-        public Builder add(Iterable<CFMetaData> tables)
+        public Builder add(Iterable<TableMetadata> tables)
         {
             tables.forEach(this::add);
             return this;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/Triggers.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Triggers.java b/src/java/org/apache/cassandra/schema/Triggers.java
index bb39f1f..5e10722 100644
--- a/src/java/org/apache/cassandra/schema/Triggers.java
+++ b/src/java/org/apache/cassandra/schema/Triggers.java
@@ -43,6 +43,16 @@ public final class Triggers implements Iterable<TriggerMetadata>
         return builder().build();
     }
 
+    public static Triggers of(TriggerMetadata... triggers)
+    {
+        return builder().add(triggers).build();
+    }
+
+    public static Triggers of(Iterable<TriggerMetadata> triggers)
+    {
+        return builder().add(triggers).build();
+    }
+
     public Iterator<TriggerMetadata> iterator()
     {
         return triggers.values().iterator();
@@ -128,6 +138,13 @@ public final class Triggers implements Iterable<TriggerMetadata>
             return this;
         }
 
+        public Builder add(TriggerMetadata... triggers)
+        {
+            for (TriggerMetadata trigger : triggers)
+                add(trigger);
+            return this;
+        }
+
         public Builder add(Iterable<TriggerMetadata> triggers)
         {
             triggers.forEach(this::add);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/UnknownIndexException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/UnknownIndexException.java b/src/java/org/apache/cassandra/schema/UnknownIndexException.java
deleted file mode 100644
index 5daf631..0000000
--- a/src/java/org/apache/cassandra/schema/UnknownIndexException.java
+++ /dev/null
@@ -1,39 +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.schema;
-
-import java.io.IOException;
-import java.util.UUID;
-
-import org.apache.cassandra.config.CFMetaData;
-
-/**
- * Exception thrown when we read an index id from a serialized ReadCommand and no corresponding IndexMetadata
- * can be found in the CFMetaData#indexes collection. Note that this is an internal exception and is not meant
- * to be user facing, the node reading the ReadCommand should proceed as if no index id were present.
- */
-public class UnknownIndexException extends IOException
-{
-    public final UUID indexId;
-    public UnknownIndexException(CFMetaData metadata, UUID id)
-    {
-        super(String.format("Unknown index %s for table %s.%s", id.toString(), metadata.ksName, metadata.cfName));
-        indexId = id;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/ViewMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/ViewMetadata.java b/src/java/org/apache/cassandra/schema/ViewMetadata.java
new file mode 100644
index 0000000..57f4092
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/ViewMetadata.java
@@ -0,0 +1,197 @@
+/*
+ * 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.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import org.antlr.runtime.*;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.view.View;
+import org.apache.cassandra.exceptions.SyntaxException;
+
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+
+public final class ViewMetadata
+{
+    public final String keyspace;
+    public final String name;
+    public final TableId baseTableId;
+    public final String baseTableName;
+    public final boolean includeAllColumns;
+    public final TableMetadata metadata;
+
+    public final SelectStatement.RawStatement select;
+    public final String whereClause;
+
+    /**
+     * @param name              Name of the view
+     * @param baseTableId       Internal ID of the table which this view is based off of
+     * @param includeAllColumns Whether to include all columns or not
+     */
+    public ViewMetadata(String keyspace,
+                        String name,
+                        TableId baseTableId,
+                        String baseTableName,
+                        boolean includeAllColumns,
+                        SelectStatement.RawStatement select,
+                        String whereClause,
+                        TableMetadata metadata)
+    {
+        this.keyspace = keyspace;
+        this.name = name;
+        this.baseTableId = baseTableId;
+        this.baseTableName = baseTableName;
+        this.includeAllColumns = includeAllColumns;
+        this.select = select;
+        this.whereClause = whereClause;
+        this.metadata = metadata;
+    }
+
+    /**
+     * @return true if the view specified by this definition will include the column, false otherwise
+     */
+    public boolean includes(ColumnIdentifier column)
+    {
+        return metadata.getColumn(column) != null;
+    }
+
+    public ViewMetadata copy(TableMetadata newMetadata)
+    {
+        return new ViewMetadata(keyspace, name, baseTableId, baseTableName, includeAllColumns, select, whereClause, newMetadata);
+    }
+
+    public TableMetadata baseTableMetadata()
+    {
+        return Schema.instance.getTableMetadata(baseTableId);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof ViewMetadata))
+            return false;
+
+        ViewMetadata other = (ViewMetadata) o;
+        return Objects.equals(keyspace, other.keyspace)
+               && Objects.equals(name, other.name)
+               && Objects.equals(baseTableId, other.baseTableId)
+               && Objects.equals(includeAllColumns, other.includeAllColumns)
+               && Objects.equals(whereClause, other.whereClause)
+               && Objects.equals(metadata, other.metadata);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return new HashCodeBuilder(29, 1597)
+               .append(keyspace)
+               .append(name)
+               .append(baseTableId)
+               .append(includeAllColumns)
+               .append(whereClause)
+               .append(metadata)
+               .toHashCode();
+    }
+
+    @Override
+    public String toString()
+    {
+        return new ToStringBuilder(this)
+               .append("keyspace", keyspace)
+               .append("name", name)
+               .append("baseTableId", baseTableId)
+               .append("baseTableName", baseTableName)
+               .append("includeAllColumns", includeAllColumns)
+               .append("whereClause", whereClause)
+               .append("metadata", metadata)
+               .toString();
+    }
+
+    /**
+     * Replace the column 'from' with 'to' in this materialized view definition's partition,
+     * clustering, or included columns.
+     * @param from the existing column
+     * @param to the new column
+     */
+    public ViewMetadata renamePrimaryKeyColumn(ColumnIdentifier from, ColumnIdentifier to)
+    {
+        // convert whereClause to Relations, rename ids in Relations, then convert back to whereClause
+        List<Relation> relations = whereClauseToRelations(whereClause);
+        ColumnMetadata.Raw fromRaw = ColumnMetadata.Raw.forQuoted(from.toString());
+        ColumnMetadata.Raw toRaw = ColumnMetadata.Raw.forQuoted(to.toString());
+        List<Relation> newRelations =
+            relations.stream()
+                     .map(r -> r.renameIdentifier(fromRaw, toRaw))
+                     .collect(Collectors.toList());
+
+        String rawSelect = View.buildSelectStatement(baseTableName, metadata.columns(), whereClause);
+
+        return new ViewMetadata(keyspace,
+                                name,
+                                baseTableId,
+                                baseTableName,
+                                includeAllColumns,
+                                (SelectStatement.RawStatement) QueryProcessor.parseStatement(rawSelect),
+                                View.relationsToWhereClause(newRelations),
+                                metadata.unbuild().renamePrimaryKeyColumn(from, to).build());
+    }
+
+    public ViewMetadata withAddedRegularColumn(ColumnMetadata column)
+    {
+        return new ViewMetadata(keyspace,
+                                name,
+                                baseTableId,
+                                baseTableName,
+                                includeAllColumns,
+                                select,
+                                whereClause,
+                                metadata.unbuild().addColumn(column).build());
+    }
+
+    public ViewMetadata withAlteredColumnType(ColumnIdentifier name, AbstractType<?> type)
+    {
+        return new ViewMetadata(keyspace,
+                                this.name,
+                                baseTableId,
+                                baseTableName,
+                                includeAllColumns,
+                                select,
+                                whereClause,
+                                metadata.unbuild().alterColumnType(name, type).build());
+    }
+
+    private static List<Relation> whereClauseToRelations(String whereClause)
+    {
+        try
+        {
+            return CQLFragmentParser.parseAnyUnhandled(CqlParser::whereClause, whereClause).build().relations;
+        }
+        catch (RecognitionException | SyntaxException exc)
+        {
+            throw new RuntimeException("Unexpected error parsing materialized view's where clause while handling column rename: ", exc);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/Views.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Views.java b/src/java/org/apache/cassandra/schema/Views.java
index b8fdd4b..6578b14 100644
--- a/src/java/org/apache/cassandra/schema/Views.java
+++ b/src/java/org/apache/cassandra/schema/Views.java
@@ -15,12 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.schema;
 
-
+import java.util.HashMap;
 import java.util.Iterator;
+import java.util.Map;
 import java.util.Optional;
+import java.util.UUID;
 
 import javax.annotation.Nullable;
 
@@ -29,14 +30,11 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.MapDifference;
 import com.google.common.collect.Maps;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ViewDefinition;
-
 import static com.google.common.collect.Iterables.filter;
 
-public final class Views implements Iterable<ViewDefinition>
+public final class Views implements Iterable<ViewMetadata>
 {
-    private final ImmutableMap<String, ViewDefinition> views;
+    private final ImmutableMap<String, ViewMetadata> views;
 
     private Views(Builder builder)
     {
@@ -53,12 +51,12 @@ public final class Views implements Iterable<ViewDefinition>
         return builder().build();
     }
 
-    public Iterator<ViewDefinition> iterator()
+    public Iterator<ViewMetadata> iterator()
     {
         return views.values().iterator();
     }
 
-    public Iterable<CFMetaData> metadatas()
+    public Iterable<TableMetadata> metadatas()
     {
         return Iterables.transform(views.values(), view -> view.metadata);
     }
@@ -73,13 +71,18 @@ public final class Views implements Iterable<ViewDefinition>
         return views.isEmpty();
     }
 
+    public Iterable<ViewMetadata> forTable(UUID tableId)
+    {
+        return Iterables.filter(this, v -> v.baseTableId.equals(tableId));
+    }
+
     /**
      * Get the materialized view with the specified name
      *
      * @param name a non-qualified materialized view name
-     * @return an empty {@link Optional} if the materialized view name is not found; a non-empty optional of {@link ViewDefinition} otherwise
+     * @return an empty {@link Optional} if the materialized view name is not found; a non-empty optional of {@link ViewMetadata} otherwise
      */
-    public Optional<ViewDefinition> get(String name)
+    public Optional<ViewMetadata> get(String name)
     {
         return Optional.ofNullable(views.get(name));
     }
@@ -88,10 +91,10 @@ public final class Views implements Iterable<ViewDefinition>
      * Get the view with the specified name
      *
      * @param name a non-qualified view name
-     * @return null if the view name is not found; the found {@link ViewDefinition} otherwise
+     * @return null if the view name is not found; the found {@link ViewMetadata} otherwise
      */
     @Nullable
-    public ViewDefinition getNullable(String name)
+    public ViewMetadata getNullable(String name)
     {
         return views.get(name);
     }
@@ -99,36 +102,39 @@ public final class Views implements Iterable<ViewDefinition>
     /**
      * Create a MaterializedViews instance with the provided materialized view added
      */
-    public Views with(ViewDefinition view)
+    public Views with(ViewMetadata view)
     {
-        if (get(view.viewName).isPresent())
-            throw new IllegalStateException(String.format("Materialized View %s already exists", view.viewName));
+        if (get(view.name).isPresent())
+            throw new IllegalStateException(String.format("Materialized View %s already exists", view.name));
 
         return builder().add(this).add(view).build();
     }
 
+    public Views withSwapped(ViewMetadata view)
+    {
+        return without(view.name).with(view);
+    }
+
     /**
      * Creates a MaterializedViews instance with the materializedView with the provided name removed
      */
     public Views without(String name)
     {
-        ViewDefinition materializedView =
+        ViewMetadata materializedView =
             get(name).orElseThrow(() -> new IllegalStateException(String.format("Materialized View %s doesn't exists", name)));
 
         return builder().add(filter(this, v -> v != materializedView)).build();
     }
 
-    /**
-     * Creates a MaterializedViews instance which contains an updated materialized view
-     */
-    public Views replace(ViewDefinition view, CFMetaData cfm)
+    MapDifference<TableId, ViewMetadata> diff(Views other)
     {
-        return without(view.viewName).with(view);
-    }
+        Map<TableId, ViewMetadata> thisViews = new HashMap<>();
+        this.forEach(v -> thisViews.put(v.metadata.id, v));
 
-    MapDifference<String, ViewDefinition> diff(Views other)
-    {
-        return Maps.difference(views, other.views);
+        Map<TableId, ViewMetadata> otherViews = new HashMap<>();
+        other.forEach(v -> otherViews.put(v.metadata.id, v));
+
+        return Maps.difference(thisViews, otherViews);
     }
 
     @Override
@@ -151,7 +157,7 @@ public final class Views implements Iterable<ViewDefinition>
 
     public static final class Builder
     {
-        final ImmutableMap.Builder<String, ViewDefinition> views = new ImmutableMap.Builder<>();
+        final ImmutableMap.Builder<String, ViewMetadata> views = new ImmutableMap.Builder<>();
 
         private Builder()
         {
@@ -163,13 +169,13 @@ public final class Views implements Iterable<ViewDefinition>
         }
 
 
-        public Builder add(ViewDefinition view)
+        public Builder add(ViewMetadata view)
         {
-            views.put(view.viewName, view);
+            views.put(view.name, view);
             return this;
         }
 
-        public Builder add(Iterable<ViewDefinition> views)
+        public Builder add(Iterable<ViewMetadata> views)
         {
             views.forEach(this::add);
             return this;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
index 8944b7c..6e0f45b 100644
--- a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
+++ b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.service;
 import java.net.InetAddress;
 import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Iterables;
@@ -28,7 +29,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.ReadRepairDecision;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.ReadCommand;
@@ -42,6 +42,7 @@ import org.apache.cassandra.metrics.ReadRepairMetrics;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.SpeculativeRetryParam;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.StorageProxy.LocalReadRunnable;
 import org.apache.cassandra.tracing.TraceState;
 import org.apache.cassandra.tracing.Tracing;
@@ -145,17 +146,29 @@ public abstract class AbstractReadExecutor
         return handler.get();
     }
 
+    private static ReadRepairDecision newReadRepairDecision(TableMetadata metadata)
+    {
+        double chance = ThreadLocalRandom.current().nextDouble();
+        if (metadata.params.readRepairChance > chance)
+            return ReadRepairDecision.GLOBAL;
+
+        if (metadata.params.dcLocalReadRepairChance > chance)
+            return ReadRepairDecision.DC_LOCAL;
+
+        return ReadRepairDecision.NONE;
+    }
+
     /**
      * @return an executor appropriate for the configured speculative read policy
      */
     public static AbstractReadExecutor getReadExecutor(SinglePartitionReadCommand command, ConsistencyLevel consistencyLevel, long queryStartNanoTime) throws UnavailableException
     {
-        Keyspace keyspace = Keyspace.open(command.metadata().ksName);
+        Keyspace keyspace = Keyspace.open(command.metadata().keyspace);
         List<InetAddress> allReplicas = StorageProxy.getLiveSortedEndpoints(keyspace, command.partitionKey());
         // 11980: Excluding EACH_QUORUM reads from potential RR, so that we do not miscount DC responses
         ReadRepairDecision repairDecision = consistencyLevel == ConsistencyLevel.EACH_QUORUM
                                             ? ReadRepairDecision.NONE
-                                            : command.metadata().newReadRepairDecision();
+                                            : newReadRepairDecision(command.metadata());
         List<InetAddress> targetReplicas = consistencyLevel.filterForQuery(keyspace, allReplicas, repairDecision);
 
         // Throw UAE early if we don't have enough replicas.
@@ -167,8 +180,8 @@ public abstract class AbstractReadExecutor
             ReadRepairMetrics.attempted.mark();
         }
 
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(command.metadata().cfId);
-        SpeculativeRetryParam retry = cfs.metadata.params.speculativeRetry;
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(command.metadata().id);
+        SpeculativeRetryParam retry = cfs.metadata().params.speculativeRetry;
 
         // Speculative retry is disabled *OR* there are simply no extra replicas to speculate.
         // 11980: Disable speculative retry if using EACH_QUORUM in order to prevent miscounting DC responses

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index 7476cd9..e7c6640 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -64,6 +64,7 @@ import org.apache.cassandra.repair.RepairJobDesc;
 import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.repair.RepairSession;
 import org.apache.cassandra.repair.messages.*;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.CassandraVersion;
 import org.apache.cassandra.utils.Clock;
 import org.apache.cassandra.utils.FBUtilities;
@@ -307,15 +308,15 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
             }
         };
 
-        List<UUID> cfIds = new ArrayList<>(columnFamilyStores.size());
+        List<TableId> tableIds = new ArrayList<>(columnFamilyStores.size());
         for (ColumnFamilyStore cfs : columnFamilyStores)
-            cfIds.add(cfs.metadata.cfId);
+            tableIds.add(cfs.metadata.id);
 
         for (InetAddress neighbour : endpoints)
         {
             if (FailureDetector.instance.isAlive(neighbour))
             {
-                PrepareMessage message = new PrepareMessage(parentRepairSession, cfIds, options.getRanges(), options.isIncremental(), timestamp, options.isGlobal());
+                PrepareMessage message = new PrepareMessage(parentRepairSession, tableIds, options.getRanges(), options.isIncremental(), timestamp, options.isGlobal());
                 MessageOut<RepairMessage> msg = message.createMessage();
                 MessagingService.instance().sendRR(msg, neighbour, callback, TimeUnit.HOURS.toMillis(1), true);
             }
@@ -357,12 +358,12 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         parentRepairSessions.put(parentRepairSession, new ParentRepairSession(coordinator, columnFamilyStores, ranges, isIncremental, timestamp, isGlobal));
     }
 
-    public Set<SSTableReader> currentlyRepairing(UUID cfId, UUID parentRepairSession)
+    public Set<SSTableReader> currentlyRepairing(TableId tableId, UUID parentRepairSession)
     {
         Set<SSTableReader> repairing = new HashSet<>();
         for (Map.Entry<UUID, ParentRepairSession> entry : parentRepairSessions.entrySet())
         {
-            Collection<SSTableReader> sstables = entry.getValue().getActiveSSTables(cfId);
+            Collection<SSTableReader> sstables = entry.getValue().getActiveSSTables(tableId);
             if (sstables != null && !entry.getKey().equals(parentRepairSession))
                 repairing.addAll(sstables);
         }
@@ -447,7 +448,7 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         // if we don't have successful repair ranges, then just skip anticompaction
         if (!successfulRanges.isEmpty())
         {
-            for (Map.Entry<UUID, ColumnFamilyStore> columnFamilyStoreEntry : prs.columnFamilyStores.entrySet())
+            for (Map.Entry<TableId, ColumnFamilyStore> columnFamilyStoreEntry : prs.columnFamilyStores.entrySet())
             {
                 Refs<SSTableReader> sstables = prs.getActiveRepairedSSTableRefsForAntiCompaction(columnFamilyStoreEntry.getKey(), parentRepairSession);
                 ColumnFamilyStore cfs = columnFamilyStoreEntry.getValue();
@@ -504,9 +505,9 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
      */
     public static class ParentRepairSession
     {
-        private final Map<UUID, ColumnFamilyStore> columnFamilyStores = new HashMap<>();
+        private final Map<TableId, ColumnFamilyStore> columnFamilyStores = new HashMap<>();
         private final Collection<Range<Token>> ranges;
-        public final Map<UUID, Set<String>> sstableMap = new HashMap<>();
+        public final Map<TableId, Set<String>> sstableMap = new HashMap<>();
         public final boolean isIncremental;
         public final boolean isGlobal;
         public final long repairedAt;
@@ -514,15 +515,15 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
         /**
          * Indicates whether we have marked sstables as repairing. Can only be done once per table per ParentRepairSession
          */
-        private final Set<UUID> marked = new HashSet<>();
+        private final Set<TableId> marked = new HashSet<>();
 
         public ParentRepairSession(InetAddress coordinator, List<ColumnFamilyStore> columnFamilyStores, Collection<Range<Token>> ranges, boolean isIncremental, long repairedAt, boolean isGlobal)
         {
             this.coordinator = coordinator;
             for (ColumnFamilyStore cfs : columnFamilyStores)
             {
-                this.columnFamilyStores.put(cfs.metadata.cfId, cfs);
-                sstableMap.put(cfs.metadata.cfId, new HashSet<String>());
+                this.columnFamilyStores.put(cfs.metadata.id, cfs);
+                sstableMap.put(cfs.metadata.id, new HashSet<>());
             }
             this.ranges = ranges;
             this.repairedAt = repairedAt;
@@ -535,22 +536,22 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
          *
          * whether this is an incremental or full repair
          *
-         * @param cfId the column family
+         * @param tableId the table
          * @param parentSessionId the parent repair session id, used to make sure we don't start multiple repairs over the same sstables
          */
-        public synchronized void markSSTablesRepairing(UUID cfId, UUID parentSessionId)
+        public synchronized void markSSTablesRepairing(TableId tableId, UUID parentSessionId)
         {
-            if (!marked.contains(cfId))
+            if (!marked.contains(tableId))
             {
-                List<SSTableReader> sstables = columnFamilyStores.get(cfId).select(View.select(SSTableSet.CANONICAL, (s) -> !isIncremental || !s.isRepaired())).sstables;
-                Set<SSTableReader> currentlyRepairing = ActiveRepairService.instance.currentlyRepairing(cfId, parentSessionId);
+                List<SSTableReader> sstables = columnFamilyStores.get(tableId).select(View.select(SSTableSet.CANONICAL, (s) -> !isIncremental || !s.isRepaired())).sstables;
+                Set<SSTableReader> currentlyRepairing = ActiveRepairService.instance.currentlyRepairing(tableId, parentSessionId);
                 if (!Sets.intersection(currentlyRepairing, Sets.newHashSet(sstables)).isEmpty())
                 {
                     logger.error("Cannot start multiple repair sessions over the same sstables");
                     throw new RuntimeException("Cannot start multiple repair sessions over the same sstables");
                 }
-                addSSTables(cfId, sstables);
-                marked.add(cfId);
+                addSSTables(tableId, sstables);
+                marked.add(tableId);
             }
         }
 
@@ -560,26 +561,26 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
          * note that validation and streaming do not call this method - they have to work on the actual active sstables on the node, we only call this
          * to know which sstables are still there that were there when we started the repair
          *
-         * @param cfId
+         * @param tableId
          * @param parentSessionId for checking if there exists a snapshot for this repair
          * @return
          */
         @SuppressWarnings("resource")
-        public synchronized Refs<SSTableReader> getActiveRepairedSSTableRefsForAntiCompaction(UUID cfId, UUID parentSessionId)
+        public synchronized Refs<SSTableReader> getActiveRepairedSSTableRefsForAntiCompaction(TableId tableId, UUID parentSessionId)
         {
-            assert marked.contains(cfId);
-            if (!columnFamilyStores.containsKey(cfId))
-                throw new RuntimeException("Not possible to get sstables for anticompaction for " + cfId);
-            boolean isSnapshotRepair = columnFamilyStores.get(cfId).snapshotExists(parentSessionId.toString());
+            assert marked.contains(tableId);
+            if (!columnFamilyStores.containsKey(tableId))
+                throw new RuntimeException("Not possible to get sstables for anticompaction for " + tableId);
+            boolean isSnapshotRepair = columnFamilyStores.get(tableId).snapshotExists(parentSessionId.toString());
             ImmutableMap.Builder<SSTableReader, Ref<SSTableReader>> references = ImmutableMap.builder();
-            Iterable<SSTableReader> sstables = isSnapshotRepair ? getSSTablesForSnapshotRepair(cfId, parentSessionId) : getActiveSSTables(cfId);
-            // we check this above - if columnFamilyStores contains the cfId sstables will not be null
+            Iterable<SSTableReader> sstables = isSnapshotRepair ? getSSTablesForSnapshotRepair(tableId, parentSessionId) : getActiveSSTables(tableId);
+            // we check this above - if columnFamilyStores contains the tableId sstables will not be null
             assert sstables != null;
             for (SSTableReader sstable : sstables)
             {
                 Ref<SSTableReader> ref = sstable.tryRef();
                 if (ref == null)
-                    sstableMap.get(cfId).remove(sstable.getFilename());
+                    sstableMap.get(tableId).remove(sstable.getFilename());
                 else
                     references.put(sstable, ref);
             }
@@ -592,14 +593,14 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
          * We use the generation of the sstables as identifiers instead of the file name to avoid having to parse out the
          * actual filename.
          *
-         * @param cfId
+         * @param tableId
          * @param parentSessionId
          * @return
          */
-        private Set<SSTableReader> getSSTablesForSnapshotRepair(UUID cfId, UUID parentSessionId)
+        private Set<SSTableReader> getSSTablesForSnapshotRepair(TableId tableId, UUID parentSessionId)
         {
             Set<SSTableReader> activeSSTables = new HashSet<>();
-            ColumnFamilyStore cfs = columnFamilyStores.get(cfId);
+            ColumnFamilyStore cfs = columnFamilyStores.get(tableId);
             if (cfs == null)
                 return null;
 
@@ -621,30 +622,30 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
             return activeSSTables;
         }
 
-        public synchronized void maybeSnapshot(UUID cfId, UUID parentSessionId)
+        public synchronized void maybeSnapshot(TableId tableId, UUID parentSessionId)
         {
             String snapshotName = parentSessionId.toString();
-            if (!columnFamilyStores.get(cfId).snapshotExists(snapshotName))
+            if (!columnFamilyStores.get(tableId).snapshotExists(snapshotName))
             {
-                Set<SSTableReader> snapshottedSSTables = columnFamilyStores.get(cfId).snapshot(snapshotName, new Predicate<SSTableReader>()
+                Set<SSTableReader> snapshottedSSTables = columnFamilyStores.get(tableId).snapshot(snapshotName, new Predicate<SSTableReader>()
                 {
                     public boolean apply(SSTableReader sstable)
                     {
                         return sstable != null &&
                                (!isIncremental || !sstable.isRepaired()) &&
-                               !(sstable.metadata.isIndex()) && // exclude SSTables from 2i
+                               !(sstable.metadata().isIndex()) && // exclude SSTables from 2i
                                new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(ranges);
                     }
                 }, true, false);
 
-                if (isAlreadyRepairing(cfId, parentSessionId, snapshottedSSTables))
+                if (isAlreadyRepairing(tableId, parentSessionId, snapshottedSSTables))
                 {
-                    columnFamilyStores.get(cfId).clearSnapshot(snapshotName);
+                    columnFamilyStores.get(tableId).clearSnapshot(snapshotName);
                     logger.error("Cannot start multiple repair sessions over the same sstables");
                     throw new RuntimeException("Cannot start multiple repair sessions over the same sstables");
                 }
-                addSSTables(cfId, snapshottedSSTables);
-                marked.add(cfId);
+                addSSTables(tableId, snapshottedSSTables);
+                marked.add(tableId);
             }
         }
 
@@ -654,14 +655,14 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
          *
          * we compare generations since the sstables have different paths due to snapshot names
          *
-         * @param cfId id of the column family store
+         * @param tableId id of table store
          * @param parentSessionId parent repair session
          * @param sstables the newly snapshotted sstables
          * @return
          */
-        private boolean isAlreadyRepairing(UUID cfId, UUID parentSessionId, Collection<SSTableReader> sstables)
+        private boolean isAlreadyRepairing(TableId tableId, UUID parentSessionId, Collection<SSTableReader> sstables)
         {
-            Set<SSTableReader> currentlyRepairing = ActiveRepairService.instance.currentlyRepairing(cfId, parentSessionId);
+            Set<SSTableReader> currentlyRepairing = ActiveRepairService.instance.currentlyRepairing(tableId, parentSessionId);
             Set<Integer> currentlyRepairingGenerations = new HashSet<>();
             Set<Integer> newRepairingGenerations = new HashSet<>();
             for (SSTableReader sstable : currentlyRepairing)
@@ -672,15 +673,15 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
             return !Sets.intersection(currentlyRepairingGenerations, newRepairingGenerations).isEmpty();
         }
 
-        private Set<SSTableReader> getActiveSSTables(UUID cfId)
+        private Set<SSTableReader> getActiveSSTables(TableId tableId)
         {
-            if (!columnFamilyStores.containsKey(cfId))
+            if (!columnFamilyStores.containsKey(tableId))
                 return null;
 
-            Set<String> repairedSSTables = sstableMap.get(cfId);
+            Set<String> repairedSSTables = sstableMap.get(tableId);
             Set<SSTableReader> activeSSTables = new HashSet<>();
             Set<String> activeSSTableNames = new HashSet<>();
-            ColumnFamilyStore cfs = columnFamilyStores.get(cfId);
+            ColumnFamilyStore cfs = columnFamilyStores.get(tableId);
             for (SSTableReader sstable : cfs.getSSTables(SSTableSet.CANONICAL))
             {
                 if (repairedSSTables.contains(sstable.getFilename()))
@@ -689,14 +690,14 @@ public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFai
                     activeSSTableNames.add(sstable.getFilename());
                 }
             }
-            sstableMap.put(cfId, activeSSTableNames);
+            sstableMap.put(tableId, activeSSTableNames);
             return activeSSTables;
         }
 
-        private void addSSTables(UUID cfId, Collection<SSTableReader> sstables)
+        private void addSSTables(TableId tableId, Collection<SSTableReader> sstables)
         {
             for (SSTableReader sstable : sstables)
-                sstableMap.get(cfId).add(sstable.getFilename());
+                sstableMap.get(tableId).add(sstable.getFilename());
         }
 
 


[08/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 9120546..4f9b12f 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -31,7 +31,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.UpdateBuilder;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.statements.IndexTarget;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.ColumnFilter;
@@ -45,9 +45,12 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -63,12 +66,7 @@ public class RangeTombstoneTest
         SchemaLoader.prepareServer();
         SchemaLoader.createKeyspace(KSNAME,
                                     KeyspaceParams.simple(1),
-                                    SchemaLoader.standardCFMD(KSNAME,
-                                                              CFNAME,
-                                                              1,
-                                                              UTF8Type.instance,
-                                                              Int32Type.instance,
-                                                              Int32Type.instance));
+                                    standardCFMD(KSNAME, CFNAME, 1, UTF8Type.instance, Int32Type.instance, Int32Type.instance));
     }
 
     @Test
@@ -82,20 +80,20 @@ public class RangeTombstoneTest
 
         UpdateBuilder builder;
 
-        builder = UpdateBuilder.create(cfs.metadata, key).withTimestamp(0);
+        builder = UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0);
         for (int i = 0; i < 40; i += 2)
             builder.newRow(i).add("val", i);
         builder.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 1, key).addRangeTombstone(10, 22).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(10, 22).build().applyUnsafe();
 
-        builder = UpdateBuilder.create(cfs.metadata, key).withTimestamp(2);
+        builder = UpdateBuilder.create(cfs.metadata(), key).withTimestamp(2);
         for (int i = 1; i < 40; i += 2)
             builder.newRow(i).add("val", i);
         builder.applyUnsafe();
 
-        new RowUpdateBuilder(cfs.metadata, 3, key).addRangeTombstone(19, 27).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 3, key).addRangeTombstone(19, 27).build().applyUnsafe();
         // We don't flush to test with both a range tomsbtone in memtable and in sstable
 
         // Queries by name
@@ -135,14 +133,14 @@ public class RangeTombstoneTest
         // Inserting data
         String key = "k111";
 
-        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, key).withTimestamp(0);
+        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0);
         for (int i = 0; i < 40; i += 2)
             builder.newRow(i).add("val", i);
         builder.applyUnsafe();
 
-        new RowUpdateBuilder(cfs.metadata, 1, key).addRangeTombstone(5, 10).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(5, 10).build().applyUnsafe();
 
-        new RowUpdateBuilder(cfs.metadata, 2, key).addRangeTombstone(15, 20).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 2, key).addRangeTombstone(15, 20).build().applyUnsafe();
 
         ImmutableBTreePartition partition;
 
@@ -210,14 +208,14 @@ public class RangeTombstoneTest
         sb.add(ClusteringBound.create(cfs.getComparator(), true, true, 1), ClusteringBound.create(cfs.getComparator(), false, true, 10));
         sb.add(ClusteringBound.create(cfs.getComparator(), true, true, 16), ClusteringBound.create(cfs.getComparator(), false, true, 20));
 
-        partition = Util.getOnlyPartitionUnfiltered(SinglePartitionReadCommand.create(cfs.metadata, FBUtilities.nowInSeconds(), Util.dk(key), sb.build()));
+        partition = Util.getOnlyPartitionUnfiltered(SinglePartitionReadCommand.create(cfs.metadata(), FBUtilities.nowInSeconds(), Util.dk(key), sb.build()));
         rt = rangeTombstones(partition);
         assertEquals(2, rt.size());
     }
 
     private Collection<RangeTombstone> rangeTombstones(ImmutableBTreePartition partition)
     {
-        List<RangeTombstone> tombstones = new ArrayList<RangeTombstone>();
+        List<RangeTombstone> tombstones = new ArrayList<>();
         Iterators.addAll(tombstones, partition.deletionInfo().rangeIterator(false));
         return tombstones;
     }
@@ -231,7 +229,7 @@ public class RangeTombstoneTest
         String key = "rt_times";
 
         int nowInSec = FBUtilities.nowInSeconds();
-        new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata, Util.dk(key), 1000, nowInSec)).apply();
+        new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata(), Util.dk(key), 1000, nowInSec)).apply();
         cfs.forceBlockingFlush();
 
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
@@ -249,11 +247,11 @@ public class RangeTombstoneTest
         cfs.truncateBlocking();
         String key = "rt_times";
 
-        UpdateBuilder.create(cfs.metadata, key).withTimestamp(999).newRow(5).add("val", 5).apply();
+        UpdateBuilder.create(cfs.metadata(), key).withTimestamp(999).newRow(5).add("val", 5).apply();
 
         key = "rt_times2";
         int nowInSec = FBUtilities.nowInSeconds();
-        new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata, Util.dk(key), 1000, nowInSec)).apply();
+        new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata(), Util.dk(key), 1000, nowInSec)).apply();
         cfs.forceBlockingFlush();
 
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
@@ -272,7 +270,7 @@ public class RangeTombstoneTest
         String key = "rt_times";
 
         int nowInSec = FBUtilities.nowInSeconds();
-        new RowUpdateBuilder(cfs.metadata, nowInSec, 1000L, key).addRangeTombstone(1, 2).build().apply();
+        new RowUpdateBuilder(cfs.metadata(), nowInSec, 1000L, key).addRangeTombstone(1, 2).build().apply();
         cfs.forceBlockingFlush();
 
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
@@ -290,11 +288,11 @@ public class RangeTombstoneTest
         cfs.truncateBlocking();
         String key = "rt_times";
 
-        UpdateBuilder.create(cfs.metadata, key).withTimestamp(999).newRow(5).add("val", 5).apply();
+        UpdateBuilder.create(cfs.metadata(), key).withTimestamp(999).newRow(5).add("val", 5).apply();
 
         key = "rt_times2";
         int nowInSec = FBUtilities.nowInSeconds();
-        new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata, Util.dk(key), 1000, nowInSec)).apply();
+        new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata(), Util.dk(key), 1000, nowInSec)).apply();
         cfs.forceBlockingFlush();
 
         cfs.forceBlockingFlush();
@@ -317,17 +315,17 @@ public class RangeTombstoneTest
     {
         Keyspace ks = Keyspace.open(KSNAME);
         ColumnFamilyStore cfs = ks.getColumnFamilyStore(CFNAME);
-        cfs.metadata.gcGraceSeconds(2);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().gcGraceSeconds(2).build(), true);
 
         String key = "7810";
 
-        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, key).withTimestamp(0);
+        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0);
         for (int i = 10; i < 20; i ++)
             builder.newRow(i).add("val", i);
         builder.apply();
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 1, key).addRangeTombstone(10, 11).build().apply();
+        new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(10, 11).build().apply();
         cfs.forceBlockingFlush();
 
         Thread.sleep(5);
@@ -340,16 +338,16 @@ public class RangeTombstoneTest
     {
         Keyspace ks = Keyspace.open(KSNAME);
         ColumnFamilyStore cfs = ks.getColumnFamilyStore(CFNAME);
-        cfs.metadata.gcGraceSeconds(2);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().gcGraceSeconds(2).build(), true);
 
         String key = "7808_1";
-        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, key).withTimestamp(0);
+        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0);
         for (int i = 0; i < 40; i += 2)
             builder.newRow(i).add("val", i);
         builder.apply();
         cfs.forceBlockingFlush();
 
-        new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata, Util.dk(key), 1, 1)).apply();
+        new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata(), Util.dk(key), 1, 1)).apply();
         cfs.forceBlockingFlush();
         Thread.sleep(5);
         cfs.forceMajorCompaction();
@@ -360,18 +358,18 @@ public class RangeTombstoneTest
     {
         Keyspace ks = Keyspace.open(KSNAME);
         ColumnFamilyStore cfs = ks.getColumnFamilyStore(CFNAME);
-        cfs.metadata.gcGraceSeconds(2);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().gcGraceSeconds(2).build(), true);
 
         String key = "7808_2";
-        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, key).withTimestamp(0);
+        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0);
         for (int i = 10; i < 20; i ++)
             builder.newRow(i).add("val", i);
         builder.apply();
         cfs.forceBlockingFlush();
 
-        new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata, Util.dk(key), 0, 0)).apply();
+        new Mutation(PartitionUpdate.fullPartitionDelete(cfs.metadata(), Util.dk(key), 0, 0)).apply();
 
-        UpdateBuilder.create(cfs.metadata, key).withTimestamp(1).newRow(5).add("val", 5).apply();
+        UpdateBuilder.create(cfs.metadata(), key).withTimestamp(1).newRow(5).add("val", 5).apply();
 
         cfs.forceBlockingFlush();
         Thread.sleep(5);
@@ -389,19 +387,19 @@ public class RangeTombstoneTest
         // Inserting data
         String key = "k2";
 
-        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, key).withTimestamp(0);
+        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0);
         for (int i = 0; i < 20; i++)
             builder.newRow(i).add("val", i);
         builder.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 1, key).addRangeTombstone(5, 15).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(5, 15).build().applyUnsafe();
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 1, key).addRangeTombstone(5, 10).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(5, 10).build().applyUnsafe();
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 2, key).addRangeTombstone(5, 8).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 2, key).addRangeTombstone(5, 8).build().applyUnsafe();
         cfs.forceBlockingFlush();
 
         Partition partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build());
@@ -435,18 +433,18 @@ public class RangeTombstoneTest
         // Inserting data
         String key = "k3";
 
-        UpdateBuilder.create(cfs.metadata, key).withTimestamp(0).newRow(2).add("val", 2).applyUnsafe();
+        UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0).newRow(2).add("val", 2).applyUnsafe();
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 1, key).addRangeTombstone(0, 10).build().applyUnsafe();
-        UpdateBuilder.create(cfs.metadata, key).withTimestamp(2).newRow(1).add("val", 1).applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(0, 10).build().applyUnsafe();
+        UpdateBuilder.create(cfs.metadata(), key).withTimestamp(2).newRow(1).add("val", 1).applyUnsafe();
         cfs.forceBlockingFlush();
 
         // Get the last value of the row
         FilteredPartition partition = Util.getOnlyPartition(Util.cmd(cfs, key).build());
         assertTrue(partition.rowCount() > 0);
 
-        int last = i(partition.unfilteredIterator(ColumnFilter.all(cfs.metadata), Slices.ALL, true).next().clustering().get(0));
+        int last = i(partition.unfilteredIterator(ColumnFilter.all(cfs.metadata()), Slices.ALL, true).next().clustering().get(0));
         assertEquals("Last column should be column 1 since column 2 has been deleted", 1, last);
     }
 
@@ -461,17 +459,25 @@ public class RangeTombstoneTest
         cfs.truncateBlocking();
         cfs.disableAutoCompaction();
 
-        ColumnDefinition cd = cfs.metadata.getColumnDefinition(indexedColumnName).copy();
+        ColumnMetadata cd = cfs.metadata().getColumn(indexedColumnName).copy();
         IndexMetadata indexDef =
-            IndexMetadata.fromIndexTargets(cfs.metadata,
-                                           Collections.singletonList(new IndexTarget(cd.name, IndexTarget.Type.VALUES)),
+            IndexMetadata.fromIndexTargets(
+            Collections.singletonList(new IndexTarget(cd.name, IndexTarget.Type.VALUES)),
                                            "test_index",
                                            IndexMetadata.Kind.CUSTOM,
                                            ImmutableMap.of(IndexTarget.CUSTOM_INDEX_OPTION_NAME,
                                                            StubIndex.class.getName()));
 
-        if (!cfs.metadata.getIndexes().get("test_index").isPresent())
-            cfs.metadata.indexes(cfs.metadata.getIndexes().with(indexDef));
+        TableMetadata current = cfs.metadata();
+
+        if (!current.indexes.get("test_index").isPresent())
+        {
+            TableMetadata updated =
+                current.unbuild()
+                       .indexes(current.indexes.with(indexDef))
+                       .build();
+            MigrationManager.announceTableUpdate(updated, true);
+        }
 
         Future<?> rebuild = cfs.indexManager.addIndex(indexDef);
         // If rebuild there is, wait for the rebuild to finish so it doesn't race with the following insertions
@@ -485,13 +491,13 @@ public class RangeTombstoneTest
                                                      .orElseThrow(() -> new RuntimeException(new AssertionError("Index not found")));
         index.reset();
 
-        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, key).withTimestamp(0);
+        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0);
         for (int i = 0; i < 10; i++)
             builder.newRow(i).add("val", i);
         builder.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 0, key).addRangeTombstone(0, 7).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, key).addRangeTombstone(0, 7).build().applyUnsafe();
         cfs.forceBlockingFlush();
 
         assertEquals(10, index.rowsInserted.size());
@@ -515,13 +521,13 @@ public class RangeTombstoneTest
         cfs.truncateBlocking();
         cfs.disableAutoCompaction();
 
-        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, key).withTimestamp(0);
+        UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0);
         for (int i = 0; i < 10; i += 2)
             builder.newRow(i).add("val", i);
         builder.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 0, key).addRangeTombstone(0, 7).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, key).addRangeTombstone(0, 7).build().applyUnsafe();
         cfs.forceBlockingFlush();
 
         // there should be 2 sstables
@@ -539,11 +545,11 @@ public class RangeTombstoneTest
             {
                 // after compaction, we should have a single RT with a single row (the row 8)
                 Unfiltered u1 = iter.next();
-                assertTrue("Expecting open marker, got " + u1.toString(cfs.metadata), u1 instanceof RangeTombstoneMarker);
+                assertTrue("Expecting open marker, got " + u1.toString(cfs.metadata()), u1 instanceof RangeTombstoneMarker);
                 Unfiltered u2 = iter.next();
-                assertTrue("Expecting close marker, got " + u2.toString(cfs.metadata), u2 instanceof RangeTombstoneMarker);
+                assertTrue("Expecting close marker, got " + u2.toString(cfs.metadata()), u2 instanceof RangeTombstoneMarker);
                 Unfiltered u3 = iter.next();
-                assertTrue("Expecting row, got " + u3.toString(cfs.metadata), u3 instanceof Row);
+                assertTrue("Expecting row, got " + u3.toString(cfs.metadata()), u3 instanceof Row);
             }
         }
     }
@@ -559,17 +565,25 @@ public class RangeTombstoneTest
         cfs.truncateBlocking();
         cfs.disableAutoCompaction();
 
-        ColumnDefinition cd = cfs.metadata.getColumnDefinition(indexedColumnName).copy();
+        ColumnMetadata cd = cfs.metadata().getColumn(indexedColumnName).copy();
         IndexMetadata indexDef =
-            IndexMetadata.fromIndexTargets(cfs.metadata,
-                                           Collections.singletonList(new IndexTarget(cd.name, IndexTarget.Type.VALUES)),
+            IndexMetadata.fromIndexTargets(
+            Collections.singletonList(new IndexTarget(cd.name, IndexTarget.Type.VALUES)),
                                            "test_index",
                                            IndexMetadata.Kind.CUSTOM,
                                            ImmutableMap.of(IndexTarget.CUSTOM_INDEX_OPTION_NAME,
                                                            StubIndex.class.getName()));
 
-        if (!cfs.metadata.getIndexes().get("test_index").isPresent())
-            cfs.metadata.indexes(cfs.metadata.getIndexes().with(indexDef));
+        TableMetadata current = cfs.metadata();
+
+        if (!current.indexes.get("test_index").isPresent())
+        {
+            TableMetadata updated =
+                current.unbuild()
+                       .indexes(current.indexes.with(indexDef))
+                       .build();
+            MigrationManager.announceTableUpdate(updated, true);
+        }
 
         Future<?> rebuild = cfs.indexManager.addIndex(indexDef);
         // If rebuild there is, wait for the rebuild to finish so it doesn't race with the following insertions
@@ -579,13 +593,13 @@ public class RangeTombstoneTest
         StubIndex index = (StubIndex)cfs.indexManager.getIndexByName("test_index");
         index.reset();
 
-        UpdateBuilder.create(cfs.metadata, key).withTimestamp(0).newRow(1).add("val", 1).applyUnsafe();
+        UpdateBuilder.create(cfs.metadata(), key).withTimestamp(0).newRow(1).add("val", 1).applyUnsafe();
 
         // add a RT which hides the column we just inserted
-        new RowUpdateBuilder(cfs.metadata, 1, key).addRangeTombstone(0, 1).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, key).addRangeTombstone(0, 1).build().applyUnsafe();
 
         // now re-insert that column
-        UpdateBuilder.create(cfs.metadata, key).withTimestamp(2).newRow(1).add("val", 1).applyUnsafe();
+        UpdateBuilder.create(cfs.metadata(), key).withTimestamp(2).newRow(1).add("val", 1).applyUnsafe();
 
         cfs.forceBlockingFlush();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/ReadCommandTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ReadCommandTest.java b/test/unit/org/apache/cassandra/db/ReadCommandTest.java
index 9b7775da..249d780 100644
--- a/test/unit/org/apache/cassandra/db/ReadCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadCommandTest.java
@@ -27,7 +27,6 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter;
 import org.apache.cassandra.db.filter.ColumnFilter;
@@ -49,6 +48,7 @@ import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -66,30 +66,28 @@ public class ReadCommandTest
     {
         DatabaseDescriptor.daemonInitialization();
 
-        CFMetaData metadata1 = SchemaLoader.standardCFMD(KEYSPACE, CF1);
-
-        CFMetaData metadata2 = CFMetaData.Builder.create(KEYSPACE, CF2)
-                                                         .addPartitionKey("key", BytesType.instance)
-                                                         .addClusteringColumn("col", AsciiType.instance)
-                                                         .addRegularColumn("a", AsciiType.instance)
-                                                         .addRegularColumn("b", AsciiType.instance).build();
-
-        CFMetaData metadata3 = CFMetaData.Builder.create(KEYSPACE, CF3)
-                                                 .addPartitionKey("key", BytesType.instance)
-                                                 .addClusteringColumn("col", AsciiType.instance)
-                                                 .addRegularColumn("a", AsciiType.instance)
-                                                 .addRegularColumn("b", AsciiType.instance)
-                                                 .addRegularColumn("c", AsciiType.instance)
-                                                 .addRegularColumn("d", AsciiType.instance)
-                                                 .addRegularColumn("e", AsciiType.instance)
-                                                 .addRegularColumn("f", AsciiType.instance).build();
+        TableMetadata.Builder metadata1 = SchemaLoader.standardCFMD(KEYSPACE, CF1);
+
+        TableMetadata.Builder metadata2 =
+            TableMetadata.builder(KEYSPACE, CF2)
+                         .addPartitionKeyColumn("key", BytesType.instance)
+                         .addClusteringColumn("col", AsciiType.instance)
+                         .addRegularColumn("a", AsciiType.instance)
+                         .addRegularColumn("b", AsciiType.instance);
+
+        TableMetadata.Builder metadata3 =
+            TableMetadata.builder(KEYSPACE, CF3)
+                         .addPartitionKeyColumn("key", BytesType.instance)
+                         .addClusteringColumn("col", AsciiType.instance)
+                         .addRegularColumn("a", AsciiType.instance)
+                         .addRegularColumn("b", AsciiType.instance)
+                         .addRegularColumn("c", AsciiType.instance)
+                         .addRegularColumn("d", AsciiType.instance)
+                         .addRegularColumn("e", AsciiType.instance)
+                         .addRegularColumn("f", AsciiType.instance);
 
         SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE,
-                                    KeyspaceParams.simple(1),
-                                    metadata1,
-                                    metadata2,
-                                    metadata3);
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), metadata1, metadata2, metadata3);
     }
 
     @Test
@@ -97,7 +95,7 @@ public class ReadCommandTest
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(CF1);
 
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key1"))
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key1"))
                 .clustering("Column1")
                 .add("val", ByteBufferUtil.bytes("abcd"))
                 .build()
@@ -105,7 +103,7 @@ public class ReadCommandTest
 
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key2"))
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key2"))
                 .clustering("Column1")
                 .add("val", ByteBufferUtil.bytes("abcd"))
                 .build()
@@ -125,7 +123,7 @@ public class ReadCommandTest
 
         cfs.truncateBlocking();
 
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key"))
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key"))
                 .clustering("cc")
                 .add("a", ByteBufferUtil.bytes("abcd"))
                 .build()
@@ -133,7 +131,7 @@ public class ReadCommandTest
 
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key"))
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key"))
                 .clustering("dd")
                 .add("a", ByteBufferUtil.bytes("abcd"))
                 .build()
@@ -156,7 +154,7 @@ public class ReadCommandTest
 
         cfs.truncateBlocking();
 
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key"))
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key"))
                 .clustering("cc")
                 .add("a", ByteBufferUtil.bytes("abcd"))
                 .build()
@@ -164,7 +162,7 @@ public class ReadCommandTest
 
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key"))
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key"))
                 .clustering("dd")
                 .add("a", ByteBufferUtil.bytes("abcd"))
                 .build()
@@ -213,10 +211,10 @@ public class ReadCommandTest
 
         List<ByteBuffer> buffers = new ArrayList<>(groups.length);
         int nowInSeconds = FBUtilities.nowInSeconds();
-        ColumnFilter columnFilter = ColumnFilter.allRegularColumnsBuilder(cfs.metadata).build();
+        ColumnFilter columnFilter = ColumnFilter.allRegularColumnsBuilder(cfs.metadata()).build();
         RowFilter rowFilter = RowFilter.create();
         Slice slice = Slice.make(ClusteringBound.BOTTOM, ClusteringBound.TOP);
-        ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfs.metadata.comparator, slice), false);
+        ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfs.metadata().comparator, slice), false);
 
         for (String[][] group : groups)
         {
@@ -228,7 +226,7 @@ public class ReadCommandTest
             {
                 if (data[0].equals("1"))
                 {
-                    new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes(data[1]))
+                    new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes(data[1]))
                     .clustering(data[2])
                     .add(data[3], ByteBufferUtil.bytes("blah"))
                     .build()
@@ -236,9 +234,9 @@ public class ReadCommandTest
                 }
                 else
                 {
-                    RowUpdateBuilder.deleteRow(cfs.metadata, FBUtilities.timestampMicros(), ByteBufferUtil.bytes(data[1]), data[2]).apply();
+                    RowUpdateBuilder.deleteRow(cfs.metadata(), FBUtilities.timestampMicros(), ByteBufferUtil.bytes(data[1]), data[2]).apply();
                 }
-                commands.add(SinglePartitionReadCommand.create(cfs.metadata, nowInSeconds, columnFilter, rowFilter, DataLimits.NONE, Util.dk(data[1]), sliceFilter));
+                commands.add(SinglePartitionReadCommand.create(cfs.metadata(), nowInSeconds, columnFilter, rowFilter, DataLimits.NONE, Util.dk(data[1]), sliceFilter));
             }
 
             cfs.forceBlockingFlush();
@@ -266,7 +264,7 @@ public class ReadCommandTest
             {
                 iterators.add(UnfilteredPartitionIterators.serializerForIntraNode().deserialize(in,
                                                                                                 MessagingService.current_version,
-                                                                                                cfs.metadata,
+                                                                                                cfs.metadata(),
                                                                                                 columnFilter,
                                                                                                 SerializationHelper.Flag.LOCAL));
             }
@@ -298,7 +296,7 @@ public class ReadCommandTest
                     while (rowIterator.hasNext())
                     {
                         Row row = rowIterator.next();
-                        assertEquals("col=" + expectedRows[i++], row.clustering().toString(cfs.metadata));
+                        assertEquals("col=" + expectedRows[i++], row.clustering().toString(cfs.metadata()));
                         //System.out.print(row.toString(cfs.metadata, true));
                     }
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/ReadMessageTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ReadMessageTest.java b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
index f76bf93..5b05253 100644
--- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
@@ -28,8 +28,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.CommitLogTestReplayer;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -59,29 +59,34 @@ public class ReadMessageTest
     {
         DatabaseDescriptor.daemonInitialization();
 
-        CFMetaData cfForReadMetadata = CFMetaData.Builder.create(KEYSPACE1, CF_FOR_READ_TEST)
-                                                            .addPartitionKey("key", BytesType.instance)
-                                                            .addClusteringColumn("col1", AsciiType.instance)
-                                                            .addClusteringColumn("col2", AsciiType.instance)
-                                                            .addRegularColumn("a", AsciiType.instance)
-                                                            .addRegularColumn("b", AsciiType.instance).build();
-
-        CFMetaData cfForCommitMetadata1 = CFMetaData.Builder.create(KEYSPACE1, CF_FOR_COMMIT_TEST)
-                                                       .addPartitionKey("key", BytesType.instance)
-                                                       .addClusteringColumn("name", AsciiType.instance)
-                                                       .addRegularColumn("commit1", AsciiType.instance).build();
-
-        CFMetaData cfForCommitMetadata2 = CFMetaData.Builder.create(KEYSPACENOCOMMIT, CF_FOR_COMMIT_TEST)
-                                                            .addPartitionKey("key", BytesType.instance)
-                                                            .addClusteringColumn("name", AsciiType.instance)
-                                                            .addRegularColumn("commit2", AsciiType.instance).build();
+        TableMetadata.Builder cfForReadMetadata =
+            TableMetadata.builder(KEYSPACE1, CF_FOR_READ_TEST)
+                         .addPartitionKeyColumn("key", BytesType.instance)
+                         .addClusteringColumn("col1", AsciiType.instance)
+                         .addClusteringColumn("col2", AsciiType.instance)
+                         .addRegularColumn("a", AsciiType.instance)
+                         .addRegularColumn("b", AsciiType.instance);
+
+        TableMetadata.Builder cfForCommitMetadata1 =
+            TableMetadata.builder(KEYSPACE1, CF_FOR_COMMIT_TEST)
+                         .addPartitionKeyColumn("key", BytesType.instance)
+                         .addClusteringColumn("name", AsciiType.instance)
+                         .addRegularColumn("commit1", AsciiType.instance);
+
+        TableMetadata.Builder cfForCommitMetadata2 =
+            TableMetadata.builder(KEYSPACENOCOMMIT, CF_FOR_COMMIT_TEST)
+                         .addPartitionKeyColumn("key", BytesType.instance)
+                         .addClusteringColumn("name", AsciiType.instance)
+                         .addRegularColumn("commit2", AsciiType.instance);
 
         SchemaLoader.prepareServer();
+
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     KeyspaceParams.simple(1),
                                     SchemaLoader.standardCFMD(KEYSPACE1, CF),
                                     cfForReadMetadata,
                                     cfForCommitMetadata1);
+
         SchemaLoader.createKeyspace(KEYSPACENOCOMMIT,
                                     KeyspaceParams.simpleTransient(1),
                                     SchemaLoader.standardCFMD(KEYSPACENOCOMMIT, CF),
@@ -158,13 +163,13 @@ public class ReadMessageTest
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF);
 
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key1"))
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key1"))
                 .clustering("Column1")
                 .add("val", ByteBufferUtil.bytes("abcd"))
                 .build()
                 .apply();
 
-        ColumnDefinition col = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
+        ColumnMetadata col = cfs.metadata().getColumn(ByteBufferUtil.bytes("val"));
         int found = 0;
         for (FilteredPartition partition : Util.getAll(Util.cmd(cfs).build()))
         {
@@ -184,20 +189,20 @@ public class ReadMessageTest
 
         ColumnFamilyStore cfsnocommit = Keyspace.open(KEYSPACENOCOMMIT).getColumnFamilyStore(CF_FOR_COMMIT_TEST);
 
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("row"))
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("row"))
                 .clustering("c")
                 .add("commit1", ByteBufferUtil.bytes("abcd"))
                 .build()
                 .apply();
 
-        new RowUpdateBuilder(cfsnocommit.metadata, 0, ByteBufferUtil.bytes("row"))
+        new RowUpdateBuilder(cfsnocommit.metadata(), 0, ByteBufferUtil.bytes("row"))
                 .clustering("c")
                 .add("commit2", ByteBufferUtil.bytes("abcd"))
                 .build()
                 .apply();
 
-        Checker checker = new Checker(cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("commit1")),
-                                      cfsnocommit.metadata.getColumnDefinition(ByteBufferUtil.bytes("commit2")));
+        Checker checker = new Checker(cfs.metadata().getColumn(ByteBufferUtil.bytes("commit1")),
+                                      cfsnocommit.metadata().getColumn(ByteBufferUtil.bytes("commit2")));
 
         CommitLogTestReplayer replayer = new CommitLogTestReplayer(checker);
         replayer.examineCommitLog();
@@ -208,13 +213,13 @@ public class ReadMessageTest
 
     static class Checker implements Predicate<Mutation>
     {
-        private final ColumnDefinition withCommit;
-        private final ColumnDefinition withoutCommit;
+        private final ColumnMetadata withCommit;
+        private final ColumnMetadata withoutCommit;
 
         boolean commitLogMessageFound = false;
         boolean noCommitLogMessageFound = false;
 
-        public Checker(ColumnDefinition withCommit, ColumnDefinition withoutCommit)
+        public Checker(ColumnMetadata withCommit, ColumnMetadata withoutCommit)
         {
             this.withCommit = withCommit;
             this.withoutCommit = withoutCommit;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/RecoveryManagerFlushedTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerFlushedTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerFlushedTest.java
index 0b20343..7225560 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerFlushedTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerFlushedTest.java
@@ -36,7 +36,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ParameterizedClass;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -126,7 +126,7 @@ public class RecoveryManagerFlushedTest
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
-        new RowUpdateBuilder(cfs.metadata, 0, key)
+        new RowUpdateBuilder(cfs.metadata(), 0, key)
             .clustering("c")
             .add("val", "val1")
             .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
index 8897700..d60574f 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerMissingHeaderTest.java
@@ -98,11 +98,11 @@ public class RecoveryManagerMissingHeaderTest
         Keyspace keyspace2 = Keyspace.open(KEYSPACE2);
 
         DecoratedKey dk = Util.dk("keymulti");
-        UnfilteredRowIterator upd1 = Util.apply(new RowUpdateBuilder(keyspace1.getColumnFamilyStore(CF_STANDARD1).metadata, 1L, 0, "keymulti")
+        UnfilteredRowIterator upd1 = Util.apply(new RowUpdateBuilder(keyspace1.getColumnFamilyStore(CF_STANDARD1).metadata(), 1L, 0, "keymulti")
                                        .clustering("col1").add("val", "1")
                                        .build());
 
-        UnfilteredRowIterator upd2 = Util.apply(new RowUpdateBuilder(keyspace2.getColumnFamilyStore(CF_STANDARD3).metadata, 1L, 0, "keymulti")
+        UnfilteredRowIterator upd2 = Util.apply(new RowUpdateBuilder(keyspace2.getColumnFamilyStore(CF_STANDARD3).metadata(), 1L, 0, "keymulti")
                                        .clustering("col1").add("val", "1")
                                        .build());
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
index f5bda4f..164253d 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
@@ -30,21 +30,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.Util;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.ParameterizedClass;
-import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.context.CounterContext;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.compress.DeflateCompressor;
-import org.apache.cassandra.io.compress.LZ4Compressor;
-import org.apache.cassandra.io.compress.SnappyCompressor;
-
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -53,16 +38,30 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import static org.junit.Assert.assertEquals;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogArchiver;
+import org.apache.cassandra.db.commitlog.CommitLogReplayer;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.security.EncryptionContext;
 import org.apache.cassandra.security.EncryptionContextGenerator;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.db.commitlog.CommitLogReplayer;
+
+import static org.junit.Assert.assertEquals;
 
 @RunWith(Parameterized.class)
 public class RecoveryManagerTest
@@ -141,11 +140,11 @@ public class RecoveryManagerTest
             Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
             Keyspace keyspace2 = Keyspace.open(KEYSPACE2);
 
-            UnfilteredRowIterator upd1 = Util.apply(new RowUpdateBuilder(keyspace1.getColumnFamilyStore(CF_STANDARD1).metadata, 1L, 0, "keymulti")
+            UnfilteredRowIterator upd1 = Util.apply(new RowUpdateBuilder(keyspace1.getColumnFamilyStore(CF_STANDARD1).metadata(), 1L, 0, "keymulti")
                 .clustering("col1").add("val", "1")
                 .build());
 
-            UnfilteredRowIterator upd2 = Util.apply(new RowUpdateBuilder(keyspace2.getColumnFamilyStore(CF_STANDARD3).metadata, 1L, 0, "keymulti")
+            UnfilteredRowIterator upd2 = Util.apply(new RowUpdateBuilder(keyspace2.getColumnFamilyStore(CF_STANDARD3).metadata(), 1L, 0, "keymulti")
                                            .clustering("col2").add("val", "1")
                                            .build());
 
@@ -204,11 +203,11 @@ public class RecoveryManagerTest
         Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
         Keyspace keyspace2 = Keyspace.open(KEYSPACE2);
 
-        UnfilteredRowIterator upd1 = Util.apply(new RowUpdateBuilder(keyspace1.getColumnFamilyStore(CF_STANDARD1).metadata, 1L, 0, "keymulti")
+        UnfilteredRowIterator upd1 = Util.apply(new RowUpdateBuilder(keyspace1.getColumnFamilyStore(CF_STANDARD1).metadata(), 1L, 0, "keymulti")
             .clustering("col1").add("val", "1")
             .build());
 
-        UnfilteredRowIterator upd2 = Util.apply(new RowUpdateBuilder(keyspace2.getColumnFamilyStore(CF_STANDARD3).metadata, 1L, 0, "keymulti")
+        UnfilteredRowIterator upd2 = Util.apply(new RowUpdateBuilder(keyspace2.getColumnFamilyStore(CF_STANDARD3).metadata(), 1L, 0, "keymulti")
                                        .clustering("col2").add("val", "1")
                                        .build());
 
@@ -231,7 +230,7 @@ public class RecoveryManagerTest
 
         for (int i = 0; i < 10; ++i)
         {
-            new CounterMutation(new RowUpdateBuilder(cfs.metadata, 1L, 0, "key")
+            new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 1L, 0, "key")
                 .clustering("cc").add("val", CounterContext.instance().createLocal(1L))
                 .build(), ConsistencyLevel.ALL).apply();
         }
@@ -240,7 +239,7 @@ public class RecoveryManagerTest
 
         int replayed = CommitLog.instance.resetUnsafe(false);
 
-        ColumnDefinition counterCol = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
+        ColumnMetadata counterCol = cfs.metadata().getColumn(ByteBufferUtil.bytes("val"));
         Row row = Util.getOnlyRow(Util.cmd(cfs).includeRow("cc").columns("val").build());
         assertEquals(10L, CounterContext.instance().total(row.getCell(counterCol).value()));
     }
@@ -257,7 +256,7 @@ public class RecoveryManagerTest
         for (int i = 0; i < 10; ++i)
         {
             long ts = TimeUnit.MILLISECONDS.toMicros(timeMS + (i * 1000));
-            new RowUpdateBuilder(cfs.metadata, ts, "name-" + i)
+            new RowUpdateBuilder(cfs.metadata(), ts, "name-" + i)
                 .clustering("cc")
                 .add("val", Integer.toString(i))
                 .build()
@@ -292,7 +291,7 @@ public class RecoveryManagerTest
             else
                 ts = TimeUnit.MILLISECONDS.toMicros(timeMS + (i * 1000));
 
-            new RowUpdateBuilder(cfs.metadata, ts, "name-" + i)
+            new RowUpdateBuilder(cfs.metadata(), ts, "name-" + i)
                 .clustering("cc")
                 .add("val", Integer.toString(i))
                 .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
index 738888f..ef00b22 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
@@ -93,7 +93,7 @@ public class RecoveryManagerTruncateTest
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         // add a single cell
-        new RowUpdateBuilder(cfs.metadata, 0, "key1")
+        new RowUpdateBuilder(cfs.metadata(), 0, "key1")
             .clustering("cc")
             .add("val", "val1")
             .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/RowCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowCacheTest.java b/test/unit/org/apache/cassandra/db/RowCacheTest.java
index 558c187..fee3f2c 100644
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@ -32,8 +32,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.RowCacheKey;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
@@ -99,9 +98,9 @@ public class RowCacheTest
 
         ByteBuffer key = ByteBufferUtil.bytes("rowcachekey");
         DecoratedKey dk = cachedStore.decorateKey(key);
-        RowCacheKey rck = new RowCacheKey(cachedStore.metadata.ksAndCFName, dk);
+        RowCacheKey rck = new RowCacheKey(cachedStore.metadata(), dk);
 
-        RowUpdateBuilder rub = new RowUpdateBuilder(cachedStore.metadata, System.currentTimeMillis(), key);
+        RowUpdateBuilder rub = new RowUpdateBuilder(cachedStore.metadata(), System.currentTimeMillis(), key);
         rub.clustering(String.valueOf(0));
         rub.add("val", ByteBufferUtil.bytes("val" + 0));
         rub.build().applyUnsafe();
@@ -414,11 +413,11 @@ public class RowCacheTest
 
         ByteBuffer key = ByteBufferUtil.bytes("rowcachekey");
         DecoratedKey dk = cachedStore.decorateKey(key);
-        RowCacheKey rck = new RowCacheKey(cachedStore.metadata.ksAndCFName, dk);
+        RowCacheKey rck = new RowCacheKey(cachedStore.metadata(), dk);
         String values[] = new String[200];
         for (int i = 0; i < 200; i++)
         {
-            RowUpdateBuilder rub = new RowUpdateBuilder(cachedStore.metadata, System.currentTimeMillis(), key);
+            RowUpdateBuilder rub = new RowUpdateBuilder(cachedStore.metadata(), System.currentTimeMillis(), key);
             rub.clustering(String.valueOf(i));
             values[i] = "val" + i;
             rub.add("val", ByteBufferUtil.bytes(values[i]));
@@ -548,12 +547,10 @@ public class RowCacheTest
     private static void readData(String keyspace, String columnFamily, int offset, int numberOfRows)
     {
         ColumnFamilyStore store = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily);
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace, columnFamily);
 
         for (int i = offset; i < offset + numberOfRows; i++)
         {
             DecoratedKey key = Util.dk("key" + i);
-            Clustering cl = Clustering.make(ByteBufferUtil.bytes("col" + i));
             Util.getAll(Util.cmd(store, key).build());
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
index 28590d8..e563070 100644
--- a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
@@ -33,7 +33,8 @@ import org.junit.Test;
 
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.IMeasurableMemory;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.columniterator.AbstractSSTableIterator;
@@ -131,18 +132,21 @@ public class RowIndexEntryTest extends CQLTester
 
     private static class DoubleSerializer implements AutoCloseable
     {
-        CFMetaData cfMeta = CFMetaData.compile("CREATE TABLE pipe.dev_null (pk bigint, ck bigint, val text, PRIMARY KEY(pk, ck))", "foo");
+        TableMetadata metadata =
+            CreateTableStatement.parse("CREATE TABLE pipe.dev_null (pk bigint, ck bigint, val text, PRIMARY KEY(pk, ck))", "foo")
+                                .build();
+
         Version version = BigFormat.latestVersion;
 
         DeletionTime deletionInfo = new DeletionTime(FBUtilities.timestampMicros(), FBUtilities.nowInSeconds());
         LivenessInfo primaryKeyLivenessInfo = LivenessInfo.EMPTY;
         Row.Deletion deletion = Row.Deletion.LIVE;
 
-        SerializationHeader header = new SerializationHeader(true, cfMeta, cfMeta.partitionColumns(), EncodingStats.NO_STATS);
+        SerializationHeader header = new SerializationHeader(true, metadata, metadata.regularAndStaticColumns(), EncodingStats.NO_STATS);
 
         // create C-11206 + old serializer instances
         RowIndexEntry.IndexSerializer rieSerializer = new RowIndexEntry.Serializer(version, header);
-        Pre_C_11206_RowIndexEntry.Serializer oldSerializer = new Pre_C_11206_RowIndexEntry.Serializer(cfMeta, version, header);
+        Pre_C_11206_RowIndexEntry.Serializer oldSerializer = new Pre_C_11206_RowIndexEntry.Serializer(metadata, version, header);
 
         @SuppressWarnings({ "resource", "IOResourceOpenedButNotSafelyClosed" })
         final DataOutputBuffer rieOutput = new DataOutputBuffer(1024);
@@ -201,7 +205,7 @@ public class RowIndexEntryTest extends CQLTester
         private AbstractUnfilteredRowIterator makeRowIter(Row staticRow, DecoratedKey partitionKey,
                                                           Iterator<Clustering> clusteringIter, SequentialWriter dataWriter)
         {
-            return new AbstractUnfilteredRowIterator(cfMeta, partitionKey, deletionInfo, cfMeta.partitionColumns(),
+            return new AbstractUnfilteredRowIterator(metadata, partitionKey, deletionInfo, metadata.regularAndStaticColumns(),
                                                      staticRow, false, new EncodingStats(0, 0, 0))
             {
                 protected Unfiltered computeNext()
@@ -225,7 +229,7 @@ public class RowIndexEntryTest extends CQLTester
         private Unfiltered buildRow(Clustering clustering)
         {
             BTree.Builder<ColumnData> builder = BTree.builder(ColumnData.comparator);
-            builder.add(BufferCell.live(cfMeta.partitionColumns().iterator().next(),
+            builder.add(BufferCell.live(metadata.regularAndStaticColumns().iterator().next(),
                                         1L,
                                         ByteBuffer.allocate(0)));
             return BTreeRow.create(clustering, primaryKeyLivenessInfo, deletion, builder.build());
@@ -404,8 +408,8 @@ public class RowIndexEntryTest extends CQLTester
         Pre_C_11206_RowIndexEntry simple = new Pre_C_11206_RowIndexEntry(123);
 
         DataOutputBuffer buffer = new DataOutputBuffer();
-        SerializationHeader header = new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS);
-        Pre_C_11206_RowIndexEntry.Serializer serializer = new Pre_C_11206_RowIndexEntry.Serializer(cfs.metadata, BigFormat.latestVersion, header);
+        SerializationHeader header = new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS);
+        Pre_C_11206_RowIndexEntry.Serializer serializer = new Pre_C_11206_RowIndexEntry.Serializer(cfs.metadata(), BigFormat.latestVersion, header);
 
         serializer.serialize(simple, buffer);
 
@@ -565,7 +569,7 @@ public class RowIndexEntryTest extends CQLTester
             private final IndexInfo.Serializer idxSerializer;
             private final Version version;
 
-            Serializer(CFMetaData metadata, Version version, SerializationHeader header)
+            Serializer(TableMetadata metadata, Version version, SerializationHeader header)
             {
                 this.idxSerializer = IndexInfo.serializer(version, header);
                 this.version = version;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/RowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowTest.java b/test/unit/org/apache/cassandra/db/RowTest.java
index 5fdb98a..5134857 100644
--- a/test/unit/org/apache/cassandra/db/RowTest.java
+++ b/test/unit/org/apache/cassandra/db/RowTest.java
@@ -29,8 +29,8 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.rows.*;
@@ -54,22 +54,23 @@ public class RowTest
     private int nowInSeconds;
     private DecoratedKey dk;
     private ColumnFamilyStore cfs;
-    private CFMetaData cfm;
+    private TableMetadata metadata;
 
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
         DatabaseDescriptor.daemonInitialization();
-        CFMetaData cfMetadata = CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD1)
-                                                  .addPartitionKey("key", BytesType.instance)
-                                                  .addClusteringColumn("col1", AsciiType.instance)
-                                                  .addRegularColumn("a", AsciiType.instance)
-                                                  .addRegularColumn("b", AsciiType.instance)
-                                                  .build();
+
+        TableMetadata.Builder metadata =
+            TableMetadata.builder(KEYSPACE1, CF_STANDARD1)
+                         .addPartitionKeyColumn("key", BytesType.instance)
+                         .addClusteringColumn("col1", AsciiType.instance)
+                         .addRegularColumn("a", AsciiType.instance)
+                         .addRegularColumn("b", AsciiType.instance);
+
         SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    cfMetadata);
+
+        SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1), metadata);
     }
 
     @Before
@@ -78,19 +79,19 @@ public class RowTest
         nowInSeconds = FBUtilities.nowInSeconds();
         dk = Util.dk("key0");
         cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
-        cfm = cfs.metadata;
+        metadata = cfs.metadata();
     }
 
     @Test
     public void testMergeRangeTombstones() throws InterruptedException
     {
-        PartitionUpdate update1 = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
+        PartitionUpdate update1 = new PartitionUpdate(metadata, dk, metadata.regularAndStaticColumns(), 1);
         writeRangeTombstone(update1, "1", "11", 123, 123);
         writeRangeTombstone(update1, "2", "22", 123, 123);
         writeRangeTombstone(update1, "3", "31", 123, 123);
         writeRangeTombstone(update1, "4", "41", 123, 123);
 
-        PartitionUpdate update2 = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
+        PartitionUpdate update2 = new PartitionUpdate(metadata, dk, metadata.regularAndStaticColumns(), 1);
         writeRangeTombstone(update2, "1", "11", 123, 123);
         writeRangeTombstone(update2, "111", "112", 1230, 123);
         writeRangeTombstone(update2, "2", "24", 123, 123);
@@ -128,17 +129,17 @@ public class RowTest
     @Test
     public void testResolve()
     {
-        ColumnDefinition defA = cfm.getColumnDefinition(new ColumnIdentifier("a", true));
-        ColumnDefinition defB = cfm.getColumnDefinition(new ColumnIdentifier("b", true));
+        ColumnMetadata defA = metadata.getColumn(new ColumnIdentifier("a", true));
+        ColumnMetadata defB = metadata.getColumn(new ColumnIdentifier("b", true));
 
         Row.Builder builder = BTreeRow.unsortedBuilder(nowInSeconds);
-        builder.newRow(cfm.comparator.make("c1"));
-        writeSimpleCellValue(builder, cfm, defA, "a1", 0);
-        writeSimpleCellValue(builder, cfm, defA, "a2", 1);
-        writeSimpleCellValue(builder, cfm, defB, "b1", 1);
+        builder.newRow(metadata.comparator.make("c1"));
+        writeSimpleCellValue(builder, defA, "a1", 0);
+        writeSimpleCellValue(builder, defA, "a2", 1);
+        writeSimpleCellValue(builder, defB, "b1", 1);
         Row row = builder.build();
 
-        PartitionUpdate update = PartitionUpdate.singleRowUpdate(cfm, dk, row);
+        PartitionUpdate update = PartitionUpdate.singleRowUpdate(metadata, dk, row);
 
         Unfiltered unfiltered = update.unfilteredIterator().next();
         assertTrue(unfiltered.kind() == Unfiltered.Kind.ROW);
@@ -152,11 +153,11 @@ public class RowTest
     public void testExpiringColumnExpiration() throws IOException
     {
         int ttl = 1;
-        ColumnDefinition def = cfm.getColumnDefinition(new ColumnIdentifier("a", true));
+        ColumnMetadata def = metadata.getColumn(new ColumnIdentifier("a", true));
 
         Cell cell = BufferCell.expiring(def, 0, ttl, nowInSeconds, ((AbstractType) def.cellValueType()).decompose("a1"));
 
-        PartitionUpdate update = PartitionUpdate.singleRowUpdate(cfm, dk, BTreeRow.singleCellRow(cfm.comparator.make("c1"), cell));
+        PartitionUpdate update = PartitionUpdate.singleRowUpdate(metadata, dk, BTreeRow.singleCellRow(metadata.comparator.make("c1"), cell));
         new Mutation(update).applyUnsafe();
 
         // when we read with a nowInSeconds before the cell has expired,
@@ -172,14 +173,14 @@ public class RowTest
     @Test
     public void testHashCode()
     {
-        ColumnDefinition defA = cfm.getColumnDefinition(new ColumnIdentifier("a", true));
-        ColumnDefinition defB = cfm.getColumnDefinition(new ColumnIdentifier("b", true));
+        ColumnMetadata defA = metadata.getColumn(new ColumnIdentifier("a", true));
+        ColumnMetadata defB = metadata.getColumn(new ColumnIdentifier("b", true));
 
         Row.Builder builder = BTreeRow.unsortedBuilder(nowInSeconds);
-        builder.newRow(cfm.comparator.make("c1"));
-        writeSimpleCellValue(builder, cfm, defA, "a1", 0);
-        writeSimpleCellValue(builder, cfm, defA, "a2", 1);
-        writeSimpleCellValue(builder, cfm, defB, "b1", 1);
+        builder.newRow(metadata.comparator.make("c1"));
+        writeSimpleCellValue(builder, defA, "a1", 0);
+        writeSimpleCellValue(builder, defA, "a2", 1);
+        writeSimpleCellValue(builder, defB, "b1", 1);
         Row row = builder.build();
 
         Map<Row, Integer> map = new HashMap<>();
@@ -189,7 +190,7 @@ public class RowTest
 
     private void assertRangeTombstoneMarkers(ClusteringBound start, ClusteringBound end, DeletionTime deletionTime, Object[] expected)
     {
-        AbstractType clusteringType = (AbstractType)cfm.comparator.subtype(0);
+        AbstractType clusteringType = (AbstractType) metadata.comparator.subtype(0);
 
         assertEquals(1, start.size());
         assertEquals(start.kind(), ClusteringPrefix.Kind.INCL_START_BOUND);
@@ -210,11 +211,10 @@ public class RowTest
     }
 
     private void writeSimpleCellValue(Row.Builder builder,
-                                      CFMetaData cfm,
-                                      ColumnDefinition columnDefinition,
+                                      ColumnMetadata columnMetadata,
                                       String value,
                                       long timestamp)
     {
-       builder.addCell(BufferCell.live(columnDefinition, timestamp, ((AbstractType) columnDefinition.cellValueType()).decompose(value)));
+       builder.addCell(BufferCell.live(columnMetadata, timestamp, ((AbstractType) columnMetadata.cellValueType()).decompose(value)));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java b/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
index 8e71d64..f4eafa3 100644
--- a/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
+++ b/test/unit/org/apache/cassandra/db/RowUpdateBuilder.java
@@ -20,16 +20,10 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.partitions.*;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.ListType;
-import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.utils.*;
 
 /**
@@ -52,22 +46,22 @@ public class RowUpdateBuilder
         this.updateBuilder = updateBuilder;
     }
 
-    public RowUpdateBuilder(CFMetaData metadata, long timestamp, Object partitionKey)
+    public RowUpdateBuilder(TableMetadata metadata, long timestamp, Object partitionKey)
     {
         this(metadata, FBUtilities.nowInSeconds(), timestamp, partitionKey);
     }
 
-    public RowUpdateBuilder(CFMetaData metadata, int localDeletionTime, long timestamp, Object partitionKey)
+    public RowUpdateBuilder(TableMetadata metadata, int localDeletionTime, long timestamp, Object partitionKey)
     {
         this(metadata, localDeletionTime, timestamp, metadata.params.defaultTimeToLive, partitionKey);
     }
 
-    public RowUpdateBuilder(CFMetaData metadata, long timestamp, int ttl, Object partitionKey)
+    public RowUpdateBuilder(TableMetadata metadata, long timestamp, int ttl, Object partitionKey)
     {
         this(metadata, FBUtilities.nowInSeconds(), timestamp, ttl, partitionKey);
     }
 
-    public RowUpdateBuilder(CFMetaData metadata, int localDeletionTime, long timestamp, int ttl, Object partitionKey)
+    public RowUpdateBuilder(TableMetadata metadata, int localDeletionTime, long timestamp, int ttl, Object partitionKey)
     {
         this(PartitionUpdate.simpleBuilder(metadata, partitionKey));
 
@@ -137,27 +131,27 @@ public class RowUpdateBuilder
         update.add(builder.build());
     }
 
-    public static Mutation deleteRow(CFMetaData metadata, long timestamp, Object key, Object... clusteringValues)
+    public static Mutation deleteRow(TableMetadata metadata, long timestamp, Object key, Object... clusteringValues)
     {
         return deleteRowAt(metadata, timestamp, FBUtilities.nowInSeconds(), key, clusteringValues);
     }
 
-    public static Mutation deleteRowAt(CFMetaData metadata, long timestamp, int localDeletionTime, Object key, Object... clusteringValues)
+    public static Mutation deleteRowAt(TableMetadata metadata, long timestamp, int localDeletionTime, Object key, Object... clusteringValues)
     {
-        PartitionUpdate update = new PartitionUpdate(metadata, makeKey(metadata, key), metadata.partitionColumns(), 0);
+        PartitionUpdate update = new PartitionUpdate(metadata, makeKey(metadata, key), metadata.regularAndStaticColumns(), 0);
         deleteRow(update, timestamp, localDeletionTime, clusteringValues);
         // note that the created mutation may get further update later on, so we don't use the ctor that create a singletonMap
         // underneath (this class if for convenience, not performance)
-        return new Mutation(update.metadata().ksName, update.partitionKey()).add(update);
+        return new Mutation(update.metadata().keyspace, update.partitionKey()).add(update);
     }
 
-    private static DecoratedKey makeKey(CFMetaData metadata, Object... partitionKey)
+    private static DecoratedKey makeKey(TableMetadata metadata, Object... partitionKey)
     {
         if (partitionKey.length == 1 && partitionKey[0] instanceof DecoratedKey)
             return (DecoratedKey)partitionKey[0];
 
-        ByteBuffer key = CFMetaData.serializePartitionKey(metadata.getKeyValidatorAsClusteringComparator().make(partitionKey));
-        return metadata.decorateKey(key);
+        ByteBuffer key = metadata.partitionKeyAsClusteringComparator().make(partitionKey).serializeAsPartitionKey();
+        return metadata.partitioner.decorateKey(key);
     }
 
     public RowUpdateBuilder addRangeTombstone(RangeTombstone rt)
@@ -178,9 +172,9 @@ public class RowUpdateBuilder
         return this;
     }
 
-    public RowUpdateBuilder add(ColumnDefinition columnDefinition, Object value)
+    public RowUpdateBuilder add(ColumnMetadata columnMetadata, Object value)
     {
-        return add(columnDefinition.name.toString(), value);
+        return add(columnMetadata.name.toString(), value);
     }
 
     public RowUpdateBuilder delete(String columnName)
@@ -189,8 +183,8 @@ public class RowUpdateBuilder
         return this;
     }
 
-    public RowUpdateBuilder delete(ColumnDefinition columnDefinition)
+    public RowUpdateBuilder delete(ColumnMetadata columnMetadata)
     {
-        return delete(columnDefinition.name.toString());
+        return delete(columnMetadata.name.toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 93ac46e..d9f6433 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -33,7 +33,6 @@ import org.junit.runner.RunWith;
 
 import org.apache.cassandra.*;
 import org.apache.cassandra.cache.ChunkCache;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.cql3.QueryProcessor;
@@ -59,8 +58,14 @@ import org.apache.cassandra.io.sstable.format.big.BigTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static org.apache.cassandra.SchemaLoader.counterCFMD;
+import static org.apache.cassandra.SchemaLoader.createKeyspace;
+import static org.apache.cassandra.SchemaLoader.getCompressionParameters;
+import static org.apache.cassandra.SchemaLoader.loadSchema;
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -91,19 +96,18 @@ public class ScrubTest
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
-        SchemaLoader.loadSchema();
-        SchemaLoader.createKeyspace(KEYSPACE,
-                                    KeyspaceParams.simple(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CF),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CF2),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CF3),
-                                    SchemaLoader.counterCFMD(KEYSPACE, COUNTER_CF)
-                                                .compression(SchemaLoader.getCompressionParameters(COMPRESSION_CHUNK_LENGTH)),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CF_UUID, 0, UUIDType.instance),
-                                    SchemaLoader.keysIndexCFMD(KEYSPACE, CF_INDEX1, true),
-                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEX2, true),
-                                    SchemaLoader.keysIndexCFMD(KEYSPACE, CF_INDEX1_BYTEORDERED, true).copy(ByteOrderedPartitioner.instance),
-                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEX2_BYTEORDERED, true).copy(ByteOrderedPartitioner.instance));
+        loadSchema();
+        createKeyspace(KEYSPACE,
+                       KeyspaceParams.simple(1),
+                       standardCFMD(KEYSPACE, CF),
+                       standardCFMD(KEYSPACE, CF2),
+                       standardCFMD(KEYSPACE, CF3),
+                       counterCFMD(KEYSPACE, COUNTER_CF).compression(getCompressionParameters(COMPRESSION_CHUNK_LENGTH)),
+                       standardCFMD(KEYSPACE, CF_UUID, 0, UUIDType.instance),
+                       SchemaLoader.keysIndexCFMD(KEYSPACE, CF_INDEX1, true),
+                       SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEX2, true),
+                       SchemaLoader.keysIndexCFMD(KEYSPACE, CF_INDEX1_BYTEORDERED, true).partitioner(ByteOrderedPartitioner.instance),
+                       SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEX2_BYTEORDERED, true).partitioner(ByteOrderedPartitioner.instance));
     }
 
     @Test
@@ -335,7 +339,7 @@ public class ScrubTest
 
                 for (String k : keys)
                 {
-                    PartitionUpdate update = UpdateBuilder.create(cfs.metadata, Util.dk(k))
+                    PartitionUpdate update = UpdateBuilder.create(cfs.metadata(), Util.dk(k))
                                                           .newRow("someName").add("val", "someValue")
                                                           .build();
 
@@ -447,7 +451,7 @@ public class ScrubTest
     {
         for (int i = 0; i < partitionsPerSSTable; i++)
         {
-            PartitionUpdate update = UpdateBuilder.create(cfs.metadata, String.valueOf(i))
+            PartitionUpdate update = UpdateBuilder.create(cfs.metadata(), String.valueOf(i))
                                                   .newRow("r1").add("val", "1")
                                                   .newRow("r1").add("val", "1")
                                                   .build();
@@ -463,7 +467,7 @@ public class ScrubTest
         assertTrue(values.length % 2 == 0);
         for (int i = 0; i < values.length; i +=2)
         {
-            UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, String.valueOf(i));
+            UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), String.valueOf(i));
             if (composite)
             {
                 builder.newRow("c" + i)
@@ -486,7 +490,7 @@ public class ScrubTest
     {
         for (int i = 0; i < partitionsPerSSTable; i++)
         {
-            PartitionUpdate update = UpdateBuilder.create(cfs.metadata, String.valueOf(i))
+            PartitionUpdate update = UpdateBuilder.create(cfs.metadata(), String.valueOf(i))
                                                   .newRow("r1").add("val", 100L)
                                                   .build();
             new CounterMutation(new Mutation(update), ConsistencyLevel.ONE).apply();
@@ -510,7 +514,7 @@ public class ScrubTest
         QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_scrub_validation (a text primary key, b int)", ConsistencyLevel.ONE);
         ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("test_scrub_validation");
 
-        new Mutation(UpdateBuilder.create(cfs2.metadata, "key").newRow().add("b", LongType.instance.decompose(1L)).build()).apply();
+        new Mutation(UpdateBuilder.create(cfs2.metadata(), "key").newRow().add("b", LongType.instance.decompose(1L)).build()).apply();
         cfs2.forceBlockingFlush();
 
         CompactionManager.instance.performScrub(cfs2, false, false, 2);
@@ -634,10 +638,10 @@ public class ScrubTest
         assertOrdered(Util.cmd(cfs).filterOn(colName, Operator.EQ, 1L).build(), numRows / 2);
     }
 
-    private static SSTableMultiWriter createTestWriter(Descriptor descriptor, long keyCount, CFMetaData metadata, LifecycleTransaction txn)
+    private static SSTableMultiWriter createTestWriter(Descriptor descriptor, long keyCount, TableMetadataRef metadata, LifecycleTransaction txn)
     {
-        SerializationHeader header = new SerializationHeader(true, metadata, metadata.partitionColumns(), EncodingStats.NO_STATS);
-        MetadataCollector collector = new MetadataCollector(metadata.comparator).sstableLevel(0);
+        SerializationHeader header = new SerializationHeader(true, metadata.get(), metadata.get().regularAndStaticColumns(), EncodingStats.NO_STATS);
+        MetadataCollector collector = new MetadataCollector(metadata.get().comparator).sstableLevel(0);
         return new TestMultiWriter(new TestWriter(descriptor, keyCount, 0, metadata, collector, header, txn), txn);
     }
 
@@ -654,7 +658,7 @@ public class ScrubTest
      */
     private static class TestWriter extends BigTableWriter
     {
-        TestWriter(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata,
+        TestWriter(Descriptor descriptor, long keyCount, long repairedAt, TableMetadataRef metadata,
                    MetadataCollector collector, SerializationHeader header, LifecycleTransaction txn)
         {
             super(descriptor, keyCount, repairedAt, metadata, collector, header, Collections.emptySet(), txn);


[20/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
index 874c679..26746ad 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -24,9 +24,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Sets;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.compaction.OperationType;
@@ -43,6 +41,9 @@ import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.concurrent.Transactional;
 
 /**
@@ -75,24 +76,24 @@ public abstract class SSTableWriter extends SSTable implements Transactional
     protected SSTableWriter(Descriptor descriptor,
                             long keyCount,
                             long repairedAt,
-                            CFMetaData metadata,
+                            TableMetadataRef metadata,
                             MetadataCollector metadataCollector,
                             SerializationHeader header,
                             Collection<SSTableFlushObserver> observers)
     {
-        super(descriptor, components(metadata), metadata, DatabaseDescriptor.getDiskOptimizationStrategy());
+        super(descriptor, components(metadata.get()), metadata, DatabaseDescriptor.getDiskOptimizationStrategy());
         this.keyCount = keyCount;
         this.repairedAt = repairedAt;
         this.metadataCollector = metadataCollector;
-        this.header = header != null ? header : SerializationHeader.makeWithoutStats(metadata); //null header indicates streaming from pre-3.0 sstable
-        this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata, descriptor.version, header);
+        this.header = header;
+        this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata.get(), descriptor.version, header);
         this.observers = observers == null ? Collections.emptySet() : observers;
     }
 
     public static SSTableWriter create(Descriptor descriptor,
                                        Long keyCount,
                                        Long repairedAt,
-                                       CFMetaData metadata,
+                                       TableMetadataRef metadata,
                                        MetadataCollector metadataCollector,
                                        SerializationHeader header,
                                        Collection<Index> indexes,
@@ -110,11 +111,11 @@ public abstract class SSTableWriter extends SSTable implements Transactional
                                        Collection<Index> indexes,
                                        LifecycleTransaction txn)
     {
-        CFMetaData metadata = Schema.instance.getCFMetaData(descriptor);
+        TableMetadataRef metadata = Schema.instance.getTableMetadataRef(descriptor);
         return create(metadata, descriptor, keyCount, repairedAt, sstableLevel, header, indexes, txn);
     }
 
-    public static SSTableWriter create(CFMetaData metadata,
+    public static SSTableWriter create(TableMetadataRef metadata,
                                        Descriptor descriptor,
                                        long keyCount,
                                        long repairedAt,
@@ -123,7 +124,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
                                        Collection<Index> indexes,
                                        LifecycleTransaction txn)
     {
-        MetadataCollector collector = new MetadataCollector(metadata.comparator).sstableLevel(sstableLevel);
+        MetadataCollector collector = new MetadataCollector(metadata.get().comparator).sstableLevel(sstableLevel);
         return create(descriptor, keyCount, repairedAt, metadata, collector, header, indexes, txn);
     }
 
@@ -138,7 +139,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
         return create(descriptor, keyCount, repairedAt, 0, header, indexes, txn);
     }
 
-    private static Set<Component> components(CFMetaData metadata)
+    private static Set<Component> components(TableMetadata metadata)
     {
         Set<Component> components = new HashSet<Component>(Arrays.asList(Component.DATA,
                 Component.PRIMARY_INDEX,
@@ -299,7 +300,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
     protected Map<MetadataType, MetadataComponent> finalizeMetadata()
     {
         return metadataCollector.finalizeMetadata(getPartitioner().getClass().getCanonicalName(),
-                                                  metadata.params.bloomFilterFpChance,
+                                                  metadata().params.bloomFilterFpChance,
                                                   repairedAt,
                                                   header);
     }
@@ -329,7 +330,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
         public abstract SSTableWriter open(Descriptor descriptor,
                                            long keyCount,
                                            long repairedAt,
-                                           CFMetaData metadata,
+                                           TableMetadataRef metadata,
                                            MetadataCollector metadataCollector,
                                            SerializationHeader header,
                                            Collection<SSTableFlushObserver> observers,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
index 02b685b..d949197 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
@@ -20,7 +20,8 @@ package org.apache.cassandra.io.sstable.format.big;
 import java.util.Collection;
 import java.util.Set;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
@@ -71,7 +72,7 @@ public class BigFormat implements SSTableFormat
     }
 
     @Override
-    public RowIndexEntry.IndexSerializer getIndexSerializer(CFMetaData metadata, Version version, SerializationHeader header)
+    public RowIndexEntry.IndexSerializer getIndexSerializer(TableMetadata metadata, Version version, SerializationHeader header)
     {
         return new RowIndexEntry.Serializer(version, header);
     }
@@ -82,7 +83,7 @@ public class BigFormat implements SSTableFormat
         public SSTableWriter open(Descriptor descriptor,
                                   long keyCount,
                                   long repairedAt,
-                                  CFMetaData metadata,
+                                  TableMetadataRef metadata,
                                   MetadataCollector metadataCollector,
                                   SerializationHeader header,
                                   Collection<SSTableFlushObserver> observers,
@@ -95,7 +96,7 @@ public class BigFormat implements SSTableFormat
     static class ReaderFactory extends SSTableReader.Factory
     {
         @Override
-        public SSTableReader open(Descriptor descriptor, Set<Component> components, CFMetaData metadata, Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason, SerializationHeader header)
+        public SSTableReader open(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata, Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason, SerializationHeader header)
         {
             return new BigTableReader(descriptor, components, metadata, maxDataAge, sstableMetadata, openReason, header);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
index 1fded2f..c29bc5d 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
@@ -17,12 +17,18 @@
  */
 package org.apache.cassandra.io.sstable.format.big;
 
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.cache.KeyCacheKey;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.columniterator.SSTableIterator;
 import org.apache.cassandra.db.columniterator.SSTableReversedIterator;
+import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.Rows;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.db.rows.UnfilteredRowIterators;
@@ -33,14 +39,9 @@ import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
 
 /**
  * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
@@ -50,7 +51,7 @@ public class BigTableReader extends SSTableReader
 {
     private static final Logger logger = LoggerFactory.getLogger(BigTableReader.class);
 
-    BigTableReader(Descriptor desc, Set<Component> components, CFMetaData metadata, Long maxDataAge, StatsMetadata sstableMetadata, OpenReason openReason, SerializationHeader header)
+    BigTableReader(Descriptor desc, Set<Component> components, TableMetadataRef metadata, Long maxDataAge, StatsMetadata sstableMetadata, OpenReason openReason, SerializationHeader header)
     {
         super(desc, components, metadata, maxDataAge, sstableMetadata, openReason, header);
     }
@@ -64,7 +65,7 @@ public class BigTableReader extends SSTableReader
     public UnfilteredRowIterator iterator(FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry, Slices slices, ColumnFilter selectedColumns, boolean reversed)
     {
         if (indexEntry == null)
-            return UnfilteredRowIterators.noRowsIterator(metadata, key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, reversed);
+            return UnfilteredRowIterators.noRowsIterator(metadata(), key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, reversed);
         return reversed
              ? new SSTableReversedIterator(this, file, key, indexEntry, slices, selectedColumns, ifile)
              : new SSTableIterator(this, file, key, indexEntry, slices, selectedColumns, ifile);
@@ -145,7 +146,9 @@ public class BigTableReader extends SSTableReader
         // next, the key cache (only make sense for valid row key)
         if ((op == Operator.EQ || op == Operator.GE) && (key instanceof DecoratedKey))
         {
-            RowIndexEntry cachedPosition = getCachedPosition((DecoratedKey)key, updateCacheAndStats);
+            DecoratedKey decoratedKey = (DecoratedKey)key;
+            KeyCacheKey cacheKey = new KeyCacheKey(metadata(), descriptor, decoratedKey.getKey());
+            RowIndexEntry cachedPosition = getCachedPosition(cacheKey, updateCacheAndStats);
             if (cachedPosition != null)
             {
                 Tracing.trace("Key cache hit for sstable {}", descriptor.generation);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java
index 1b33f5b..716ef4c 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java
@@ -21,10 +21,10 @@ import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.AbstractIterator;
 import com.google.common.collect.Iterators;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.filter.*;
@@ -68,7 +68,7 @@ public class BigTableScanner implements ISSTableScanner
     // Full scan of the sstables
     public static ISSTableScanner getScanner(SSTableReader sstable)
     {
-        return new BigTableScanner(sstable, ColumnFilter.all(sstable.metadata), null, Iterators.singletonIterator(fullRange(sstable)));
+        return new BigTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, Iterators.singletonIterator(fullRange(sstable)));
     }
 
     public static ISSTableScanner getScanner(SSTableReader sstable, ColumnFilter columns, DataRange dataRange)
@@ -83,12 +83,12 @@ public class BigTableScanner implements ISSTableScanner
         if (positions.isEmpty())
             return new EmptySSTableScanner(sstable);
 
-        return new BigTableScanner(sstable, ColumnFilter.all(sstable.metadata), null, makeBounds(sstable, tokenRanges).iterator());
+        return new BigTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, makeBounds(sstable, tokenRanges).iterator());
     }
 
     public static ISSTableScanner getScanner(SSTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
     {
-        return new BigTableScanner(sstable, ColumnFilter.all(sstable.metadata), null, rangeIterator);
+        return new BigTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator);
     }
 
     private BigTableScanner(SSTableReader sstable, ColumnFilter columns, DataRange dataRange, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
@@ -100,7 +100,7 @@ public class BigTableScanner implements ISSTableScanner
         this.sstable = sstable;
         this.columns = columns;
         this.dataRange = dataRange;
-        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata,
+        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata(),
                                                                                                         sstable.descriptor.version,
                                                                                                         sstable.header);
         this.rangeIterator = rangeIterator;
@@ -237,9 +237,10 @@ public class BigTableScanner implements ISSTableScanner
         return sstable.toString();
     }
 
-    public CFMetaData metadata()
+
+    public TableMetadata metadata()
     {
-        return sstable.metadata;
+        return sstable.metadata();
     }
 
     public boolean hasNext()
@@ -414,9 +415,9 @@ public class BigTableScanner implements ISSTableScanner
             return sstable.getFilename();
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
-            return sstable.metadata;
+            return sstable.metadata();
         }
 
         public boolean hasNext()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index 018edac..e134f2d 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -29,7 +29,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cache.ChunkCache;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
@@ -47,6 +46,7 @@ import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.Transactional;
 
@@ -71,7 +71,7 @@ public class BigTableWriter extends SSTableWriter
     public BigTableWriter(Descriptor descriptor,
                           long keyCount,
                           long repairedAt,
-                          CFMetaData metadata,
+                          TableMetadataRef metadata,
                           MetadataCollector metadataCollector, 
                           SerializationHeader header,
                           Collection<SSTableFlushObserver> observers,
@@ -86,7 +86,7 @@ public class BigTableWriter extends SSTableWriter
                                              descriptor.filenameFor(Component.COMPRESSION_INFO),
                                              new File(descriptor.filenameFor(Component.DIGEST)),
                                              writerOption,
-                                             metadata.params.compression,
+                                             metadata().params.compression,
                                              metadataCollector);
         }
         else
@@ -207,8 +207,8 @@ public class BigTableWriter extends SSTableWriter
     {
         if (rowSize > DatabaseDescriptor.getCompactionLargePartitionWarningThreshold())
         {
-            String keyString = metadata.getKeyValidator().getString(key.getKey());
-            logger.warn("Writing large partition {}/{}:{} ({}) to sstable {}", metadata.ksName, metadata.cfName, keyString, FBUtilities.prettyPrintMemory(rowSize), getFilename());
+            String keyString = metadata().partitionKeyType.getString(key.getKey());
+            logger.warn("Writing large partition {}/{}:{} ({}) to sstable {}", metadata.keyspace, metadata.name, keyString, FBUtilities.prettyPrintMemory(rowSize), getFilename());
         }
     }
 
@@ -280,7 +280,7 @@ public class BigTableWriter extends SSTableWriter
         StatsMetadata stats = statsMetadata();
         assert boundary.indexLength > 0 && boundary.dataLength > 0;
         // open the reader early
-        IndexSummary indexSummary = iwriter.summary.build(metadata.partitioner, boundary);
+        IndexSummary indexSummary = iwriter.summary.build(metadata().partitioner, boundary);
         long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length();
         int indexBufferSize = optimizationStrategy.bufferSize(indexFileLength / indexSummary.size());
         FileHandle ifile = iwriter.builder.bufferSize(indexBufferSize).complete(boundary.indexLength);
@@ -326,7 +326,7 @@ public class BigTableWriter extends SSTableWriter
 
         StatsMetadata stats = statsMetadata();
         // finalize in-memory state for the reader
-        IndexSummary indexSummary = iwriter.summary.build(this.metadata.partitioner);
+        IndexSummary indexSummary = iwriter.summary.build(metadata().partitioner);
         long indexFileLength = new File(descriptor.filenameFor(Component.PRIMARY_INDEX)).length();
         int dataBufferSize = optimizationStrategy.bufferSize(stats.estimatedPartitionSize.percentile(DatabaseDescriptor.getDiskOptimizationEstimatePercentile()));
         int indexBufferSize = optimizationStrategy.bufferSize(indexFileLength / indexSummary.size());
@@ -337,7 +337,7 @@ public class BigTableWriter extends SSTableWriter
         invalidateCacheAtBoundary(dfile);
         SSTableReader sstable = SSTableReader.internalOpen(descriptor,
                                                            components,
-                                                           this.metadata,
+                                                           metadata,
                                                            ifile,
                                                            dfile,
                                                            indexSummary,
@@ -441,8 +441,8 @@ public class BigTableWriter extends SSTableWriter
             indexFile = new SequentialWriter(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)), writerOption);
             builder = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX)).mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap);
             chunkCache.ifPresent(builder::withChunkCache);
-            summary = new IndexSummaryBuilder(keyCount, metadata.params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL);
-            bf = FilterFactory.getFilter(keyCount, metadata.params.bloomFilterFpChance, true);
+            summary = new IndexSummaryBuilder(keyCount, metadata().params.minIndexInterval, Downsampling.BASE_SAMPLING_LEVEL);
+            bf = FilterFactory.getFilter(keyCount, metadata().params.bloomFilterFpChance, true);
             // register listeners to be alerted when the data files are flushed
             indexFile.setPostFlushListener(() -> summary.markIndexSynced(indexFile.getLastFlushOffset()));
             dataFile.setPostFlushListener(() -> summary.markDataSynced(dataFile.getLastFlushOffset()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
index 96ca38c..d54090b 100644
--- a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
@@ -24,12 +24,12 @@ import com.codahale.metrics.Counter;
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Meter;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 
 import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
@@ -110,25 +110,25 @@ public class CompactionMetrics implements CompactionManager.CompactionExecutorSt
                 // currently running compactions
                 for (CompactionInfo.Holder compaction : compactions)
                 {
-                    CFMetaData metaData = compaction.getCompactionInfo().getCFMetaData();
+                    TableMetadata metaData = compaction.getCompactionInfo().getTableMetadata();
                     if (metaData == null)
                     {
                         continue;
                     }
-                    if (!resultMap.containsKey(metaData.ksName))
+                    if (!resultMap.containsKey(metaData.keyspace))
                     {
-                        resultMap.put(metaData.ksName, new HashMap<>());
+                        resultMap.put(metaData.keyspace, new HashMap<>());
                     }
 
-                    Map<String, Integer> tableNameToCountMap = resultMap.get(metaData.ksName);
-                    if (tableNameToCountMap.containsKey(metaData.cfName))
+                    Map<String, Integer> tableNameToCountMap = resultMap.get(metaData.keyspace);
+                    if (tableNameToCountMap.containsKey(metaData.name))
                     {
-                        tableNameToCountMap.put(metaData.cfName,
-                                                tableNameToCountMap.get(metaData.cfName) + 1);
+                        tableNameToCountMap.put(metaData.name,
+                                                tableNameToCountMap.get(metaData.name) + 1);
                     }
                     else
                     {
-                        tableNameToCountMap.put(metaData.cfName, 1);
+                        tableNameToCountMap.put(metaData.name, 1);
                     }
                 }
                 return resultMap;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/metrics/TableMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/TableMetrics.java b/src/java/org/apache/cassandra/metrics/TableMetrics.java
index 0a726d4..126abed 100644
--- a/src/java/org/apache/cassandra/metrics/TableMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/TableMetrics.java
@@ -27,8 +27,8 @@ import com.google.common.collect.Maps;
 
 import com.codahale.metrics.*;
 import com.codahale.metrics.Timer;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Memtable;
@@ -668,7 +668,7 @@ public class TableMetrics
 
         // We do not want to capture view mutation specific metrics for a view
         // They only makes sense to capture on the base table
-        if (cfs.metadata.isView())
+        if (cfs.metadata().isView())
         {
             viewLockAcquireTime = null;
             viewReadTime = null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
index 53e53a4..67e54c8 100644
--- a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
@@ -36,9 +36,8 @@ import net.jpountz.lz4.LZ4Factory;
 import net.jpountz.xxhash.XXHashFactory;
 
 import org.apache.cassandra.config.Config;
-import org.xerial.snappy.SnappyInputStream;
+import org.apache.cassandra.exceptions.UnknownTableException;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.UnknownColumnFamilyException;
 import org.apache.cassandra.db.monitoring.ApproximateTime;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
@@ -98,9 +97,9 @@ public class IncomingTcpConnection extends FastThreadLocalThread implements Clos
             logger.trace("eof reading from socket; closing", e);
             // connection will be reset so no need to throw an exception.
         }
-        catch (UnknownColumnFamilyException e)
+        catch (UnknownTableException e)
         {
-            logger.warn("UnknownColumnFamilyException reading from socket; closing", e);
+            logger.warn("UnknownTableException reading from socket; closing", e);
         }
         catch (IOException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 7215397..ef521ef 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -74,6 +74,8 @@ import org.apache.cassandra.metrics.ConnectionMetrics;
 import org.apache.cassandra.metrics.DroppedMessageMetrics;
 import org.apache.cassandra.metrics.MessagingMetrics;
 import org.apache.cassandra.repair.messages.RepairMessage;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.service.*;
 import org.apache.cassandra.service.paxos.Commit;
@@ -1119,9 +1121,9 @@ public final class MessagingService implements MessagingServiceMBean
     {
         assert mutation != null : "Mutation should not be null when updating dropped mutations count";
 
-        for (UUID columnFamilyId : mutation.getColumnFamilyIds())
+        for (TableId tableId : mutation.getTableIds())
         {
-            ColumnFamilyStore cfs = Keyspace.open(mutation.getKeyspaceName()).getColumnFamilyStore(columnFamilyId);
+            ColumnFamilyStore cfs = Keyspace.open(mutation.getKeyspaceName()).getColumnFamilyStore(tableId);
             if (cfs != null)
             {
                 cfs.metric.droppedMutations.inc();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
index 52625bf..d7736f0 100644
--- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
+++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
@@ -35,6 +35,7 @@ import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.*;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.ActiveRepairService;
 
 /**
@@ -56,14 +57,14 @@ public class RepairMessageVerbHandler implements IVerbHandler<RepairMessage>
                 case PREPARE_MESSAGE:
                     PrepareMessage prepareMessage = (PrepareMessage) message.payload;
                     logger.debug("Preparing, {}", prepareMessage);
-                    List<ColumnFamilyStore> columnFamilyStores = new ArrayList<>(prepareMessage.cfIds.size());
-                    for (UUID cfId : prepareMessage.cfIds)
+                    List<ColumnFamilyStore> columnFamilyStores = new ArrayList<>(prepareMessage.tableIds.size());
+                    for (TableId tableId : prepareMessage.tableIds)
                     {
-                        ColumnFamilyStore columnFamilyStore = ColumnFamilyStore.getIfExists(cfId);
+                        ColumnFamilyStore columnFamilyStore = ColumnFamilyStore.getIfExists(tableId);
                         if (columnFamilyStore == null)
                         {
                             logErrorAndSendFailureResponse(String.format("Table with id %s was dropped during prepare phase of repair",
-                                                                         cfId.toString()), message.from, id);
+                                                                         tableId), message.from, id);
                             return;
                         }
                         columnFamilyStores.add(columnFamilyStore);
@@ -91,7 +92,7 @@ public class RepairMessageVerbHandler implements IVerbHandler<RepairMessage>
                     ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId);
                     if (prs.isGlobal)
                     {
-                        prs.maybeSnapshot(cfs.metadata.cfId, desc.parentSessionId);
+                        prs.maybeSnapshot(cfs.metadata.id, desc.parentSessionId);
                     }
                     else
                     {
@@ -100,7 +101,7 @@ public class RepairMessageVerbHandler implements IVerbHandler<RepairMessage>
                             public boolean apply(SSTableReader sstable)
                             {
                                 return sstable != null &&
-                                       !sstable.metadata.isIndex() && // exclude SSTables from 2i
+                                       !sstable.metadata().isIndex() && // exclude SSTables from 2i
                                        new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(desc.ranges);
                             }
                         }, true, false); //ephemeral snapshot, if repair fails, it will be cleaned next startup

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/repair/RepairRunnable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
index c9eed54..6f7297b 100644
--- a/src/java/org/apache/cassandra/repair/RepairRunnable.java
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -35,7 +35,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.JMXConfigurableThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
index 638cf38..fc7aab4 100644
--- a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
+++ b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
@@ -24,7 +24,6 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.concurrent.TimeUnit;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -38,11 +37,10 @@ import com.google.common.collect.Sets;
 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.SchemaConstants;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
@@ -50,9 +48,14 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Tables;
 import org.apache.cassandra.utils.FBUtilities;
 
+import static java.lang.String.format;
+
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
 public final class SystemDistributedKeyspace
@@ -69,56 +72,58 @@ public final class SystemDistributedKeyspace
 
     public static final String VIEW_BUILD_STATUS = "view_build_status";
 
-    private static final CFMetaData RepairHistory =
-        compile(REPAIR_HISTORY,
-                "Repair history",
-                "CREATE TABLE %s ("
-                     + "keyspace_name text,"
-                     + "columnfamily_name text,"
-                     + "id timeuuid,"
-                     + "parent_id timeuuid,"
-                     + "range_begin text,"
-                     + "range_end text,"
-                     + "coordinator inet,"
-                     + "participants set<inet>,"
-                     + "exception_message text,"
-                     + "exception_stacktrace text,"
-                     + "status text,"
-                     + "started_at timestamp,"
-                     + "finished_at timestamp,"
-                     + "PRIMARY KEY ((keyspace_name, columnfamily_name), id))");
-
-    private static final CFMetaData ParentRepairHistory =
-        compile(PARENT_REPAIR_HISTORY,
-                "Repair history",
-                "CREATE TABLE %s ("
-                     + "parent_id timeuuid,"
-                     + "keyspace_name text,"
-                     + "columnfamily_names set<text>,"
-                     + "started_at timestamp,"
-                     + "finished_at timestamp,"
-                     + "exception_message text,"
-                     + "exception_stacktrace text,"
-                     + "requested_ranges set<text>,"
-                     + "successful_ranges set<text>,"
-                     + "options map<text, text>,"
-                     + "PRIMARY KEY (parent_id))");
-
-    private static final CFMetaData ViewBuildStatus =
-    compile(VIEW_BUILD_STATUS,
-            "Materialized View build status",
-            "CREATE TABLE %s ("
-                     + "keyspace_name text,"
-                     + "view_name text,"
-                     + "host_id uuid,"
-                     + "status text,"
-                     + "PRIMARY KEY ((keyspace_name, view_name), host_id))");
-
-    private static CFMetaData compile(String name, String description, String schema)
+    private static final TableMetadata RepairHistory =
+        parse(REPAIR_HISTORY,
+              "Repair history",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "columnfamily_name text,"
+              + "id timeuuid,"
+              + "parent_id timeuuid,"
+              + "range_begin text,"
+              + "range_end text,"
+              + "coordinator inet,"
+              + "participants set<inet>,"
+              + "exception_message text,"
+              + "exception_stacktrace text,"
+              + "status text,"
+              + "started_at timestamp,"
+              + "finished_at timestamp,"
+              + "PRIMARY KEY ((keyspace_name, columnfamily_name), id))");
+
+    private static final TableMetadata ParentRepairHistory =
+        parse(PARENT_REPAIR_HISTORY,
+              "Repair history",
+              "CREATE TABLE %s ("
+              + "parent_id timeuuid,"
+              + "keyspace_name text,"
+              + "columnfamily_names set<text>,"
+              + "started_at timestamp,"
+              + "finished_at timestamp,"
+              + "exception_message text,"
+              + "exception_stacktrace text,"
+              + "requested_ranges set<text>,"
+              + "successful_ranges set<text>,"
+              + "options map<text, text>,"
+              + "PRIMARY KEY (parent_id))");
+
+    private static final TableMetadata ViewBuildStatus =
+        parse(VIEW_BUILD_STATUS,
+              "Materialized View build status",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "view_name text,"
+              + "host_id uuid,"
+              + "status text,"
+              + "PRIMARY KEY ((keyspace_name, view_name), host_id))");
+
+    private static TableMetadata parse(String table, String description, String cql)
     {
-        return CFMetaData.compile(String.format(schema, name), SchemaConstants.DISTRIBUTED_KEYSPACE_NAME)
-                         .comment(description)
-                         .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(10));
+        return CreateTableStatement.parse(format(cql, table), SchemaConstants.DISTRIBUTED_KEYSPACE_NAME)
+                                   .id(TableId.forSystemTable(SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, table))
+                                   .dcLocalReadRepairChance(0.0)
+                                   .comment(description)
+                                   .build();
     }
 
     public static KeyspaceMetadata metadata()
@@ -131,7 +136,7 @@ public final class SystemDistributedKeyspace
         Collection<Range<Token>> ranges = options.getRanges();
         String query = "INSERT INTO %s.%s (parent_id, keyspace_name, columnfamily_names, requested_ranges, started_at,          options)"+
                                  " VALUES (%s,        '%s',          { '%s' },           { '%s' },          toTimestamp(now()), { %s })";
-        String fmtQry = String.format(query,
+        String fmtQry = format(query,
                                       SchemaConstants.DISTRIBUTED_KEYSPACE_NAME,
                                       PARENT_REPAIR_HISTORY,
                                       parent_id.toString(),
@@ -154,7 +159,7 @@ public final class SystemDistributedKeyspace
                 if (!first)
                     map.append(',');
                 first = false;
-                map.append(String.format("'%s': '%s'", entry.getKey(), entry.getValue()));
+                map.append(format("'%s': '%s'", entry.getKey(), entry.getValue()));
             }
         }
         return map.toString();
@@ -167,14 +172,14 @@ public final class SystemDistributedKeyspace
         StringWriter sw = new StringWriter();
         PrintWriter pw = new PrintWriter(sw);
         t.printStackTrace(pw);
-        String fmtQuery = String.format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, PARENT_REPAIR_HISTORY, parent_id.toString());
+        String fmtQuery = format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, PARENT_REPAIR_HISTORY, parent_id.toString());
         processSilent(fmtQuery, t.getMessage(), sw.toString());
     }
 
     public static void successfulParentRepair(UUID parent_id, Collection<Range<Token>> successfulRanges)
     {
         String query = "UPDATE %s.%s SET finished_at = toTimestamp(now()), successful_ranges = {'%s'} WHERE parent_id=%s";
-        String fmtQuery = String.format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, PARENT_REPAIR_HISTORY, Joiner.on("','").join(successfulRanges), parent_id.toString());
+        String fmtQuery = format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, PARENT_REPAIR_HISTORY, Joiner.on("','").join(successfulRanges), parent_id.toString());
         processSilent(fmtQuery);
     }
 
@@ -194,7 +199,7 @@ public final class SystemDistributedKeyspace
         {
             for (Range<Token> range : ranges)
             {
-                String fmtQry = String.format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, REPAIR_HISTORY,
+                String fmtQry = format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, REPAIR_HISTORY,
                                               keyspaceName,
                                               cfname,
                                               id.toString(),
@@ -218,7 +223,7 @@ public final class SystemDistributedKeyspace
     public static void successfulRepairJob(UUID id, String keyspaceName, String cfname)
     {
         String query = "UPDATE %s.%s SET status = '%s', finished_at = toTimestamp(now()) WHERE keyspace_name = '%s' AND columnfamily_name = '%s' AND id = %s";
-        String fmtQuery = String.format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, REPAIR_HISTORY,
+        String fmtQuery = format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, REPAIR_HISTORY,
                                         RepairState.SUCCESS.toString(),
                                         keyspaceName,
                                         cfname,
@@ -232,7 +237,7 @@ public final class SystemDistributedKeyspace
         StringWriter sw = new StringWriter();
         PrintWriter pw = new PrintWriter(sw);
         t.printStackTrace(pw);
-        String fmtQry = String.format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, REPAIR_HISTORY,
+        String fmtQry = format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, REPAIR_HISTORY,
                                       RepairState.FAILED.toString(),
                                       keyspaceName,
                                       cfname,
@@ -243,7 +248,7 @@ public final class SystemDistributedKeyspace
     public static void startViewBuild(String keyspace, String view, UUID hostId)
     {
         String query = "INSERT INTO %s.%s (keyspace_name, view_name, host_id, status) VALUES (?, ?, ?, ?)";
-        QueryProcessor.process(String.format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, VIEW_BUILD_STATUS),
+        QueryProcessor.process(format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, VIEW_BUILD_STATUS),
                                ConsistencyLevel.ONE,
                                Lists.newArrayList(bytes(keyspace),
                                                   bytes(view),
@@ -254,7 +259,7 @@ public final class SystemDistributedKeyspace
     public static void successfulViewBuild(String keyspace, String view, UUID hostId)
     {
         String query = "UPDATE %s.%s SET status = ? WHERE keyspace_name = ? AND view_name = ? AND host_id = ?";
-        QueryProcessor.process(String.format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, VIEW_BUILD_STATUS),
+        QueryProcessor.process(format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, VIEW_BUILD_STATUS),
                                ConsistencyLevel.ONE,
                                Lists.newArrayList(bytes(BuildStatus.SUCCESS.toString()),
                                                   bytes(keyspace),
@@ -268,7 +273,7 @@ public final class SystemDistributedKeyspace
         UntypedResultSet results;
         try
         {
-            results = QueryProcessor.execute(String.format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, VIEW_BUILD_STATUS),
+            results = QueryProcessor.execute(format(query, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, VIEW_BUILD_STATUS),
                                              ConsistencyLevel.ONE,
                                              keyspace,
                                              view);
@@ -290,7 +295,7 @@ public final class SystemDistributedKeyspace
     public static void setViewRemoved(String keyspaceName, String viewName)
     {
         String buildReq = "DELETE FROM %s.%s WHERE keyspace_name = ? AND view_name = ?";
-        QueryProcessor.executeInternal(String.format(buildReq, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, VIEW_BUILD_STATUS), keyspaceName, viewName);
+        QueryProcessor.executeInternal(format(buildReq, SchemaConstants.DISTRIBUTED_KEYSPACE_NAME, VIEW_BUILD_STATUS), keyspaceName, viewName);
         forceBlockingFlush(VIEW_BUILD_STATUS);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java b/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java
index b3efeae..9903114 100644
--- a/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java
+++ b/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java
@@ -30,13 +30,14 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 
 public class PrepareMessage extends RepairMessage
 {
     public final static MessageSerializer serializer = new PrepareMessageSerializer();
-    public final List<UUID> cfIds;
+    public final List<TableId> tableIds;
     public final Collection<Range<Token>> ranges;
 
     public final UUID parentRepairSession;
@@ -44,11 +45,11 @@ public class PrepareMessage extends RepairMessage
     public final long timestamp;
     public final boolean isGlobal;
 
-    public PrepareMessage(UUID parentRepairSession, List<UUID> cfIds, Collection<Range<Token>> ranges, boolean isIncremental, long timestamp, boolean isGlobal)
+    public PrepareMessage(UUID parentRepairSession, List<TableId> tableIds, Collection<Range<Token>> ranges, boolean isIncremental, long timestamp, boolean isGlobal)
     {
         super(Type.PREPARE_MESSAGE, null);
         this.parentRepairSession = parentRepairSession;
-        this.cfIds = cfIds;
+        this.tableIds = tableIds;
         this.ranges = ranges;
         this.isIncremental = isIncremental;
         this.timestamp = timestamp;
@@ -66,23 +67,23 @@ public class PrepareMessage extends RepairMessage
                isIncremental == other.isIncremental &&
                isGlobal == other.isGlobal &&
                timestamp == other.timestamp &&
-               cfIds.equals(other.cfIds) &&
+               tableIds.equals(other.tableIds) &&
                ranges.equals(other.ranges);
     }
 
     @Override
     public int hashCode()
     {
-        return Objects.hash(messageType, parentRepairSession, isGlobal, isIncremental, timestamp, cfIds, ranges);
+        return Objects.hash(messageType, parentRepairSession, isGlobal, isIncremental, timestamp, tableIds, ranges);
     }
 
     public static class PrepareMessageSerializer implements MessageSerializer<PrepareMessage>
     {
         public void serialize(PrepareMessage message, DataOutputPlus out, int version) throws IOException
         {
-            out.writeInt(message.cfIds.size());
-            for (UUID cfId : message.cfIds)
-                UUIDSerializer.serializer.serialize(cfId, out, version);
+            out.writeInt(message.tableIds.size());
+            for (TableId tableId : message.tableIds)
+                tableId.serialize(out);
             UUIDSerializer.serializer.serialize(message.parentRepairSession, out, version);
             out.writeInt(message.ranges.size());
             for (Range<Token> r : message.ranges)
@@ -97,10 +98,10 @@ public class PrepareMessage extends RepairMessage
 
         public PrepareMessage deserialize(DataInputPlus in, int version) throws IOException
         {
-            int cfIdCount = in.readInt();
-            List<UUID> cfIds = new ArrayList<>(cfIdCount);
-            for (int i = 0; i < cfIdCount; i++)
-                cfIds.add(UUIDSerializer.serializer.deserialize(in, version));
+            int tableIdCount = in.readInt();
+            List<TableId> tableIds = new ArrayList<>(tableIdCount);
+            for (int i = 0; i < tableIdCount; i++)
+                tableIds.add(TableId.deserialize(in));
             UUID parentRepairSession = UUIDSerializer.serializer.deserialize(in, version);
             int rangeCount = in.readInt();
             List<Range<Token>> ranges = new ArrayList<>(rangeCount);
@@ -109,15 +110,15 @@ public class PrepareMessage extends RepairMessage
             boolean isIncremental = in.readBoolean();
             long timestamp = in.readLong();
             boolean isGlobal = in.readBoolean();
-            return new PrepareMessage(parentRepairSession, cfIds, ranges, isIncremental, timestamp, isGlobal);
+            return new PrepareMessage(parentRepairSession, tableIds, ranges, isIncremental, timestamp, isGlobal);
         }
 
         public long serializedSize(PrepareMessage message, int version)
         {
             long size;
-            size = TypeSizes.sizeof(message.cfIds.size());
-            for (UUID cfId : message.cfIds)
-                size += UUIDSerializer.serializer.serializedSize(cfId, version);
+            size = TypeSizes.sizeof(message.tableIds.size());
+            for (TableId tableId : message.tableIds)
+                size += tableId.serializedSize();
             size += UUIDSerializer.serializer.serializedSize(message.parentRepairSession, version);
             size += TypeSizes.sizeof(message.ranges.size());
             for (Range<Token> r : message.ranges)
@@ -133,12 +134,12 @@ public class PrepareMessage extends RepairMessage
     public String toString()
     {
         return "PrepareMessage{" +
-                "cfIds='" + cfIds + '\'' +
-                ", ranges=" + ranges +
-                ", parentRepairSession=" + parentRepairSession +
-                ", isIncremental="+isIncremental +
-                ", timestamp=" + timestamp +
-                ", isGlobal=" + isGlobal +
-                '}';
+               "tableIds='" + tableIds + '\'' +
+               ", ranges=" + ranges +
+               ", parentRepairSession=" + parentRepairSession +
+               ", isIncremental=" + isIncremental +
+               ", timestamp=" + timestamp +
+               ", isGlobal=" + isGlobal +
+               '}';
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/ColumnMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/ColumnMetadata.java b/src/java/org/apache/cassandra/schema/ColumnMetadata.java
new file mode 100644
index 0000000..788351e
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/ColumnMetadata.java
@@ -0,0 +1,617 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Objects;
+import com.google.common.collect.Collections2;
+
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.selection.Selectable;
+import org.apache.cassandra.cql3.selection.Selector;
+import org.apache.cassandra.cql3.selection.SimpleSelector;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public final class ColumnMetadata extends ColumnSpecification implements Selectable, Comparable<ColumnMetadata>
+{
+    public static final Comparator<Object> asymmetricColumnDataComparator =
+        (a, b) -> ((ColumnData) a).column().compareTo((ColumnMetadata) b);
+
+    public static final int NO_POSITION = -1;
+
+    public enum ClusteringOrder
+    {
+        ASC, DESC, NONE
+    }
+
+    /**
+     * The type of CQL3 column this definition represents.
+     * There is 4 main type of CQL3 columns: those parts of the partition key,
+     * those parts of the clustering columns and amongst the others, regular and
+     * static ones.
+     *
+     * IMPORTANT: this enum is serialized as toString() and deserialized by calling
+     * Kind.valueOf(), so do not override toString() or rename existing values.
+     */
+    public enum Kind
+    {
+        // NOTE: if adding a new type, must modify comparisonOrder
+        PARTITION_KEY,
+        CLUSTERING,
+        REGULAR,
+        STATIC;
+
+        public boolean isPrimaryKeyKind()
+        {
+            return this == PARTITION_KEY || this == CLUSTERING;
+        }
+
+    }
+
+    public final Kind kind;
+
+    /*
+     * If the column is a partition key or clustering column, its position relative to
+     * other columns of the same kind. Otherwise,  NO_POSITION (-1).
+     *
+     * Note that partition key and clustering columns are numbered separately so
+     * the first clustering column is 0.
+     */
+    private final int position;
+
+    private final Comparator<CellPath> cellPathComparator;
+    private final Comparator<Object> asymmetricCellPathComparator;
+    private final Comparator<? super Cell> cellComparator;
+
+    private int hash;
+
+    /**
+     * These objects are compared frequently, so we encode several of their comparison components
+     * into a single long value so that this can be done efficiently
+     */
+    private final long comparisonOrder;
+
+    private static long comparisonOrder(Kind kind, boolean isComplex, long position, ColumnIdentifier name)
+    {
+        assert position >= 0 && position < 1 << 12;
+        return   (((long) kind.ordinal()) << 61)
+               | (isComplex ? 1L << 60 : 0)
+               | (position << 48)
+               | (name.prefixComparison >>> 16);
+    }
+
+    public static ColumnMetadata partitionKeyColumn(TableMetadata table, ByteBuffer name, AbstractType<?> type, int position)
+    {
+        return new ColumnMetadata(table, name, type, position, Kind.PARTITION_KEY);
+    }
+
+    public static ColumnMetadata partitionKeyColumn(String keyspace, String table, String name, AbstractType<?> type, int position)
+    {
+        return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, position, Kind.PARTITION_KEY);
+    }
+
+    public static ColumnMetadata clusteringColumn(TableMetadata table, ByteBuffer name, AbstractType<?> type, int position)
+    {
+        return new ColumnMetadata(table, name, type, position, Kind.CLUSTERING);
+    }
+
+    public static ColumnMetadata clusteringColumn(String keyspace, String table, String name, AbstractType<?> type, int position)
+    {
+        return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, position, Kind.CLUSTERING);
+    }
+
+    public static ColumnMetadata regularColumn(TableMetadata table, ByteBuffer name, AbstractType<?> type)
+    {
+        return new ColumnMetadata(table, name, type, NO_POSITION, Kind.REGULAR);
+    }
+
+    public static ColumnMetadata regularColumn(String keyspace, String table, String name, AbstractType<?> type)
+    {
+        return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, NO_POSITION, Kind.REGULAR);
+    }
+
+    public static ColumnMetadata staticColumn(TableMetadata table, ByteBuffer name, AbstractType<?> type)
+    {
+        return new ColumnMetadata(table, name, type, NO_POSITION, Kind.STATIC);
+    }
+
+    public static ColumnMetadata staticColumn(String keyspace, String table, String name, AbstractType<?> type)
+    {
+        return new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, NO_POSITION, Kind.STATIC);
+    }
+
+    public ColumnMetadata(TableMetadata table, ByteBuffer name, AbstractType<?> type, int position, Kind kind)
+    {
+        this(table.keyspace,
+             table.name,
+             ColumnIdentifier.getInterned(name, table.columnDefinitionNameComparator(kind)),
+             type,
+             position,
+             kind);
+    }
+
+    @VisibleForTesting
+    public ColumnMetadata(String ksName,
+                          String cfName,
+                          ColumnIdentifier name,
+                          AbstractType<?> type,
+                          int position,
+                          Kind kind)
+    {
+        super(ksName, cfName, name, type);
+        assert name != null && type != null && kind != null;
+        assert name.isInterned();
+        assert (position == NO_POSITION) == !kind.isPrimaryKeyKind(); // The position really only make sense for partition and clustering columns (and those must have one),
+                                                                      // so make sure we don't sneak it for something else since it'd breaks equals()
+        this.kind = kind;
+        this.position = position;
+        this.cellPathComparator = makeCellPathComparator(kind, type);
+        this.cellComparator = cellPathComparator == null ? ColumnData.comparator : (a, b) -> cellPathComparator.compare(a.path(), b.path());
+        this.asymmetricCellPathComparator = cellPathComparator == null ? null : (a, b) -> cellPathComparator.compare(((Cell)a).path(), (CellPath) b);
+        this.comparisonOrder = comparisonOrder(kind, isComplex(), Math.max(0, position), name);
+    }
+
+    private static Comparator<CellPath> makeCellPathComparator(Kind kind, AbstractType<?> type)
+    {
+        if (kind.isPrimaryKeyKind() || !type.isMultiCell())
+            return null;
+
+        AbstractType<?> nameComparator = type.isCollection()
+                                       ? ((CollectionType) type).nameComparator()
+                                       : ((UserType) type).nameComparator();
+
+
+        return (path1, path2) ->
+        {
+            if (path1.size() == 0 || path2.size() == 0)
+            {
+                if (path1 == CellPath.BOTTOM)
+                    return path2 == CellPath.BOTTOM ? 0 : -1;
+                if (path1 == CellPath.TOP)
+                    return path2 == CellPath.TOP ? 0 : 1;
+                return path2 == CellPath.BOTTOM ? 1 : -1;
+            }
+
+            // This will get more complicated once we have non-frozen UDT and nested collections
+            assert path1.size() == 1 && path2.size() == 1;
+            return nameComparator.compare(path1.get(0), path2.get(0));
+        };
+    }
+
+    public ColumnMetadata copy()
+    {
+        return new ColumnMetadata(ksName, cfName, name, type, position, kind);
+    }
+
+    public ColumnMetadata withNewName(ColumnIdentifier newName)
+    {
+        return new ColumnMetadata(ksName, cfName, newName, type, position, kind);
+    }
+
+    public ColumnMetadata withNewType(AbstractType<?> newType)
+    {
+        return new ColumnMetadata(ksName, cfName, name, newType, position, kind);
+    }
+
+    public boolean isPartitionKey()
+    {
+        return kind == Kind.PARTITION_KEY;
+    }
+
+    public boolean isClusteringColumn()
+    {
+        return kind == Kind.CLUSTERING;
+    }
+
+    public boolean isStatic()
+    {
+        return kind == Kind.STATIC;
+    }
+
+    public boolean isRegular()
+    {
+        return kind == Kind.REGULAR;
+    }
+
+    public ClusteringOrder clusteringOrder()
+    {
+        if (!isClusteringColumn())
+            return ClusteringOrder.NONE;
+
+        return type.isReversed() ? ClusteringOrder.DESC : ClusteringOrder.ASC;
+    }
+
+    public int position()
+    {
+        return position;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof ColumnMetadata))
+            return false;
+
+        ColumnMetadata cd = (ColumnMetadata) o;
+
+        return Objects.equal(ksName, cd.ksName)
+            && Objects.equal(cfName, cd.cfName)
+            && Objects.equal(name, cd.name)
+            && Objects.equal(type, cd.type)
+            && Objects.equal(kind, cd.kind)
+            && Objects.equal(position, cd.position);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        // This achieves the same as Objects.hashcode, but avoids the object array allocation
+        // which features significantly in the allocation profile and caches the result.
+        int result = hash;
+        if (result == 0)
+        {
+            result = 31 + (ksName == null ? 0 : ksName.hashCode());
+            result = 31 * result + (cfName == null ? 0 : cfName.hashCode());
+            result = 31 * result + (name == null ? 0 : name.hashCode());
+            result = 31 * result + (type == null ? 0 : type.hashCode());
+            result = 31 * result + (kind == null ? 0 : kind.hashCode());
+            result = 31 * result + position;
+            hash = result;
+        }
+        return result;
+    }
+
+    @Override
+    public String toString()
+    {
+        return name.toString();
+    }
+
+    public String debugString()
+    {
+        return MoreObjects.toStringHelper(this)
+                          .add("name", name)
+                          .add("type", type)
+                          .add("kind", kind)
+                          .add("position", position)
+                          .toString();
+    }
+
+    public boolean isPrimaryKeyColumn()
+    {
+        return kind.isPrimaryKeyKind();
+    }
+
+    /**
+     * Converts the specified column definitions into column identifiers.
+     *
+     * @param definitions the column definitions to convert.
+     * @return the column identifiers corresponding to the specified definitions
+     */
+    public static Collection<ColumnIdentifier> toIdentifiers(Collection<ColumnMetadata> definitions)
+    {
+        return Collections2.transform(definitions, columnDef -> columnDef.name);
+    }
+
+    public int compareTo(ColumnMetadata other)
+    {
+        if (this == other)
+            return 0;
+
+        if (comparisonOrder != other.comparisonOrder)
+            return Long.compare(comparisonOrder, other.comparisonOrder);
+
+        return this.name.compareTo(other.name);
+    }
+
+    public Comparator<CellPath> cellPathComparator()
+    {
+        return cellPathComparator;
+    }
+
+    public Comparator<Object> asymmetricCellPathComparator()
+    {
+        return asymmetricCellPathComparator;
+    }
+
+    public Comparator<? super Cell> cellComparator()
+    {
+        return cellComparator;
+    }
+
+    public boolean isComplex()
+    {
+        return cellPathComparator != null;
+    }
+
+    public boolean isSimple()
+    {
+        return !isComplex();
+    }
+
+    public CellPath.Serializer cellPathSerializer()
+    {
+        // Collections are our only complex so far, so keep it simple
+        return CollectionType.cellPathSerializer;
+    }
+
+    public void validateCell(Cell cell)
+    {
+        if (cell.isTombstone())
+        {
+            if (cell.value().hasRemaining())
+                throw new MarshalException("A tombstone should not have a value");
+            if (cell.path() != null)
+                validateCellPath(cell.path());
+        }
+        else if(type.isUDT())
+        {
+            // To validate a non-frozen UDT field, both the path and the value
+            // are needed, the path being an index into an array of value types.
+            ((UserType)type).validateCell(cell);
+        }
+        else
+        {
+            type.validateCellValue(cell.value());
+            if (cell.path() != null)
+                validateCellPath(cell.path());
+        }
+    }
+
+    private void validateCellPath(CellPath path)
+    {
+        if (!isComplex())
+            throw new MarshalException("Only complex cells should have a cell path");
+
+        assert type.isMultiCell();
+        if (type.isCollection())
+            ((CollectionType)type).nameComparator().validate(path.get(0));
+        else
+            ((UserType)type).nameComparator().validate(path.get(0));
+    }
+
+    public static String toCQLString(Iterable<ColumnMetadata> defs)
+    {
+        return toCQLString(defs.iterator());
+    }
+
+    public static String toCQLString(Iterator<ColumnMetadata> defs)
+    {
+        if (!defs.hasNext())
+            return "";
+
+        StringBuilder sb = new StringBuilder();
+        sb.append(defs.next().name);
+        while (defs.hasNext())
+            sb.append(", ").append(defs.next().name);
+        return sb.toString();
+    }
+
+    /**
+     * The type of the cell values for cell belonging to this column.
+     *
+     * This is the same than the column type, except for collections where it's the 'valueComparator'
+     * of the collection.
+     */
+    public AbstractType<?> cellValueType()
+    {
+        return type instanceof CollectionType
+             ? ((CollectionType)type).valueComparator()
+             : type;
+    }
+
+    public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType<?> expectedType, List<ColumnMetadata> defs, VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        return SimpleSelector.newFactory(this, addAndGetIndex(this, defs));
+    }
+
+    public AbstractType<?> getExactTypeIfKnown(String keyspace)
+    {
+        return type;
+    }
+
+    /**
+     * Because legacy-created tables may have a non-text comparator, we cannot determine the proper 'key' until
+     * we know the comparator. ColumnMetadata.Raw is a placeholder that can be converted to a real ColumnIdentifier
+     * once the comparator is known with prepare(). This should only be used with identifiers that are actual
+     * column names. See CASSANDRA-8178 for more background.
+     */
+    public static abstract class Raw extends Selectable.Raw
+    {
+        /**
+         * Creates a {@code ColumnMetadata.Raw} from an unquoted identifier string.
+         */
+        public static Raw forUnquoted(String text)
+        {
+            return new Literal(text, false);
+        }
+
+        /**
+         * Creates a {@code ColumnMetadata.Raw} from a quoted identifier string.
+         */
+        public static Raw forQuoted(String text)
+        {
+            return new Literal(text, true);
+        }
+
+        /**
+         * Creates a {@code ColumnMetadata.Raw} from a pre-existing {@code ColumnMetadata}
+         * (useful in the rare cases where we already have the column but need
+         * a {@code ColumnMetadata.Raw} for typing purposes).
+         */
+        public static Raw forColumn(ColumnMetadata column)
+        {
+            return new ForColumn(column);
+        }
+
+        /**
+         * Get the identifier corresponding to this raw column, without assuming this is an
+         * existing column (unlike {@link Selectable.Raw#prepare}).
+         */
+        public abstract ColumnIdentifier getIdentifier(TableMetadata table);
+
+        public abstract String rawText();
+
+        @Override
+        public abstract ColumnMetadata prepare(TableMetadata table);
+
+        @Override
+        public boolean processesSelection()
+        {
+            return false;
+        }
+
+        @Override
+        public final int hashCode()
+        {
+            return toString().hashCode();
+        }
+
+        @Override
+        public final boolean equals(Object o)
+        {
+            if(!(o instanceof Raw))
+                return false;
+
+            Raw that = (Raw)o;
+            return this.toString().equals(that.toString());
+        }
+
+        private static class Literal extends Raw
+        {
+            private final String text;
+
+            public Literal(String rawText, boolean keepCase)
+            {
+                this.text =  keepCase ? rawText : rawText.toLowerCase(Locale.US);
+            }
+
+            public ColumnIdentifier getIdentifier(TableMetadata table)
+            {
+                if (!table.isStaticCompactTable())
+                    return ColumnIdentifier.getInterned(text, true);
+
+                AbstractType<?> columnNameType = table.staticCompactOrSuperTableColumnNameType();
+                if (columnNameType instanceof UTF8Type)
+                    return ColumnIdentifier.getInterned(text, true);
+
+                // We have a legacy-created table with a non-text comparator. Check if we have a matching column, otherwise assume we should use
+                // columnNameType
+                ByteBuffer bufferName = ByteBufferUtil.bytes(text);
+                for (ColumnMetadata def : table.columns())
+                {
+                    if (def.name.bytes.equals(bufferName))
+                        return def.name;
+                }
+                return ColumnIdentifier.getInterned(columnNameType, columnNameType.fromString(text), text);
+            }
+
+            public ColumnMetadata prepare(TableMetadata table)
+            {
+                if (!table.isStaticCompactTable())
+                    return find(table);
+
+                AbstractType<?> columnNameType = table.staticCompactOrSuperTableColumnNameType();
+                if (columnNameType instanceof UTF8Type)
+                    return find(table);
+
+                // We have a legacy-created table with a non-text comparator. Check if we have a match column, otherwise assume we should use
+                // columnNameType
+                ByteBuffer bufferName = ByteBufferUtil.bytes(text);
+                for (ColumnMetadata def : table.columns())
+                {
+                    if (def.name.bytes.equals(bufferName))
+                        return def;
+                }
+                return find(columnNameType.fromString(text), table);
+            }
+
+            private ColumnMetadata find(TableMetadata table)
+            {
+                return find(ByteBufferUtil.bytes(text), table);
+            }
+
+            private ColumnMetadata find(ByteBuffer id, TableMetadata table)
+            {
+                ColumnMetadata def = table.getColumn(id);
+                if (def == null)
+                    throw new InvalidRequestException(String.format("Undefined column name %s", toString()));
+                return def;
+            }
+
+            public String rawText()
+            {
+                return text;
+            }
+
+            @Override
+            public String toString()
+            {
+                return ColumnIdentifier.maybeQuote(text);
+            }
+        }
+
+        // Use internally in the rare case where we need a ColumnMetadata.Raw for type-checking but
+        // actually already have the column itself.
+        private static class ForColumn extends Raw
+        {
+            private final ColumnMetadata column;
+
+            private ForColumn(ColumnMetadata column)
+            {
+                this.column = column;
+            }
+
+            public ColumnIdentifier getIdentifier(TableMetadata table)
+            {
+                return column.name;
+            }
+
+            public ColumnMetadata prepare(TableMetadata table)
+            {
+                assert table.getColumn(column.name) != null; // Sanity check that we're not doing something crazy
+                return column;
+            }
+
+            public String rawText()
+            {
+                return column.name.toString();
+            }
+
+            @Override
+            public String toString()
+            {
+                return column.name.toCQLString();
+            }
+        }
+    }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/CompactionParams.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/CompactionParams.java b/src/java/org/apache/cassandra/schema/CompactionParams.java
index 73271f1..1a4a5f1 100644
--- a/src/java/org/apache/cassandra/schema/CompactionParams.java
+++ b/src/java/org/apache/cassandra/schema/CompactionParams.java
@@ -249,7 +249,7 @@ public final class CompactionParams
         return create(classFromName(className), options);
     }
 
-    private static Class<? extends AbstractCompactionStrategy> classFromName(String name)
+    public static Class<? extends AbstractCompactionStrategy> classFromName(String name)
     {
         String className = name.contains(".")
                          ? name

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/DroppedColumn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/DroppedColumn.java b/src/java/org/apache/cassandra/schema/DroppedColumn.java
new file mode 100644
index 0000000..90dfe65
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/DroppedColumn.java
@@ -0,0 +1,59 @@
+/*
+ * 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 com.google.common.base.MoreObjects;
+import com.google.common.base.Objects;
+
+public final class DroppedColumn
+{
+    public final ColumnMetadata column;
+    public final long droppedTime; // drop timestamp, in microseconds, yet with millisecond granularity
+
+    public DroppedColumn(ColumnMetadata column, long droppedTime)
+    {
+        this.column = column;
+        this.droppedTime = droppedTime;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof DroppedColumn))
+            return false;
+
+        DroppedColumn dc = (DroppedColumn) o;
+
+        return column.equals(dc.column) && droppedTime == dc.droppedTime;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(column, droppedTime);
+    }
+
+    @Override
+    public String toString()
+    {
+        return MoreObjects.toStringHelper(this).add("column", column).add("droppedTime", droppedTime).toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/Functions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Functions.java b/src/java/org/apache/cassandra/schema/Functions.java
index a936d81..8e3a3f1 100644
--- a/src/java/org/apache/cassandra/schema/Functions.java
+++ b/src/java/org/apache/cassandra/schema/Functions.java
@@ -22,9 +22,12 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
 import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.Maps;
 
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.utils.Pair;
 
 import static com.google.common.collect.Iterables.filter;
 
@@ -82,6 +85,28 @@ public final class Functions implements Iterable<Function>
         return stream().filter(f -> f instanceof UDAggregate).map(f -> (UDAggregate) f);
     }
 
+    MapDifference<Pair<FunctionName, List<String>>, UDFunction> udfsDiff(Functions other)
+    {
+        Map<Pair<FunctionName, List<String>>, UDFunction> before = new HashMap<>();
+        udfs().forEach(f -> before.put(Pair.create(f.name(), f.argumentsList()), f));
+
+        Map<Pair<FunctionName, List<String>>, UDFunction> after = new HashMap<>();
+        other.udfs().forEach(f -> after.put(Pair.create(f.name(), f.argumentsList()), f));
+
+        return Maps.difference(before, after);
+    }
+
+    MapDifference<Pair<FunctionName, List<String>>, UDAggregate> udasDiff(Functions other)
+    {
+        Map<Pair<FunctionName, List<String>>, UDAggregate> before = new HashMap<>();
+        udas().forEach(f -> before.put(Pair.create(f.name(), f.argumentsList()), f));
+
+        Map<Pair<FunctionName, List<String>>, UDAggregate> after = new HashMap<>();
+        other.udas().forEach(f -> after.put(Pair.create(f.name(), f.argumentsList()), f));
+
+        return Maps.difference(before, after);
+    }
+
     /**
      * @return a collection of aggregates that use the provided function as either a state or a final function
      * @param function the referree function
@@ -206,7 +231,7 @@ public final class Functions implements Iterable<Function>
         private Builder()
         {
             // we need deterministic iteration order; otherwise Functions.equals() breaks down
-            functions.orderValuesBy((f1, f2) -> Integer.compare(f1.hashCode(), f2.hashCode()));
+            functions.orderValuesBy(Comparator.comparingInt(Object::hashCode));
         }
 
         public Functions build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/IndexMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/IndexMetadata.java b/src/java/org/apache/cassandra/schema/IndexMetadata.java
index b8c4854..190871a 100644
--- a/src/java/org/apache/cassandra/schema/IndexMetadata.java
+++ b/src/java/org/apache/cassandra/schema/IndexMetadata.java
@@ -31,10 +31,10 @@ import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.statements.IndexTarget;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.UnknownIndexException;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -81,15 +81,14 @@ public final class IndexMetadata
         return new IndexMetadata(name, options, kind);
     }
 
-    public static IndexMetadata fromIndexTargets(CFMetaData cfm,
-                                                 List<IndexTarget> targets,
+    public static IndexMetadata fromIndexTargets(List<IndexTarget> targets,
                                                  String name,
                                                  Kind kind,
                                                  Map<String, String> options)
     {
         Map<String, String> newOptions = new HashMap<>(options);
         newOptions.put(IndexTarget.TARGET_OPTION_NAME, targets.stream()
-                                                              .map(target -> target.asCqlString(cfm))
+                                                              .map(target -> target.asCqlString())
                                                               .collect(Collectors.joining(", ")));
         return new IndexMetadata(name, newOptions, kind);
     }
@@ -107,7 +106,7 @@ public final class IndexMetadata
             return PATTERN_NON_WORD_CHAR.matcher(cfName + "_" + root + "_idx").replaceAll("");
     }
 
-    public void validate(CFMetaData cfm)
+    public void validate(TableMetadata table)
     {
         if (!isNameValid(name))
             throw new ConfigurationException("Illegal index name " + name);
@@ -122,29 +121,25 @@ public final class IndexMetadata
                                                                name, IndexTarget.CUSTOM_INDEX_OPTION_NAME));
             String className = options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME);
             Class<Index> indexerClass = FBUtilities.classForName(className, "custom indexer");
-            if(!Index.class.isAssignableFrom(indexerClass))
+            if (!Index.class.isAssignableFrom(indexerClass))
                 throw new ConfigurationException(String.format("Specified Indexer class (%s) does not implement the Indexer interface", className));
-            validateCustomIndexOptions(cfm, indexerClass, options);
+            validateCustomIndexOptions(table, indexerClass, options);
         }
     }
 
-    private void validateCustomIndexOptions(CFMetaData cfm,
-                                            Class<? extends Index> indexerClass,
-                                            Map<String, String> options)
-    throws ConfigurationException
+    private void validateCustomIndexOptions(TableMetadata table, Class<? extends Index> indexerClass, Map<String, String> options)
     {
         try
         {
-            Map<String, String> filteredOptions =
-                Maps.filterKeys(options,key -> !key.equals(IndexTarget.CUSTOM_INDEX_OPTION_NAME));
+            Map<String, String> filteredOptions = Maps.filterKeys(options, key -> !key.equals(IndexTarget.CUSTOM_INDEX_OPTION_NAME));
 
             if (filteredOptions.isEmpty())
                 return;
 
-            Map<?,?> unknownOptions;
+            Map<?, ?> unknownOptions;
             try
             {
-                unknownOptions = (Map) indexerClass.getMethod("validateOptions", Map.class, CFMetaData.class).invoke(null, filteredOptions, cfm);
+                unknownOptions = (Map) indexerClass.getMethod("validateOptions", Map.class, TableMetadata.class).invoke(null, filteredOptions, table);
             }
             catch (NoSuchMethodException e)
             {
@@ -190,6 +185,7 @@ public final class IndexMetadata
         return kind == Kind.COMPOSITES;
     }
 
+    @Override
     public int hashCode()
     {
         return Objects.hashCode(id, name, kind, options);
@@ -198,9 +194,10 @@ public final class IndexMetadata
     public boolean equalsWithoutName(IndexMetadata other)
     {
         return Objects.equal(kind, other.kind)
-            && Objects.equal(options, other.options);
+               && Objects.equal(options, other.options);
     }
 
+    @Override
     public boolean equals(Object obj)
     {
         if (obj == this)
@@ -209,19 +206,25 @@ public final class IndexMetadata
         if (!(obj instanceof IndexMetadata))
             return false;
 
-        IndexMetadata other = (IndexMetadata)obj;
+        IndexMetadata other = (IndexMetadata) obj;
 
         return Objects.equal(id, other.id) && Objects.equal(name, other.name) && equalsWithoutName(other);
     }
 
+    @Override
     public String toString()
     {
         return new ToStringBuilder(this)
-            .append("id", id.toString())
-            .append("name", name)
-            .append("kind", kind)
-            .append("options", options)
-            .build();
+               .append("id", id.toString())
+               .append("name", name)
+               .append("kind", kind)
+               .append("options", options)
+               .build();
+    }
+
+    public String toCQLString()
+    {
+        return ColumnIdentifier.maybeQuote(name);
     }
 
     public static class Serializer
@@ -231,10 +234,10 @@ public final class IndexMetadata
             UUIDSerializer.serializer.serialize(metadata.id, out, version);
         }
 
-        public IndexMetadata deserialize(DataInputPlus in, int version, CFMetaData cfm) throws IOException
+        public IndexMetadata deserialize(DataInputPlus in, int version, TableMetadata table) throws IOException
         {
             UUID id = UUIDSerializer.serializer.deserialize(in, version);
-            return cfm.getIndexes().get(id).orElseThrow(() -> new UnknownIndexException(cfm, id));
+            return table.indexes.get(id).orElseThrow(() -> new UnknownIndexException(table, id));
         }
 
         public long serializedSize(IndexMetadata metadata, int version)


[33/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java
index 1ff45d0..b1edf94 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeyRestrictions.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.cql3.restrictions;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.statements.Bound;
 
@@ -53,16 +53,16 @@ interface PartitionKeyRestrictions extends Restrictions
     /**
      * checks if specified restrictions require filtering
      *
-     * @param cfm column family metadata
+     * @param table column family metadata
      * @return <code>true</code> if filtering is required, <code>false</code> otherwise
      */
-    public boolean needFiltering(CFMetaData cfm);
+    public boolean needFiltering(TableMetadata table);
 
     /**
      * Checks if the partition key has unrestricted components.
      *
-     * @param cfm column family metadata
+     * @param table column family metadata
      * @return <code>true</code> if the partition key has unrestricted components, <code>false</code> otherwise.
      */
-    public boolean hasUnrestrictedPartitionKeyComponents(CFMetaData cfm);
+    public boolean hasUnrestrictedPartitionKeyComponents(TableMetadata table);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeySingleRestrictionSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeySingleRestrictionSet.java b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeySingleRestrictionSet.java
index b34ff54..5113667 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeySingleRestrictionSet.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/PartitionKeySingleRestrictionSet.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.cql3.restrictions;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.statements.Bound;
 import org.apache.cassandra.db.ClusteringComparator;
@@ -59,7 +59,7 @@ final class PartitionKeySingleRestrictionSet extends RestrictionSetWrapper imple
     {
         List<ByteBuffer> l = new ArrayList<>(clusterings.size());
         for (ClusteringPrefix clustering : clusterings)
-            l.add(CFMetaData.serializePartitionKey(clustering));
+            l.add(clustering.serializeAsPartitionKey());
         return l;
     }
 
@@ -131,18 +131,18 @@ final class PartitionKeySingleRestrictionSet extends RestrictionSetWrapper imple
     }
 
     @Override
-    public boolean needFiltering(CFMetaData cfm)
+    public boolean needFiltering(TableMetadata table)
     {
         if (isEmpty())
             return false;
         // slice or has unrestricted key component
-        return hasUnrestrictedPartitionKeyComponents(cfm) || hasSlice();
+        return hasUnrestrictedPartitionKeyComponents(table) || hasSlice();
     }
 
     @Override
-    public boolean hasUnrestrictedPartitionKeyComponents(CFMetaData cfm)
+    public boolean hasUnrestrictedPartitionKeyComponents(TableMetadata table)
     {
-        return size() < cfm.partitionKeyColumns().size();
+        return size() < table.partitionKeyColumns().size();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
index fc7f5bc..daace46 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.cql3.restrictions;
 
 import java.util.List;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.filter.RowFilter;
@@ -39,19 +39,19 @@ public interface Restriction
      * Returns the definition of the first column.
      * @return the definition of the first column.
      */
-    public ColumnDefinition getFirstColumn();
+    public ColumnMetadata getFirstColumn();
 
     /**
      * Returns the definition of the last column.
      * @return the definition of the last column.
      */
-    public ColumnDefinition getLastColumn();
+    public ColumnMetadata getLastColumn();
 
     /**
      * Returns the column definitions in position order.
      * @return the column definitions in position order.
      */
-    public List<ColumnDefinition> getColumnDefs();
+    public List<ColumnMetadata> getColumnDefs();
 
     /**
      * Adds all functions (native and user-defined) used by any component of the restriction

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
index 3a1bcb1..ee3df98 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
@@ -21,7 +21,7 @@ import java.util.*;
 
 import com.google.common.collect.AbstractIterator;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction.ContainsRestriction;
@@ -39,10 +39,10 @@ final class RestrictionSet implements Restrictions, Iterable<SingleRestriction>
     /**
      * The comparator used to sort the <code>Restriction</code>s.
      */
-    private static final Comparator<ColumnDefinition> COLUMN_DEFINITION_COMPARATOR = new Comparator<ColumnDefinition>()
+    private static final Comparator<ColumnMetadata> COLUMN_DEFINITION_COMPARATOR = new Comparator<ColumnMetadata>()
     {
         @Override
-        public int compare(ColumnDefinition column, ColumnDefinition otherColumn)
+        public int compare(ColumnMetadata column, ColumnMetadata otherColumn)
         {
             int value = Integer.compare(column.position(), otherColumn.position());
             return value != 0 ? value : column.name.bytes.compareTo(otherColumn.name.bytes);
@@ -52,7 +52,7 @@ final class RestrictionSet implements Restrictions, Iterable<SingleRestriction>
     /**
      * The restrictions per column.
      */
-    protected final TreeMap<ColumnDefinition, SingleRestriction> restrictions;
+    protected final TreeMap<ColumnMetadata, SingleRestriction> restrictions;
 
     /**
      * {@code true} if it contains multi-column restrictions, {@code false} otherwise.
@@ -61,10 +61,10 @@ final class RestrictionSet implements Restrictions, Iterable<SingleRestriction>
 
     public RestrictionSet()
     {
-        this(new TreeMap<ColumnDefinition, SingleRestriction>(COLUMN_DEFINITION_COMPARATOR), false);
+        this(new TreeMap<ColumnMetadata, SingleRestriction>(COLUMN_DEFINITION_COMPARATOR), false);
     }
 
-    private RestrictionSet(TreeMap<ColumnDefinition, SingleRestriction> restrictions,
+    private RestrictionSet(TreeMap<ColumnMetadata, SingleRestriction> restrictions,
                            boolean hasMultiColumnRestrictions)
     {
         this.restrictions = restrictions;
@@ -79,7 +79,7 @@ final class RestrictionSet implements Restrictions, Iterable<SingleRestriction>
     }
 
     @Override
-    public List<ColumnDefinition> getColumnDefs()
+    public List<ColumnMetadata> getColumnDefs()
     {
         return new ArrayList<>(restrictions.keySet());
     }
@@ -108,9 +108,9 @@ final class RestrictionSet implements Restrictions, Iterable<SingleRestriction>
      * @param kind the column kind
      * @return {@code true} if one of the restrictions applies to a column of the specific kind, {@code false} otherwise.
      */
-    public boolean hasRestrictionFor(ColumnDefinition.Kind kind)
+    public boolean hasRestrictionFor(ColumnMetadata.Kind kind)
     {
-        for (ColumnDefinition column : restrictions.keySet())
+        for (ColumnMetadata column : restrictions.keySet())
         {
             if (column.kind == kind)
                 return true;
@@ -127,19 +127,19 @@ final class RestrictionSet implements Restrictions, Iterable<SingleRestriction>
     public RestrictionSet addRestriction(SingleRestriction restriction)
     {
         // RestrictionSet is immutable so we need to clone the restrictions map.
-        TreeMap<ColumnDefinition, SingleRestriction> newRestrictions = new TreeMap<>(this.restrictions);
+        TreeMap<ColumnMetadata, SingleRestriction> newRestrictions = new TreeMap<>(this.restrictions);
         return new RestrictionSet(mergeRestrictions(newRestrictions, restriction), hasMultiColumnRestrictions || restriction.isMultiColumn());
     }
 
-    private TreeMap<ColumnDefinition, SingleRestriction> mergeRestrictions(TreeMap<ColumnDefinition, SingleRestriction> restrictions,
-                                                                           SingleRestriction restriction)
+    private TreeMap<ColumnMetadata, SingleRestriction> mergeRestrictions(TreeMap<ColumnMetadata, SingleRestriction> restrictions,
+                                                                         SingleRestriction restriction)
     {
-        Collection<ColumnDefinition> columnDefs = restriction.getColumnDefs();
+        Collection<ColumnMetadata> columnDefs = restriction.getColumnDefs();
         Set<SingleRestriction> existingRestrictions = getRestrictions(columnDefs);
 
         if (existingRestrictions.isEmpty())
         {
-            for (ColumnDefinition columnDef : columnDefs)
+            for (ColumnMetadata columnDef : columnDefs)
                 restrictions.put(columnDef, restriction);
         }
         else
@@ -148,7 +148,7 @@ final class RestrictionSet implements Restrictions, Iterable<SingleRestriction>
             {
                 SingleRestriction newRestriction = mergeRestrictions(existing, restriction);
 
-                for (ColumnDefinition columnDef : columnDefs)
+                for (ColumnMetadata columnDef : columnDefs)
                     restrictions.put(columnDef, newRestriction);
             }
         }
@@ -157,7 +157,7 @@ final class RestrictionSet implements Restrictions, Iterable<SingleRestriction>
     }
 
     @Override
-    public Set<Restriction> getRestrictions(ColumnDefinition columnDef)
+    public Set<Restriction> getRestrictions(ColumnMetadata columnDef)
     {
         Restriction existing = restrictions.get(columnDef);
         return existing == null ? Collections.emptySet() : Collections.singleton(existing);
@@ -169,10 +169,10 @@ final class RestrictionSet implements Restrictions, Iterable<SingleRestriction>
      * @param columnDefs the column definitions
      * @return all the restrictions applied to the specified columns
      */
-    private Set<SingleRestriction> getRestrictions(Collection<ColumnDefinition> columnDefs)
+    private Set<SingleRestriction> getRestrictions(Collection<ColumnMetadata> columnDefs)
     {
         Set<SingleRestriction> set = new HashSet<>();
-        for (ColumnDefinition columnDef : columnDefs)
+        for (ColumnMetadata columnDef : columnDefs)
         {
             SingleRestriction existing = restrictions.get(columnDef);
             if (existing != null)
@@ -198,19 +198,19 @@ final class RestrictionSet implements Restrictions, Iterable<SingleRestriction>
      * @param columnDef the column for which the next one need to be found
      * @return the column after the specified one.
      */
-    ColumnDefinition nextColumn(ColumnDefinition columnDef)
+    ColumnMetadata nextColumn(ColumnMetadata columnDef)
     {
         return restrictions.tailMap(columnDef, false).firstKey();
     }
 
     @Override
-    public ColumnDefinition getFirstColumn()
+    public ColumnMetadata getFirstColumn()
     {
         return isEmpty() ? null : this.restrictions.firstKey();
     }
 
     @Override
-    public ColumnDefinition getLastColumn()
+    public ColumnMetadata getLastColumn()
     {
         return isEmpty() ? null : this.restrictions.lastKey();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSetWrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSetWrapper.java b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSetWrapper.java
index 996a1c4..e9a62de 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSetWrapper.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSetWrapper.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.cql3.restrictions;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.filter.RowFilter;
@@ -49,7 +49,7 @@ class RestrictionSetWrapper implements Restrictions
         restrictions.addRowFilterTo(filter, indexManager, options);
     }
 
-    public List<ColumnDefinition> getColumnDefs()
+    public List<ColumnMetadata> getColumnDefs()
     {
         return restrictions.getColumnDefs();
     }
@@ -74,12 +74,12 @@ class RestrictionSetWrapper implements Restrictions
         return restrictions.hasSupportingIndex(indexManager);
     }
 
-    public ColumnDefinition getFirstColumn()
+    public ColumnMetadata getFirstColumn()
     {
         return restrictions.getFirstColumn();
     }
 
-    public ColumnDefinition getLastColumn()
+    public ColumnMetadata getLastColumn()
     {
         return restrictions.getLastColumn();
     }
@@ -99,7 +99,7 @@ class RestrictionSetWrapper implements Restrictions
         return restrictions.hasOnlyEqualityRestrictions();
     }
 
-    public Set<Restriction> getRestrictions(ColumnDefinition columnDef)
+    public Set<Restriction> getRestrictions(ColumnMetadata columnDef)
     {
         return restrictions.getRestrictions(columnDef);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
index 8a5140a..8ccb1b2 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.cql3.restrictions;
 
 import java.util.Set;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 
 /**
  * Sets of restrictions
@@ -32,7 +32,7 @@ public interface Restrictions extends Restriction
      * @param columnDef the column definition
      * @return the restrictions applied to the specified column
      */
-    Set<Restriction> getRestrictions(ColumnDefinition columnDef);
+    Set<Restriction> getRestrictions(ColumnMetadata columnDef);
 
     /**
      * Checks if this <code>Restrictions</code> is empty or not.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
index 1d84331..44f95a8 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.Term.Terminal;
 import org.apache.cassandra.cql3.functions.Function;
@@ -45,27 +45,27 @@ public abstract class SingleColumnRestriction implements SingleRestriction
     /**
      * The definition of the column to which apply the restriction.
      */
-    protected final ColumnDefinition columnDef;
+    protected final ColumnMetadata columnDef;
 
-    public SingleColumnRestriction(ColumnDefinition columnDef)
+    public SingleColumnRestriction(ColumnMetadata columnDef)
     {
         this.columnDef = columnDef;
     }
 
     @Override
-    public List<ColumnDefinition> getColumnDefs()
+    public List<ColumnMetadata> getColumnDefs()
     {
         return Collections.singletonList(columnDef);
     }
 
     @Override
-    public ColumnDefinition getFirstColumn()
+    public ColumnMetadata getFirstColumn()
     {
         return columnDef;
     }
 
     @Override
-    public ColumnDefinition getLastColumn()
+    public ColumnMetadata getLastColumn()
     {
         return columnDef;
     }
@@ -125,7 +125,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
     {
         private final Term value;
 
-        public EQRestriction(ColumnDefinition columnDef, Term value)
+        public EQRestriction(ColumnMetadata columnDef, Term value)
         {
             super(columnDef);
             this.value = value;
@@ -187,7 +187,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
 
     public static abstract class INRestriction extends SingleColumnRestriction
     {
-        public INRestriction(ColumnDefinition columnDef)
+        public INRestriction(ColumnMetadata columnDef)
         {
             super(columnDef);
         }
@@ -234,7 +234,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
     {
         protected final List<Term> values;
 
-        public InRestrictionWithValues(ColumnDefinition columnDef, List<Term> values)
+        public InRestrictionWithValues(ColumnMetadata columnDef, List<Term> values)
         {
             super(columnDef);
             this.values = values;
@@ -272,7 +272,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
     {
         protected final AbstractMarker marker;
 
-        public InRestrictionWithMarker(ColumnDefinition columnDef, AbstractMarker marker)
+        public InRestrictionWithMarker(ColumnMetadata columnDef, AbstractMarker marker)
         {
             super(columnDef);
             this.marker = marker;
@@ -310,7 +310,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
     {
         private final TermSlice slice;
 
-        public SliceRestriction(ColumnDefinition columnDef, Bound bound, boolean inclusive, Term term)
+        public SliceRestriction(ColumnMetadata columnDef, Bound bound, boolean inclusive, Term term)
         {
             super(columnDef);
             slice = TermSlice.newInstance(bound, inclusive, term);
@@ -404,7 +404,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
             return String.format("SLICE%s", slice);
         }
 
-        private SliceRestriction(ColumnDefinition columnDef, TermSlice slice)
+        private SliceRestriction(ColumnMetadata columnDef, TermSlice slice)
         {
             super(columnDef);
             this.slice = slice;
@@ -419,7 +419,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
         private List<Term> entryKeys = new ArrayList<>(); // for map[key] = value
         private List<Term> entryValues = new ArrayList<>(); // for map[key] = value
 
-        public ContainsRestriction(ColumnDefinition columnDef, Term t, boolean isKey)
+        public ContainsRestriction(ColumnMetadata columnDef, Term t, boolean isKey)
         {
             super(columnDef);
             if (isKey)
@@ -428,7 +428,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
                 values.add(t);
         }
 
-        public ContainsRestriction(ColumnDefinition columnDef, Term mapKey, Term mapValue)
+        public ContainsRestriction(ColumnMetadata columnDef, Term mapKey, Term mapValue)
         {
             super(columnDef);
             entryKeys.add(mapKey);
@@ -583,7 +583,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
             to.entryValues.addAll(from.entryValues);
         }
 
-        private ContainsRestriction(ColumnDefinition columnDef)
+        private ContainsRestriction(ColumnMetadata columnDef)
         {
             super(columnDef);
         }
@@ -591,7 +591,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
 
     public static final class IsNotNullRestriction extends SingleColumnRestriction
     {
-        public IsNotNullRestriction(ColumnDefinition columnDef)
+        public IsNotNullRestriction(ColumnMetadata columnDef)
         {
             super(columnDef);
         }
@@ -652,7 +652,7 @@ public abstract class SingleColumnRestriction implements SingleRestriction
         private final Operator operator;
         private final Term value;
 
-        public LikeRestriction(ColumnDefinition columnDef, Operator operator, Term value)
+        public LikeRestriction(ColumnMetadata columnDef, Operator operator, Term value)
         {
             super(columnDef);
             this.operator = operator;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
index 38aeb18..8431490 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@ -22,9 +22,6 @@ import java.util.*;
 
 import com.google.common.base.Joiner;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.ColumnDefinition.Kind;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.statements.Bound;
@@ -36,7 +33,8 @@ import org.apache.cassandra.dht.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.index.SecondaryIndexManager;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.btree.BTreeSet;
 
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
@@ -61,7 +59,7 @@ public final class StatementRestrictions
     /**
      * The Column Family meta data
      */
-    public final CFMetaData cfm;
+    public final TableMetadata table;
 
     /**
      * Restrictions on partitioning columns
@@ -78,7 +76,7 @@ public final class StatementRestrictions
      */
     private RestrictionSet nonPrimaryKeyRestrictions;
 
-    private Set<ColumnDefinition> notNullColumns;
+    private Set<ColumnMetadata> notNullColumns;
 
     /**
      * The restrictions used to build the row filter
@@ -105,40 +103,40 @@ public final class StatementRestrictions
      * Creates a new empty <code>StatementRestrictions</code>.
      *
      * @param type the type of statement
-     * @param cfm the column family meta data
+     * @param table the column family meta data
      * @return a new empty <code>StatementRestrictions</code>.
      */
-    public static StatementRestrictions empty(StatementType type, CFMetaData cfm)
+    public static StatementRestrictions empty(StatementType type, TableMetadata table)
     {
-        return new StatementRestrictions(type, cfm, false);
+        return new StatementRestrictions(type, table, false);
     }
 
-    private StatementRestrictions(StatementType type, CFMetaData cfm, boolean allowFiltering)
+    private StatementRestrictions(StatementType type, TableMetadata table, boolean allowFiltering)
     {
         this.type = type;
-        this.cfm = cfm;
-        this.partitionKeyRestrictions = new PartitionKeySingleRestrictionSet(cfm.getKeyValidatorAsClusteringComparator());
-        this.clusteringColumnsRestrictions = new ClusteringColumnRestrictions(cfm, allowFiltering);
+        this.table = table;
+        this.partitionKeyRestrictions = new PartitionKeySingleRestrictionSet(table.partitionKeyAsClusteringComparator());
+        this.clusteringColumnsRestrictions = new ClusteringColumnRestrictions(table, allowFiltering);
         this.nonPrimaryKeyRestrictions = new RestrictionSet();
         this.notNullColumns = new HashSet<>();
     }
 
     public StatementRestrictions(StatementType type,
-                                 CFMetaData cfm,
+                                 TableMetadata table,
                                  WhereClause whereClause,
                                  VariableSpecifications boundNames,
                                  boolean selectsOnlyStaticColumns,
                                  boolean allowFiltering,
                                  boolean forView)
     {
-        this(type, cfm, allowFiltering);
+        this(type, table, allowFiltering);
 
         ColumnFamilyStore cfs;
         SecondaryIndexManager secondaryIndexManager = null;
 
         if (type.allowUseOfSecondaryIndices())
         {
-            cfs = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+            cfs = Keyspace.open(table.keyspace).getColumnFamilyStore(table.name);
             secondaryIndexManager = cfs.indexManager;
         }
 
@@ -158,12 +156,12 @@ public final class StatementRestrictions
                 if (!forView)
                     throw new InvalidRequestException("Unsupported restriction: " + relation);
 
-                for (ColumnDefinition def : relation.toRestriction(cfm, boundNames).getColumnDefs())
+                for (ColumnMetadata def : relation.toRestriction(table, boundNames).getColumnDefs())
                     this.notNullColumns.add(def);
             }
             else if (relation.isLIKE())
             {
-                Restriction restriction = relation.toRestriction(cfm, boundNames);
+                Restriction restriction = relation.toRestriction(table, boundNames);
 
                 if (!type.allowUseOfSecondaryIndices() || !restriction.hasSupportingIndex(secondaryIndexManager))
                     throw new InvalidRequestException(String.format("LIKE restriction is only supported on properly " +
@@ -174,11 +172,11 @@ public final class StatementRestrictions
             }
             else
             {
-                addRestriction(relation.toRestriction(cfm, boundNames));
+                addRestriction(relation.toRestriction(table, boundNames));
             }
         }
 
-        hasRegularColumnsRestrictions = nonPrimaryKeyRestrictions.hasRestrictionFor(Kind.REGULAR);
+        hasRegularColumnsRestrictions = nonPrimaryKeyRestrictions.hasRestrictionFor(ColumnMetadata.Kind.REGULAR);
 
         boolean hasQueriableClusteringColumnIndex = false;
         boolean hasQueriableIndex = false;
@@ -200,7 +198,7 @@ public final class StatementRestrictions
 
         // Some but not all of the partition key columns have been specified;
         // hence we need turn these restrictions into a row filter.
-        if (usesSecondaryIndexing || partitionKeyRestrictions.needFiltering(cfm))
+        if (usesSecondaryIndexing || partitionKeyRestrictions.needFiltering(table))
             filterRestrictions.add(partitionKeyRestrictions);
 
         if (selectsOnlyStaticColumns && hasClusteringColumnsRestrictions())
@@ -241,7 +239,7 @@ public final class StatementRestrictions
             if (!type.allowNonPrimaryKeyInWhereClause())
             {
                 Collection<ColumnIdentifier> nonPrimaryKeyColumns =
-                        ColumnDefinition.toIdentifiers(nonPrimaryKeyRestrictions.getColumnDefs());
+                        ColumnMetadata.toIdentifiers(nonPrimaryKeyRestrictions.getColumnDefs());
 
                 throw invalidRequest("Non PRIMARY KEY columns found in where clause: %s ",
                                      Joiner.on(", ").join(nonPrimaryKeyColumns));
@@ -260,7 +258,7 @@ public final class StatementRestrictions
 
     private void addRestriction(Restriction restriction)
     {
-        ColumnDefinition def = restriction.getFirstColumn();
+        ColumnMetadata def = restriction.getFirstColumn();
         if (def.isPartitionKey())
             partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
         else if (def.isClusteringColumn())
@@ -287,19 +285,19 @@ public final class StatementRestrictions
      * by an IS NOT NULL restriction will be included, otherwise they will not be included (unless another restriction
      * applies to them).
      */
-    public Set<ColumnDefinition> nonPKRestrictedColumns(boolean includeNotNullRestrictions)
+    public Set<ColumnMetadata> nonPKRestrictedColumns(boolean includeNotNullRestrictions)
     {
-        Set<ColumnDefinition> columns = new HashSet<>();
+        Set<ColumnMetadata> columns = new HashSet<>();
         for (Restrictions r : filterRestrictions.getRestrictions())
         {
-            for (ColumnDefinition def : r.getColumnDefs())
+            for (ColumnMetadata def : r.getColumnDefs())
                 if (!def.isPrimaryKeyColumn())
                     columns.add(def);
         }
 
         if (includeNotNullRestrictions)
         {
-            for (ColumnDefinition def : notNullColumns)
+            for (ColumnMetadata def : notNullColumns)
             {
                 if (!def.isPrimaryKeyColumn())
                     columns.add(def);
@@ -312,7 +310,7 @@ public final class StatementRestrictions
     /**
      * @return the set of columns that have an IS NOT NULL restriction on them
      */
-    public Set<ColumnDefinition> notNullColumns()
+    public Set<ColumnMetadata> notNullColumns()
     {
         return notNullColumns;
     }
@@ -320,7 +318,7 @@ public final class StatementRestrictions
     /**
      * @return true if column is restricted by some restriction, false otherwise
      */
-    public boolean isRestricted(ColumnDefinition column)
+    public boolean isRestricted(ColumnMetadata column)
     {
         if (notNullColumns.contains(column))
             return true;
@@ -356,7 +354,7 @@ public final class StatementRestrictions
      * @return <code>true</code> if the specified column is restricted by an EQ restiction, <code>false</code>
      * otherwise.
      */
-    public boolean isColumnRestrictedByEq(ColumnDefinition columnDef)
+    public boolean isColumnRestrictedByEq(ColumnMetadata columnDef)
     {
         Set<Restriction> restrictions = getRestrictions(columnDef.kind).getRestrictions(columnDef);
         return restrictions.stream()
@@ -370,7 +368,7 @@ public final class StatementRestrictions
      * @param kind the column type
      * @return the <code>Restrictions</code> for the specified type of columns
      */
-    private Restrictions getRestrictions(ColumnDefinition.Kind kind)
+    private Restrictions getRestrictions(ColumnMetadata.Kind kind)
     {
         switch (kind)
         {
@@ -397,7 +395,7 @@ public final class StatementRestrictions
             checkFalse(partitionKeyRestrictions.isOnToken(),
                        "The token function cannot be used in WHERE clauses for %s statements", type);
 
-            if (partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(cfm))
+            if (partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(table))
                 throw invalidRequest("Some partition key parts are missing: %s",
                                      Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents()));
 
@@ -412,7 +410,7 @@ public final class StatementRestrictions
             if (partitionKeyRestrictions.isOnToken())
                 isKeyRange = true;
 
-            if (partitionKeyRestrictions.isEmpty() && partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(cfm))
+            if (partitionKeyRestrictions.isEmpty() && partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(table))
             {
                 isKeyRange = true;
                 usesSecondaryIndexing = hasQueriableIndex;
@@ -424,10 +422,10 @@ public final class StatementRestrictions
             // - Is it queriable without 2ndary index, which is always more efficient
             // If a component of the partition key is restricted by a relation, all preceding
             // components must have a EQ. Only the last partition key component can be in IN relation.
-            if (partitionKeyRestrictions.needFiltering(cfm))
+            if (partitionKeyRestrictions.needFiltering(table))
             {
                 if (!allowFiltering && !forView && !hasQueriableIndex
-                    && (partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(cfm) || partitionKeyRestrictions.hasSlice()))
+                    && (partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(table) || partitionKeyRestrictions.hasSlice()))
                     throw new InvalidRequestException(REQUIRES_ALLOW_FILTERING_MESSAGE);
 
                 if (partitionKeyRestrictions.hasIN())
@@ -459,9 +457,9 @@ public final class StatementRestrictions
      */
     private Collection<ColumnIdentifier> getPartitionKeyUnrestrictedComponents()
     {
-        List<ColumnDefinition> list = new ArrayList<>(cfm.partitionKeyColumns());
+        List<ColumnMetadata> list = new ArrayList<>(table.partitionKeyColumns());
         list.removeAll(partitionKeyRestrictions.getColumnDefs());
-        return ColumnDefinition.toIdentifiers(list);
+        return ColumnMetadata.toIdentifiers(list);
     }
 
     /**
@@ -502,7 +500,7 @@ public final class StatementRestrictions
                    "Slice restrictions are not supported on the clustering columns in %s statements", type);
 
         if (!type.allowClusteringColumnSlices()
-               && (!cfm.isCompactTable() || (cfm.isCompactTable() && !hasClusteringColumnsRestrictions())))
+               && (!table.isCompactTable() || (table.isCompactTable() && !hasClusteringColumnsRestrictions())))
         {
             if (!selectsOnlyStaticColumns && hasUnrestrictedClusteringColumns())
                 throw invalidRequest("Some clustering keys are missing: %s",
@@ -522,13 +520,13 @@ public final class StatementRestrictions
                 }
                 else if (!allowFiltering)
                 {
-                    List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
-                    List<ColumnDefinition> restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs());
+                    List<ColumnMetadata> clusteringColumns = table.clusteringColumns();
+                    List<ColumnMetadata> restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs());
 
                     for (int i = 0, m = restrictedColumns.size(); i < m; i++)
                     {
-                        ColumnDefinition clusteringColumn = clusteringColumns.get(i);
-                        ColumnDefinition restrictedColumn = restrictedColumns.get(i);
+                        ColumnMetadata clusteringColumn = clusteringColumns.get(i);
+                        ColumnMetadata restrictedColumn = restrictedColumns.get(i);
 
                         if (!clusteringColumn.equals(restrictedColumn))
                         {
@@ -550,9 +548,9 @@ public final class StatementRestrictions
      */
     private Collection<ColumnIdentifier> getUnrestrictedClusteringColumns()
     {
-        List<ColumnDefinition> missingClusteringColumns = new ArrayList<>(cfm.clusteringColumns());
+        List<ColumnMetadata> missingClusteringColumns = new ArrayList<>(table.clusteringColumns());
         missingClusteringColumns.removeAll(new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs()));
-        return ColumnDefinition.toIdentifiers(missingClusteringColumns);
+        return ColumnMetadata.toIdentifiers(missingClusteringColumns);
     }
 
     /**
@@ -561,7 +559,7 @@ public final class StatementRestrictions
      */
     private boolean hasUnrestrictedClusteringColumns()
     {
-        return cfm.clusteringColumns().size() != clusteringColumnsRestrictions.size();
+        return table.clusteringColumns().size() != clusteringColumnsRestrictions.size();
     }
 
     private void processCustomIndexExpressions(List<CustomIndexExpression> expressions,
@@ -575,16 +573,16 @@ public final class StatementRestrictions
 
         CFName cfName = expression.targetIndex.getCfName();
         if (cfName.hasKeyspace()
-            && !expression.targetIndex.getKeyspace().equals(cfm.ksName))
-            throw IndexRestrictions.invalidIndex(expression.targetIndex, cfm);
+            && !expression.targetIndex.getKeyspace().equals(table.keyspace))
+            throw IndexRestrictions.invalidIndex(expression.targetIndex, table);
 
-        if (cfName.getColumnFamily() != null && !cfName.getColumnFamily().equals(cfm.cfName))
-            throw IndexRestrictions.invalidIndex(expression.targetIndex, cfm);
+        if (cfName.getColumnFamily() != null && !cfName.getColumnFamily().equals(table.name))
+            throw IndexRestrictions.invalidIndex(expression.targetIndex, table);
 
-        if (!cfm.getIndexes().has(expression.targetIndex.getIdx()))
-            throw IndexRestrictions.indexNotFound(expression.targetIndex, cfm);
+        if (!table.indexes.has(expression.targetIndex.getIdx()))
+            throw IndexRestrictions.indexNotFound(expression.targetIndex, table);
 
-        Index index = indexManager.getIndex(cfm.getIndexes().get(expression.targetIndex.getIdx()).get());
+        Index index = indexManager.getIndex(table.indexes.get(expression.targetIndex.getIdx()).get());
 
         if (!index.getIndexMetadata().isCustom())
             throw IndexRestrictions.nonCustomIndexInExpression(expression.targetIndex);
@@ -593,7 +591,7 @@ public final class StatementRestrictions
         if (expressionType == null)
             throw IndexRestrictions.customExpressionNotSupported(expression.targetIndex);
 
-        expression.prepareValue(cfm, expressionType, boundNames);
+        expression.prepareValue(table, expressionType, boundNames);
 
         filterRestrictions.add(expression);
     }
@@ -608,7 +606,7 @@ public final class StatementRestrictions
             restrictions.addRowFilterTo(filter, indexManager, options);
 
         for (CustomIndexExpression expression : filterRestrictions.getCustomIndexExpressions())
-            expression.addToRowFilter(filter, cfm, options);
+            expression.addToRowFilter(filter, table, options);
 
         return filter;
     }
@@ -645,7 +643,7 @@ public final class StatementRestrictions
      */
     public AbstractBounds<PartitionPosition> getPartitionKeyBounds(QueryOptions options)
     {
-        IPartitioner p = cfm.partitioner;
+        IPartitioner p = table.partitioner;
 
         if (partitionKeyRestrictions.isOnToken())
         {
@@ -660,7 +658,7 @@ public final class StatementRestrictions
     {
         // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the
         // first component of a composite partition key) queries that filter on the partition key.
-        if (partitionKeyRestrictions.needFiltering(cfm))
+        if (partitionKeyRestrictions.needFiltering(table))
             return new Range<>(p.getMinimumToken().minKeyBound(), p.getMinimumToken().maxKeyBound());
 
         ByteBuffer startKeyBytes = getPartitionKeyBound(Bound.START, options);
@@ -746,8 +744,8 @@ public final class StatementRestrictions
         // If this is a names command and the table is a static compact one, then as far as CQL is concerned we have
         // only a single row which internally correspond to the static parts. In which case we want to return an empty
         // set (since that's what ClusteringIndexNamesFilter expects).
-        if (cfm.isStaticCompactTable())
-            return BTreeSet.empty(cfm.comparator);
+        if (table.isStaticCompactTable())
+            return BTreeSet.empty(table.comparator);
 
         return clusteringColumnsRestrictions.valuesAsClustering(options);
     }
@@ -774,7 +772,7 @@ public final class StatementRestrictions
         // For static compact tables we want to ignore the fake clustering column (note that if we weren't special casing,
         // this would mean a 'SELECT *' on a static compact table would query whole partitions, even though we'll only return
         // the static part as far as CQL is concerned. This is thus mostly an optimization to use the query-by-name path).
-        int numberOfClusteringColumns = cfm.isStaticCompactTable() ? 0 : cfm.clusteringColumns().size();
+        int numberOfClusteringColumns = table.isStaticCompactTable() ? 0 : table.clusteringColumns().size();
         // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ or IN.
         return clusteringColumnsRestrictions.size() < numberOfClusteringColumns
             || !clusteringColumnsRestrictions.hasOnlyEqualityRestrictions();
@@ -816,7 +814,7 @@ public final class StatementRestrictions
     public boolean hasAllPKColumnsRestrictedByEqualities()
     {
         return !isPartitionKeyRestrictionsOnToken()
-                && !partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(cfm)
+                && !partitionKeyRestrictions.hasUnrestrictedPartitionKeyComponents(table)
                 && (partitionKeyRestrictions.hasOnlyEqualityRestrictions())
                 && !hasUnrestrictedClusteringColumns()
                 && (clusteringColumnsRestrictions.hasOnlyEqualityRestrictions());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java b/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java
index d6543b8..100fcef 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.cql3.restrictions;
 
 import java.util.List;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.cql3.functions.Function;
@@ -155,7 +155,7 @@ final class TermSlice
      * @return <code>true</code> this type of <code>TermSlice</code> is supported by the specified index,
      * <code>false</code> otherwise.
      */
-    public boolean isSupportedBy(ColumnDefinition column, Index index)
+    public boolean isSupportedBy(ColumnMetadata column, Index index)
     {
         boolean supported = false;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java b/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java
index 400706b..b2d8509 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java
@@ -25,8 +25,8 @@ import com.google.common.collect.ImmutableRangeSet;
 import com.google.common.collect.Range;
 import com.google.common.collect.RangeSet;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.statements.Bound;
@@ -73,7 +73,7 @@ final class TokenFilter implements PartitionKeyRestrictions
     }
 
     @Override
-    public Set<Restriction> getRestrictions(ColumnDefinition columnDef)
+    public Set<Restriction> getRestrictions(ColumnMetadata columnDef)
     {
         Set<Restriction> set = new HashSet<>();
         set.addAll(restrictions.getRestrictions(columnDef));
@@ -243,19 +243,19 @@ final class TokenFilter implements PartitionKeyRestrictions
     }
 
     @Override
-    public ColumnDefinition getFirstColumn()
+    public ColumnMetadata getFirstColumn()
     {
         return restrictions.getFirstColumn();
     }
 
     @Override
-    public ColumnDefinition getLastColumn()
+    public ColumnMetadata getLastColumn()
     {
         return restrictions.getLastColumn();
     }
 
     @Override
-    public List<ColumnDefinition> getColumnDefs()
+    public List<ColumnMetadata> getColumnDefs()
     {
         return restrictions.getColumnDefs();
     }
@@ -291,15 +291,15 @@ final class TokenFilter implements PartitionKeyRestrictions
     }
 
     @Override
-    public boolean needFiltering(CFMetaData cfm)
+    public boolean needFiltering(TableMetadata table)
     {
-        return restrictions.needFiltering(cfm);
+        return restrictions.needFiltering(table);
     }
 
     @Override
-    public boolean hasUnrestrictedPartitionKeyComponents(CFMetaData cfm)
+    public boolean hasUnrestrictedPartitionKeyComponents(TableMetadata table)
     {
-        return restrictions.hasUnrestrictedPartitionKeyComponents(cfm);
+        return restrictions.hasUnrestrictedPartitionKeyComponents(table);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
index e90319d..7f4b9ff 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
@@ -22,8 +22,8 @@ import java.util.*;
 
 import com.google.common.base.Joiner;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.cql3.functions.Function;
@@ -42,16 +42,16 @@ public abstract class TokenRestriction implements PartitionKeyRestrictions
     /**
      * The definition of the columns to which apply the token restriction.
      */
-    protected final List<ColumnDefinition> columnDefs;
+    protected final List<ColumnMetadata> columnDefs;
 
-    protected final CFMetaData metadata;
+    protected final TableMetadata metadata;
 
     /**
      * Creates a new <code>TokenRestriction</code> that apply to the specified columns.
      *
      * @param columnDefs the definition of the columns to which apply the token restriction
      */
-    public TokenRestriction(CFMetaData metadata, List<ColumnDefinition> columnDefs)
+    public TokenRestriction(TableMetadata metadata, List<ColumnMetadata> columnDefs)
     {
         this.columnDefs = columnDefs;
         this.metadata = metadata;
@@ -68,7 +68,7 @@ public abstract class TokenRestriction implements PartitionKeyRestrictions
     }
 
     @Override
-    public Set<Restriction> getRestrictions(ColumnDefinition columnDef)
+    public Set<Restriction> getRestrictions(ColumnMetadata columnDef)
     {
         return Collections.singleton(this);
     }
@@ -80,7 +80,7 @@ public abstract class TokenRestriction implements PartitionKeyRestrictions
     }
 
     @Override
-    public boolean needFiltering(CFMetaData cfm)
+    public boolean needFiltering(TableMetadata table)
     {
         return false;
     }
@@ -92,25 +92,25 @@ public abstract class TokenRestriction implements PartitionKeyRestrictions
     }
 
     @Override
-    public boolean hasUnrestrictedPartitionKeyComponents(CFMetaData cfm)
+    public boolean hasUnrestrictedPartitionKeyComponents(TableMetadata table)
     {
         return false;
     }
 
     @Override
-    public List<ColumnDefinition> getColumnDefs()
+    public List<ColumnMetadata> getColumnDefs()
     {
         return columnDefs;
     }
 
     @Override
-    public ColumnDefinition getFirstColumn()
+    public ColumnMetadata getFirstColumn()
     {
         return columnDefs.get(0);
     }
 
     @Override
-    public ColumnDefinition getLastColumn()
+    public ColumnMetadata getLastColumn()
     {
         return columnDefs.get(columnDefs.size() - 1);
     }
@@ -146,7 +146,7 @@ public abstract class TokenRestriction implements PartitionKeyRestrictions
      */
     protected final String getColumnNamesAsString()
     {
-        return Joiner.on(", ").join(ColumnDefinition.toIdentifiers(columnDefs));
+        return Joiner.on(", ").join(ColumnMetadata.toIdentifiers(columnDefs));
     }
 
     @Override
@@ -176,16 +176,16 @@ public abstract class TokenRestriction implements PartitionKeyRestrictions
         if (restriction instanceof PartitionKeyRestrictions)
             return (PartitionKeyRestrictions) restriction;
 
-        return new PartitionKeySingleRestrictionSet(metadata.getKeyValidatorAsClusteringComparator()).mergeWith(restriction);
+        return new PartitionKeySingleRestrictionSet(metadata.partitionKeyAsClusteringComparator()).mergeWith(restriction);
     }
 
     public static final class EQRestriction extends TokenRestriction
     {
         private final Term value;
 
-        public EQRestriction(CFMetaData cfm, List<ColumnDefinition> columnDefs, Term value)
+        public EQRestriction(TableMetadata table, List<ColumnMetadata> columnDefs, Term value)
         {
-            super(cfm, columnDefs);
+            super(table, columnDefs);
             this.value = value;
         }
 
@@ -199,7 +199,7 @@ public abstract class TokenRestriction implements PartitionKeyRestrictions
         protected PartitionKeyRestrictions doMergeWith(TokenRestriction otherRestriction) throws InvalidRequestException
         {
             throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal",
-                                 Joiner.on(", ").join(ColumnDefinition.toIdentifiers(columnDefs)));
+                                 Joiner.on(", ").join(ColumnMetadata.toIdentifiers(columnDefs)));
         }
 
         @Override
@@ -231,9 +231,9 @@ public abstract class TokenRestriction implements PartitionKeyRestrictions
     {
         private final TermSlice slice;
 
-        public SliceRestriction(CFMetaData cfm, List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
+        public SliceRestriction(TableMetadata table, List<ColumnMetadata> columnDefs, Bound bound, boolean inclusive, Term term)
         {
-            super(cfm, columnDefs);
+            super(table, columnDefs);
             slice = TermSlice.newInstance(bound, inclusive, term);
         }
 
@@ -291,7 +291,7 @@ public abstract class TokenRestriction implements PartitionKeyRestrictions
                 throw invalidRequest("More than one restriction was found for the end bound on %s",
                                      getColumnNamesAsString());
 
-            return new SliceRestriction(metadata, columnDefs,  slice.merge(otherSlice.slice));
+            return new SliceRestriction(metadata, columnDefs, slice.merge(otherSlice.slice));
         }
 
         @Override
@@ -299,9 +299,9 @@ public abstract class TokenRestriction implements PartitionKeyRestrictions
         {
             return String.format("SLICE%s", slice);
         }
-        private SliceRestriction(CFMetaData cfm, List<ColumnDefinition> columnDefs, TermSlice slice)
+        private SliceRestriction(TableMetadata table, List<ColumnMetadata> columnDefs, TermSlice slice)
         {
-            super(cfm, columnDefs);
+            super(table, columnDefs);
             this.slice = slice;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
index 498cf0f..115729a 100644
--- a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
@@ -22,7 +22,7 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.lang3.text.StrBuilder;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
@@ -70,7 +70,7 @@ abstract class AbstractFunctionSelector<T extends Function> extends Selector
                 if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
                     // add a null mapping for cases where there are no
                     // further selectors, such as no-arg functions and count
-                    mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+                    mapping.addMapping(resultsColumn, (ColumnMetadata)null);
                 else
                     // collate the mapped columns from the child factories & add those
                     mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java b/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
index de5fd93..88885dd 100644
--- a/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
+++ b/src/java/org/apache/cassandra/cql3/selection/CollectionFactory.java
@@ -19,11 +19,11 @@ package org.apache.cassandra.cql3.selection;
 
 import java.util.List;
 
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.selection.Selector.Factory;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.ColumnMetadata;
 
 /**
  * A base <code>Selector.Factory</code> for collections or tuples.
@@ -83,7 +83,7 @@ abstract class CollectionFactory extends Factory
 
         if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
             // add a null mapping for cases where the collection is empty
-            mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+            mapping.addMapping(resultsColumn, (ColumnMetadata)null);
         else
             // collate the mapped columns from the child factories & add those
             mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/ListSelector.java b/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
index 940bd9c..b540ec9 100644
--- a/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/ListSelector.java
@@ -21,8 +21,6 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.Lists;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/MapSelector.java b/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
index 8bbae8c..97dfc30 100644
--- a/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/MapSelector.java
@@ -24,7 +24,6 @@ import java.util.Map;
 import java.util.TreeMap;
 import java.util.stream.Collectors;
 
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.Maps;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -33,6 +32,7 @@ import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.serializers.CollectionSerializer;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.Pair;
@@ -78,7 +78,7 @@ final class MapSelector extends Selector
 
                 if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
                     // add a null mapping for cases where the collection is empty
-                    mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+                    mapping.addMapping(resultsColumn, (ColumnMetadata)null);
                 else
                     // collate the mapped columns from the child factories & add those
                     mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/RawSelector.java b/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
index 7d5543f..fbf7c30 100644
--- a/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
@@ -20,11 +20,11 @@ package org.apache.cassandra.cql3.selection;
 
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 
-import com.google.common.base.Function;
-import com.google.common.collect.Lists;
 
 public class RawSelector
 {
@@ -43,15 +43,9 @@ public class RawSelector
      * @param raws the <code>RawSelector</code>s to converts.
      * @return a list of <code>Selectable</code>s
      */
-    public static List<Selectable> toSelectables(List<RawSelector> raws, final CFMetaData cfm)
+    public static List<Selectable> toSelectables(List<RawSelector> raws, final TableMetadata table)
     {
-        return Lists.transform(raws, new Function<RawSelector, Selectable>()
-        {
-            public Selectable apply(RawSelector raw)
-            {
-                return raw.selectable.prepare(cfm);
-            }
-        });
+        return Lists.transform(raws, raw -> raw.selectable.prepare(table));
     }
 
     public boolean processesSelection()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/Selectable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
index 20719ea..b2526a5 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selectable.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@ -23,13 +23,13 @@ import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.text.StrBuilder;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.cql3.selection.Selector.Factory;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.Pair;
 
 import static org.apache.cassandra.cql3.selection.SelectorFactories.createFactoriesAndCollectColumnDefinitions;
@@ -37,7 +37,7 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.invalidReq
 
 public interface Selectable extends AssignmentTestable
 {
-    public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames);
+    public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType<?> expectedType, List<ColumnMetadata> defs, VariableSpecifications boundNames);
 
     /**
      * The type of the {@code Selectable} if it can be infered.
@@ -58,7 +58,7 @@ public interface Selectable extends AssignmentTestable
         return type == null ? TestResult.NOT_ASSIGNABLE : type.testAssignment(keyspace, receiver);
     }
 
-    default int addAndGetIndex(ColumnDefinition def, List<ColumnDefinition> l)
+    default int addAndGetIndex(ColumnMetadata def, List<ColumnMetadata> l)
     {
         int idx = l.indexOf(def);
         if (idx < 0)
@@ -71,7 +71,7 @@ public interface Selectable extends AssignmentTestable
 
     public static abstract class Raw
     {
-        public abstract Selectable prepare(CFMetaData cfm);
+        public abstract Selectable prepare(TableMetadata table);
 
         /**
          * Returns true if any processing is performed on the selected column.
@@ -111,7 +111,7 @@ public interface Selectable extends AssignmentTestable
             return rawTerm.testAssignment(keyspace, receiver);
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames) throws InvalidRequestException
+        public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType<?> expectedType, List<ColumnMetadata> defs, VariableSpecifications boundNames) throws InvalidRequestException
         {
             /*
              * expectedType will be null if we have no constraint on what the type should be. For instance, if this term is a bind marker:
@@ -133,7 +133,7 @@ public interface Selectable extends AssignmentTestable
              * Lastly, note that if the term is a terminal literal, we don't have to check it's compatibility with 'expectedType' as any incompatibility
              * would have been found at preparation time.
              */
-            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            AbstractType<?> type = getExactTypeIfKnown(table.keyspace);
             if (type == null)
             {
                 type = expectedType;
@@ -145,7 +145,7 @@ public interface Selectable extends AssignmentTestable
             // selection will have this name. Which isn't terribly helpful, but it's unclear how to provide
             // something a lot more helpful and in practice user can bind those markers by position or, even better,
             // use bind markers.
-            Term term = rawTerm.prepare(cfm.ksName, new ColumnSpecification(cfm.ksName, cfm.cfName, bindMarkerNameInSelection, type));
+            Term term = rawTerm.prepare(table.keyspace, new ColumnSpecification(table.keyspace, table.name, bindMarkerNameInSelection, type));
             term.collectMarkerSpecification(boundNames);
             return TermSelector.newFactory(rawTerm.getText(), term, type);
         }
@@ -171,7 +171,7 @@ public interface Selectable extends AssignmentTestable
                 this.term = term;
             }
 
-            public Selectable prepare(CFMetaData cfm)
+            public Selectable prepare(TableMetadata table)
             {
                 return new WithTerm(term);
             }
@@ -180,10 +180,10 @@ public interface Selectable extends AssignmentTestable
 
     public static class WritetimeOrTTL implements Selectable
     {
-        public final ColumnDefinition column;
+        public final ColumnMetadata column;
         public final boolean isWritetime;
 
-        public WritetimeOrTTL(ColumnDefinition column, boolean isWritetime)
+        public WritetimeOrTTL(ColumnMetadata column, boolean isWritetime)
         {
             this.column = column;
             this.isWritetime = isWritetime;
@@ -195,9 +195,9 @@ public interface Selectable extends AssignmentTestable
             return (isWritetime ? "writetime" : "ttl") + "(" + column.name + ")";
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm,
+        public Selector.Factory newSelectorFactory(TableMetadata table,
                                                    AbstractType<?> expectedType,
-                                                   List<ColumnDefinition> defs,
+                                                   List<ColumnMetadata> defs,
                                                    VariableSpecifications boundNames)
         {
             if (column.isPrimaryKeyColumn())
@@ -219,18 +219,18 @@ public interface Selectable extends AssignmentTestable
 
         public static class Raw extends Selectable.Raw
         {
-            private final ColumnDefinition.Raw id;
+            private final ColumnMetadata.Raw id;
             private final boolean isWritetime;
 
-            public Raw(ColumnDefinition.Raw id, boolean isWritetime)
+            public Raw(ColumnMetadata.Raw id, boolean isWritetime)
             {
                 this.id = id;
                 this.isWritetime = isWritetime;
             }
 
-            public WritetimeOrTTL prepare(CFMetaData cfm)
+            public WritetimeOrTTL prepare(TableMetadata table)
             {
-                return new WritetimeOrTTL(id.prepare(cfm), isWritetime);
+                return new WritetimeOrTTL(id.prepare(table), isWritetime);
             }
         }
     }
@@ -252,9 +252,9 @@ public interface Selectable extends AssignmentTestable
             return function.columnName(args.stream().map(Object::toString).collect(Collectors.toList()));
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
+        public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType<?> expectedType, List<ColumnMetadata> defs, VariableSpecifications boundNames)
         {
-            SelectorFactories factories = SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, function.argTypes(), cfm, defs, boundNames);
+            SelectorFactories factories = SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, function.argTypes(), table, defs, boundNames);
             return AbstractFunctionSelector.newFactory(function, factories);
         }
 
@@ -292,11 +292,11 @@ public interface Selectable extends AssignmentTestable
                                Collections.singletonList(arg));
             }
 
-            public Selectable prepare(CFMetaData cfm)
+            public Selectable prepare(TableMetadata table)
             {
                 List<Selectable> preparedArgs = new ArrayList<>(args.size());
                 for (Selectable.Raw arg : args)
-                    preparedArgs.add(arg.prepare(cfm));
+                    preparedArgs.add(arg.prepare(table));
 
                 FunctionName name = functionName;
                 // We need to circumvent the normal function lookup process for toJson() because instances of the function
@@ -319,7 +319,7 @@ public interface Selectable extends AssignmentTestable
                     preparedArgs = Collections.emptyList();
                 }
 
-                Function fun = FunctionResolver.get(cfm.ksName, name, preparedArgs, cfm.ksName, cfm.cfName, null);
+                Function fun = FunctionResolver.get(table.keyspace, name, preparedArgs, table.keyspace, table.name, null);
 
                 if (fun == null)
                     throw new InvalidRequestException(String.format("Unknown function '%s'", functionName));
@@ -351,9 +351,9 @@ public interface Selectable extends AssignmentTestable
                                    .toString();
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
+        public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType<?> expectedType, List<ColumnMetadata> defs, VariableSpecifications boundNames)
         {
-            SelectorFactories factories = SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, null, cfm, defs, boundNames);
+            SelectorFactories factories = SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, null, table, defs, boundNames);
             Function fun = ToJsonFct.getInstance(factories.getReturnTypes());
             return AbstractFunctionSelector.newFactory(fun, factories);
         }
@@ -381,10 +381,10 @@ public interface Selectable extends AssignmentTestable
             return String.format("cast(%s as %s)", arg, type.toString().toLowerCase());
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
+        public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType<?> expectedType, List<ColumnMetadata> defs, VariableSpecifications boundNames)
         {
             List<Selectable> args = Collections.singletonList(arg);
-            SelectorFactories factories = SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, null, cfm, defs, boundNames);
+            SelectorFactories factories = SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, null, table, defs, boundNames);
 
             Selector.Factory factory = factories.get(0);
 
@@ -393,7 +393,7 @@ public interface Selectable extends AssignmentTestable
                 return factory;
 
             FunctionName name = FunctionName.nativeFunction(CastFcts.getFunctionName(type));
-            Function fun = FunctionResolver.get(cfm.ksName, name, args, cfm.ksName, cfm.cfName, null);
+            Function fun = FunctionResolver.get(table.keyspace, name, args, table.keyspace, table.name, null);
 
             if (fun == null)
             {
@@ -420,9 +420,9 @@ public interface Selectable extends AssignmentTestable
                 this.type = type;
             }
 
-            public WithCast prepare(CFMetaData cfm)
+            public WithCast prepare(TableMetadata table)
             {
-                return new WithCast(arg.prepare(cfm), type);
+                return new WithCast(arg.prepare(table), type);
             }
         }
     }
@@ -444,7 +444,7 @@ public interface Selectable extends AssignmentTestable
             return String.format("%s.%s", selected, field);
         }
 
-        public Selector.Factory newSelectorFactory(CFMetaData cfm, AbstractType<?> expectedType, List<ColumnDefinition> defs, VariableSpecifications boundNames)
+        public Selector.Factory newSelectorFactory(TableMetadata table, AbstractType<?> expectedType, List<ColumnMetadata> defs, VariableSpecifications boundNames)
         {
             AbstractType<?> expectedUdtType = null;
 
@@ -452,11 +452,11 @@ public interface Selectable extends AssignmentTestable
             if (selected instanceof BetweenParenthesesOrWithTuple)
             {
                 BetweenParenthesesOrWithTuple betweenParentheses = (BetweenParenthesesOrWithTuple) selected;
-                expectedUdtType = betweenParentheses.selectables.get(0).getExactTypeIfKnown(cfm.ksName);
+                expectedUdtType = betweenParentheses.selectables.get(0).getExactTypeIfKnown(table.keyspace);
             }
 
-            Selector.Factory factory = selected.newSelectorFactory(cfm, expectedUdtType, defs, boundNames);
-            AbstractType<?> type = factory.getColumnSpecification(cfm).type;
+            Selector.Factory factory = selected.newSelectorFactory(table, expectedUdtType, defs, boundNames);
+            AbstractType<?> type = factory.getColumnSpecification(table).type;
             if (!type.isUDT())
             {
                 throw new InvalidRequestException(
@@ -501,9 +501,9 @@ public interface Selectable extends AssignmentTestable
                 this.field = field;
             }
 
-            public WithFieldSelection prepare(CFMetaData cfm)
+            public WithFieldSelection prepare(TableMetadata table)
             {
-                return new WithFieldSelection(selected.prepare(cfm), field);
+                return new WithFieldSelection(selected.prepare(table), field);
             }
         }
     }
@@ -535,12 +535,12 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
-        public Factory newSelectorFactory(CFMetaData cfm,
+        public Factory newSelectorFactory(TableMetadata cfm,
                                           AbstractType<?> expectedType,
-                                          List<ColumnDefinition> defs,
+                                          List<ColumnMetadata> defs,
                                           VariableSpecifications boundNames)
         {
-            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            AbstractType<?> type = getExactTypeIfKnown(cfm.keyspace);
             if (type == null)
             {
                 type = expectedType;
@@ -555,9 +555,9 @@ public interface Selectable extends AssignmentTestable
             return newTupleSelectorFactory(cfm, (TupleType) type, defs, boundNames);
         }
 
-        private Factory newBetweenParenthesesSelectorFactory(CFMetaData cfm,
+        private Factory newBetweenParenthesesSelectorFactory(TableMetadata cfm,
                                                              AbstractType<?> expectedType,
-                                                             List<ColumnDefinition> defs,
+                                                             List<ColumnMetadata> defs,
                                                              VariableSpecifications boundNames)
         {
             Selectable selectable = selectables.get(0);
@@ -577,9 +577,9 @@ public interface Selectable extends AssignmentTestable
             };
         }
 
-        private Factory newTupleSelectorFactory(CFMetaData cfm,
+        private Factory newTupleSelectorFactory(TableMetadata cfm,
                                                 TupleType tupleType,
-                                                List<ColumnDefinition> defs,
+                                                List<ColumnMetadata> defs,
                                                 VariableSpecifications boundNames)
         {
             SelectorFactories factories = createFactoriesAndCollectColumnDefinitions(selectables,
@@ -617,7 +617,7 @@ public interface Selectable extends AssignmentTestable
                 this.raws = raws;
             }
 
-            public Selectable prepare(CFMetaData cfm)
+            public Selectable prepare(TableMetadata cfm)
             {
                 return new BetweenParenthesesOrWithTuple(raws.stream().map(p -> p.prepare(cfm)).collect(Collectors.toList()));
             }
@@ -646,12 +646,12 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
-        public Factory newSelectorFactory(CFMetaData cfm,
+        public Factory newSelectorFactory(TableMetadata cfm,
                                           AbstractType<?> expectedType,
-                                          List<ColumnDefinition> defs,
+                                          List<ColumnMetadata> defs,
                                           VariableSpecifications boundNames)
         {
-            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            AbstractType<?> type = getExactTypeIfKnown(cfm.keyspace);
             if (type == null)
             {
                 type = expectedType;
@@ -695,7 +695,7 @@ public interface Selectable extends AssignmentTestable
                 this.raws = raws;
             }
 
-            public Selectable prepare(CFMetaData cfm)
+            public Selectable prepare(TableMetadata cfm)
             {
                 return new WithList(raws.stream().map(p -> p.prepare(cfm)).collect(Collectors.toList()));
             }
@@ -724,12 +724,12 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
-        public Factory newSelectorFactory(CFMetaData cfm,
+        public Factory newSelectorFactory(TableMetadata cfm,
                                           AbstractType<?> expectedType,
-                                          List<ColumnDefinition> defs,
+                                          List<ColumnMetadata> defs,
                                           VariableSpecifications boundNames)
         {
-            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            AbstractType<?> type = getExactTypeIfKnown(cfm.keyspace);
             if (type == null)
             {
                 type = expectedType;
@@ -781,7 +781,7 @@ public interface Selectable extends AssignmentTestable
                 this.raws = raws;
             }
 
-            public Selectable prepare(CFMetaData cfm)
+            public Selectable prepare(TableMetadata cfm)
             {
                 return new WithSet(raws.stream().map(p -> p.prepare(cfm)).collect(Collectors.toList()));
             }
@@ -791,7 +791,7 @@ public interface Selectable extends AssignmentTestable
     /**
      * {@code Selectable} for literal Maps or UDTs.
      * <p>The parser cannot differentiate between a Map or a UDT in the selection cause because a
-     * {@code ColumnDefinition} is equivalent to a {@code FieldIdentifier} from a syntax point of view.
+     * {@code ColumnMetadata} is equivalent to a {@code FieldIdentifier} from a syntax point of view.
      * By consequence, we are forced to wait until the type is known to be able to differentiate them.</p>
      */
     public static class WithMapOrUdt implements Selectable
@@ -800,14 +800,14 @@ public interface Selectable extends AssignmentTestable
          * The column family metadata. We need to store them to be able to build the proper data once the type has been
          * identified.
          */
-        private final CFMetaData cfm;
+        private final TableMetadata cfm;
 
         /**
          * The Map or UDT raw elements.
          */
         private final List<Pair<Selectable.Raw, Selectable.Raw>> raws;
 
-        public WithMapOrUdt(CFMetaData cfm, List<Pair<Selectable.Raw, Selectable.Raw>> raws)
+        public WithMapOrUdt(TableMetadata cfm, List<Pair<Selectable.Raw, Selectable.Raw>> raws)
         {
             this.cfm = cfm;
             this.raws = raws;
@@ -821,12 +821,12 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
-        public Factory newSelectorFactory(CFMetaData cfm,
+        public Factory newSelectorFactory(TableMetadata cfm,
                                           AbstractType<?> expectedType,
-                                          List<ColumnDefinition> defs,
+                                          List<ColumnMetadata> defs,
                                           VariableSpecifications boundNames)
         {
-            AbstractType<?> type = getExactTypeIfKnown(cfm.ksName);
+            AbstractType<?> type = getExactTypeIfKnown(cfm.keyspace);
             if (type == null)
             {
                 type = expectedType;
@@ -841,8 +841,8 @@ public interface Selectable extends AssignmentTestable
             return newMapSelectorFactory(cfm, defs, boundNames, type);
         }
 
-        private Factory newMapSelectorFactory(CFMetaData cfm,
-                                              List<ColumnDefinition> defs,
+        private Factory newMapSelectorFactory(TableMetadata cfm,
+                                              List<ColumnMetadata> defs,
                                               VariableSpecifications boundNames,
                                               AbstractType<?> type)
         {
@@ -857,9 +857,9 @@ public interface Selectable extends AssignmentTestable
                                                                   .collect(Collectors.toList()));
         }
 
-        private Factory newUdtSelectorFactory(CFMetaData cfm,
+        private Factory newUdtSelectorFactory(TableMetadata cfm,
                                               AbstractType<?> expectedType,
-                                              List<ColumnDefinition> defs,
+                                              List<ColumnMetadata> defs,
                                               VariableSpecifications boundNames)
         {
             UserType ut = (UserType) expectedType;
@@ -905,7 +905,7 @@ public interface Selectable extends AssignmentTestable
                        .collect(Collectors.joining(", ", "{", "}"));
         }
 
-        private List<Pair<Selectable, Selectable>> getMapEntries(CFMetaData cfm)
+        private List<Pair<Selectable, Selectable>> getMapEntries(TableMetadata cfm)
         {
             return raws.stream()
                        .map(p -> Pair.create(p.left.prepare(cfm), p.right.prepare(cfm)))
@@ -946,7 +946,7 @@ public interface Selectable extends AssignmentTestable
                 this.raws = raws;
             }
 
-            public Selectable prepare(CFMetaData cfm)
+            public Selectable prepare(TableMetadata cfm)
             {
                 return new WithMapOrUdt(cfm, raws);
             }
@@ -993,14 +993,14 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
-        public Factory newSelectorFactory(CFMetaData cfm,
+        public Factory newSelectorFactory(TableMetadata cfm,
                                           AbstractType<?> expectedType,
-                                          List<ColumnDefinition> defs,
+                                          List<ColumnMetadata> defs,
                                           VariableSpecifications boundNames)
         {
-            final ColumnSpecification receiver = new ColumnSpecification(cfm.ksName, cfm.cfName, new ColumnIdentifier(toString(), true), type);
+            final ColumnSpecification receiver = new ColumnSpecification(cfm.keyspace, cfm.name, new ColumnIdentifier(toString(), true), type);
 
-            if (!selectable.testAssignment(cfm.ksName, receiver).isAssignable())
+            if (!selectable.testAssignment(cfm.keyspace, receiver).isAssignable())
                 throw new InvalidRequestException(String.format("Cannot assign value %s to %s of type %s", this, receiver.name, receiver.type.asCQL3Type()));
 
             final Factory factory = selectable.newSelectorFactory(cfm, type, defs, boundNames);
@@ -1048,10 +1048,10 @@ public interface Selectable extends AssignmentTestable
                 this.raw = raw;
             }
 
-            public Selectable prepare(CFMetaData cfm)
+            public Selectable prepare(TableMetadata cfm)
             {
                 Selectable selectable = raw.prepare(cfm);
-                AbstractType<?> type = this.typeRaw.prepare(cfm.ksName).getType();
+                AbstractType<?> type = this.typeRaw.prepare(cfm.keyspace).getType();
                 if (type.isFreezable())
                     type = type.freeze();
                 return new WithTypeHint(typeRaw.toString(), type, selectable);
@@ -1062,7 +1062,7 @@ public interface Selectable extends AssignmentTestable
     /**
      * In the selection clause, the parser cannot differentiate between Maps and UDTs as a column identifier and field
      * identifier have the same syntax. By consequence, we need to wait until the type is known to create the proper
-     * Object: {@code ColumnDefinition} or {@code FieldIdentifier}.
+     * Object: {@code ColumnMetadata} or {@code FieldIdentifier}.
      */
     public static final class RawIdentifier extends Selectable.Raw
     {
@@ -1093,10 +1093,10 @@ public interface Selectable extends AssignmentTestable
         }
 
         @Override
-        public Selectable prepare(CFMetaData cfm)
+        public Selectable prepare(TableMetadata cfm)
         {
-            ColumnDefinition.Raw raw = quoted ? ColumnDefinition.Raw.forQuoted(text)
-                                              : ColumnDefinition.Raw.forUnquoted(text);
+            ColumnMetadata.Raw raw = quoted ? ColumnMetadata.Raw.forQuoted(text)
+                                            : ColumnMetadata.Raw.forUnquoted(text);
             return raw.prepare(cfm);
         }
 


[12/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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
deleted file mode 100644
index b1249a6..0000000
--- a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
+++ /dev/null
@@ -1,193 +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.util.*;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.db.rows.UnfilteredRowIterators;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.schema.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class CFMetaDataTest
-{
-    private static final String KEYSPACE1 = "CFMetaDataTest1";
-    private static final String CF_STANDARD1 = "Standard1";
-
-    @BeforeClass
-    public static void defineSchema() throws ConfigurationException
-    {
-        SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
-    }
-
-    @Test
-    public void testConversionsInverses() throws Exception
-    {
-        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
-        {
-            for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
-            {
-                CFMetaData cfm = cfs.metadata;
-                checkInverses(cfm);
-
-                // Testing with compression to catch #3558
-                CFMetaData withCompression = cfm.copy();
-                withCompression.compression(CompressionParams.snappy(32768));
-                checkInverses(withCompression);
-            }
-        }
-    }
-
-    private void checkInverses(CFMetaData cfm) throws Exception
-    {
-        KeyspaceMetadata keyspace = Schema.instance.getKSMetaData(cfm.ksName);
-
-        // Test schema conversion
-        Mutation rm = SchemaKeyspace.makeCreateTableMutation(keyspace, cfm, FBUtilities.timestampMicros()).build();
-        PartitionUpdate cfU = rm.getPartitionUpdate(Schema.instance.getId(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.TABLES));
-        PartitionUpdate cdU = rm.getPartitionUpdate(Schema.instance.getId(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.COLUMNS));
-
-        UntypedResultSet.Row tableRow = QueryProcessor.resultify(String.format("SELECT * FROM %s.%s", SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.TABLES),
-                                                                 UnfilteredRowIterators.filter(cfU.unfilteredIterator(), FBUtilities.nowInSeconds()))
-                                                      .one();
-        TableParams params = SchemaKeyspace.createTableParamsFromRow(tableRow);
-
-        UntypedResultSet columnsRows = QueryProcessor.resultify(String.format("SELECT * FROM %s.%s", SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.COLUMNS),
-                                                                UnfilteredRowIterators.filter(cdU.unfilteredIterator(), FBUtilities.nowInSeconds()));
-        Set<ColumnDefinition> columns = new HashSet<>();
-        for (UntypedResultSet.Row row : columnsRows)
-            columns.add(SchemaKeyspace.createColumnFromRow(row, Types.none()));
-
-        assertEquals(cfm.params, params);
-        assertEquals(new HashSet<>(cfm.allColumns()), columns);
-    }
-    
-    @Test
-    public void testIsNameValidPositive()
-    {
-         assertTrue(CFMetaData.isNameValid("abcdefghijklmnopqrstuvwxyz"));
-         assertTrue(CFMetaData.isNameValid("ABCDEFGHIJKLMNOPQRSTUVWXYZ"));
-         assertTrue(CFMetaData.isNameValid("_01234567890"));
-    }
-    
-    @Test
-    public void testIsNameValidNegative()
-    {
-        assertFalse(CFMetaData.isNameValid(null));
-        assertFalse(CFMetaData.isNameValid(""));
-        assertFalse(CFMetaData.isNameValid(" "));
-        assertFalse(CFMetaData.isNameValid("@"));
-        assertFalse(CFMetaData.isNameValid("!"));
-    }
-
-    private static Set<String> primitiveTypes = new HashSet<String>(Arrays.asList(new String[] { "ascii", "bigint", "blob", "boolean", "date",
-                                                                                                 "duration", "decimal", "double", "float",
-                                                                                                 "inet", "int", "smallint", "text", "time",
-                                                                                                 "timestamp", "timeuuid", "tinyint", "uuid",
-                                                                                                 "varchar", "varint" }));
-
-    @Test
-    public void typeCompatibilityTest() throws Throwable
-    {
-        Map<String, Set<String>> compatibilityMap = new HashMap<>();
-        compatibilityMap.put("bigint", new HashSet<>(Arrays.asList(new String[] {"timestamp"})));
-        compatibilityMap.put("blob", new HashSet<>(Arrays.asList(new String[] {"ascii", "bigint", "boolean", "date", "decimal", "double", "duration",
-                                                                               "float", "inet", "int", "smallint", "text", "time", "timestamp",
-                                                                               "timeuuid", "tinyint", "uuid", "varchar", "varint"})));
-        compatibilityMap.put("date", new HashSet<>(Arrays.asList(new String[] {"int"})));
-        compatibilityMap.put("time", new HashSet<>(Arrays.asList(new String[] {"bigint"})));
-        compatibilityMap.put("text", new HashSet<>(Arrays.asList(new String[] {"ascii", "varchar"})));
-        compatibilityMap.put("timestamp", new HashSet<>(Arrays.asList(new String[] {"bigint"})));
-        compatibilityMap.put("varchar", new HashSet<>(Arrays.asList(new String[] {"ascii", "text"})));
-        compatibilityMap.put("varint", new HashSet<>(Arrays.asList(new String[] {"bigint", "int", "timestamp"})));
-        compatibilityMap.put("uuid", new HashSet<>(Arrays.asList(new String[] {"timeuuid"})));
-
-        for (String sourceTypeString: primitiveTypes)
-        {
-            AbstractType sourceType = CQLTypeParser.parse("KEYSPACE", sourceTypeString, Types.none());
-            for (String destinationTypeString: primitiveTypes)
-            {
-                AbstractType destinationType = CQLTypeParser.parse("KEYSPACE", destinationTypeString, Types.none());
-
-                if (compatibilityMap.get(destinationTypeString) != null &&
-                    compatibilityMap.get(destinationTypeString).contains(sourceTypeString) ||
-                    sourceTypeString.equals(destinationTypeString))
-                {
-                    assertTrue(sourceTypeString + " should be compatible with " + destinationTypeString,
-                               destinationType.isValueCompatibleWith(sourceType));
-                }
-                else
-                {
-                    assertFalse(sourceTypeString + " should not be compatible with " + destinationTypeString,
-                                destinationType.isValueCompatibleWith(sourceType));
-                }
-            }
-        }
-    }
-
-    @Test
-    public void clusteringColumnTypeCompatibilityTest() throws Throwable
-    {
-        Map<String, Set<String>> compatibilityMap = new HashMap<>();
-        compatibilityMap.put("blob", new HashSet<>(Arrays.asList(new String[] {"ascii", "text", "varchar"})));
-        compatibilityMap.put("text", new HashSet<>(Arrays.asList(new String[] {"ascii", "varchar"})));
-        compatibilityMap.put("varchar", new HashSet<>(Arrays.asList(new String[] {"ascii", "text" })));
-
-        for (String sourceTypeString: primitiveTypes)
-        {
-            AbstractType sourceType = CQLTypeParser.parse("KEYSPACE", sourceTypeString, Types.none());
-            for (String destinationTypeString: primitiveTypes)
-            {
-                AbstractType destinationType = CQLTypeParser.parse("KEYSPACE", destinationTypeString, Types.none());
-
-                if (compatibilityMap.get(destinationTypeString) != null &&
-                    compatibilityMap.get(destinationTypeString).contains(sourceTypeString) ||
-                    sourceTypeString.equals(destinationTypeString))
-                {
-                    assertTrue(sourceTypeString + " should be compatible with " + destinationTypeString,
-                               destinationType.isCompatibleWith(sourceType));
-                }
-                else
-                {
-                    assertFalse(sourceTypeString + " should not be compatible with " + destinationTypeString,
-                                destinationType.isCompatibleWith(sourceType));
-                }
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 bce9182..c1dc268 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -18,7 +18,6 @@
 */
 package org.apache.cassandra.config;
 
-import java.io.IOException;
 import java.net.Inet4Address;
 import java.net.Inet6Address;
 import java.net.InetAddress;
@@ -30,20 +29,12 @@ import java.util.Enumeration;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
+
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.service.MigrationManager;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
 import static org.junit.Assert.assertTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
@@ -57,42 +48,6 @@ public class DatabaseDescriptorTest
 
     // this came as a result of CASSANDRA-995
     @Test
-    public void testTransKsMigration() throws ConfigurationException, IOException
-    {
-        SchemaLoader.cleanupAndLeaveDirs();
-        Schema.instance.loadFromDisk();
-        assertEquals(0, Schema.instance.getNonSystemKeyspaces().size());
-
-        Gossiper.instance.start((int)(System.currentTimeMillis() / 1000));
-        Keyspace.setInitialized();
-
-        try
-        {
-            // add a few.
-            MigrationManager.announceNewKeyspace(KeyspaceMetadata.create("ks0", KeyspaceParams.simple(3)));
-            MigrationManager.announceNewKeyspace(KeyspaceMetadata.create("ks1", KeyspaceParams.simple(3)));
-
-            assertNotNull(Schema.instance.getKSMetaData("ks0"));
-            assertNotNull(Schema.instance.getKSMetaData("ks1"));
-
-            Schema.instance.clearKeyspaceMetadata(Schema.instance.getKSMetaData("ks0"));
-            Schema.instance.clearKeyspaceMetadata(Schema.instance.getKSMetaData("ks1"));
-
-            assertNull(Schema.instance.getKSMetaData("ks0"));
-            assertNull(Schema.instance.getKSMetaData("ks1"));
-
-            Schema.instance.loadFromDisk();
-
-            assertNotNull(Schema.instance.getKSMetaData("ks0"));
-            assertNotNull(Schema.instance.getKSMetaData("ks1"));
-        }
-        finally
-        {
-            Gossiper.instance.stop();
-        }
-    }
-
-    @Test
     public void testConfigurationLoader() throws Exception
     {
         // By default, we should load from the yaml
@@ -254,7 +209,5 @@ public class DatabaseDescriptorTest
         Collection<String> tokens = DatabaseDescriptor.tokensFromString(" a,b ,c , d, f,g,h");
         assertEquals(7, tokens.size());
         assertTrue(tokens.containsAll(Arrays.asList(new String[]{"a", "b", "c", "d", "f", "g", "h"})));
-
-        
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 a3e2021..68d8f66 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -42,10 +42,8 @@ import com.datastax.driver.core.DataType;
 import com.datastax.driver.core.ResultSet;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
 import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.cql3.functions.ThreadAwareSecurityManager;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
@@ -58,6 +56,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.locator.AbstractEndpointSnitch;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
@@ -725,9 +724,9 @@ public abstract class CQLTester
         }
     }
 
-    protected CFMetaData currentTableMetadata()
+    protected TableMetadata currentTableMetadata()
     {
-        return Schema.instance.getCFMetaData(KEYSPACE, currentTable());
+        return Schema.instance.getTableMetadata(KEYSPACE, currentTable());
     }
 
     protected com.datastax.driver.core.ResultSet executeNet(ProtocolVersion protocolVersion, String query, Object... values) throws Throwable

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java b/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
index 32e6aec..408d489 100644
--- a/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
+++ b/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
@@ -30,12 +30,13 @@ import org.junit.Test;
 
 import org.apache.cassandra.cache.KeyCacheKey;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.metrics.CacheMetrics;
 import org.apache.cassandra.metrics.CassandraMetricsRegistry;
 import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.StorageService;
 
@@ -43,7 +44,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNull;
-import org.apache.cassandra.utils.Pair;
 
 
 public class KeyCacheCqlTest extends CQLTester
@@ -301,11 +301,6 @@ public class KeyCacheCqlTest extends CQLTester
             assertEquals(500, result.size());
         }
 
-        //Test Schema.getColumnFamilyStoreIncludingIndexes, several null check paths
-        //are defensive and unreachable
-        assertNull(Schema.instance.getColumnFamilyStoreIncludingIndexes(Pair.create("foo", "bar")));
-        assertNull(Schema.instance.getColumnFamilyStoreIncludingIndexes(Pair.create(KEYSPACE_PER_TEST, "bar")));
-
         dropTable("DROP TABLE %s");
         Schema.instance.updateVersion();
 
@@ -386,8 +381,9 @@ public class KeyCacheCqlTest extends CQLTester
         while(iter.hasNext())
         {
             KeyCacheKey key = iter.next();
-            Assert.assertFalse(key.ksAndCFName.left.equals("KEYSPACE_PER_TEST"));
-            Assert.assertFalse(key.ksAndCFName.right.startsWith(table));
+            TableMetadataRef tableMetadataRef = Schema.instance.getTableMetadataRef(key.tableId);
+            Assert.assertFalse(tableMetadataRef.keyspace.equals("KEYSPACE_PER_TEST"));
+            Assert.assertFalse(tableMetadataRef.name.startsWith(table));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
index 24efc5e..b4fe0f5 100644
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.cql3;
 import static junit.framework.Assert.fail;
 
 import java.io.Closeable;
-import java.util.UUID;
 import java.util.concurrent.ExecutionException;
 
 import org.junit.Assert;
@@ -34,6 +33,7 @@ import org.apache.cassandra.db.commitlog.CommitLogSegment;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.KillerForTests;
 
@@ -125,9 +125,9 @@ public class OutOfSpaceTest extends CQLTester
         }
 
         // Make sure commit log wasn't discarded.
-        UUID cfid = currentTableMetadata().cfId;
+        TableId tableId = currentTableMetadata().id;
         for (CommitLogSegment segment : CommitLog.instance.segmentManager.getActiveSegments())
-            if (segment.getDirtyCFIDs().contains(cfid))
+            if (segment.getDirtyTableIds().contains(tableId))
                 return;
         fail("Expected commit log to remain dirty for the affected table.");
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java b/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java
index e01b812..cd7e381 100644
--- a/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/PreparedStatementsTest.java
@@ -27,7 +27,7 @@ import com.datastax.driver.core.Session;
 import com.datastax.driver.core.exceptions.SyntaxError;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.index.StubIndex;
 import org.apache.cassandra.service.EmbeddedCassandraService;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
index 0bcb877..fb577ec 100644
--- a/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
+++ b/test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.junit.Test;
 
 import junit.framework.Assert;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.Int32Type;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java b/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
index cc8bfe9..ea07f90 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
@@ -31,9 +31,6 @@ import junit.framework.Assert;
 import org.apache.cassandra.concurrent.SEPExecutor;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.transport.ProtocolVersion;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java b/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
index 86e00dc..8d2f3da 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
@@ -34,9 +34,9 @@ import junit.framework.Assert;
 import org.apache.cassandra.concurrent.SEPExecutor;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.serializers.SimpleDateSerializer;
 import org.apache.cassandra.serializers.TimeSerializer;
@@ -176,8 +176,8 @@ public class ViewSchemaTest extends CQLTester
 
         //Test alter add
         executeNet(protocolVersion, "ALTER TABLE %s ADD foo text");
-        CFMetaData metadata = Schema.instance.getCFMetaData(keyspace(), "mv1_test");
-        Assert.assertNotNull(metadata.getColumnDefinition(ByteBufferUtil.bytes("foo")));
+        TableMetadata metadata = Schema.instance.getTableMetadata(keyspace(), "mv1_test");
+        Assert.assertNotNull(metadata.getColumn(ByteBufferUtil.bytes("foo")));
 
         updateView("INSERT INTO %s(k,asciival,bigintval,foo)VALUES(?,?,?,?)", 0, "foo", 1L, "bar");
         assertRows(execute("SELECT foo from %s"), row("bar"));
@@ -186,8 +186,8 @@ public class ViewSchemaTest extends CQLTester
         executeNet(protocolVersion, "ALTER TABLE %s RENAME asciival TO bar");
 
         assertRows(execute("SELECT bar from %s"), row("foo"));
-        metadata = Schema.instance.getCFMetaData(keyspace(), "mv1_test");
-        Assert.assertNotNull(metadata.getColumnDefinition(ByteBufferUtil.bytes("bar")));
+        metadata = Schema.instance.getTableMetadata(keyspace(), "mv1_test");
+        Assert.assertNotNull(metadata.getColumn(ByteBufferUtil.bytes("bar")));
     }
 
 
@@ -286,12 +286,12 @@ public class ViewSchemaTest extends CQLTester
                     "tupleval frozen<tuple<int, ascii, uuid>>," +
                     "udtval frozen<" + myType + ">)");
 
-        CFMetaData metadata = currentTableMetadata();
+        TableMetadata metadata = currentTableMetadata();
 
         execute("USE " + keyspace());
         executeNet(protocolVersion, "USE " + keyspace());
 
-        for (ColumnDefinition def : new HashSet<>(metadata.allColumns()))
+        for (ColumnMetadata def : new HashSet<>(metadata.columns()))
         {
             try
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewTest.java b/test/unit/org/apache/cassandra/cql3/ViewTest.java
index ac065e6..b5573c2 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@ -35,8 +35,8 @@ import com.datastax.driver.core.exceptions.InvalidQueryException;
 import org.apache.cassandra.concurrent.SEPExecutor;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;
@@ -636,12 +636,12 @@ public class ViewTest extends CQLTester
                     "bigintval bigint, " +
                     "PRIMARY KEY((k, asciival)))");
 
-        CFMetaData metadata = currentTableMetadata();
+        TableMetadata metadata = currentTableMetadata();
 
         execute("USE " + keyspace());
         executeNet(protocolVersion, "USE " + keyspace());
 
-        for (ColumnDefinition def : new HashSet<>(metadata.allColumns()))
+        for (ColumnMetadata def : new HashSet<>(metadata.columns()))
         {
             try
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/conditions/ColumnConditionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/conditions/ColumnConditionTest.java b/test/unit/org/apache/cassandra/cql3/conditions/ColumnConditionTest.java
index 5822027..0035fa1 100644
--- a/test/unit/org/apache/cassandra/cql3/conditions/ColumnConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/conditions/ColumnConditionTest.java
@@ -23,10 +23,7 @@ import java.util.*;
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.Constants.Value;
-import org.apache.cassandra.cql3.conditions.ColumnCondition;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.ListType;
@@ -34,6 +31,7 @@ import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.db.marshal.SetType;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.serializers.TimeUUIDSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
@@ -52,13 +50,13 @@ public class ColumnConditionTest
     public static final ByteBuffer ONE = Int32Type.instance.fromString("1");
     public static final ByteBuffer TWO = Int32Type.instance.fromString("2");
 
-    private static Row newRow(ColumnDefinition definition, ByteBuffer value)
+    private static Row newRow(ColumnMetadata definition, ByteBuffer value)
     {
         BufferCell cell = new BufferCell(definition, 0L, Cell.NO_TTL, Cell.NO_DELETION_TIME, value, null);
         return BTreeRow.singleCellRow(Clustering.EMPTY, cell);
     }
 
-    private static Row newRow(ColumnDefinition definition, List<ByteBuffer> values)
+    private static Row newRow(ColumnMetadata definition, List<ByteBuffer> values)
     {
         Row.Builder builder = BTreeRow.sortedBuilder();
         builder.newRow(Clustering.EMPTY);
@@ -82,7 +80,7 @@ public class ColumnConditionTest
         return builder.build();
     }
 
-    private static Row newRow(ColumnDefinition definition, SortedSet<ByteBuffer> values)
+    private static Row newRow(ColumnMetadata definition, SortedSet<ByteBuffer> values)
     {
         Row.Builder builder = BTreeRow.sortedBuilder();
         builder.newRow(Clustering.EMPTY);
@@ -102,7 +100,7 @@ public class ColumnConditionTest
         return builder.build();
     }
 
-    private static Row newRow(ColumnDefinition definition, Map<ByteBuffer, ByteBuffer> values)
+    private static Row newRow(ColumnMetadata definition, Map<ByteBuffer, ByteBuffer> values)
     {
         Row.Builder builder = BTreeRow.sortedBuilder();
         builder.newRow(Clustering.EMPTY);
@@ -124,7 +122,7 @@ public class ColumnConditionTest
 
     private static boolean conditionApplies(ByteBuffer rowValue, Operator op, ByteBuffer conditionValue)
     {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", Int32Type.instance);
+        ColumnMetadata definition = ColumnMetadata.regularColumn("ks", "cf", "c", Int32Type.instance);
         ColumnCondition condition = ColumnCondition.condition(definition, op, Terms.of(new Constants.Value(conditionValue)));
         ColumnCondition.Bound bound = condition.bind(QueryOptions.DEFAULT);
         return bound.appliesTo(newRow(definition, rowValue));
@@ -132,7 +130,7 @@ public class ColumnConditionTest
 
     private static boolean conditionApplies(List<ByteBuffer> rowValue, Operator op, List<ByteBuffer> conditionValue)
     {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true));
+        ColumnMetadata definition = ColumnMetadata.regularColumn("ks", "cf", "c", ListType.getInstance(Int32Type.instance, true));
         ColumnCondition condition = ColumnCondition.condition(definition, op, Terms.of(new Lists.Value(conditionValue)));
         ColumnCondition.Bound bound = condition.bind(QueryOptions.DEFAULT);
         return bound.appliesTo(newRow(definition, rowValue));
@@ -140,7 +138,7 @@ public class ColumnConditionTest
 
     private static boolean conditionApplies(SortedSet<ByteBuffer> rowValue, Operator op, SortedSet<ByteBuffer> conditionValue)
     {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", SetType.getInstance(Int32Type.instance, true));
+        ColumnMetadata definition = ColumnMetadata.regularColumn("ks", "cf", "c", SetType.getInstance(Int32Type.instance, true));
         ColumnCondition condition = ColumnCondition.condition(definition, op, Terms.of(new Sets.Value(conditionValue)));
         ColumnCondition.Bound bound = condition.bind(QueryOptions.DEFAULT);
         return bound.appliesTo(newRow(definition, rowValue));
@@ -148,7 +146,7 @@ public class ColumnConditionTest
 
     private static boolean conditionApplies(Map<ByteBuffer, ByteBuffer> rowValue, Operator op, Map<ByteBuffer, ByteBuffer> conditionValue)
     {
-        ColumnDefinition definition = ColumnDefinition.regularDef("ks", "cf", "c", MapType.getInstance(Int32Type.instance, Int32Type.instance, true));
+        ColumnMetadata definition = ColumnMetadata.regularColumn("ks", "cf", "c", MapType.getInstance(Int32Type.instance, Int32Type.instance, true));
         ColumnCondition condition = ColumnCondition.condition(definition, op, Terms.of(new Maps.Value(conditionValue)));
         ColumnCondition.Bound bound = condition.bind(QueryOptions.DEFAULT);
         return bound.appliesTo(newRow(definition, rowValue));


[14/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
index 0b40fcb..ba702dd 100644
--- a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
+++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
@@ -22,23 +22,20 @@ import java.util.*;
 
 import com.datastax.driver.core.*;
 
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.ColumnDefinition.ClusteringOrder;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.ColumnMetadata.ClusteringOrder;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.dht.Token.TokenFactory;
 import org.apache.cassandra.io.sstable.SSTableLoader;
-import org.apache.cassandra.schema.CQLTypeParser;
-import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.schema.Types;
+import org.apache.cassandra.schema.TableMetadata;
 
 public class NativeSSTableLoaderClient extends SSTableLoader.Client
 {
-    protected final Map<String, CFMetaData> tables;
+    protected final Map<String, TableMetadataRef> tables;
     private final Collection<InetAddress> hosts;
     private final int port;
     private final AuthProvider authProvider;
@@ -90,20 +87,20 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
             Types types = fetchTypes(keyspace, session);
 
             tables.putAll(fetchTables(keyspace, session, partitioner, types));
-            // We only need the CFMetaData for the views, so we only load that.
+            // We only need the TableMetadata for the views, so we only load that.
             tables.putAll(fetchViews(keyspace, session, partitioner, types));
         }
     }
 
-    public CFMetaData getTableMetadata(String tableName)
+    public TableMetadataRef getTableMetadata(String tableName)
     {
         return tables.get(tableName);
     }
 
     @Override
-    public void setTableMetadata(CFMetaData cfm)
+    public void setTableMetadata(TableMetadataRef cfm)
     {
-        tables.put(cfm.cfName, cfm);
+        tables.put(cfm.name, cfm);
     }
 
     private static Types fetchTypes(String keyspace, Session session)
@@ -130,9 +127,9 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
      * Note: It is not safe for this class to use static methods from SchemaKeyspace (static final fields are ok)
      * as that triggers initialization of the class, which fails in client mode.
      */
-    private static Map<String, CFMetaData> fetchTables(String keyspace, Session session, IPartitioner partitioner, Types types)
+    private static Map<String, TableMetadataRef> fetchTables(String keyspace, Session session, IPartitioner partitioner, Types types)
     {
-        Map<String, CFMetaData> tables = new HashMap<>();
+        Map<String, TableMetadataRef> tables = new HashMap<>();
         String query = String.format("SELECT * FROM %s.%s WHERE keyspace_name = ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.TABLES);
 
         for (Row row : session.execute(query, keyspace))
@@ -144,12 +141,9 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
         return tables;
     }
 
-    /*
-     * In the case where we are creating View CFMetaDatas, we
-     */
-    private static Map<String, CFMetaData> fetchViews(String keyspace, Session session, IPartitioner partitioner, Types types)
+    private static Map<String, TableMetadataRef> fetchViews(String keyspace, Session session, IPartitioner partitioner, Types types)
     {
-        Map<String, CFMetaData> tables = new HashMap<>();
+        Map<String, TableMetadataRef> tables = new HashMap<>();
         String query = String.format("SELECT * FROM %s.%s WHERE keyspace_name = ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.VIEWS);
 
         for (Row row : session.execute(query, keyspace))
@@ -161,43 +155,31 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
         return tables;
     }
 
-    private static CFMetaData createTableMetadata(String keyspace,
-                                                  Session session,
-                                                  IPartitioner partitioner,
-                                                  boolean isView,
-                                                  Row row,
-                                                  String name,
-                                                  Types types)
+    private static TableMetadataRef createTableMetadata(String keyspace,
+                                                        Session session,
+                                                        IPartitioner partitioner,
+                                                        boolean isView,
+                                                        Row row,
+                                                        String name,
+                                                        Types types)
     {
-        UUID id = row.getUUID("id");
-        Set<CFMetaData.Flag> flags = isView ? Collections.emptySet() : CFMetaData.flagsFromStrings(row.getSet("flags", String.class));
+        TableMetadata.Builder builder = TableMetadata.builder(keyspace, name, TableId.fromUUID(row.getUUID("id")))
+                                                     .partitioner(partitioner);
 
-        boolean isSuper = flags.contains(CFMetaData.Flag.SUPER);
-        boolean isCounter = flags.contains(CFMetaData.Flag.COUNTER);
-        boolean isDense = flags.contains(CFMetaData.Flag.DENSE);
-        boolean isCompound = isView || flags.contains(CFMetaData.Flag.COMPOUND);
+        if (!isView)
+            builder.flags(TableMetadata.Flag.fromStringSet(row.getSet("flags", String.class)));
 
         String columnsQuery = String.format("SELECT * FROM %s.%s WHERE keyspace_name = ? AND table_name = ?",
                                             SchemaConstants.SCHEMA_KEYSPACE_NAME,
                                             SchemaKeyspace.COLUMNS);
 
-        List<ColumnDefinition> defs = new ArrayList<>();
         for (Row colRow : session.execute(columnsQuery, keyspace, name))
-            defs.add(createDefinitionFromRow(colRow, keyspace, name, types));
-
-        return CFMetaData.create(keyspace,
-                                 name,
-                                 id,
-                                 isDense,
-                                 isCompound,
-                                 isSuper,
-                                 isCounter,
-                                 isView,
-                                 defs,
-                                 partitioner);
+            builder.addColumn(createDefinitionFromRow(colRow, keyspace, name, types));
+
+        return TableMetadataRef.forOfflineTools(builder.build());
     }
 
-    private static ColumnDefinition createDefinitionFromRow(Row row, String keyspace, String table, Types types)
+    private static ColumnMetadata createDefinitionFromRow(Row row, String keyspace, String table, Types types)
     {
         ClusteringOrder order = ClusteringOrder.valueOf(row.getString("clustering_order").toUpperCase());
         AbstractType<?> type = CQLTypeParser.parse(keyspace, row.getString("type"), types);
@@ -209,7 +191,7 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
                                                              row.getString("column_name"));
 
         int position = row.getInt("position");
-        ColumnDefinition.Kind kind = ColumnDefinition.Kind.valueOf(row.getString("kind").toUpperCase());
-        return new ColumnDefinition(keyspace, table, name, type, position, kind);
+        org.apache.cassandra.schema.ColumnMetadata.Kind kind = ColumnMetadata.Kind.valueOf(row.getString("kind").toUpperCase());
+        return new ColumnMetadata(keyspace, table, name, type, position, kind);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/long/org/apache/cassandra/cql3/CorruptionTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/cql3/CorruptionTest.java b/test/long/org/apache/cassandra/cql3/CorruptionTest.java
index 43cf5e0..f2ed36a 100644
--- a/test/long/org/apache/cassandra/cql3/CorruptionTest.java
+++ b/test/long/org/apache/cassandra/cql3/CorruptionTest.java
@@ -37,7 +37,7 @@ import com.datastax.driver.core.policies.Policies;
 import com.datastax.driver.core.utils.Bytes;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.service.EmbeddedCassandraService;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index 3f5be03..eb1be8b 100644
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -47,6 +47,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.security.EncryptionContext;
 import org.apache.cassandra.security.EncryptionContextGenerator;
 
@@ -262,7 +263,7 @@ public class CommitLogStressTest
             }
             verifySizes(commitLog);
 
-            commitLog.discardCompletedSegments(Schema.instance.getCFMetaData("Keyspace1", "Standard1").cfId,
+            commitLog.discardCompletedSegments(Schema.instance.getTableMetadata("Keyspace1", "Standard1").id,
                     CommitLogPosition.NONE, discardedPos);
             threads.clear();
 
@@ -444,7 +445,7 @@ public class CommitLogStressTest
                     rl.acquire();
                 ByteBuffer key = randomBytes(16, rand);
 
-                UpdateBuilder builder = UpdateBuilder.create(Schema.instance.getCFMetaData("Keyspace1", "Standard1"), Util.dk(key));
+                UpdateBuilder builder = UpdateBuilder.create(Schema.instance.getTableMetadata("Keyspace1", "Standard1"), Util.dk(key));
                 for (int ii = 0; ii < numCells; ii++)
                 {
                     int sz = randomSize ? rand.nextInt(cellSize) : cellSize;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
index d684e11..f204303 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -28,7 +28,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.UpdateBuilder;
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.*;
@@ -108,7 +108,7 @@ public class LongCompactionsTest
             {
                 String key = String.valueOf(j);
                 // last sstable has highest timestamps
-                UpdateBuilder builder = UpdateBuilder.create(store.metadata, String.valueOf(j))
+                UpdateBuilder builder = UpdateBuilder.create(store.metadata(), String.valueOf(j))
                                                      .withTimestamp(k);
                 for (int i = 0; i < rowsPerPartition; i++)
                     builder.newRow(String.valueOf(i)).add("val", String.valueOf(i));
@@ -123,7 +123,7 @@ public class LongCompactionsTest
         Thread.sleep(1000);
 
         long start = System.nanoTime();
-        final int gcBefore = (int) (System.currentTimeMillis() / 1000) - Schema.instance.getCFMetaData(KEYSPACE1, "Standard1").params.gcGraceSeconds;
+        final int gcBefore = (int) (System.currentTimeMillis() / 1000) - Schema.instance.getTableMetadata(KEYSPACE1, "Standard1").params.gcGraceSeconds;
         try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.COMPACTION))
         {
             assert txn != null : "Cannot markCompacting all sstables";
@@ -146,7 +146,7 @@ public class LongCompactionsTest
         cfs.clearUnsafe();
 
         final int ROWS_PER_SSTABLE = 10;
-        final int SSTABLES = cfs.metadata.params.minIndexInterval * 3 / ROWS_PER_SSTABLE;
+        final int SSTABLES = cfs.metadata().params.minIndexInterval * 3 / ROWS_PER_SSTABLE;
 
         // disable compaction while flushing
         cfs.disableAutoCompaction();
@@ -158,7 +158,7 @@ public class LongCompactionsTest
                 DecoratedKey key = Util.dk(String.valueOf(i % 2));
                 long timestamp = j * ROWS_PER_SSTABLE + i;
                 maxTimestampExpected = Math.max(timestamp, maxTimestampExpected);
-                UpdateBuilder.create(cfs.metadata, key)
+                UpdateBuilder.create(cfs.metadata(), key)
                              .withTimestamp(timestamp)
                              .newRow(String.valueOf(i / 2)).add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                              .apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
index cf4aba2..3d18d85 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
@@ -82,7 +82,7 @@ public class LongLeveledCompactionStrategyTest
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            UpdateBuilder builder = UpdateBuilder.create(store.metadata, key);
+            UpdateBuilder builder = UpdateBuilder.create(store.metadata(), key);
             for (int c = 0; c < columns; c++)
                 builder.newRow("column" + c).add("val", value);
 
@@ -160,7 +160,7 @@ public class LongLeveledCompactionStrategyTest
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            UpdateBuilder builder = UpdateBuilder.create(store.metadata, key);
+            UpdateBuilder builder = UpdateBuilder.create(store.metadata(), key);
             for (int c = 0; c < columns; c++)
                 builder.newRow("column" + c).add("val", value);
 
@@ -179,7 +179,7 @@ public class LongLeveledCompactionStrategyTest
         for (int r = 0; r < 10; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            UpdateBuilder builder = UpdateBuilder.create(store.metadata, key);
+            UpdateBuilder builder = UpdateBuilder.create(store.metadata(), key);
             for (int c = 0; c < 10; c++)
                 builder.newRow("column" + c).add("val", value);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/long/org/apache/cassandra/hints/HintsWriteThenReadTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/hints/HintsWriteThenReadTest.java b/test/long/org/apache/cassandra/hints/HintsWriteThenReadTest.java
index fd880cb..a905f9a 100644
--- a/test/long/org/apache/cassandra/hints/HintsWriteThenReadTest.java
+++ b/test/long/org/apache/cassandra/hints/HintsWriteThenReadTest.java
@@ -31,8 +31,8 @@ import com.google.common.collect.Iterables;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.rows.Cell;
@@ -167,7 +167,7 @@ public class HintsWriteThenReadTest
 
     private static Mutation createMutation(int index, long timestamp)
     {
-        CFMetaData table = Schema.instance.getCFMetaData(KEYSPACE, TABLE);
+        TableMetadata table = Schema.instance.getTableMetadata(KEYSPACE, TABLE);
         return new RowUpdateBuilder(table, timestamp, bytes(index))
                .clustering(bytes(index))
                .add("val", bytes(index))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/streaming/LongStreamingTest.java b/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
index 1340224..a30d6c9 100644
--- a/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
+++ b/test/long/org/apache/cassandra/streaming/LongStreamingTest.java
@@ -24,14 +24,13 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.io.Files;
-import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.Keyspace;
@@ -110,9 +109,9 @@ public class LongStreamingTest
                 this.ks = keyspace;
             }
 
-            public CFMetaData getTableMetadata(String cfName)
+            public TableMetadataRef getTableMetadata(String cfName)
             {
-                return Schema.instance.getCFMetaData(ks, cfName);
+                return Schema.instance.getTableMetadataRef(ks, cfName);
             }
         }, new OutputHandler.SystemOutput(false, false));
 
@@ -137,9 +136,9 @@ public class LongStreamingTest
                 this.ks = keyspace;
             }
 
-            public CFMetaData getTableMetadata(String cfName)
+            public TableMetadataRef getTableMetadata(String cfName)
             {
-                return Schema.instance.getCFMetaData(ks, cfName);
+                return Schema.instance.getTableMetadataRef(ks, cfName);
             }
         }, new OutputHandler.SystemOutput(false, false));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java
----------------------------------------------------------------------
diff --git a/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java b/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java
index d8dfd66..41220a2 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/CompactionBench.java
@@ -21,51 +21,16 @@ package org.apache.cassandra.test.microbench;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.*;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.Uninterruptibles;
-
-import org.apache.cassandra.UpdateBuilder;
-import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.util.DataInputBuffer;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.service.CassandraDaemon;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.hadoop.util.bloom.Key;
 import org.openjdk.jmh.annotations.*;
-import org.openjdk.jmh.profile.StackProfiler;
-import org.openjdk.jmh.results.Result;
-import org.openjdk.jmh.results.RunResult;
-import org.openjdk.jmh.runner.Runner;
-import org.openjdk.jmh.runner.options.Options;
-import org.openjdk.jmh.runner.options.OptionsBuilder;
 
 @BenchmarkMode(Mode.AverageTime)
 @OutputTimeUnit(TimeUnit.MILLISECONDS)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java
----------------------------------------------------------------------
diff --git a/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java b/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java
index 8c177cf..45ad258 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/MutationBench.java
@@ -25,10 +25,10 @@ import java.util.Collection;
 import java.util.concurrent.*;
 
 import org.apache.cassandra.UpdateBuilder;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Config;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.io.util.DataInputBuffer;
@@ -91,16 +91,16 @@ public class MutationBench
     public void setup() throws IOException
     {
         Schema.instance.load(KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1)));
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
-        CFMetaData metadata = CFMetaData.compile("CREATE TABLE userpics " +
-                                                   "( userid bigint," +
-                                                   "picid bigint," +
-                                                   "commentid bigint, " +
-                                                   "PRIMARY KEY(userid, picid))", keyspace);
-
-        Schema.instance.load(metadata);
-        Schema.instance.setKeyspaceMetadata(ksm.withSwapped(ksm.tables.with(metadata)));
-
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
+        TableMetadata metadata =
+            CreateTableStatement.parse("CREATE TABLE userpics " +
+                                       "( userid bigint," +
+                                       "picid bigint," +
+                                       "commentid bigint, " +
+                                       "PRIMARY KEY(userid, picid))", keyspace)
+                                .build();
+
+        Schema.instance.load(ksm.withSwapped(ksm.tables.with(metadata)));
 
         mutation = (Mutation)UpdateBuilder.create(metadata, 1L).newRow(1L).add("commentid", 32L).makeMutation();
         messageOut = mutation.createMessage();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/microbench/org/apache/cassandra/test/microbench/ReadWriteTest.java
----------------------------------------------------------------------
diff --git a/test/microbench/org/apache/cassandra/test/microbench/ReadWriteTest.java b/test/microbench/org/apache/cassandra/test/microbench/ReadWriteTest.java
index 89973fd..066c289 100644
--- a/test/microbench/org/apache/cassandra/test/microbench/ReadWriteTest.java
+++ b/test/microbench/org/apache/cassandra/test/microbench/ReadWriteTest.java
@@ -19,44 +19,13 @@
 package org.apache.cassandra.test.microbench;
 
 
-import java.io.File;
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.List;
 import java.util.concurrent.*;
 
-import org.apache.cassandra.UpdateBuilder;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.dht.Murmur3Partitioner;
-import org.apache.cassandra.io.util.DataInputBuffer;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.io.util.DataOutputBufferFixed;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.service.CassandraDaemon;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.transport.messages.ResultMessage;
 import org.openjdk.jmh.annotations.*;
-import org.openjdk.jmh.profile.StackProfiler;
-import org.openjdk.jmh.results.Result;
-import org.openjdk.jmh.results.RunResult;
-import org.openjdk.jmh.runner.Runner;
-import org.openjdk.jmh.runner.options.Options;
-import org.openjdk.jmh.runner.options.OptionsBuilder;
 
 @BenchmarkMode(Mode.Throughput)
 @OutputTimeUnit(TimeUnit.MILLISECONDS)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/MockSchema.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/MockSchema.java b/test/unit/org/apache/cassandra/MockSchema.java
deleted file mode 100644
index 804bccb..0000000
--- a/test/unit/org/apache/cassandra/MockSchema.java
+++ /dev/null
@@ -1,187 +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;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import com.google.common.collect.ImmutableSet;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.dht.Murmur3Partitioner;
-import org.apache.cassandra.io.sstable.Component;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.IndexSummary;
-import org.apache.cassandra.io.sstable.format.SSTableFormat;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-import org.apache.cassandra.io.sstable.metadata.MetadataType;
-import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.Memory;
-import org.apache.cassandra.io.util.FileHandle;
-import org.apache.cassandra.schema.CachingParams;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.utils.AlwaysPresentFilter;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class MockSchema
-{
-    static
-    {
-        Memory offsets = Memory.allocate(4);
-        offsets.setInt(0, 0);
-        indexSummary = new IndexSummary(Murmur3Partitioner.instance, offsets, 0, Memory.allocate(4), 0, 0, 0, 1);
-    }
-    private static final AtomicInteger id = new AtomicInteger();
-    public static final Keyspace ks = Keyspace.mockKS(KeyspaceMetadata.create("mockks", KeyspaceParams.simpleTransient(1)));
-
-    public static final IndexSummary indexSummary;
-    private static final FileHandle RANDOM_ACCESS_READER_FACTORY = new FileHandle.Builder(temp("mocksegmentedfile").getAbsolutePath()).complete();
-
-    public static Memtable memtable(ColumnFamilyStore cfs)
-    {
-        return new Memtable(cfs.metadata);
-    }
-
-    public static SSTableReader sstable(int generation, ColumnFamilyStore cfs)
-    {
-        return sstable(generation, false, cfs);
-    }
-
-    public static SSTableReader sstable(int generation, boolean keepRef, ColumnFamilyStore cfs)
-    {
-        return sstable(generation, 0, keepRef, cfs);
-    }
-
-    public static SSTableReader sstable(int generation, int size, ColumnFamilyStore cfs)
-    {
-        return sstable(generation, size, false, cfs);
-    }
-
-    public static SSTableReader sstable(int generation, int size, boolean keepRef, ColumnFamilyStore cfs)
-    {
-        Descriptor descriptor = new Descriptor(cfs.getDirectories().getDirectoryForNewSSTables(),
-                                               cfs.keyspace.getName(),
-                                               cfs.getColumnFamilyName(),
-                                               generation, SSTableFormat.Type.BIG);
-        Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
-        for (Component component : components)
-        {
-            File file = new File(descriptor.filenameFor(component));
-            try
-            {
-                file.createNewFile();
-            }
-            catch (IOException e)
-            {
-            }
-        }
-        if (size > 0)
-        {
-            try
-            {
-                File file = new File(descriptor.filenameFor(Component.DATA));
-                try (RandomAccessFile raf = new RandomAccessFile(file, "rw"))
-                {
-                    raf.setLength(size);
-                }
-            }
-            catch (IOException e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-        SerializationHeader header = SerializationHeader.make(cfs.metadata, Collections.emptyList());
-        StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
-                                                 .finalizeMetadata(cfs.metadata.partitioner.getClass().getCanonicalName(), 0.01f, -1, header)
-                                                 .get(MetadataType.STATS);
-        SSTableReader reader = SSTableReader.internalOpen(descriptor, components, cfs.metadata,
-                                                          RANDOM_ACCESS_READER_FACTORY.sharedCopy(), RANDOM_ACCESS_READER_FACTORY.sharedCopy(), indexSummary.sharedCopy(),
-                                                          new AlwaysPresentFilter(), 1L, metadata, SSTableReader.OpenReason.NORMAL, header);
-        reader.first = reader.last = readerBounds(generation);
-        if (!keepRef)
-            reader.selfRef().release();
-        return reader;
-    }
-
-    public static ColumnFamilyStore newCFS()
-    {
-        return newCFS(ks.getName());
-    }
-
-    public static ColumnFamilyStore newCFS(String ksname)
-    {
-        String cfname = "mockcf" + (id.incrementAndGet());
-        CFMetaData metadata = newCFMetaData(ksname, cfname);
-        return new ColumnFamilyStore(ks, cfname, 0, metadata, new Directories(metadata), false, false, false);
-    }
-
-    public static CFMetaData newCFMetaData(String ksname, String cfname)
-    {
-        CFMetaData metadata = CFMetaData.Builder.create(ksname, cfname)
-                                                .addPartitionKey("key", UTF8Type.instance)
-                                                .addClusteringColumn("col", UTF8Type.instance)
-                                                .addRegularColumn("value", UTF8Type.instance)
-                                                .withPartitioner(Murmur3Partitioner.instance)
-                                                .build();
-        metadata.caching(CachingParams.CACHE_NOTHING);
-        return metadata;
-    }
-
-    public static BufferDecoratedKey readerBounds(int generation)
-    {
-        return new BufferDecoratedKey(new Murmur3Partitioner.LongToken(generation), ByteBufferUtil.EMPTY_BYTE_BUFFER);
-    }
-
-    private static File temp(String id)
-    {
-        try
-        {
-            File file = File.createTempFile(id, "tmp");
-            file.deleteOnExit();
-            return file;
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public static void cleanup()
-    {
-        // clean up data directory which are stored as data directory/keyspace/data files
-        for (String dirName : DatabaseDescriptor.getAllDataFileLocations())
-        {
-            File dir = new File(dirName);
-            if (!dir.exists())
-                continue;
-            String[] children = dir.list();
-            for (String child : children)
-                FileUtils.deleteRecursive(new File(dir, child));
-        }
-    }
-}


[16/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/CacheService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CacheService.java b/src/java/org/apache/cassandra/service/CacheService.java
index e64ec75..4cb7470 100644
--- a/src/java/org/apache/cassandra/service/CacheService.java
+++ b/src/java/org/apache/cassandra/service/CacheService.java
@@ -32,23 +32,25 @@ import javax.management.ObjectName;
 
 import com.google.common.util.concurrent.Futures;
 
-import org.apache.cassandra.db.lifecycle.SSTableSet;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.cache.*;
 import org.apache.cassandra.cache.AutoSavingCache.CacheSerializer;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.*;
+import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.partitions.CachedBTreePartition;
 import org.apache.cassandra.db.partitions.CachedPartition;
-import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
@@ -265,13 +267,13 @@ public class CacheService implements CacheServiceMBean
         keyCache.clear();
     }
 
-    public void invalidateKeyCacheForCf(Pair<String, String> ksAndCFName)
+    public void invalidateKeyCacheForCf(TableMetadata tableMetadata)
     {
         Iterator<KeyCacheKey> keyCacheIterator = keyCache.keyIterator();
         while (keyCacheIterator.hasNext())
         {
             KeyCacheKey key = keyCacheIterator.next();
-            if (key.ksAndCFName.equals(ksAndCFName))
+            if (key.sameTable(tableMetadata))
                 keyCacheIterator.remove();
         }
     }
@@ -281,24 +283,24 @@ public class CacheService implements CacheServiceMBean
         rowCache.clear();
     }
 
-    public void invalidateRowCacheForCf(Pair<String, String> ksAndCFName)
+    public void invalidateRowCacheForCf(TableMetadata tableMetadata)
     {
         Iterator<RowCacheKey> rowCacheIterator = rowCache.keyIterator();
         while (rowCacheIterator.hasNext())
         {
-            RowCacheKey rowCacheKey = rowCacheIterator.next();
-            if (rowCacheKey.ksAndCFName.equals(ksAndCFName))
+            RowCacheKey key = rowCacheIterator.next();
+            if (key.sameTable(tableMetadata))
                 rowCacheIterator.remove();
         }
     }
 
-    public void invalidateCounterCacheForCf(Pair<String, String> ksAndCFName)
+    public void invalidateCounterCacheForCf(TableMetadata tableMetadata)
     {
         Iterator<CounterCacheKey> counterCacheIterator = counterCache.keyIterator();
         while (counterCacheIterator.hasNext())
         {
-            CounterCacheKey counterCacheKey = counterCacheIterator.next();
-            if (counterCacheKey.ksAndCFName.equals(ksAndCFName))
+            CounterCacheKey key = counterCacheIterator.next();
+            if (key.sameTable(tableMetadata))
                 counterCacheIterator.remove();
         }
     }
@@ -353,8 +355,10 @@ public class CacheService implements CacheServiceMBean
     {
         public void serialize(CounterCacheKey key, DataOutputPlus out, ColumnFamilyStore cfs) throws IOException
         {
-            assert(cfs.metadata.isCounter());
-            out.write(cfs.metadata.ksAndCFBytes);
+            assert(cfs.metadata().isCounter());
+            TableMetadata tableMetadata = cfs.metadata();
+            tableMetadata.id.serialize(out);
+            out.writeUTF(tableMetadata.indexName().orElse(""));
             key.write(out);
         }
 
@@ -362,8 +366,10 @@ public class CacheService implements CacheServiceMBean
         {
             //Keyspace and CF name are deserialized by AutoSaving cache and used to fetch the CFS provided as a
             //parameter so they aren't deserialized here, even though they are serialized by this serializer
-            final CounterCacheKey cacheKey = CounterCacheKey.read(cfs.metadata.ksAndCFName, in);
-            if (cfs == null || !cfs.metadata.isCounter() || !cfs.isCounterCacheEnabled())
+            if (cfs == null)
+                return null;
+            final CounterCacheKey cacheKey = CounterCacheKey.read(cfs.metadata(), in);
+            if (!cfs.metadata().isCounter() || !cfs.isCounterCacheEnabled())
                 return null;
 
             return StageManager.getStage(Stage.READ).submit(new Callable<Pair<CounterCacheKey, ClockAndCount>>()
@@ -384,7 +390,9 @@ public class CacheService implements CacheServiceMBean
         public void serialize(RowCacheKey key, DataOutputPlus out, ColumnFamilyStore cfs) throws IOException
         {
             assert(!cfs.isIndex());//Shouldn't have row cache entries for indexes
-            out.write(cfs.metadata.ksAndCFBytes);
+            TableMetadata tableMetadata = cfs.metadata();
+            tableMetadata.id.serialize(out);
+            out.writeUTF(tableMetadata.indexName().orElse(""));
             ByteBufferUtil.writeWithLength(key.key, out);
         }
 
@@ -395,7 +403,7 @@ public class CacheService implements CacheServiceMBean
             final ByteBuffer buffer = ByteBufferUtil.readWithLength(in);
             if (cfs == null  || !cfs.isRowCacheEnabled())
                 return null;
-            final int rowsToCache = cfs.metadata.params.caching.rowsPerPartitionToCache();
+            final int rowsToCache = cfs.metadata().params.caching.rowsPerPartitionToCache();
             assert(!cfs.isIndex());//Shouldn't have row cache entries for indexes
 
             return StageManager.getStage(Stage.READ).submit(new Callable<Pair<RowCacheKey, IRowCacheEntry>>()
@@ -404,11 +412,11 @@ public class CacheService implements CacheServiceMBean
                 {
                     DecoratedKey key = cfs.decorateKey(buffer);
                     int nowInSec = FBUtilities.nowInSeconds();
-                    SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(cfs.metadata, nowInSec, key);
+                    SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(cfs.metadata(), nowInSec, key);
                     try (ReadExecutionController controller = cmd.executionController(); UnfilteredRowIterator iter = cmd.queryMemtableAndDisk(cfs, controller))
                     {
                         CachedPartition toCache = CachedBTreePartition.create(DataLimits.cqlLimits(rowsToCache).filter(iter, nowInSec), nowInSec);
-                        return Pair.create(new RowCacheKey(cfs.metadata.ksAndCFName, key), (IRowCacheEntry)toCache);
+                        return Pair.create(new RowCacheKey(cfs.metadata(), key), toCache);
                     }
                 }
             });
@@ -423,13 +431,15 @@ public class CacheService implements CacheServiceMBean
             if (entry == null)
                 return;
 
-            out.write(cfs.metadata.ksAndCFBytes);
+            TableMetadata tableMetadata = cfs.metadata();
+            tableMetadata.id.serialize(out);
+            out.writeUTF(tableMetadata.indexName().orElse(""));
             ByteBufferUtil.writeWithLength(key.key, out);
             out.writeInt(key.desc.generation);
             out.writeBoolean(true);
 
-            SerializationHeader header = new SerializationHeader(false, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS);
-            key.desc.getFormat().getIndexSerializer(cfs.metadata, key.desc.version, header).serializeForCache(entry, out);
+            SerializationHeader header = new SerializationHeader(false, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS);
+            key.desc.getFormat().getIndexSerializer(cfs.metadata(), key.desc.version, header).serializeForCache(entry, out);
         }
 
         public Future<Pair<KeyCacheKey, RowIndexEntry>> deserialize(DataInputPlus input, ColumnFamilyStore cfs) throws IOException
@@ -455,11 +465,11 @@ public class CacheService implements CacheServiceMBean
                 RowIndexEntry.Serializer.skipForCache(input);
                 return null;
             }
-            RowIndexEntry.IndexSerializer<?> indexSerializer = reader.descriptor.getFormat().getIndexSerializer(reader.metadata,
+            RowIndexEntry.IndexSerializer<?> indexSerializer = reader.descriptor.getFormat().getIndexSerializer(reader.metadata(),
                                                                                                                 reader.descriptor.version,
                                                                                                                 reader.header);
             RowIndexEntry<?> entry = indexSerializer.deserializeForCache(input);
-            return Futures.immediateFuture(Pair.create(new KeyCacheKey(cfs.metadata.ksAndCFName, reader.descriptor, key), entry));
+            return Futures.immediateFuture(Pair.create(new KeyCacheKey(cfs.metadata(), reader.descriptor, key), entry));
         }
 
         private SSTableReader findDesc(int generation, Iterable<SSTableReader> collection)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 a026d6e..2beb039 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -46,10 +46,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.functions.ThreadAwareSecurityManager;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.db.*;
@@ -244,7 +245,7 @@ public class CassandraDaemon
             if (keyspaceName.equals(SchemaConstants.SYSTEM_KEYSPACE_NAME))
                 continue;
 
-            for (CFMetaData cfm : Schema.instance.getTablesAndViews(keyspaceName))
+            for (TableMetadata cfm : Schema.instance.getTablesAndViews(keyspaceName))
             {
                 try
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 52e71ae..80cf810 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -28,13 +28,13 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.*;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.QueryHandler;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.Validation;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.AuthenticationException;
@@ -255,7 +255,7 @@ public class ClientState
     {
         // Skip keyspace validation for non-authenticated users. Apparently, some client libraries
         // call set_keyspace() before calling login(), and we have to handle that.
-        if (user != null && Schema.instance.getKSMetaData(ks) == null)
+        if (user != null && Schema.instance.getKeyspaceMetadata(ks) == null)
             throw new InvalidRequestException("Keyspace '" + ks + "' does not exist");
         keyspace = ks;
     }
@@ -290,14 +290,20 @@ public class ClientState
     public void hasColumnFamilyAccess(String keyspace, String columnFamily, Permission perm)
     throws UnauthorizedException, InvalidRequestException
     {
-        Validation.validateColumnFamily(keyspace, columnFamily);
+        Schema.instance.validateTable(keyspace, columnFamily);
         hasAccess(keyspace, perm, DataResource.table(keyspace, columnFamily));
     }
 
-    public void hasColumnFamilyAccess(CFMetaData cfm, Permission perm)
+    public void hasColumnFamilyAccess(TableMetadataRef tableRef, Permission perm)
     throws UnauthorizedException, InvalidRequestException
     {
-        hasAccess(cfm.ksName, perm, cfm.resource);
+        hasColumnFamilyAccess(tableRef.get(), perm);
+    }
+
+    public void hasColumnFamilyAccess(TableMetadata table, Permission perm)
+    throws UnauthorizedException, InvalidRequestException
+    {
+        hasAccess(table.keyspace, perm, table.resource);
     }
 
     private void hasAccess(String keyspace, Permission perm, DataResource resource)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/DataResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DataResolver.java b/src/java/org/apache/cassandra/service/DataResolver.java
index 991c768..cc7ab2d 100644
--- a/src/java/org/apache/cassandra/service/DataResolver.java
+++ b/src/java/org/apache/cassandra/service/DataResolver.java
@@ -25,8 +25,8 @@ import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
@@ -122,7 +122,7 @@ public class DataResolver extends ResponseResolver
             return new MergeListener(partitionKey, columns(versions), isReversed(versions));
         }
 
-        private PartitionColumns columns(List<UnfilteredRowIterator> versions)
+        private RegularAndStaticColumns columns(List<UnfilteredRowIterator> versions)
         {
             Columns statics = Columns.NONE;
             Columns regulars = Columns.NONE;
@@ -131,11 +131,11 @@ public class DataResolver extends ResponseResolver
                 if (iter == null)
                     continue;
 
-                PartitionColumns cols = iter.columns();
+                RegularAndStaticColumns cols = iter.columns();
                 statics = statics.mergeTo(cols.statics);
                 regulars = regulars.mergeTo(cols.regulars);
             }
-            return new PartitionColumns(statics, regulars);
+            return new RegularAndStaticColumns(statics, regulars);
         }
 
         private boolean isReversed(List<UnfilteredRowIterator> versions)
@@ -175,7 +175,7 @@ public class DataResolver extends ResponseResolver
         private class MergeListener implements UnfilteredRowIterators.MergeListener
         {
             private final DecoratedKey partitionKey;
-            private final PartitionColumns columns;
+            private final RegularAndStaticColumns columns;
             private final boolean isReversed;
             private final PartitionUpdate[] repairs = new PartitionUpdate[sources.length];
 
@@ -191,7 +191,7 @@ public class DataResolver extends ResponseResolver
             // For each source, record if there is an open range to send as repair, and from where.
             private final ClusteringBound[] markerToRepair = new ClusteringBound[sources.length];
 
-            public MergeListener(DecoratedKey partitionKey, PartitionColumns columns, boolean isReversed)
+            public MergeListener(DecoratedKey partitionKey, RegularAndStaticColumns columns, boolean isReversed)
             {
                 this.partitionKey = partitionKey;
                 this.columns = columns;
@@ -211,7 +211,7 @@ public class DataResolver extends ResponseResolver
                             currentRow(i, clustering).addRowDeletion(merged);
                     }
 
-                    public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
+                    public void onComplexDeletion(int i, Clustering clustering, ColumnMetadata column, DeletionTime merged, DeletionTime original)
                     {
                         if (merged != null && !merged.equals(original))
                             currentRow(i, clustering).addComplexDeletion(column, merged);
@@ -231,7 +231,7 @@ public class DataResolver extends ResponseResolver
                         // semantic (making sure we can always distinguish between a row that doesn't exist from one that do exist but has
                         /// no value for the column requested by the user) and so it won't be unexpected by the user that those columns are
                         // not repaired.
-                        ColumnDefinition column = cell.column();
+                        ColumnMetadata column = cell.column();
                         ColumnFilter filter = command.columnFilter();
                         return column.isComplex() ? filter.fetchedCellIsQueried(column, cell.path()) : filter.fetchedColumnIsQueried(column);
                     }
@@ -422,12 +422,12 @@ public class DataResolver extends ResponseResolver
 
         private class ShortReadRowProtection extends Transformation implements MoreRows<UnfilteredRowIterator>
         {
-            final CFMetaData metadata;
+            final TableMetadata metadata;
             final DecoratedKey partitionKey;
             Clustering lastClustering;
             int lastCount = 0;
 
-            private ShortReadRowProtection(CFMetaData metadata, DecoratedKey partitionKey)
+            private ShortReadRowProtection(TableMetadata metadata, DecoratedKey partitionKey)
             {
                 this.metadata = metadata;
                 this.partitionKey = partitionKey;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/MigrationListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationListener.java b/src/java/org/apache/cassandra/service/MigrationListener.java
deleted file mode 100644
index 19d2592..0000000
--- a/src/java/org/apache/cassandra/service/MigrationListener.java
+++ /dev/null
@@ -1,102 +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.service;
-
-import java.util.List;
-
-import org.apache.cassandra.db.marshal.AbstractType;
-
-public abstract class MigrationListener
-{
-    public void onCreateKeyspace(String ksName)
-    {
-    }
-
-    public void onCreateColumnFamily(String ksName, String cfName)
-    {
-    }
-
-    public void onCreateView(String ksName, String viewName)
-    {
-        onCreateColumnFamily(ksName, viewName);
-    }
-
-    public void onCreateUserType(String ksName, String typeName)
-    {
-    }
-
-    public void onCreateFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
-    {
-    }
-
-    public void onCreateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
-    {
-    }
-
-    public void onUpdateKeyspace(String ksName)
-    {
-    }
-
-    // the boolean flag indicates whether the change that triggered this event may have a substantive
-    // impact on statements using the column family.
-    public void onUpdateColumnFamily(String ksName, String cfName, boolean affectsStatements)
-    {
-    }
-
-    public void onUpdateView(String ksName, String viewName, boolean columnsDidChange)
-    {
-        onUpdateColumnFamily(ksName, viewName, columnsDidChange);
-    }
-
-    public void onUpdateUserType(String ksName, String typeName)
-    {
-    }
-
-    public void onUpdateFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
-    {
-    }
-
-    public void onUpdateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
-    {
-    }
-
-    public void onDropKeyspace(String ksName)
-    {
-    }
-
-    public void onDropColumnFamily(String ksName, String cfName)
-    {
-    }
-
-    public void onDropView(String ksName, String viewName)
-    {
-        onDropColumnFamily(ksName, viewName);
-    }
-
-    public void onDropUserType(String ksName, String typeName)
-    {
-    }
-
-    public void onDropFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
-    {
-    }
-
-    public void onDropAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
-    {
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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
deleted file mode 100644
index f9e4aff..0000000
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ /dev/null
@@ -1,626 +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.service;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.util.*;
-import java.util.concurrent.*;
-import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
-import org.apache.cassandra.config.ViewDefinition;
-import org.apache.cassandra.cql3.functions.UDAggregate;
-import org.apache.cassandra.cql3.functions.UDFunction;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.marshal.UserType;
-import org.apache.cassandra.exceptions.AlreadyExistsException;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.gms.*;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.WrappedRunnable;
-
-public class MigrationManager
-{
-    private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
-
-    public static final MigrationManager instance = new MigrationManager();
-
-    private static final RuntimeMXBean runtimeMXBean = ManagementFactory.getRuntimeMXBean();
-
-    public static final int MIGRATION_DELAY_IN_MS = 60000;
-
-    private static final int MIGRATION_TASK_WAIT_IN_SECONDS = Integer.parseInt(System.getProperty("cassandra.migration_task_wait_in_seconds", "1"));
-
-    private final List<MigrationListener> listeners = new CopyOnWriteArrayList<>();
-
-    private MigrationManager() {}
-
-    public void register(MigrationListener listener)
-    {
-        listeners.add(listener);
-    }
-
-    public void unregister(MigrationListener listener)
-    {
-        listeners.remove(listener);
-    }
-
-    public static void scheduleSchemaPull(InetAddress endpoint, EndpointState state)
-    {
-        VersionedValue value = state.getApplicationState(ApplicationState.SCHEMA);
-
-        if (!endpoint.equals(FBUtilities.getBroadcastAddress()) && value != null)
-            maybeScheduleSchemaPull(UUID.fromString(value.value), endpoint);
-    }
-
-    /**
-     * If versions differ this node sends request with local migration list to the endpoint
-     * and expecting to receive a list of migrations to apply locally.
-     */
-    private static void maybeScheduleSchemaPull(final UUID theirVersion, final InetAddress endpoint)
-    {
-        if ((Schema.instance.getVersion() != null && Schema.instance.getVersion().equals(theirVersion)) || !shouldPullSchemaFrom(endpoint))
-        {
-            logger.debug("Not pulling schema because versions match or shouldPullSchemaFrom returned false");
-            return;
-        }
-
-        if (SchemaConstants.emptyVersion.equals(Schema.instance.getVersion()) || runtimeMXBean.getUptime() < MIGRATION_DELAY_IN_MS)
-        {
-            // If we think we may be bootstrapping or have recently started, submit MigrationTask immediately
-            logger.debug("Submitting migration task for {}", endpoint);
-            submitMigrationTask(endpoint);
-        }
-        else
-        {
-            // Include a delay to make sure we have a chance to apply any changes being
-            // pushed out simultaneously. See CASSANDRA-5025
-            Runnable runnable = () ->
-            {
-                // grab the latest version of the schema since it may have changed again since the initial scheduling
-                EndpointState epState = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
-                if (epState == null)
-                {
-                    logger.debug("epState vanished for {}, not submitting migration task", endpoint);
-                    return;
-                }
-                VersionedValue value = epState.getApplicationState(ApplicationState.SCHEMA);
-                UUID currentVersion = UUID.fromString(value.value);
-                if (Schema.instance.getVersion().equals(currentVersion))
-                {
-                    logger.debug("not submitting migration task for {} because our versions match", endpoint);
-                    return;
-                }
-                logger.debug("submitting migration task for {}", endpoint);
-                submitMigrationTask(endpoint);
-            };
-            ScheduledExecutors.nonPeriodicTasks.schedule(runnable, MIGRATION_DELAY_IN_MS, TimeUnit.MILLISECONDS);
-        }
-    }
-
-    private static Future<?> submitMigrationTask(InetAddress endpoint)
-    {
-        /*
-         * Do not de-ref the future because that causes distributed deadlock (CASSANDRA-3832) because we are
-         * running in the gossip stage.
-         */
-        return StageManager.getStage(Stage.MIGRATION).submit(new MigrationTask(endpoint));
-    }
-
-    public static boolean shouldPullSchemaFrom(InetAddress endpoint)
-    {
-        /*
-         * Don't request schema from nodes with a differnt or unknonw major version (may have incompatible schema)
-         * Don't request schema from fat clients
-         */
-        return MessagingService.instance().knowsVersion(endpoint)
-                && MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version
-                && !Gossiper.instance.isGossipOnlyMember(endpoint);
-    }
-
-    public static boolean isReadyForBootstrap()
-    {
-        return MigrationTask.getInflightTasks().isEmpty();
-    }
-
-    public static void waitUntilReadyForBootstrap()
-    {
-        CountDownLatch completionLatch;
-        while ((completionLatch = MigrationTask.getInflightTasks().poll()) != null)
-        {
-            try
-            {
-                if (!completionLatch.await(MIGRATION_TASK_WAIT_IN_SECONDS, TimeUnit.SECONDS))
-                    logger.error("Migration task failed to complete");
-            }
-            catch (InterruptedException e)
-            {
-                Thread.currentThread().interrupt();
-                logger.error("Migration task was interrupted");
-            }
-        }
-    }
-
-    public void notifyCreateKeyspace(KeyspaceMetadata ksm)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onCreateKeyspace(ksm.name);
-    }
-
-    public void notifyCreateColumnFamily(CFMetaData cfm)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onCreateColumnFamily(cfm.ksName, cfm.cfName);
-    }
-
-    public void notifyCreateView(ViewDefinition view)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onCreateView(view.ksName, view.viewName);
-    }
-
-    public void notifyCreateUserType(UserType ut)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onCreateUserType(ut.keyspace, ut.getNameAsString());
-    }
-
-    public void notifyCreateFunction(UDFunction udf)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onCreateFunction(udf.name().keyspace, udf.name().name, udf.argTypes());
-    }
-
-    public void notifyCreateAggregate(UDAggregate udf)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onCreateAggregate(udf.name().keyspace, udf.name().name, udf.argTypes());
-    }
-
-    public void notifyUpdateKeyspace(KeyspaceMetadata ksm)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onUpdateKeyspace(ksm.name);
-    }
-
-    public void notifyUpdateColumnFamily(CFMetaData cfm, boolean columnsDidChange)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onUpdateColumnFamily(cfm.ksName, cfm.cfName, columnsDidChange);
-    }
-
-    public void notifyUpdateView(ViewDefinition view, boolean columnsDidChange)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onUpdateView(view.ksName, view.viewName, columnsDidChange);
-    }
-
-    public void notifyUpdateUserType(UserType ut)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onUpdateUserType(ut.keyspace, ut.getNameAsString());
-
-        // FIXME: remove when we get rid of AbstractType in metadata. Doesn't really belong anywhere.
-        Schema.instance.getKSMetaData(ut.keyspace).functions.udfs().forEach(f -> f.userTypeUpdated(ut.keyspace, ut.getNameAsString()));
-    }
-
-    public void notifyUpdateFunction(UDFunction udf)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onUpdateFunction(udf.name().keyspace, udf.name().name, udf.argTypes());
-    }
-
-    public void notifyUpdateAggregate(UDAggregate udf)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onUpdateAggregate(udf.name().keyspace, udf.name().name, udf.argTypes());
-    }
-
-    public void notifyDropKeyspace(KeyspaceMetadata ksm)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onDropKeyspace(ksm.name);
-    }
-
-    public void notifyDropColumnFamily(CFMetaData cfm)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onDropColumnFamily(cfm.ksName, cfm.cfName);
-    }
-
-    public void notifyDropView(ViewDefinition view)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onDropView(view.ksName, view.viewName);
-    }
-
-    public void notifyDropUserType(UserType ut)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onDropUserType(ut.keyspace, ut.getNameAsString());
-    }
-
-    public void notifyDropFunction(UDFunction udf)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onDropFunction(udf.name().keyspace, udf.name().name, udf.argTypes());
-    }
-
-    public void notifyDropAggregate(UDAggregate udf)
-    {
-        for (MigrationListener listener : listeners)
-            listener.onDropAggregate(udf.name().keyspace, udf.name().name, udf.argTypes());
-    }
-
-    public static void announceNewKeyspace(KeyspaceMetadata ksm) throws ConfigurationException
-    {
-        announceNewKeyspace(ksm, false);
-    }
-
-    public static void announceNewKeyspace(KeyspaceMetadata ksm, boolean announceLocally) throws ConfigurationException
-    {
-        announceNewKeyspace(ksm, FBUtilities.timestampMicros(), announceLocally);
-    }
-
-    public static void announceNewKeyspace(KeyspaceMetadata ksm, long timestamp, boolean announceLocally) throws ConfigurationException
-    {
-        ksm.validate();
-
-        if (Schema.instance.getKSMetaData(ksm.name) != null)
-            throw new AlreadyExistsException(ksm.name);
-
-        logger.info("Create new Keyspace: {}", ksm);
-        announce(SchemaKeyspace.makeCreateKeyspaceMutation(ksm, timestamp), announceLocally);
-    }
-
-    public static void announceNewColumnFamily(CFMetaData cfm) throws ConfigurationException
-    {
-        announceNewColumnFamily(cfm, false);
-    }
-
-    public static void announceNewColumnFamily(CFMetaData cfm, boolean announceLocally) throws ConfigurationException
-    {
-        announceNewColumnFamily(cfm, announceLocally, true);
-    }
-
-    /**
-     * Announces the table even if the definition is already know locally.
-     * This should generally be avoided but is used internally when we want to force the most up to date version of
-     * a system table schema (Note that we don't know if the schema we force _is_ the most recent version or not, we
-     * just rely on idempotency to basically ignore that announce if it's not. That's why we can't use announceUpdateColumnFamily,
-     * it would for instance delete new columns if this is not called with the most up-to-date version)
-     *
-     * Note that this is only safe for system tables where we know the cfId is fixed and will be the same whatever version
-     * of the definition is used.
-     */
-    public static void forceAnnounceNewColumnFamily(CFMetaData cfm) throws ConfigurationException
-    {
-        announceNewColumnFamily(cfm, false, false);
-    }
-
-    private static void announceNewColumnFamily(CFMetaData cfm, boolean announceLocally, boolean throwOnDuplicate) throws ConfigurationException
-    {
-        cfm.validate();
-
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(cfm.ksName);
-        if (ksm == null)
-            throw new ConfigurationException(String.format("Cannot add table '%s' to non existing keyspace '%s'.", cfm.cfName, cfm.ksName));
-        // If we have a table or a view which has the same name, we can't add a new one
-        else if (throwOnDuplicate && ksm.getTableOrViewNullable(cfm.cfName) != null)
-            throw new AlreadyExistsException(cfm.ksName, cfm.cfName);
-
-        logger.info("Create new table: {}", cfm);
-        announce(SchemaKeyspace.makeCreateTableMutation(ksm, cfm, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceNewView(ViewDefinition view, boolean announceLocally) throws ConfigurationException
-    {
-        view.metadata.validate();
-
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(view.ksName);
-        if (ksm == null)
-            throw new ConfigurationException(String.format("Cannot add table '%s' to non existing keyspace '%s'.", view.viewName, view.ksName));
-        else if (ksm.getTableOrViewNullable(view.viewName) != null)
-            throw new AlreadyExistsException(view.ksName, view.viewName);
-
-        logger.info("Create new view: {}", view);
-        announce(SchemaKeyspace.makeCreateViewMutation(ksm, view, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceNewType(UserType newType, boolean announceLocally)
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(newType.keyspace);
-        announce(SchemaKeyspace.makeCreateTypeMutation(ksm, newType, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceNewFunction(UDFunction udf, boolean announceLocally)
-    {
-        logger.info("Create scalar function '{}'", udf.name());
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
-        announce(SchemaKeyspace.makeCreateFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceNewAggregate(UDAggregate udf, boolean announceLocally)
-    {
-        logger.info("Create aggregate function '{}'", udf.name());
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
-        announce(SchemaKeyspace.makeCreateAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceKeyspaceUpdate(KeyspaceMetadata ksm) throws ConfigurationException
-    {
-        announceKeyspaceUpdate(ksm, false);
-    }
-
-    public static void announceKeyspaceUpdate(KeyspaceMetadata ksm, boolean announceLocally) throws ConfigurationException
-    {
-        ksm.validate();
-
-        KeyspaceMetadata oldKsm = Schema.instance.getKSMetaData(ksm.name);
-        if (oldKsm == null)
-            throw new ConfigurationException(String.format("Cannot update non existing keyspace '%s'.", ksm.name));
-
-        logger.info("Update Keyspace '{}' From {} To {}", ksm.name, oldKsm, ksm);
-        announce(SchemaKeyspace.makeCreateKeyspaceMutation(ksm.name, ksm.params, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceColumnFamilyUpdate(CFMetaData cfm) throws ConfigurationException
-    {
-        announceColumnFamilyUpdate(cfm, false);
-    }
-
-    public static void announceColumnFamilyUpdate(CFMetaData cfm, boolean announceLocally) throws ConfigurationException
-    {
-        cfm.validate();
-
-        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));
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(cfm.ksName);
-
-        oldCfm.validateCompatibility(cfm);
-
-        logger.info("Update table '{}/{}' From {} To {}", cfm.ksName, cfm.cfName, oldCfm, cfm);
-        announce(SchemaKeyspace.makeUpdateTableMutation(ksm, oldCfm, cfm, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceViewUpdate(ViewDefinition view, boolean announceLocally) throws ConfigurationException
-    {
-        view.metadata.validate();
-
-        ViewDefinition oldView = Schema.instance.getView(view.ksName, view.viewName);
-        if (oldView == null)
-            throw new ConfigurationException(String.format("Cannot update non existing materialized view '%s' in keyspace '%s'.", view.viewName, view.ksName));
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(view.ksName);
-
-        oldView.metadata.validateCompatibility(view.metadata);
-
-        logger.info("Update view '{}/{}' From {} To {}", view.ksName, view.viewName, oldView, view);
-        announce(SchemaKeyspace.makeUpdateViewMutation(ksm, oldView, view, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceTypeUpdate(UserType updatedType, boolean announceLocally)
-    {
-        logger.info("Update type '{}.{}' to {}", updatedType.keyspace, updatedType.getNameAsString(), updatedType);
-        announceNewType(updatedType, announceLocally);
-    }
-
-    public static void announceKeyspaceDrop(String ksName) throws ConfigurationException
-    {
-        announceKeyspaceDrop(ksName, false);
-    }
-
-    public static void announceKeyspaceDrop(String ksName, boolean announceLocally) throws ConfigurationException
-    {
-        KeyspaceMetadata oldKsm = Schema.instance.getKSMetaData(ksName);
-        if (oldKsm == null)
-            throw new ConfigurationException(String.format("Cannot drop non existing keyspace '%s'.", ksName));
-
-        logger.info("Drop Keyspace '{}'", oldKsm.name);
-        announce(SchemaKeyspace.makeDropKeyspaceMutation(oldKsm, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceColumnFamilyDrop(String ksName, String cfName) throws ConfigurationException
-    {
-        announceColumnFamilyDrop(ksName, cfName, false);
-    }
-
-    public static void announceColumnFamilyDrop(String ksName, String cfName, boolean announceLocally) throws ConfigurationException
-    {
-        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));
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(ksName);
-
-        logger.info("Drop table '{}/{}'", oldCfm.ksName, oldCfm.cfName);
-        announce(SchemaKeyspace.makeDropTableMutation(ksm, oldCfm, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceViewDrop(String ksName, String viewName, boolean announceLocally) throws ConfigurationException
-    {
-        ViewDefinition view = Schema.instance.getView(ksName, viewName);
-        if (view == null)
-            throw new ConfigurationException(String.format("Cannot drop non existing materialized view '%s' in keyspace '%s'.", viewName, ksName));
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(ksName);
-
-        logger.info("Drop table '{}/{}'", view.ksName, view.viewName);
-        announce(SchemaKeyspace.makeDropViewMutation(ksm, view, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceTypeDrop(UserType droppedType)
-    {
-        announceTypeDrop(droppedType, false);
-    }
-
-    public static void announceTypeDrop(UserType droppedType, boolean announceLocally)
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(droppedType.keyspace);
-        announce(SchemaKeyspace.dropTypeFromSchemaMutation(ksm, droppedType, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceFunctionDrop(UDFunction udf, boolean announceLocally)
-    {
-        logger.info("Drop scalar function overload '{}' args '{}'", udf.name(), udf.argTypes());
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
-        announce(SchemaKeyspace.makeDropFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    public static void announceAggregateDrop(UDAggregate udf, boolean announceLocally)
-    {
-        logger.info("Drop aggregate function overload '{}' args '{}'", udf.name(), udf.argTypes());
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
-        announce(SchemaKeyspace.makeDropAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
-    }
-
-    /**
-     * actively announce a new version to active hosts via rpc
-     * @param schema The schema mutation to be applied
-     */
-    private static void announce(Mutation.SimpleBuilder schema, boolean announceLocally)
-    {
-        List<Mutation> mutations = Collections.singletonList(schema.build());
-
-        if (announceLocally)
-            SchemaKeyspace.mergeSchema(mutations);
-        else
-            FBUtilities.waitOnFuture(announce(mutations));
-    }
-
-    private static void pushSchemaMutation(InetAddress endpoint, Collection<Mutation> schema)
-    {
-        MessageOut<Collection<Mutation>> msg = new MessageOut<>(MessagingService.Verb.DEFINITIONS_UPDATE,
-                                                                schema,
-                                                                MigrationsSerializer.instance);
-        MessagingService.instance().sendOneWay(msg, endpoint);
-    }
-
-    // Returns a future on the local application of the schema
-    private static Future<?> announce(final Collection<Mutation> schema)
-    {
-        Future<?> f = StageManager.getStage(Stage.MIGRATION).submit(new WrappedRunnable()
-        {
-            protected void runMayThrow() throws ConfigurationException
-            {
-                SchemaKeyspace.mergeSchemaAndAnnounceVersion(schema);
-            }
-        });
-
-        for (InetAddress endpoint : Gossiper.instance.getLiveMembers())
-        {
-            // only push schema to nodes with known and equal versions
-            if (!endpoint.equals(FBUtilities.getBroadcastAddress()) &&
-                    MessagingService.instance().knowsVersion(endpoint) &&
-                    MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version)
-                pushSchemaMutation(endpoint, schema);
-        }
-
-        return f;
-    }
-
-    /**
-     * Announce my version passively over gossip.
-     * Used to notify nodes as they arrive in the cluster.
-     *
-     * @param version The schema version to announce
-     */
-    public static void passiveAnnounce(UUID version)
-    {
-        Gossiper.instance.addLocalApplicationState(ApplicationState.SCHEMA, StorageService.instance.valueFactory.schema(version));
-        logger.debug("Gossiping my schema version {}", version);
-    }
-
-    /**
-     * Clear all locally stored schema information and reset schema to initial state.
-     * Called by user (via JMX) who wants to get rid of schema disagreement.
-     */
-    public static void resetLocalSchema()
-    {
-        logger.info("Starting local schema reset...");
-
-        logger.debug("Truncating schema tables...");
-
-        SchemaKeyspace.truncate();
-
-        logger.debug("Clearing local schema keyspace definitions...");
-
-        Schema.instance.clear();
-
-        Set<InetAddress> liveEndpoints = Gossiper.instance.getLiveMembers();
-        liveEndpoints.remove(FBUtilities.getBroadcastAddress());
-
-        // force migration if there are nodes around
-        for (InetAddress node : liveEndpoints)
-        {
-            if (shouldPullSchemaFrom(node))
-            {
-                logger.debug("Requesting schema from {}", node);
-                FBUtilities.waitOnFuture(submitMigrationTask(node));
-                break;
-            }
-        }
-
-        logger.info("Local schema reset is complete.");
-    }
-
-    public static class MigrationsSerializer implements IVersionedSerializer<Collection<Mutation>>
-    {
-        public static MigrationsSerializer instance = new MigrationsSerializer();
-
-        public void serialize(Collection<Mutation> schema, DataOutputPlus out, int version) throws IOException
-        {
-            out.writeInt(schema.size());
-            for (Mutation mutation : schema)
-                Mutation.serializer.serialize(mutation, out, version);
-        }
-
-        public Collection<Mutation> deserialize(DataInputPlus in, int version) throws IOException
-        {
-            int count = in.readInt();
-            Collection<Mutation> schema = new ArrayList<>(count);
-
-            for (int i = 0; i < count; i++)
-                schema.add(Mutation.serializer.deserialize(in, version));
-
-            return schema;
-        }
-
-        public long serializedSize(Collection<Mutation> schema, int version)
-        {
-            int size = TypeSizes.sizeof(schema.size());
-            for (Mutation mutation : schema)
-                size += Mutation.serializer.serializedSize(mutation, version);
-            return size;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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
deleted file mode 100644
index 052b89e..0000000
--- a/src/java/org/apache/cassandra/service/MigrationTask.java
+++ /dev/null
@@ -1,115 +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.service;
-
-import java.net.InetAddress;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.Set;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.CountDownLatch;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.db.SystemKeyspace.BootstrapState;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.gms.FailureDetector;
-import org.apache.cassandra.net.IAsyncCallback;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.utils.WrappedRunnable;
-
-
-class MigrationTask extends WrappedRunnable
-{
-    private static final Logger logger = LoggerFactory.getLogger(MigrationTask.class);
-
-    private static final ConcurrentLinkedQueue<CountDownLatch> inflightTasks = new ConcurrentLinkedQueue<>();
-
-    private static final Set<BootstrapState> monitoringBootstrapStates = EnumSet.of(BootstrapState.NEEDS_BOOTSTRAP, BootstrapState.IN_PROGRESS);
-
-    private final InetAddress endpoint;
-
-    MigrationTask(InetAddress endpoint)
-    {
-        this.endpoint = endpoint;
-    }
-
-    public static ConcurrentLinkedQueue<CountDownLatch> getInflightTasks()
-    {
-        return inflightTasks;
-    }
-
-    public void runMayThrow() throws Exception
-    {
-        // There is a chance that quite some time could have passed between now and the MM#maybeScheduleSchemaPull(),
-        // potentially enough for the endpoint node to restart - which is an issue if it does restart upgraded, with
-        // a higher major.
-        if (!MigrationManager.shouldPullSchemaFrom(endpoint))
-        {
-            logger.info("Skipped sending a migration request: node {} has a higher major version now.", endpoint);
-            return;
-        }
-
-        if (!FailureDetector.instance.isAlive(endpoint))
-        {
-            logger.debug("Can't send schema pull request: node {} is down.", endpoint);
-            return;
-        }
-
-        MessageOut message = new MessageOut<>(MessagingService.Verb.MIGRATION_REQUEST, null, MigrationManager.MigrationsSerializer.instance);
-
-        final CountDownLatch completionLatch = new CountDownLatch(1);
-
-        IAsyncCallback<Collection<Mutation>> cb = new IAsyncCallback<Collection<Mutation>>()
-        {
-            @Override
-            public void response(MessageIn<Collection<Mutation>> message)
-            {
-                try
-                {
-                    SchemaKeyspace.mergeSchemaAndAnnounceVersion(message.payload);
-                }
-                catch (ConfigurationException e)
-                {
-                    logger.error("Configuration exception merging remote schema", e);
-                }
-                finally
-                {
-                    completionLatch.countDown();
-                }
-            }
-
-            public boolean isLatencyForSnitch()
-            {
-                return false;
-            }
-        };
-
-        // Only save the latches if we need bootstrap or are bootstrapping
-        if (monitoringBootstrapStates.contains(SystemKeyspace.getBootstrapState()))
-            inflightTasks.offer(completionLatch);
-
-        MessagingService.instance().sendRR(message, endpoint, cb);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
index 5b1aa0d..297774a 100644
--- a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
+++ b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.service;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/ReadCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ReadCallback.java b/src/java/org/apache/cassandra/service/ReadCallback.java
index 440e35a..d63e860 100644
--- a/src/java/org/apache/cassandra/service/ReadCallback.java
+++ b/src/java/org/apache/cassandra/service/ReadCallback.java
@@ -77,9 +77,9 @@ public class ReadCallback implements IAsyncCallbackWithFailure<ReadResponse>
     {
         this(resolver,
              consistencyLevel,
-             consistencyLevel.blockFor(Keyspace.open(command.metadata().ksName)),
+             consistencyLevel.blockFor(Keyspace.open(command.metadata().keyspace)),
              command,
-             Keyspace.open(command.metadata().ksName),
+             Keyspace.open(command.metadata().keyspace),
              filteredEndpoints,
              queryStartNanoTime);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/ReadRepairDecision.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ReadRepairDecision.java b/src/java/org/apache/cassandra/service/ReadRepairDecision.java
new file mode 100644
index 0000000..8d2ced7
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/ReadRepairDecision.java
@@ -0,0 +1,23 @@
+/*
+ * 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.service;
+
+public enum ReadRepairDecision
+{
+    NONE, GLOBAL, DC_LOCAL;
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java
index 75f7788..767bfc6 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -29,10 +29,11 @@ import com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.SystemKeyspace;
@@ -319,7 +320,7 @@ public class StartupChecks
             // we do a one-off scrub of the system keyspace first; we can't load the list of the rest of the keyspaces,
             // until system keyspace is opened.
 
-            for (CFMetaData cfm : Schema.instance.getTablesAndViews(SchemaConstants.SYSTEM_KEYSPACE_NAME))
+            for (TableMetadata cfm : Schema.instance.getTablesAndViews(SchemaConstants.SYSTEM_KEYSPACE_NAME))
                 ColumnFamilyStore.scrubDataDirectories(cfm);
 
             try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 d5a9f47..0585717 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -44,10 +44,10 @@ import org.apache.cassandra.batchlog.Batch;
 import org.apache.cassandra.batchlog.BatchlogManager;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
@@ -236,7 +236,7 @@ public class StorageProxy implements StorageProxyMBean
             consistencyForPaxos.validateForCas();
             consistencyForCommit.validateForCasCommit(keyspaceName);
 
-            CFMetaData metadata = Schema.instance.getCFMetaData(keyspaceName, cfName);
+            TableMetadata metadata = Schema.instance.getTableMetadata(keyspaceName, cfName);
 
             long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout());
             while (System.nanoTime() - queryStartNanoTime < timeout)
@@ -348,11 +348,11 @@ public class StorageProxy implements StorageProxyMBean
         };
     }
 
-    private static Pair<List<InetAddress>, Integer> getPaxosParticipants(CFMetaData cfm, DecoratedKey key, ConsistencyLevel consistencyForPaxos) throws UnavailableException
+    private static Pair<List<InetAddress>, Integer> getPaxosParticipants(TableMetadata metadata, DecoratedKey key, ConsistencyLevel consistencyForPaxos) throws UnavailableException
     {
         Token tk = key.getToken();
-        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(cfm.ksName, tk);
-        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, cfm.ksName);
+        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(metadata.keyspace, tk);
+        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, metadata.keyspace);
         if (consistencyForPaxos == ConsistencyLevel.LOCAL_SERIAL)
         {
             // Restrict naturalEndpoints and pendingEndpoints to node in the local DC only
@@ -387,7 +387,7 @@ public class StorageProxy implements StorageProxyMBean
      */
     private static Pair<UUID, Integer> beginAndRepairPaxos(long queryStartNanoTime,
                                                            DecoratedKey key,
-                                                           CFMetaData metadata,
+                                                           TableMetadata metadata,
                                                            List<InetAddress> liveEndpoints,
                                                            int requiredParticipants,
                                                            ConsistencyLevel consistencyForPaxos,
@@ -482,7 +482,7 @@ public class StorageProxy implements StorageProxyMBean
         }
 
         recordCasContention(contentions);
-        throw new WriteTimeoutException(WriteType.CAS, consistencyForPaxos, 0, consistencyForPaxos.blockFor(Keyspace.open(metadata.ksName)));
+        throw new WriteTimeoutException(WriteType.CAS, consistencyForPaxos, 0, consistencyForPaxos.blockFor(Keyspace.open(metadata.keyspace)));
     }
 
     /**
@@ -528,7 +528,7 @@ public class StorageProxy implements StorageProxyMBean
     private static void commitPaxos(Commit proposal, ConsistencyLevel consistencyLevel, boolean shouldHint, long queryStartNanoTime) throws WriteTimeoutException
     {
         boolean shouldBlock = consistencyLevel != ConsistencyLevel.ANY;
-        Keyspace keyspace = Keyspace.open(proposal.update.metadata().ksName);
+        Keyspace keyspace = Keyspace.open(proposal.update.metadata().keyspace);
 
         Token tk = proposal.update.partitionKey().getToken();
         List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspace.getName(), tk);
@@ -1513,7 +1513,7 @@ public class StorageProxy implements StorageProxyMBean
     private static boolean systemKeyspaceQuery(List<? extends ReadCommand> cmds)
     {
         for (ReadCommand cmd : cmds)
-            if (!SchemaConstants.isSystemKeyspace(cmd.metadata().ksName))
+            if (!SchemaConstants.isSystemKeyspace(cmd.metadata().keyspace))
                 return false;
         return true;
     }
@@ -1566,7 +1566,7 @@ public class StorageProxy implements StorageProxyMBean
 
         long start = System.nanoTime();
         SinglePartitionReadCommand command = group.commands.get(0);
-        CFMetaData metadata = command.metadata();
+        TableMetadata metadata = command.metadata();
         DecoratedKey key = command.partitionKey();
 
         PartitionIterator result = null;
@@ -1590,7 +1590,7 @@ public class StorageProxy implements StorageProxyMBean
             }
             catch (WriteTimeoutException e)
             {
-                throw new ReadTimeoutException(consistencyLevel, 0, consistencyLevel.blockFor(Keyspace.open(metadata.ksName)), false);
+                throw new ReadTimeoutException(consistencyLevel, 0, consistencyLevel.blockFor(Keyspace.open(metadata.keyspace)), false);
             }
             catch (WriteFailureException e)
             {
@@ -1626,7 +1626,7 @@ public class StorageProxy implements StorageProxyMBean
             readMetrics.addNano(latency);
             casReadMetrics.addNano(latency);
             readMetricsMap.get(consistencyLevel).addNano(latency);
-            Keyspace.open(metadata.ksName).getColumnFamilyStore(metadata.cfName).metric.coordinatorReadLatency.update(latency, TimeUnit.NANOSECONDS);
+            Keyspace.open(metadata.keyspace).getColumnFamilyStore(metadata.name).metric.coordinatorReadLatency.update(latency, TimeUnit.NANOSECONDS);
         }
 
         return result;
@@ -1764,7 +1764,7 @@ public class StorageProxy implements StorageProxyMBean
                 ReadRepairMetrics.repairedBlocking.mark();
 
                 // Do a full data read to resolve the correct response (and repair node that need be)
-                Keyspace keyspace = Keyspace.open(command.metadata().ksName);
+                Keyspace keyspace = Keyspace.open(command.metadata().keyspace);
                 DataResolver resolver = new DataResolver(keyspace, command, ConsistencyLevel.ALL, executor.handler.endpoints.size(), queryStartNanoTime);
                 repairHandler = new ReadCallback(resolver,
                                                  ConsistencyLevel.ALL,
@@ -1805,7 +1805,7 @@ public class StorageProxy implements StorageProxyMBean
                     logger.trace("Timed out waiting on digest mismatch repair requests");
                 // the caught exception here will have CL.ALL from the repair command,
                 // not whatever CL the initial command was at (CASSANDRA-7947)
-                int blockFor = consistency.blockFor(Keyspace.open(command.metadata().ksName));
+                int blockFor = consistency.blockFor(Keyspace.open(command.metadata().keyspace));
                 throw new ReadTimeoutException(consistency, blockFor-1, blockFor, true);
             }
         }
@@ -1903,7 +1903,7 @@ public class StorageProxy implements StorageProxyMBean
      */
     private static float estimateResultsPerRange(PartitionRangeReadCommand command, Keyspace keyspace)
     {
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(command.metadata().cfId);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(command.metadata().id);
         Index index = command.getIndex(cfs);
         float maxExpectedResults = index == null
                                  ? command.limits().estimateTotalResults(cfs)
@@ -2223,7 +2223,7 @@ public class StorageProxy implements StorageProxyMBean
     {
         Tracing.trace("Computing ranges to query");
 
-        Keyspace keyspace = Keyspace.open(command.metadata().ksName);
+        Keyspace keyspace = Keyspace.open(command.metadata().keyspace);
         RangeIterator ranges = new RangeIterator(command, keyspace, consistencyLevel);
 
         // our estimate of how many result rows there will be per-range
@@ -2274,7 +2274,7 @@ public class StorageProxy implements StorageProxyMBean
                 return false;
             }
         };
-        // an empty message acts as a request to the SchemaCheckVerbHandler.
+        // an empty message acts as a request to the SchemaVersionVerbHandler.
         MessageOut message = new MessageOut(MessagingService.Verb.SCHEMA_CHECK);
         for (InetAddress endpoint : liveHosts)
             MessagingService.instance().sendRR(message, endpoint, cb);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 6cb67fc..62f4871 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -52,7 +52,7 @@ import ch.qos.logback.classic.spi.ILoggingEvent;
 import ch.qos.logback.core.Appender;
 import ch.qos.logback.core.hook.DelayingShutdownHook;
 import org.apache.cassandra.auth.AuthKeyspace;
-import org.apache.cassandra.auth.AuthMigrationListener;
+import org.apache.cassandra.auth.AuthSchemaChangeListener;
 import org.apache.cassandra.batchlog.BatchRemoveVerbHandler;
 import org.apache.cassandra.batchlog.BatchStoreVerbHandler;
 import org.apache.cassandra.batchlog.BatchlogManager;
@@ -60,11 +60,7 @@ import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Stage;
 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.config.SchemaConstants;
-import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.compaction.CompactionManager;
@@ -85,6 +81,15 @@ import org.apache.cassandra.repair.*;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.schema.CompactionParams.TombstoneOption;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.SchemaPullVerbHandler;
+import org.apache.cassandra.schema.SchemaPushVerbHandler;
+import org.apache.cassandra.schema.SchemaVersionVerbHandler;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.schema.ViewMetadata;
 import org.apache.cassandra.service.paxos.CommitVerbHandler;
 import org.apache.cassandra.service.paxos.PrepareVerbHandler;
 import org.apache.cassandra.service.paxos.ProposeVerbHandler;
@@ -285,9 +290,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.GOSSIP_DIGEST_ACK, new GossipDigestAckVerbHandler());
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.GOSSIP_DIGEST_ACK2, new GossipDigestAck2VerbHandler());
 
-        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.DEFINITIONS_UPDATE, new DefinitionsUpdateVerbHandler());
-        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.SCHEMA_CHECK, new SchemaCheckVerbHandler());
-        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.MIGRATION_REQUEST, new MigrationRequestVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.DEFINITIONS_UPDATE, new SchemaPushVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.SCHEMA_CHECK, new SchemaVersionVerbHandler());
+        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.MIGRATION_REQUEST, new SchemaPullVerbHandler());
 
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.SNAPSHOT, new SnapshotVerbHandler());
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.ECHO, new EchoVerbHandler());
@@ -1045,7 +1050,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         DatabaseDescriptor.getRoleManager().setup();
         DatabaseDescriptor.getAuthenticator().setup();
         DatabaseDescriptor.getAuthorizer().setup();
-        MigrationManager.instance.register(new AuthMigrationListener());
+        Schema.instance.registerListener(new AuthSchemaChangeListener());
         authSetupComplete = true;
     }
 
@@ -1079,23 +1084,23 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         // (#8162 being an example), so even if the table definition exists, we still need to force the "current"
         // version of the schema, the one the node will be expecting.
 
-        KeyspaceMetadata defined = Schema.instance.getKSMetaData(expected.name);
+        KeyspaceMetadata defined = Schema.instance.getKeyspaceMetadata(expected.name);
         // If the keyspace doesn't exist, create it
         if (defined == null)
         {
             maybeAddKeyspace(expected);
-            defined = Schema.instance.getKSMetaData(expected.name);
+            defined = Schema.instance.getKeyspaceMetadata(expected.name);
         }
 
         // While the keyspace exists, it might miss table or have outdated one
         // There is also the potential for a race, as schema migrations add the bare
         // keyspace into Schema.instance before adding its tables, so double check that
         // all the expected tables are present
-        for (CFMetaData expectedTable : expected.tables)
+        for (TableMetadata expectedTable : expected.tables)
         {
-            CFMetaData definedTable = defined.tables.get(expectedTable.cfName).orElse(null);
+            TableMetadata definedTable = defined.tables.get(expectedTable.name).orElse(null);
             if (definedTable == null || !definedTable.equals(expectedTable))
-                MigrationManager.forceAnnounceNewColumnFamily(expectedTable);
+                MigrationManager.forceAnnounceNewTable(expectedTable);
         }
     }
 
@@ -1474,8 +1479,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     private void markViewsAsBuilt() {
         for (String keyspace : Schema.instance.getUserKeyspaces())
         {
-            for (ViewDefinition view: Schema.instance.getKSMetaData(keyspace).views)
-                SystemKeyspace.finishViewBuildStatus(view.ksName, view.viewName);
+            for (ViewMetadata view: Schema.instance.getKeyspaceMetadata(keyspace).views)
+                SystemKeyspace.finishViewBuildStatus(view.keyspace, view.name);
         }
     }
 
@@ -3616,15 +3621,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public List<InetAddress> getNaturalEndpoints(String keyspaceName, String cf, String key)
     {
-        KeyspaceMetadata ksMetaData = Schema.instance.getKSMetaData(keyspaceName);
+        KeyspaceMetadata ksMetaData = Schema.instance.getKeyspaceMetadata(keyspaceName);
         if (ksMetaData == null)
             throw new IllegalArgumentException("Unknown keyspace '" + keyspaceName + "'");
 
-        CFMetaData cfMetaData = ksMetaData.getTableOrViewNullable(cf);
-        if (cfMetaData == null)
+        TableMetadata metadata = ksMetaData.getTableOrViewNullable(cf);
+        if (metadata == null)
             throw new IllegalArgumentException("Unknown table '" + cf + "' in keyspace '" + keyspaceName + "'");
 
-        return getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(cfMetaData.getKeyValidator().fromString(key)));
+        return getNaturalEndpoints(keyspaceName, tokenMetadata.partitioner.getToken(metadata.partitionKeyType.fromString(key)));
     }
 
     public List<InetAddress> getNaturalEndpoints(String keyspaceName, ByteBuffer key)
@@ -3760,7 +3765,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             Token token = tokens.get(index);
             Range<Token> range = new Range<>(prevToken, token);
             // always return an estimate > 0 (see CASSANDRA-7322)
-            splits.add(Pair.create(range, Math.max(cfs.metadata.params.minIndexInterval, cfs.estimatedKeysForRange(range))));
+            splits.add(Pair.create(range, Math.max(cfs.metadata().params.minIndexInterval, cfs.estimatedKeysForRange(range))));
             prevToken = token;
         }
         return splits;
@@ -4709,7 +4714,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public List<String> getNonSystemKeyspaces()
     {
-        return Collections.unmodifiableList(Schema.instance.getNonSystemKeyspaces());
+        return Schema.instance.getNonSystemKeyspaces();
     }
 
     public List<String> getNonLocalStrategyKeyspaces()
@@ -4991,9 +4996,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 }
             }
 
-            public CFMetaData getTableMetadata(String tableName)
+            public TableMetadataRef getTableMetadata(String tableName)
             {
-                return Schema.instance.getCFMetaData(keyspace, tableName);
+                return Schema.instance.getTableMetadataRef(keyspace, tableName);
             }
         };
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java b/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
index 8a04108..0682b15 100644
--- a/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
@@ -17,12 +17,12 @@
  */
 package org.apache.cassandra.service.pager;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.transform.Transformation;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.transport.ProtocolVersion;
 
@@ -77,10 +77,10 @@ abstract class AbstractQueryPager implements QueryPager
         return Transformation.apply(nextPageReadCommand(pageSize).executeInternal(executionController), pager);
     }
 
-    public UnfilteredPartitionIterator fetchPageUnfiltered(CFMetaData cfm, int pageSize, ReadExecutionController executionController)
+    public UnfilteredPartitionIterator fetchPageUnfiltered(TableMetadata metadata, int pageSize, ReadExecutionController executionController)
     {
         if (isExhausted())
-            return EmptyIterators.unfilteredPartition(cfm);
+            return EmptyIterators.unfilteredPartition(metadata);
 
         pageSize = Math.min(pageSize, remaining);
         UnfilteredPager pager = new UnfilteredPager(limits.forPaging(pageSize), command.nowInSec());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/pager/AggregationQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/AggregationQueryPager.java b/src/java/org/apache/cassandra/service/pager/AggregationQueryPager.java
index f9a8cda..5ac01b2 100644
--- a/src/java/org/apache/cassandra/service/pager/AggregationQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/AggregationQueryPager.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.service.pager;
 import java.nio.ByteBuffer;
 import java.util.NoSuchElementException;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.aggregation.GroupingState;
 import org.apache.cassandra.db.filter.DataLimits;
@@ -319,7 +319,7 @@ public final class AggregationQueryPager implements QueryPager
                 this.rowIterator = delegate;
             }
 
-            public CFMetaData metadata()
+            public TableMetadata metadata()
             {
                 return rowIterator.metadata();
             }
@@ -329,7 +329,7 @@ public final class AggregationQueryPager implements QueryPager
                 return rowIterator.isReverseOrder();
             }
 
-            public PartitionColumns columns()
+            public RegularAndStaticColumns columns()
             {
                 return rowIterator.columns();
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/pager/PagingState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/PagingState.java b/src/java/org/apache/cassandra/service/pager/PagingState.java
index bcf3979..f036f96 100644
--- a/src/java/org/apache/cassandra/service/pager/PagingState.java
+++ b/src/java/org/apache/cassandra/service/pager/PagingState.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.CompactTables;
 import org.apache.cassandra.db.TypeSizes;
@@ -189,7 +189,7 @@ public class PagingState
             this.protocolVersion = protocolVersion;
         }
 
-        private static List<AbstractType<?>> makeClusteringTypes(CFMetaData metadata)
+        private static List<AbstractType<?>> makeClusteringTypes(TableMetadata metadata)
         {
             // This is the types that will be used when serializing the clustering in the paging state. We can't really use the actual clustering
             // types however because we can't guarantee that there won't be a schema change between when we send the paging state and get it back,
@@ -203,7 +203,7 @@ public class PagingState
             return l;
         }
 
-        public static RowMark create(CFMetaData metadata, Row row, ProtocolVersion protocolVersion)
+        public static RowMark create(TableMetadata metadata, Row row, ProtocolVersion protocolVersion)
         {
             ByteBuffer mark;
             if (protocolVersion.isSmallerOrEqualTo(ProtocolVersion.V3))
@@ -234,7 +234,7 @@ public class PagingState
             return new RowMark(mark, protocolVersion);
         }
 
-        public Clustering clustering(CFMetaData metadata)
+        public Clustering clustering(TableMetadata metadata)
         {
             if (mark == null)
                 return null;
@@ -245,7 +245,7 @@ public class PagingState
         }
 
         // Old (pre-3.0) encoding of cells. We need that for the protocol v3 as that is how things where encoded
-        private static ByteBuffer encodeCellName(CFMetaData metadata, Clustering clustering, ByteBuffer columnName, ByteBuffer collectionElement)
+        private static ByteBuffer encodeCellName(TableMetadata metadata, Clustering clustering, ByteBuffer columnName, ByteBuffer collectionElement)
         {
             boolean isStatic = clustering == Clustering.STATIC_CLUSTERING;
 
@@ -302,7 +302,7 @@ public class PagingState
             return CompositeType.build(isStatic, values);
         }
 
-        private static Clustering decodeClustering(CFMetaData metadata, ByteBuffer value)
+        private static Clustering decodeClustering(TableMetadata metadata, ByteBuffer value)
         {
             int csize = metadata.comparator.size();
             if (csize == 0)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java b/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
index 68547be..4f09f97 100644
--- a/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/PartitionRangeQueryPager.java
@@ -42,7 +42,7 @@ public class PartitionRangeQueryPager extends AbstractQueryPager
 
         if (state != null)
         {
-            lastReturnedKey = command.metadata().decorateKey(state.partitionKey);
+            lastReturnedKey = command.metadata().partitioner.decorateKey(state.partitionKey);
             lastReturnedRow = state.rowMark;
             restoreState(lastReturnedKey, state.remaining, state.remainingInPartition);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/paxos/Commit.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/paxos/Commit.java b/src/java/org/apache/cassandra/service/paxos/Commit.java
index 3b0364c..6e23c8b 100644
--- a/src/java/org/apache/cassandra/service/paxos/Commit.java
+++ b/src/java/org/apache/cassandra/service/paxos/Commit.java
@@ -27,7 +27,7 @@ import java.util.UUID;
 
 import com.google.common.base.Objects;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -55,7 +55,7 @@ public class Commit
         this.update = update;
     }
 
-    public static Commit newPrepare(DecoratedKey key, CFMetaData metadata, UUID ballot)
+    public static Commit newPrepare(DecoratedKey key, TableMetadata metadata, UUID ballot)
     {
         return new Commit(ballot, PartitionUpdate.emptyUpdate(metadata, key));
     }
@@ -66,7 +66,7 @@ public class Commit
         return new Commit(ballot, update);
     }
 
-    public static Commit emptyCommit(DecoratedKey key, CFMetaData metadata)
+    public static Commit emptyCommit(DecoratedKey key, TableMetadata metadata)
     {
         return new Commit(UUIDGen.minTimeUUID(0), PartitionUpdate.emptyUpdate(metadata, key));
     }


[03/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
index 930db50..353b1ad 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
@@ -24,14 +24,13 @@ import java.util.Collection;
 import java.util.List;
 
 import com.google.common.collect.Iterables;
-import com.google.common.util.concurrent.RateLimiter;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DataRange;
 import org.apache.cassandra.db.DecoratedKey;
@@ -74,7 +73,7 @@ public class SSTableScannerTest
     }
 
     // we produce all DataRange variations that produce an inclusive start and exclusive end range
-    private static Iterable<DataRange> dataRanges(CFMetaData metadata, int start, int end)
+    private static Iterable<DataRange> dataRanges(TableMetadata metadata, int start, int end)
     {
         if (end < start)
             return dataRanges(metadata, start, end, false, true);
@@ -85,7 +84,7 @@ public class SSTableScannerTest
         );
     }
 
-    private static Iterable<DataRange> dataRanges(CFMetaData metadata, int start, int end, boolean inclusiveStart, boolean inclusiveEnd)
+    private static Iterable<DataRange> dataRanges(TableMetadata metadata, int start, int end, boolean inclusiveStart, boolean inclusiveEnd)
     {
         List<DataRange> ranges = new ArrayList<>();
         if (start == end + 1)
@@ -143,7 +142,7 @@ public class SSTableScannerTest
         return token(key).maxKeyBound();
     }
 
-    private static DataRange dataRange(CFMetaData metadata, PartitionPosition start, boolean startInclusive, PartitionPosition end, boolean endInclusive)
+    private static DataRange dataRange(TableMetadata metadata, PartitionPosition start, boolean startInclusive, PartitionPosition end, boolean endInclusive)
     {
         Slices.Builder sb = new Slices.Builder(metadata.comparator);
         ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(sb.build(), false);
@@ -165,7 +164,7 @@ public class SSTableScannerTest
         return ranges;
     }
 
-    private static void insertRowWithKey(CFMetaData metadata, int key)
+    private static void insertRowWithKey(TableMetadata metadata, int key)
     {
         long timestamp = System.currentTimeMillis();
 
@@ -180,9 +179,9 @@ public class SSTableScannerTest
     private static void assertScanMatches(SSTableReader sstable, int scanStart, int scanEnd, int ... boundaries)
     {
         assert boundaries.length % 2 == 0;
-        for (DataRange range : dataRanges(sstable.metadata, scanStart, scanEnd))
+        for (DataRange range : dataRanges(sstable.metadata(), scanStart, scanEnd))
         {
-            try(ISSTableScanner scanner = sstable.getScanner(ColumnFilter.all(sstable.metadata), range))
+            try(ISSTableScanner scanner = sstable.getScanner(ColumnFilter.all(sstable.metadata()), range))
             {
                 for (int b = 0; b < boundaries.length; b += 2)
                     for (int i = boundaries[b]; i <= boundaries[b + 1]; i++)
@@ -212,7 +211,7 @@ public class SSTableScannerTest
         store.disableAutoCompaction();
 
         for (int i = 2; i < 10; i++)
-            insertRowWithKey(store.metadata, i);
+            insertRowWithKey(store.metadata(), i);
         store.forceBlockingFlush();
 
         assertEquals(1, store.getLiveSSTables().size());
@@ -318,7 +317,7 @@ public class SSTableScannerTest
 
         for (int i = 0; i < 3; i++)
             for (int j = 2; j < 10; j++)
-                insertRowWithKey(store.metadata, i * 100 + j);
+                insertRowWithKey(store.metadata(), i * 100 + j);
         store.forceBlockingFlush();
 
         assertEquals(1, store.getLiveSSTables().size());
@@ -438,7 +437,7 @@ public class SSTableScannerTest
         // disable compaction while flushing
         store.disableAutoCompaction();
 
-        insertRowWithKey(store.metadata, 205);
+        insertRowWithKey(store.metadata(), 205);
         store.forceBlockingFlush();
 
         assertEquals(1, store.getLiveSSTables().size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
index 90b1857..189782c 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -23,8 +23,8 @@ import java.io.File;
 import java.io.IOException;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
@@ -171,7 +171,7 @@ public class SSTableUtils
             Map<String, PartitionUpdate> map = new HashMap<>();
             for (String key : keys)
             {
-                RowUpdateBuilder builder = new RowUpdateBuilder(Schema.instance.getCFMetaData(ksname, cfname), 0, key);
+                RowUpdateBuilder builder = new RowUpdateBuilder(Schema.instance.getTableMetadata(ksname, cfname), 0, key);
                 builder.clustering(key).add("val", key);
                 map.put(key, builder.buildUpdate());
             }
@@ -180,7 +180,7 @@ public class SSTableUtils
 
         public Collection<SSTableReader> write(SortedMap<DecoratedKey, PartitionUpdate> sorted) throws IOException
         {
-            PartitionColumns.Builder builder = PartitionColumns.builder();
+            RegularAndStaticColumns.Builder builder = RegularAndStaticColumns.builder();
             for (PartitionUpdate update : sorted.values())
                 builder.addAll(update.columns());
             final Iterator<Map.Entry<DecoratedKey, PartitionUpdate>> iter = sorted.entrySet().iterator();
@@ -188,7 +188,7 @@ public class SSTableUtils
             {
                 public SerializationHeader header()
                 {
-                    return new SerializationHeader(true, Schema.instance.getCFMetaData(ksname, cfname), builder.build(), EncodingStats.NO_STATS);
+                    return new SerializationHeader(true, Schema.instance.getTableMetadata(ksname, cfname), builder.build(), EncodingStats.NO_STATS);
                 }
 
                 @Override
@@ -214,8 +214,8 @@ public class SSTableUtils
         public Collection<SSTableReader> write(int expectedSize, Appender appender) throws IOException
         {
             File datafile = (dest == null) ? tempSSTableFile(ksname, cfname, generation) : new File(dest.filenameFor(Component.DATA));
-            CFMetaData cfm = Schema.instance.getCFMetaData(ksname, cfname);
-            ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(cfm.cfId);
+            TableMetadata metadata = Schema.instance.getTableMetadata(ksname, cfname);
+            ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.id);
             SerializationHeader header = appender.header();
             SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, Descriptor.fromFilename(datafile.getAbsolutePath()), expectedSize, ActiveRepairService.UNREPAIRED_SSTABLE, 0, header);
             while (appender.append(writer)) { /* pass */ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
index 1a50d70..fd93ca1 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
@@ -52,7 +52,7 @@ public class SSTableWriterTest extends SSTableWriterTestBase
         {
             for (int i = 0; i < 10000; i++)
             {
-                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, random(i, 10)).withTimestamp(1);
+                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), random(i, 10)).withTimestamp(1);
                 for (int j = 0; j < 100; j++)
                     builder.newRow("" + j).add("val", ByteBuffer.allocate(1000));
                 writer.append(builder.build().unfilteredIterator());
@@ -63,7 +63,7 @@ public class SSTableWriterTest extends SSTableWriterTestBase
             assertFileCounts(dir.list());
             for (int i = 10000; i < 20000; i++)
             {
-                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, random(i, 10)).withTimestamp(1);
+                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), random(i, 10)).withTimestamp(1);
                 for (int j = 0; j < 100; j++)
                     builder.newRow("" + j).add("val", ByteBuffer.allocate(1000));
                 writer.append(builder.build().unfilteredIterator());
@@ -107,7 +107,7 @@ public class SSTableWriterTest extends SSTableWriterTestBase
         {
             for (int i = 0; i < 10000; i++)
             {
-                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, random(i, 10)).withTimestamp(1);
+                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), random(i, 10)).withTimestamp(1);
                 for (int j = 0; j < 100; j++)
                     builder.newRow("" + j).add("val", ByteBuffer.allocate(1000));
                 writer.append(builder.build().unfilteredIterator());
@@ -116,7 +116,7 @@ public class SSTableWriterTest extends SSTableWriterTestBase
             assertFileCounts(dir.list());
             for (int i = 10000; i < 20000; i++)
             {
-                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, random(i, 10)).withTimestamp(1);
+                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), random(i, 10)).withTimestamp(1);
                 for (int j = 0; j < 100; j++)
                     builder.newRow("" + j).add("val", ByteBuffer.allocate(1000));
                 writer.append(builder.build().unfilteredIterator());
@@ -158,7 +158,7 @@ public class SSTableWriterTest extends SSTableWriterTestBase
         {
             for (int i = 0; i < 10000; i++)
             {
-                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, random(i, 10)).withTimestamp(1);
+                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), random(i, 10)).withTimestamp(1);
                 for (int j = 0; j < 100; j++)
                     builder.newRow("" + j).add("val", ByteBuffer.allocate(1000));
                 writer1.append(builder.build().unfilteredIterator());
@@ -167,7 +167,7 @@ public class SSTableWriterTest extends SSTableWriterTestBase
             assertFileCounts(dir.list());
             for (int i = 10000; i < 20000; i++)
             {
-                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, random(i, 10)).withTimestamp(1);
+                UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), random(i, 10)).withTimestamp(1);
                 for (int j = 0; j < 100; j++)
                     builder.newRow("" + j).add("val", ByteBuffer.allocate(1000));
                 writer2.append(builder.build().unfilteredIterator());
@@ -212,7 +212,7 @@ public class SSTableWriterTest extends SSTableWriterTestBase
 
         try (SSTableWriter writer1 = getWriter(cfs, dir, txn))
         {
-            UpdateBuilder largeValue = UpdateBuilder.create(cfs.metadata, "large_value").withTimestamp(1);
+            UpdateBuilder largeValue = UpdateBuilder.create(cfs.metadata(), "large_value").withTimestamp(1);
             largeValue.newRow("clustering").add("val", ByteBuffer.allocate(2 * 1024 * 1024));
             writer1.append(largeValue.build().unfilteredIterator());
 
@@ -223,7 +223,7 @@ public class SSTableWriterTest extends SSTableWriterTestBase
             try
             {
                 DecoratedKey dk = Util.dk("large_value");
-                UnfilteredRowIterator rowIter = sstable.iterator(dk, Slices.ALL, ColumnFilter.all(cfs.metadata), false);
+                UnfilteredRowIterator rowIter = sstable.iterator(dk, Slices.ALL, ColumnFilter.all(cfs.metadata()), false);
                 while (rowIter.hasNext())
                 {
                     rowIter.next();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
index c1f11b6..cc92b2c 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
@@ -164,7 +164,7 @@ public class SSTableWriterTestBase extends SchemaLoader
     public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory, LifecycleTransaction txn)
     {
         Descriptor desc = cfs.newSSTableDescriptor(directory);
-        return SSTableWriter.create(desc, 0, 0, new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS), cfs.indexManager.listIndexes(), txn);
+        return SSTableWriter.create(desc, 0, 0, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS), cfs.indexManager.listIndexes(), txn);
     }
 
     public static ByteBuffer random(int i, int size)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java b/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
index 505d45d..18defdf 100644
--- a/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
@@ -25,8 +25,8 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.DecoratedKey;
@@ -45,6 +45,7 @@ import org.apache.cassandra.io.sstable.format.big.BigTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 
@@ -70,12 +71,13 @@ public class SSTableFlushObserverTest
     @Test
     public void testFlushObserver()
     {
-        CFMetaData cfm = CFMetaData.Builder.create(KS_NAME, CF_NAME)
-                                           .addPartitionKey("id", UTF8Type.instance)
-                                           .addRegularColumn("first_name", UTF8Type.instance)
-                                           .addRegularColumn("age", Int32Type.instance)
-                                           .addRegularColumn("height", LongType.instance)
-                                           .build();
+        TableMetadata cfm =
+            TableMetadata.builder(KS_NAME, CF_NAME)
+                         .addPartitionKeyColumn("id", UTF8Type.instance)
+                         .addRegularColumn("first_name", UTF8Type.instance)
+                         .addRegularColumn("age", Int32Type.instance)
+                         .addRegularColumn("height", LongType.instance)
+                         .build();
 
         LifecycleTransaction transaction = LifecycleTransaction.offline(OperationType.COMPACTION);
         FlushObserver observer = new FlushObserver();
@@ -94,9 +96,9 @@ public class SSTableFlushObserverTest
                                                                   KS_NAME, CF_NAME,
                                                                   0,
                                                                   sstableFormat),
-                                                   10L, 0L, cfm,
+                                                   10L, 0L, TableMetadataRef.forOfflineTools(cfm),
                                                    new MetadataCollector(cfm.comparator).sstableLevel(0),
-                                                   new SerializationHeader(true, cfm, cfm.partitionColumns(), EncodingStats.NO_STATS),
+                                                   new SerializationHeader(true, cfm, cfm.regularAndStaticColumns(), EncodingStats.NO_STATS),
                                                    Collections.singletonList(observer),
                                                    transaction);
 
@@ -161,12 +163,12 @@ public class SSTableFlushObserverTest
     {
         private final Iterator<Unfiltered> rows;
 
-        public RowIterator(CFMetaData cfm, ByteBuffer key, Collection<Unfiltered> content)
+        public RowIterator(TableMetadata cfm, ByteBuffer key, Collection<Unfiltered> content)
         {
             super(cfm,
                   DatabaseDescriptor.getPartitioner().decorateKey(key),
                   DeletionTime.LIVE,
-                  cfm.partitionColumns(),
+                  cfm.regularAndStaticColumns(),
                   BTreeRow.emptyRow(Clustering.STATIC_CLUSTERING),
                   false,
                   EncodingStats.NO_STATS);
@@ -219,8 +221,8 @@ public class SSTableFlushObserverTest
         return rowBuilder.build();
     }
 
-    private static ColumnDefinition getColumn(CFMetaData cfm, String name)
+    private static ColumnMetadata getColumn(TableMetadata cfm, String name)
     {
-        return cfm.getColumnDefinition(UTF8Type.instance.fromString(name));
+        return cfm.getColumn(UTF8Type.instance.fromString(name));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index 79249b6..b03f275 100644
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.io.sstable.metadata;
 
 import java.io.File;
@@ -29,7 +28,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.commitlog.CommitLogPosition;
@@ -91,7 +90,7 @@ public class MetadataSerializerTest
         CommitLogPosition club = new CommitLogPosition(11L, 12);
         CommitLogPosition cllb = new CommitLogPosition(9L, 12);
 
-        CFMetaData cfm = SchemaLoader.standardCFMD("ks1", "cf1");
+        TableMetadata cfm = SchemaLoader.standardCFMD("ks1", "cf1").build();
         MetadataCollector collector = new MetadataCollector(cfm.comparator)
                                           .commitLogIntervals(new IntervalSet<>(cllb, club));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
index c811811..b83194b 100644
--- a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
+++ b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
@@ -45,8 +45,7 @@ public class ReplicationStrategyEndpointCacheTest
     public static void defineSchema() throws Exception
     {
         SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE,
-                                    KeyspaceParams.simple(5));
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(5));
     }
 
     public void setup(Class stratClass, Map<String, String> strategyOptions) throws Exception

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
index 0955985..f97a6e5 100644
--- a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
@@ -28,7 +28,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.OrderPreservingPartitioner;
@@ -181,7 +181,7 @@ public class SimpleStrategyTest
 
     private AbstractReplicationStrategy getStrategy(String keyspaceName, TokenMetadata tmd)
     {
-        KeyspaceMetadata ksmd = Schema.instance.getKSMetaData(keyspaceName);
+        KeyspaceMetadata ksmd = Schema.instance.getKeyspaceMetadata(keyspaceName);
         return AbstractReplicationStrategy.createReplicationStrategy(
                                                                     keyspaceName,
                                                                     ksmd.params.replication.klass,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java b/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java
index 60ee725..968480b 100644
--- a/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java
@@ -31,8 +31,8 @@ import com.datastax.driver.core.Session;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.service.EmbeddedCassandraService;
 
 import static junit.framework.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java b/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
index 099a530..e186998 100644
--- a/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
@@ -30,7 +30,7 @@ import com.datastax.driver.core.Session;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.service.EmbeddedCassandraService;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java b/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
index 70e5add..d3257d7 100644
--- a/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
+++ b/test/unit/org/apache/cassandra/net/WriteCallbackInfoTest.java
@@ -25,15 +25,15 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import junit.framework.Assert;
-import org.apache.cassandra.MockSchema;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.PartitionColumns;
+import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.net.MessagingService.Verb;
+import org.apache.cassandra.schema.MockSchema;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -60,7 +60,7 @@ public class WriteCallbackInfoTest
     private void testShouldHint(Verb verb, ConsistencyLevel cl, boolean allowHints, boolean expectHint) throws Exception
     {
         Object payload = verb == Verb.PAXOS_COMMIT
-                         ? new Commit(UUID.randomUUID(), new PartitionUpdate(MockSchema.newCFMetaData("", ""), ByteBufferUtil.EMPTY_BYTE_BUFFER, PartitionColumns.NONE, 1))
+                         ? new Commit(UUID.randomUUID(), new PartitionUpdate(MockSchema.newTableMetadata("", ""), ByteBufferUtil.EMPTY_BYTE_BUFFER, RegularAndStaticColumns.NONE, 1))
                          : new Mutation("", new BufferDecoratedKey(new Murmur3Partitioner.LongToken(0), ByteBufferUtil.EMPTY_BYTE_BUFFER));
 
         WriteCallbackInfo wcbi = new WriteCallbackInfo(InetAddress.getByName("192.168.1.1"), null, new MessageOut(verb, payload, null), null, cl, allowHints);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/repair/ValidatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/ValidatorTest.java b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
index 9c32cef..c514d7c 100644
--- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java
+++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
@@ -26,9 +26,6 @@ import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.SettableFuture;
-
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.CompactionsTest;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
@@ -37,7 +34,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.EmptyIterators;
@@ -80,7 +77,7 @@ public class ValidatorTest
         SchemaLoader.createKeyspace(keyspace,
                                     KeyspaceParams.simple(1),
                                     SchemaLoader.standardCFMD(keyspace, columnFamily));
-        partitioner = Schema.instance.getCFMetaData(keyspace, columnFamily).partitioner;
+        partitioner = Schema.instance.getTableMetadata(keyspace, columnFamily).partitioner;
     }
 
     @After
@@ -111,7 +108,7 @@ public class ValidatorTest
 
         // add a row
         Token mid = partitioner.midpoint(range.left, range.right);
-        validator.add(EmptyIterators.unfilteredRow(cfs.metadata, new BufferDecoratedKey(mid, ByteBufferUtil.bytes("inconceivable!")), false));
+        validator.add(EmptyIterators.unfilteredRow(cfs.metadata(), new BufferDecoratedKey(mid, ByteBufferUtil.bytes("inconceivable!")), false));
         validator.complete();
 
         // confirm that the tree was validated

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java
index 028d899..3924045 100644
--- a/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java
+++ b/test/unit/org/apache/cassandra/repair/messages/RepairMessageSerializationsTest.java
@@ -44,6 +44,7 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.NodePair;
 import org.apache.cassandra.repair.RepairJobDesc;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.MerkleTrees;
 
@@ -167,7 +168,7 @@ public class RepairMessageSerializationsTest
     @Test
     public void prepareMessage() throws IOException
     {
-        PrepareMessage msg = new PrepareMessage(UUID.randomUUID(), new ArrayList<UUID>() {{add(UUID.randomUUID());}},
+        PrepareMessage msg = new PrepareMessage(UUID.randomUUID(), new ArrayList<TableId>() {{add(TableId.generate());}},
                                                 buildTokenRanges(), true, 100000L, false);
         serializeRoundTrip(msg, PrepareMessage.serializer);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/schema/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/DefsTest.java b/test/unit/org/apache/cassandra/schema/DefsTest.java
deleted file mode 100644
index d4ac1dc..0000000
--- a/test/unit/org/apache/cassandra/schema/DefsTest.java
+++ /dev/null
@@ -1,545 +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.schema;
-
-import java.io.File;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.function.Supplier;
-
-import com.google.common.collect.ImmutableMap;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.Component;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.utils.FBUtilities;
-
-import static org.apache.cassandra.Util.throwAssert;
-import static org.apache.cassandra.cql3.CQLTester.assertRows;
-import static org.apache.cassandra.cql3.CQLTester.row;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-
-@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 EMPTY_KEYSPACE = "test_empty_keyspace";
-    private static final String TABLE1 = "standard1";
-    private static final String TABLE2 = "standard2";
-    private static final String TABLE1i = "indexed1";
-
-    @BeforeClass
-    public static void defineSchema() throws ConfigurationException
-    {
-        SchemaLoader.prepareServer();
-        SchemaLoader.startGossiper();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, TABLE1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, TABLE2));
-        SchemaLoader.createKeyspace(KEYSPACE3,
-                                    KeyspaceParams.simple(5),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, TABLE1),
-                                    SchemaLoader.compositeIndexCFMD(KEYSPACE3, TABLE1i, true));
-        SchemaLoader.createKeyspace(KEYSPACE6,
-                                    KeyspaceParams.simple(1),
-                                    SchemaLoader.compositeIndexCFMD(KEYSPACE6, TABLE1i, true));
-    }
-
-    @Test
-    public void testCFMetaDataApply() throws ConfigurationException
-    {
-        CFMetaData cfm = CFMetaData.Builder.create(KEYSPACE1, "TestApplyCFM_CF")
-                                           .addPartitionKey("keys", BytesType.instance)
-                                           .addClusteringColumn("col", BytesType.instance).build();
-
-
-        for (int i = 0; i < 5; i++)
-        {
-            ByteBuffer name = ByteBuffer.wrap(new byte[] { (byte)i });
-            cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, name, BytesType.instance));
-        }
-
-        cfm.comment("No comment")
-           .readRepairChance(0.5)
-           .gcGraceSeconds(100000)
-           .compaction(CompactionParams.scts(ImmutableMap.of("min_threshold", "500",
-                                                             "max_threshold", "500")));
-
-        // we'll be adding this one later. make sure it's not already there.
-        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);
-        cfNew.addColumnDefinition(addIndexDef);
-
-        // remove one.
-        ColumnDefinition removeIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 0 }), BytesType.instance);
-        assertTrue(cfNew.removeColumnDefinition(removeIndexDef));
-
-        cfm.apply(cfNew);
-
-        for (int i = 1; i < cfm.allColumns().size(); i++)
-            assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[]{ 1 })));
-        assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[]{ 0 })));
-        assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[]{ 5 })));
-    }
-
-    @Test
-    public void testInvalidNames()
-    {
-        String[] valid = {"1", "a", "_1", "b_", "__", "1_a"};
-        for (String s : valid)
-            assertTrue(CFMetaData.isNameValid(s));
-
-        String[] invalid = {"b@t", "dash-y", "", " ", "dot.s", ".hidden"};
-        for (String s : invalid)
-            assertFalse(CFMetaData.isNameValid(s));
-    }
-
-    @Test
-    public void addNewCfToBogusKeyspace()
-    {
-        CFMetaData newCf = addTestTable("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 addNewTable() throws ConfigurationException
-    {
-        final String ksName = KEYSPACE1;
-        final String tableName = "anewtable";
-        KeyspaceMetadata original = Schema.instance.getKSMetaData(ksName);
-
-        CFMetaData cfm = addTestTable(original.name, tableName, "A New Table");
-
-        assertFalse(Schema.instance.getKSMetaData(ksName).tables.get(cfm.cfName).isPresent());
-        MigrationManager.announceNewColumnFamily(cfm);
-
-        assertTrue(Schema.instance.getKSMetaData(ksName).tables.get(cfm.cfName).isPresent());
-        assertEquals(cfm, Schema.instance.getKSMetaData(ksName).tables.get(cfm.cfName).get());
-
-        // now read and write to it.
-        QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, col, val) VALUES (?, ?, ?)",
-                                                     ksName, tableName),
-                                       "key0", "col0", "val0");
-
-        // flush to exercise more than just hitting the memtable
-        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(tableName);
-        assertNotNull(cfs);
-        cfs.forceBlockingFlush();
-
-        // and make sure we get out what we put in
-        UntypedResultSet rows = QueryProcessor.executeInternal(String.format("SELECT * FROM %s.%s", ksName, tableName));
-        assertRows(rows, row("key0", "col0", "val0"));
-    }
-
-    @Test
-    public void dropCf() throws ConfigurationException
-    {
-        // sanity
-        final KeyspaceMetadata ks = Schema.instance.getKSMetaData(KEYSPACE1);
-        assertNotNull(ks);
-        final CFMetaData cfm = ks.tables.getNullable(TABLE1);
-        assertNotNull(cfm);
-
-        // write some data, force a flush, then verify that files exist on disk.
-        for (int i = 0; i < 100; i++)
-            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
-                                                         KEYSPACE1, TABLE1),
-                                           "dropCf", "col" + i, "anyvalue");
-        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
-        assertNotNull(store);
-        store.forceBlockingFlush();
-        assertTrue(store.getDirectories().sstableLister(Directories.OnTxnErr.THROW).list().size() > 0);
-
-        MigrationManager.announceColumnFamilyDrop(ks.name, cfm.cfName);
-
-        assertFalse(Schema.instance.getKSMetaData(ks.name).tables.get(cfm.cfName).isPresent());
-
-        // any write should fail.
-        boolean success = true;
-        try
-        {
-            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
-                                                         KEYSPACE1, TABLE1),
-                                           "dropCf", "col0", "anyvalue");
-        }
-        catch (Throwable th)
-        {
-            success = false;
-        }
-        assertFalse("This mutation should have failed since the CF no longer exists.", success);
-
-        // verify that the files are gone.
-        Supplier<Object> lambda = () -> {
-            for (File file : store.getDirectories().sstableLister(Directories.OnTxnErr.THROW).listFiles())
-            {
-                if (file.getPath().endsWith("Data.db") && !new File(file.getPath().replace("Data.db", "Compacted")).exists())
-                    return false;
-            }
-            return true;
-        };
-        Util.spinAssertEquals(true, lambda, 30);
-
-    }
-
-    @Test
-    public void addNewKS() throws ConfigurationException
-    {
-        CFMetaData cfm = addTestTable("newkeyspace1", "newstandard1", "A new cf for a new ks");
-        KeyspaceMetadata newKs = KeyspaceMetadata.create(cfm.ksName, KeyspaceParams.simple(5), Tables.of(cfm));
-        MigrationManager.announceNewKeyspace(newKs);
-
-        assertNotNull(Schema.instance.getKSMetaData(cfm.ksName));
-        assertEquals(Schema.instance.getKSMetaData(cfm.ksName), newKs);
-
-        // test reads and writes.
-        QueryProcessor.executeInternal("INSERT INTO newkeyspace1.newstandard1 (key, col, val) VALUES (?, ?, ?)",
-                                       "key0", "col0", "val0");
-        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
-        assertNotNull(store);
-        store.forceBlockingFlush();
-
-        UntypedResultSet rows = QueryProcessor.executeInternal("SELECT * FROM newkeyspace1.newstandard1");
-        assertRows(rows, row("key0", "col0", "val0"));
-    }
-
-    @Test
-    public void dropKS() throws ConfigurationException
-    {
-        // sanity
-        final KeyspaceMetadata ks = Schema.instance.getKSMetaData(KEYSPACE1);
-        assertNotNull(ks);
-        final CFMetaData cfm = ks.tables.getNullable(TABLE2);
-        assertNotNull(cfm);
-
-        // write some data, force a flush, then verify that files exist on disk.
-        for (int i = 0; i < 100; i++)
-            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
-                                                         KEYSPACE1, TABLE2),
-                                           "dropKs", "col" + i, "anyvalue");
-        ColumnFamilyStore cfs = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
-        assertNotNull(cfs);
-        cfs.forceBlockingFlush();
-        assertTrue(!cfs.getDirectories().sstableLister(Directories.OnTxnErr.THROW).list().isEmpty());
-
-        MigrationManager.announceKeyspaceDrop(ks.name);
-
-        assertNull(Schema.instance.getKSMetaData(ks.name));
-
-        // write should fail.
-        boolean success = true;
-        try
-        {
-            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
-                                                         KEYSPACE1, TABLE2),
-                                           "dropKs", "col0", "anyvalue");
-        }
-        catch (Throwable th)
-        {
-            success = false;
-        }
-        assertFalse("This mutation should have failed since the KS no longer exists.", success);
-
-        // reads should fail too.
-        boolean threw = false;
-        try
-        {
-            Keyspace.open(ks.name);
-        }
-        catch (Throwable th)
-        {
-            threw = true;
-        }
-        assertTrue(threw);
-    }
-
-    @Test
-    public void dropKSUnflushed() throws ConfigurationException
-    {
-        // sanity
-        final KeyspaceMetadata ks = Schema.instance.getKSMetaData(KEYSPACE3);
-        assertNotNull(ks);
-        final CFMetaData cfm = ks.tables.getNullable(TABLE1);
-        assertNotNull(cfm);
-
-        // write some data
-        for (int i = 0; i < 100; i++)
-            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
-                                                         KEYSPACE3, TABLE1),
-                                           "dropKs", "col" + i, "anyvalue");
-
-        MigrationManager.announceKeyspaceDrop(ks.name);
-
-        assertNull(Schema.instance.getKSMetaData(ks.name));
-    }
-
-    @Test
-    public void createEmptyKsAddNewCf() throws ConfigurationException
-    {
-        assertNull(Schema.instance.getKSMetaData(EMPTY_KEYSPACE));
-        KeyspaceMetadata newKs = KeyspaceMetadata.create(EMPTY_KEYSPACE, KeyspaceParams.simple(5));
-        MigrationManager.announceNewKeyspace(newKs);
-        assertNotNull(Schema.instance.getKSMetaData(EMPTY_KEYSPACE));
-
-        String tableName = "added_later";
-        CFMetaData newCf = addTestTable(EMPTY_KEYSPACE, tableName, "A new CF to add to an empty KS");
-
-        //should not exist until apply
-        assertFalse(Schema.instance.getKSMetaData(newKs.name).tables.get(newCf.cfName).isPresent());
-
-        //add the new CF to the empty space
-        MigrationManager.announceNewColumnFamily(newCf);
-
-        assertTrue(Schema.instance.getKSMetaData(newKs.name).tables.get(newCf.cfName).isPresent());
-        assertEquals(Schema.instance.getKSMetaData(newKs.name).tables.get(newCf.cfName).get(), newCf);
-
-        // now read and write to it.
-        QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, col, val) VALUES (?, ?, ?)",
-                                                     EMPTY_KEYSPACE, tableName),
-                                       "key0", "col0", "val0");
-
-        ColumnFamilyStore cfs = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
-        assertNotNull(cfs);
-        cfs.forceBlockingFlush();
-
-        UntypedResultSet rows = QueryProcessor.executeInternal(String.format("SELECT * FROM %s.%s", EMPTY_KEYSPACE, tableName));
-        assertRows(rows, row("key0", "col0", "val0"));
-    }
-
-    @Test
-    public void testUpdateKeyspace() throws ConfigurationException
-    {
-        // create a keyspace to serve as existing.
-        CFMetaData cf = addTestTable("UpdatedKeyspace", "AddedStandard1", "A new cf for a new ks");
-        KeyspaceMetadata oldKs = KeyspaceMetadata.create(cf.ksName, KeyspaceParams.simple(5), Tables.of(cf));
-
-        MigrationManager.announceNewKeyspace(oldKs);
-
-        assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
-        assertEquals(Schema.instance.getKSMetaData(cf.ksName), oldKs);
-
-        // names should match.
-        KeyspaceMetadata newBadKs2 = KeyspaceMetadata.create(cf.ksName + "trash", KeyspaceParams.simple(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.
-        }
-
-        Map<String, String> replicationMap = new HashMap<>();
-        replicationMap.put(ReplicationParams.CLASS, OldNetworkTopologyStrategy.class.getName());
-        replicationMap.put("replication_factor", "1");
-
-        KeyspaceMetadata newKs = KeyspaceMetadata.create(cf.ksName, KeyspaceParams.create(true, replicationMap));
-        MigrationManager.announceKeyspaceUpdate(newKs);
-
-        KeyspaceMetadata newFetchedKs = Schema.instance.getKSMetaData(newKs.name);
-        assertEquals(newFetchedKs.params.replication.klass, newKs.params.replication.klass);
-        assertFalse(newFetchedKs.params.replication.klass.equals(oldKs.params.replication.klass));
-    }
-
-    /*
-    @Test
-    public void testUpdateColumnFamilyNoIndexes() throws ConfigurationException
-    {
-        // create a keyspace with a cf to update.
-        CFMetaData cf = addTestTable("UpdatedCfKs", "Standard1added", "A new cf that will be updated");
-        KSMetaData ksm = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(1), cf);
-        MigrationManager.announceNewKeyspace(ksm);
-
-        assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
-        assertEquals(Schema.instance.getKSMetaData(cf.ksName), ksm);
-        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); // doesn't get set back here.
-
-        newCfm.readRepairChance(0.23);
-        MigrationManager.announceColumnFamilyUpdate(newCfm);
-
-        newCfm.gcGraceSeconds(12);
-        MigrationManager.announceColumnFamilyUpdate(newCfm);
-
-        newCfm.defaultValidator(UTF8Type.instance);
-        MigrationManager.announceColumnFamilyUpdate(newCfm);
-
-        newCfm.minCompactionThreshold(3);
-        MigrationManager.announceColumnFamilyUpdate(newCfm);
-
-        newCfm.maxCompactionThreshold(33);
-        MigrationManager.announceColumnFamilyUpdate(newCfm);
-
-        // can't test changing the reconciler because there is only one impl.
-
-        // check the cumulative affect.
-        assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getComment(), newCfm.getComment());
-        assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getReadRepairChance(), newCfm.getReadRepairChance(), 0.0001);
-        assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getGcGraceSeconds(), newCfm.getGcGraceSeconds());
-        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
-        SchemaKeyspace.makeCreateKeyspaceMutation(Schema.instance.getKSMetaData(KEYSPACE6), FBUtilities.timestampMicros()).build().applyUnsafe();
-        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE6).getColumnFamilyStore(TABLE1i);
-        String indexName = "birthdate_key_index";
-
-        // insert some data.  save the sstable descriptor so we can make sure it's marked for delete after the drop
-        QueryProcessor.executeInternal(String.format(
-                                                    "INSERT INTO %s.%s (key, c1, birthdate, notbirthdate) VALUES (?, ?, ?, ?)",
-                                                    KEYSPACE6,
-                                                    TABLE1i),
-                                       "key0", "col0", 1L, 1L);
-
-        cfs.forceBlockingFlush();
-        ColumnFamilyStore indexCfs = cfs.indexManager.getIndexByName(indexName)
-                                                     .getBackingTable()
-                                                     .orElseThrow(throwAssert("Cannot access index cfs"));
-        Descriptor desc = indexCfs.getLiveSSTables().iterator().next().descriptor;
-
-        // drop the index
-        CFMetaData meta = cfs.metadata.copy();
-        IndexMetadata existing = cfs.metadata.getIndexes()
-                                             .get(indexName)
-                                             .orElseThrow(throwAssert("Index not found"));
-
-        meta.indexes(meta.getIndexes().without(existing.name));
-        MigrationManager.announceColumnFamilyUpdate(meta);
-
-        // check
-        assertTrue(cfs.indexManager.listIndexes().isEmpty());
-        LifecycleTransaction.waitForDeletions();
-        assertFalse(new File(desc.filenameFor(Component.DATA)).exists());
-    }
-
-    private CFMetaData addTestTable(String ks, String cf, String comment)
-    {
-        CFMetaData newCFMD = CFMetaData.Builder.create(ks, cf)
-                                               .addPartitionKey("key", UTF8Type.instance)
-                                               .addClusteringColumn("col", UTF8Type.instance)
-                                               .addRegularColumn("val", UTF8Type.instance).build();
-
-        newCFMD.comment(comment)
-               .readRepairChance(0.0);
-
-        return newCFMD;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java b/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java
new file mode 100644
index 0000000..07640c3
--- /dev/null
+++ b/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java
@@ -0,0 +1,539 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+import java.util.function.Supplier;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.Util.throwAssert;
+import static org.apache.cassandra.cql3.CQLTester.assertRows;
+import static org.apache.cassandra.cql3.CQLTester.row;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+
+@RunWith(OrderedJUnit4ClassRunner.class)
+public class MigrationManagerTest
+{
+    private static final String KEYSPACE1 = "keyspace1";
+    private static final String KEYSPACE3 = "keyspace3";
+    private static final String KEYSPACE6 = "keyspace6";
+    private static final String EMPTY_KEYSPACE = "test_empty_keyspace";
+    private static final String TABLE1 = "standard1";
+    private static final String TABLE2 = "standard2";
+    private static final String TABLE1i = "indexed1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.startGossiper();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, TABLE1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, TABLE2));
+        SchemaLoader.createKeyspace(KEYSPACE3,
+                                    KeyspaceParams.simple(5),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, TABLE1),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE3, TABLE1i, true));
+        SchemaLoader.createKeyspace(KEYSPACE6,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE6, TABLE1i, true));
+    }
+
+    @Test
+    public void testTableMetadataBuilder() throws ConfigurationException
+    {
+        TableMetadata.Builder builder =
+            TableMetadata.builder(KEYSPACE1, "TestApplyCFM_CF")
+                         .addPartitionKeyColumn("keys", BytesType.instance)
+                         .addClusteringColumn("col", BytesType.instance)
+                         .comment("No comment")
+                         .readRepairChance(0.5)
+                         .gcGraceSeconds(100000)
+                         .compaction(CompactionParams.scts(ImmutableMap.of("min_threshold", "500", "max_threshold", "500")));
+
+        for (int i = 0; i < 5; i++)
+        {
+            ByteBuffer name = ByteBuffer.wrap(new byte[] { (byte)i });
+            builder.addRegularColumn(ColumnIdentifier.getInterned(name, BytesType.instance), ByteType.instance);
+        }
+
+
+        TableMetadata table = builder.build();
+        // we'll be adding this one later. make sure it's not already there.
+        assertNull(table.getColumn(ByteBuffer.wrap(new byte[]{ 5 })));
+
+        // add one.
+        ColumnMetadata addIndexDef = ColumnMetadata.regularColumn(table, ByteBuffer.wrap(new byte[] { 5 }), BytesType.instance);
+        builder.addColumn(addIndexDef);
+
+        // remove one.
+        ColumnMetadata removeIndexDef = ColumnMetadata.regularColumn(table, ByteBuffer.wrap(new byte[] { 0 }), BytesType.instance);
+        builder.removeRegularOrStaticColumn(removeIndexDef.name);
+
+        TableMetadata table2 = builder.build();
+
+        for (int i = 1; i < table2.columns().size(); i++)
+            assertNotNull(table2.getColumn(ByteBuffer.wrap(new byte[]{ 1 })));
+        assertNull(table2.getColumn(ByteBuffer.wrap(new byte[]{ 0 })));
+        assertNotNull(table2.getColumn(ByteBuffer.wrap(new byte[]{ 5 })));
+    }
+
+    @Test
+    public void testInvalidNames()
+    {
+        String[] valid = {"1", "a", "_1", "b_", "__", "1_a"};
+        for (String s : valid)
+            assertTrue(SchemaConstants.isValidName(s));
+
+        String[] invalid = {"b@t", "dash-y", "", " ", "dot.s", ".hidden"};
+        for (String s : invalid)
+            assertFalse(SchemaConstants.isValidName(s));
+    }
+
+    @Test
+    public void addNewCfToBogusKeyspace()
+    {
+        TableMetadata newCf = addTestTable("MadeUpKeyspace", "NewCF", "new cf");
+        try
+        {
+            MigrationManager.announceNewTable(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 addNewTable() throws ConfigurationException
+    {
+        final String ksName = KEYSPACE1;
+        final String tableName = "anewtable";
+        KeyspaceMetadata original = Schema.instance.getKeyspaceMetadata(ksName);
+
+        TableMetadata cfm = addTestTable(original.name, tableName, "A New Table");
+
+        assertFalse(Schema.instance.getKeyspaceMetadata(ksName).tables.get(cfm.name).isPresent());
+        MigrationManager.announceNewTable(cfm);
+
+        assertTrue(Schema.instance.getKeyspaceMetadata(ksName).tables.get(cfm.name).isPresent());
+        assertEquals(cfm, Schema.instance.getKeyspaceMetadata(ksName).tables.get(cfm.name).get());
+
+        // now read and write to it.
+        QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, col, val) VALUES (?, ?, ?)",
+                                                     ksName, tableName),
+                                       "key0", "col0", "val0");
+
+        // flush to exercise more than just hitting the memtable
+        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(tableName);
+        assertNotNull(cfs);
+        cfs.forceBlockingFlush();
+
+        // and make sure we get out what we put in
+        UntypedResultSet rows = QueryProcessor.executeInternal(String.format("SELECT * FROM %s.%s", ksName, tableName));
+        assertRows(rows, row("key0", "col0", "val0"));
+    }
+
+    @Test
+    public void dropCf() throws ConfigurationException
+    {
+        // sanity
+        final KeyspaceMetadata ks = Schema.instance.getKeyspaceMetadata(KEYSPACE1);
+        assertNotNull(ks);
+        final TableMetadata cfm = ks.tables.getNullable(TABLE1);
+        assertNotNull(cfm);
+
+        // write some data, force a flush, then verify that files exist on disk.
+        for (int i = 0; i < 100; i++)
+            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
+                                                         KEYSPACE1, TABLE1),
+                                           "dropCf", "col" + i, "anyvalue");
+        ColumnFamilyStore store = Keyspace.open(cfm.keyspace).getColumnFamilyStore(cfm.name);
+        assertNotNull(store);
+        store.forceBlockingFlush();
+        assertTrue(store.getDirectories().sstableLister(Directories.OnTxnErr.THROW).list().size() > 0);
+
+        MigrationManager.announceTableDrop(ks.name, cfm.name);
+
+        assertFalse(Schema.instance.getKeyspaceMetadata(ks.name).tables.get(cfm.name).isPresent());
+
+        // any write should fail.
+        boolean success = true;
+        try
+        {
+            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
+                                                         KEYSPACE1, TABLE1),
+                                           "dropCf", "col0", "anyvalue");
+        }
+        catch (Throwable th)
+        {
+            success = false;
+        }
+        assertFalse("This mutation should have failed since the CF no longer exists.", success);
+
+        // verify that the files are gone.
+        Supplier<Object> lambda = () -> {
+            for (File file : store.getDirectories().sstableLister(Directories.OnTxnErr.THROW).listFiles())
+            {
+                if (file.getPath().endsWith("Data.db") && !new File(file.getPath().replace("Data.db", "Compacted")).exists())
+                    return false;
+            }
+            return true;
+        };
+        Util.spinAssertEquals(true, lambda, 30);
+
+    }
+
+    @Test
+    public void addNewKS() throws ConfigurationException
+    {
+        TableMetadata cfm = addTestTable("newkeyspace1", "newstandard1", "A new cf for a new ks");
+        KeyspaceMetadata newKs = KeyspaceMetadata.create(cfm.keyspace, KeyspaceParams.simple(5), Tables.of(cfm));
+        MigrationManager.announceNewKeyspace(newKs);
+
+        assertNotNull(Schema.instance.getKeyspaceMetadata(cfm.keyspace));
+        assertEquals(Schema.instance.getKeyspaceMetadata(cfm.keyspace), newKs);
+
+        // test reads and writes.
+        QueryProcessor.executeInternal("INSERT INTO newkeyspace1.newstandard1 (key, col, val) VALUES (?, ?, ?)",
+                                       "key0", "col0", "val0");
+        ColumnFamilyStore store = Keyspace.open(cfm.keyspace).getColumnFamilyStore(cfm.name);
+        assertNotNull(store);
+        store.forceBlockingFlush();
+
+        UntypedResultSet rows = QueryProcessor.executeInternal("SELECT * FROM newkeyspace1.newstandard1");
+        assertRows(rows, row("key0", "col0", "val0"));
+    }
+
+    @Test
+    public void dropKS() throws ConfigurationException
+    {
+        // sanity
+        final KeyspaceMetadata ks = Schema.instance.getKeyspaceMetadata(KEYSPACE1);
+        assertNotNull(ks);
+        final TableMetadata cfm = ks.tables.getNullable(TABLE2);
+        assertNotNull(cfm);
+
+        // write some data, force a flush, then verify that files exist on disk.
+        for (int i = 0; i < 100; i++)
+            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
+                                                         KEYSPACE1, TABLE2),
+                                           "dropKs", "col" + i, "anyvalue");
+        ColumnFamilyStore cfs = Keyspace.open(cfm.keyspace).getColumnFamilyStore(cfm.name);
+        assertNotNull(cfs);
+        cfs.forceBlockingFlush();
+        assertTrue(!cfs.getDirectories().sstableLister(Directories.OnTxnErr.THROW).list().isEmpty());
+
+        MigrationManager.announceKeyspaceDrop(ks.name);
+
+        assertNull(Schema.instance.getKeyspaceMetadata(ks.name));
+
+        // write should fail.
+        boolean success = true;
+        try
+        {
+            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
+                                                         KEYSPACE1, TABLE2),
+                                           "dropKs", "col0", "anyvalue");
+        }
+        catch (Throwable th)
+        {
+            success = false;
+        }
+        assertFalse("This mutation should have failed since the KS no longer exists.", success);
+
+        // reads should fail too.
+        boolean threw = false;
+        try
+        {
+            Keyspace.open(ks.name);
+        }
+        catch (Throwable th)
+        {
+            threw = true;
+        }
+        assertTrue(threw);
+    }
+
+    @Test
+    public void dropKSUnflushed() throws ConfigurationException
+    {
+        // sanity
+        final KeyspaceMetadata ks = Schema.instance.getKeyspaceMetadata(KEYSPACE3);
+        assertNotNull(ks);
+        final TableMetadata cfm = ks.tables.getNullable(TABLE1);
+        assertNotNull(cfm);
+
+        // write some data
+        for (int i = 0; i < 100; i++)
+            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
+                                                         KEYSPACE3, TABLE1),
+                                           "dropKs", "col" + i, "anyvalue");
+
+        MigrationManager.announceKeyspaceDrop(ks.name);
+
+        assertNull(Schema.instance.getKeyspaceMetadata(ks.name));
+    }
+
+    @Test
+    public void createEmptyKsAddNewCf() throws ConfigurationException
+    {
+        assertNull(Schema.instance.getKeyspaceMetadata(EMPTY_KEYSPACE));
+        KeyspaceMetadata newKs = KeyspaceMetadata.create(EMPTY_KEYSPACE, KeyspaceParams.simple(5));
+        MigrationManager.announceNewKeyspace(newKs);
+        assertNotNull(Schema.instance.getKeyspaceMetadata(EMPTY_KEYSPACE));
+
+        String tableName = "added_later";
+        TableMetadata newCf = addTestTable(EMPTY_KEYSPACE, tableName, "A new CF to add to an empty KS");
+
+        //should not exist until apply
+        assertFalse(Schema.instance.getKeyspaceMetadata(newKs.name).tables.get(newCf.name).isPresent());
+
+        //add the new CF to the empty space
+        MigrationManager.announceNewTable(newCf);
+
+        assertTrue(Schema.instance.getKeyspaceMetadata(newKs.name).tables.get(newCf.name).isPresent());
+        assertEquals(Schema.instance.getKeyspaceMetadata(newKs.name).tables.get(newCf.name).get(), newCf);
+
+        // now read and write to it.
+        QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, col, val) VALUES (?, ?, ?)",
+                                                     EMPTY_KEYSPACE, tableName),
+                                       "key0", "col0", "val0");
+
+        ColumnFamilyStore cfs = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.name);
+        assertNotNull(cfs);
+        cfs.forceBlockingFlush();
+
+        UntypedResultSet rows = QueryProcessor.executeInternal(String.format("SELECT * FROM %s.%s", EMPTY_KEYSPACE, tableName));
+        assertRows(rows, row("key0", "col0", "val0"));
+    }
+
+    @Test
+    public void testUpdateKeyspace() throws ConfigurationException
+    {
+        // create a keyspace to serve as existing.
+        TableMetadata cf = addTestTable("UpdatedKeyspace", "AddedStandard1", "A new cf for a new ks");
+        KeyspaceMetadata oldKs = KeyspaceMetadata.create(cf.keyspace, KeyspaceParams.simple(5), Tables.of(cf));
+
+        MigrationManager.announceNewKeyspace(oldKs);
+
+        assertNotNull(Schema.instance.getKeyspaceMetadata(cf.keyspace));
+        assertEquals(Schema.instance.getKeyspaceMetadata(cf.keyspace), oldKs);
+
+        // names should match.
+        KeyspaceMetadata newBadKs2 = KeyspaceMetadata.create(cf.keyspace + "trash", KeyspaceParams.simple(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.
+        }
+
+        Map<String, String> replicationMap = new HashMap<>();
+        replicationMap.put(ReplicationParams.CLASS, OldNetworkTopologyStrategy.class.getName());
+        replicationMap.put("replication_factor", "1");
+
+        KeyspaceMetadata newKs = KeyspaceMetadata.create(cf.keyspace, KeyspaceParams.create(true, replicationMap));
+        MigrationManager.announceKeyspaceUpdate(newKs);
+
+        KeyspaceMetadata newFetchedKs = Schema.instance.getKeyspaceMetadata(newKs.name);
+        assertEquals(newFetchedKs.params.replication.klass, newKs.params.replication.klass);
+        assertFalse(newFetchedKs.params.replication.klass.equals(oldKs.params.replication.klass));
+    }
+
+    /*
+    @Test
+    public void testUpdateColumnFamilyNoIndexes() throws ConfigurationException
+    {
+        // create a keyspace with a cf to update.
+        CFMetaData cf = addTestTable("UpdatedCfKs", "Standard1added", "A new cf that will be updated");
+        KSMetaData ksm = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(1), cf);
+        MigrationManager.announceNewKeyspace(ksm);
+
+        assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
+        assertEquals(Schema.instance.getKSMetaData(cf.ksName), ksm);
+        assertNotNull(Schema.instance.getTableMetadataRef(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.announceTableUpdate(newCfm); // doesn't get set back here.
+
+        newCfm.readRepairChance(0.23);
+        MigrationManager.announceTableUpdate(newCfm);
+
+        newCfm.gcGraceSeconds(12);
+        MigrationManager.announceTableUpdate(newCfm);
+
+        newCfm.defaultValidator(UTF8Type.instance);
+        MigrationManager.announceTableUpdate(newCfm);
+
+        newCfm.minCompactionThreshold(3);
+        MigrationManager.announceTableUpdate(newCfm);
+
+        newCfm.maxCompactionThreshold(33);
+        MigrationManager.announceTableUpdate(newCfm);
+
+        // can't test changing the reconciler because there is only one impl.
+
+        // check the cumulative affect.
+        assertEquals(Schema.instance.getTableMetadataRef(cf.ksName, cf.cfName).getComment(), newCfm.getComment());
+        assertEquals(Schema.instance.getTableMetadataRef(cf.ksName, cf.cfName).getReadRepairChance(), newCfm.getReadRepairChance(), 0.0001);
+        assertEquals(Schema.instance.getTableMetadataRef(cf.ksName, cf.cfName).getGcGraceSeconds(), newCfm.getGcGraceSeconds());
+        assertEquals(UTF8Type.instance, Schema.instance.getTableMetadataRef(cf.ksName, cf.cfName).getDefaultValidator());
+
+        // Change tableId
+        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
+        SchemaKeyspace.makeCreateKeyspaceMutation(Schema.instance.getKeyspaceMetadata(KEYSPACE6), FBUtilities.timestampMicros()).build().applyUnsafe();
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE6).getColumnFamilyStore(TABLE1i);
+        String indexName = TABLE1i + "_birthdate_key_index";
+
+        // insert some data.  save the sstable descriptor so we can make sure it's marked for delete after the drop
+        QueryProcessor.executeInternal(String.format(
+                                                    "INSERT INTO %s.%s (key, c1, birthdate, notbirthdate) VALUES (?, ?, ?, ?)",
+                                                    KEYSPACE6,
+                                                    TABLE1i),
+                                       "key0", "col0", 1L, 1L);
+
+        cfs.forceBlockingFlush();
+        ColumnFamilyStore indexCfs = cfs.indexManager.getIndexByName(indexName)
+                                                     .getBackingTable()
+                                                     .orElseThrow(throwAssert("Cannot access index cfs"));
+        Descriptor desc = indexCfs.getLiveSSTables().iterator().next().descriptor;
+
+        // drop the index
+        TableMetadata meta = cfs.metadata();
+        IndexMetadata existing = meta.indexes
+                                     .get(indexName)
+                                     .orElseThrow(throwAssert("Index not found"));
+
+        MigrationManager.announceTableUpdate(meta.unbuild().indexes(meta.indexes.without(existing.name)).build());
+
+        // check
+        assertTrue(cfs.indexManager.listIndexes().isEmpty());
+        LifecycleTransaction.waitForDeletions();
+        assertFalse(new File(desc.filenameFor(Component.DATA)).exists());
+    }
+
+    private TableMetadata addTestTable(String ks, String cf, String comment)
+    {
+        return
+            TableMetadata.builder(ks, cf)
+                         .addPartitionKeyColumn("key", UTF8Type.instance)
+                         .addClusteringColumn("col", UTF8Type.instance)
+                         .addRegularColumn("val", UTF8Type.instance)
+                         .comment(comment)
+                         .readRepairChance(0.0)
+                         .build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/schema/MockSchema.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/MockSchema.java b/test/unit/org/apache/cassandra/schema/MockSchema.java
new file mode 100644
index 0000000..023015b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/schema/MockSchema.java
@@ -0,0 +1,187 @@
+/*
+* 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.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.Memory;
+import org.apache.cassandra.schema.CachingParams;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.AlwaysPresentFilter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class MockSchema
+{
+    static
+    {
+        Memory offsets = Memory.allocate(4);
+        offsets.setInt(0, 0);
+        indexSummary = new IndexSummary(Murmur3Partitioner.instance, offsets, 0, Memory.allocate(4), 0, 0, 0, 1);
+    }
+    private static final AtomicInteger id = new AtomicInteger();
+    public static final Keyspace ks = Keyspace.mockKS(KeyspaceMetadata.create("mockks", KeyspaceParams.simpleTransient(1)));
+
+    public static final IndexSummary indexSummary;
+    private static final FileHandle RANDOM_ACCESS_READER_FACTORY = new FileHandle.Builder(temp("mocksegmentedfile").getAbsolutePath()).complete();
+
+    public static Memtable memtable(ColumnFamilyStore cfs)
+    {
+        return new Memtable(cfs.metadata());
+    }
+
+    public static SSTableReader sstable(int generation, ColumnFamilyStore cfs)
+    {
+        return sstable(generation, false, cfs);
+    }
+
+    public static SSTableReader sstable(int generation, boolean keepRef, ColumnFamilyStore cfs)
+    {
+        return sstable(generation, 0, keepRef, cfs);
+    }
+
+    public static SSTableReader sstable(int generation, int size, ColumnFamilyStore cfs)
+    {
+        return sstable(generation, size, false, cfs);
+    }
+
+    public static SSTableReader sstable(int generation, int size, boolean keepRef, ColumnFamilyStore cfs)
+    {
+        Descriptor descriptor = new Descriptor(cfs.getDirectories().getDirectoryForNewSSTables(),
+                                               cfs.keyspace.getName(),
+                                               cfs.getColumnFamilyName(),
+                                               generation, SSTableFormat.Type.BIG);
+        Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
+        for (Component component : components)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            try
+            {
+                file.createNewFile();
+            }
+            catch (IOException e)
+            {
+            }
+        }
+        if (size > 0)
+        {
+            try
+            {
+                File file = new File(descriptor.filenameFor(Component.DATA));
+                try (RandomAccessFile raf = new RandomAccessFile(file, "rw"))
+                {
+                    raf.setLength(size);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+        SerializationHeader header = SerializationHeader.make(cfs.metadata(), Collections.emptyList());
+        StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata().comparator)
+                                                 .finalizeMetadata(cfs.metadata().partitioner.getClass().getCanonicalName(), 0.01f, -1, header)
+                                                 .get(MetadataType.STATS);
+        SSTableReader reader = SSTableReader.internalOpen(descriptor, components, cfs.metadata,
+                                                          RANDOM_ACCESS_READER_FACTORY.sharedCopy(), RANDOM_ACCESS_READER_FACTORY.sharedCopy(), indexSummary.sharedCopy(),
+                                                          new AlwaysPresentFilter(), 1L, metadata, SSTableReader.OpenReason.NORMAL, header);
+        reader.first = reader.last = readerBounds(generation);
+        if (!keepRef)
+            reader.selfRef().release();
+        return reader;
+    }
+
+    public static ColumnFamilyStore newCFS()
+    {
+        return newCFS(ks.getName());
+    }
+
+    public static ColumnFamilyStore newCFS(String ksname)
+    {
+        String cfname = "mockcf" + (id.incrementAndGet());
+        TableMetadata metadata = newTableMetadata(ksname, cfname);
+        return new ColumnFamilyStore(ks, cfname, 0, new TableMetadataRef(metadata), new Directories(metadata), false, false, false);
+    }
+
+    public static TableMetadata newTableMetadata(String ksname, String cfname)
+    {
+        return TableMetadata.builder(ksname, cfname)
+                            .partitioner(Murmur3Partitioner.instance)
+                            .addPartitionKeyColumn("key", UTF8Type.instance)
+                            .addClusteringColumn("col", UTF8Type.instance)
+                            .addRegularColumn("value", UTF8Type.instance)
+                            .caching(CachingParams.CACHE_NOTHING)
+                            .build();
+    }
+
+    public static BufferDecoratedKey readerBounds(int generation)
+    {
+        return new BufferDecoratedKey(new Murmur3Partitioner.LongToken(generation), ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    }
+
+    private static File temp(String id)
+    {
+        try
+        {
+            File file = File.createTempFile(id, "tmp");
+            file.deleteOnExit();
+            return file;
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static void cleanup()
+    {
+        // clean up data directory which are stored as data directory/keyspace/data files
+        for (String dirName : DatabaseDescriptor.getAllDataFileLocations())
+        {
+            File dir = new File(dirName);
+            if (!dir.exists())
+                continue;
+            String[] children = dir.list();
+            for (String child : children)
+                FileUtils.deleteRecursive(new File(dir, child));
+        }
+    }
+}


[02/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java b/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
index 2fd7b06..d4c4bb4 100644
--- a/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/schema/SchemaKeyspaceTest.java
@@ -20,10 +20,8 @@ package org.apache.cassandra.schema;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableMap;
@@ -32,21 +30,15 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.UnfilteredRowIterators;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
@@ -73,12 +65,10 @@ public class SchemaKeyspaceTest
         {
             for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
             {
-                CFMetaData cfm = cfs.metadata;
-                checkInverses(cfm);
+                checkInverses(cfs.metadata());
 
                 // Testing with compression to catch #3558
-                CFMetaData withCompression = cfm.copy();
-                withCompression.compression(CompressionParams.snappy(32768));
+                TableMetadata withCompression = cfs.metadata().unbuild().compression(CompressionParams.snappy(32768)).build();
                 checkInverses(withCompression);
             }
         }
@@ -91,44 +81,44 @@ public class SchemaKeyspaceTest
 
         createTable(keyspace, "CREATE TABLE test (a text primary key, b int, c int)");
 
-        CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, "test");
+        TableMetadata metadata = Schema.instance.getTableMetadata(keyspace, "test");
         assertTrue("extensions should be empty", metadata.params.extensions.isEmpty());
 
         ImmutableMap<String, ByteBuffer> extensions = ImmutableMap.of("From ... with Love",
                                                                       ByteBuffer.wrap(new byte[]{0, 0, 7}));
 
-        CFMetaData copy = metadata.copy().extensions(extensions);
+        TableMetadata copy = metadata.unbuild().extensions(extensions).build();
 
         updateTable(keyspace, metadata, copy);
 
-        metadata = Schema.instance.getCFMetaData(keyspace, "test");
+        metadata = Schema.instance.getTableMetadata(keyspace, "test");
         assertEquals(extensions, metadata.params.extensions);
     }
 
-    private static void updateTable(String keyspace, CFMetaData oldTable, CFMetaData newTable)
+    private static void updateTable(String keyspace, TableMetadata oldTable, TableMetadata newTable)
     {
         KeyspaceMetadata ksm = Schema.instance.getKeyspaceInstance(keyspace).getMetadata();
         Mutation mutation = SchemaKeyspace.makeUpdateTableMutation(ksm, oldTable, newTable, FBUtilities.timestampMicros()).build();
-        SchemaKeyspace.mergeSchema(Collections.singleton(mutation));
+        Schema.instance.merge(Collections.singleton(mutation));
     }
 
     private static void createTable(String keyspace, String cql)
     {
-        CFMetaData table = CFMetaData.compile(cql, keyspace);
+        TableMetadata table = CreateTableStatement.parse(cql, keyspace).build();
 
         KeyspaceMetadata ksm = KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1), Tables.of(table));
         Mutation mutation = SchemaKeyspace.makeCreateTableMutation(ksm, table, FBUtilities.timestampMicros()).build();
-        SchemaKeyspace.mergeSchema(Collections.singleton(mutation));
+        Schema.instance.merge(Collections.singleton(mutation));
     }
 
-    private static void checkInverses(CFMetaData cfm) throws Exception
+    private static void checkInverses(TableMetadata metadata) throws Exception
     {
-        KeyspaceMetadata keyspace = Schema.instance.getKSMetaData(cfm.ksName);
+        KeyspaceMetadata keyspace = Schema.instance.getKeyspaceMetadata(metadata.keyspace);
 
         // Test schema conversion
-        Mutation rm = SchemaKeyspace.makeCreateTableMutation(keyspace, cfm, FBUtilities.timestampMicros()).build();
-        PartitionUpdate serializedCf = rm.getPartitionUpdate(Schema.instance.getId(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.TABLES));
-        PartitionUpdate serializedCD = rm.getPartitionUpdate(Schema.instance.getId(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.COLUMNS));
+        Mutation rm = SchemaKeyspace.makeCreateTableMutation(keyspace, metadata, FBUtilities.timestampMicros()).build();
+        PartitionUpdate serializedCf = rm.getPartitionUpdate(Schema.instance.getTableMetadata(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.TABLES));
+        PartitionUpdate serializedCD = rm.getPartitionUpdate(Schema.instance.getTableMetadata(SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.COLUMNS));
 
         UntypedResultSet.Row tableRow = QueryProcessor.resultify(String.format("SELECT * FROM %s.%s", SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.TABLES),
                                                                  UnfilteredRowIterators.filter(serializedCf.unfilteredIterator(), FBUtilities.nowInSeconds()))
@@ -137,11 +127,11 @@ public class SchemaKeyspaceTest
 
         UntypedResultSet columnsRows = QueryProcessor.resultify(String.format("SELECT * FROM %s.%s", SchemaConstants.SCHEMA_KEYSPACE_NAME, SchemaKeyspace.COLUMNS),
                                                                 UnfilteredRowIterators.filter(serializedCD.unfilteredIterator(), FBUtilities.nowInSeconds()));
-        Set<ColumnDefinition> columns = new HashSet<>();
+        Set<ColumnMetadata> columns = new HashSet<>();
         for (UntypedResultSet.Row row : columnsRows)
             columns.add(SchemaKeyspace.createColumnFromRow(row, Types.none()));
 
-        assertEquals(cfm.params, params);
-        assertEquals(new HashSet<>(cfm.allColumns()), columns);
+        assertEquals(metadata.params, params);
+        assertEquals(new HashSet<>(metadata.columns()), columns);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/schema/SchemaTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/SchemaTest.java b/test/unit/org/apache/cassandra/schema/SchemaTest.java
new file mode 100644
index 0000000..32a5620
--- /dev/null
+++ b/test/unit/org/apache/cassandra/schema/SchemaTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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 org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.gms.Gossiper;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class SchemaTest
+{
+    @BeforeClass
+    public static void setupDatabaseDescriptor()
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    @Test
+    public void testTransKsMigration() throws IOException
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        Schema.instance.loadFromDisk();
+        assertEquals(0, Schema.instance.getNonSystemKeyspaces().size());
+
+        Gossiper.instance.start((int)(System.currentTimeMillis() / 1000));
+        Keyspace.setInitialized();
+
+        try
+        {
+            // add a few.
+            MigrationManager.announceNewKeyspace(KeyspaceMetadata.create("ks0", KeyspaceParams.simple(3)));
+            MigrationManager.announceNewKeyspace(KeyspaceMetadata.create("ks1", KeyspaceParams.simple(3)));
+
+            assertNotNull(Schema.instance.getKeyspaceMetadata("ks0"));
+            assertNotNull(Schema.instance.getKeyspaceMetadata("ks1"));
+
+            Schema.instance.unload(Schema.instance.getKeyspaceMetadata("ks0"));
+            Schema.instance.unload(Schema.instance.getKeyspaceMetadata("ks1"));
+
+            assertNull(Schema.instance.getKeyspaceMetadata("ks0"));
+            assertNull(Schema.instance.getKeyspaceMetadata("ks1"));
+
+            Schema.instance.loadFromDisk();
+
+            assertNotNull(Schema.instance.getKeyspaceMetadata("ks0"));
+            assertNotNull(Schema.instance.getKeyspaceMetadata("ks1"));
+        }
+        finally
+        {
+            Gossiper.instance.stop();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/schema/ValidationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/ValidationTest.java b/test/unit/org/apache/cassandra/schema/ValidationTest.java
new file mode 100644
index 0000000..8eb1247
--- /dev/null
+++ b/test/unit/org/apache/cassandra/schema/ValidationTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.util.*;
+
+import org.apache.cassandra.db.marshal.*;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ValidationTest
+{
+    @Test
+    public void testIsNameValidPositive()
+    {
+         assertTrue(SchemaConstants.isValidName("abcdefghijklmnopqrstuvwxyz"));
+         assertTrue(SchemaConstants.isValidName("ABCDEFGHIJKLMNOPQRSTUVWXYZ"));
+         assertTrue(SchemaConstants.isValidName("_01234567890"));
+    }
+    
+    @Test
+    public void testIsNameValidNegative()
+    {
+        assertFalse(SchemaConstants.isValidName(null));
+        assertFalse(SchemaConstants.isValidName(""));
+        assertFalse(SchemaConstants.isValidName(" "));
+        assertFalse(SchemaConstants.isValidName("@"));
+        assertFalse(SchemaConstants.isValidName("!"));
+    }
+
+    private static Set<String> primitiveTypes =
+        new HashSet<>(Arrays.asList(new String[] { "ascii", "bigint", "blob", "boolean", "date",
+                                                   "duration", "decimal", "double", "float",
+                                                   "inet", "int", "smallint", "text", "time",
+                                                   "timestamp", "timeuuid", "tinyint", "uuid",
+                                                   "varchar", "varint" }));
+
+    @Test
+    public void typeCompatibilityTest()
+    {
+        Map<String, Set<String>> compatibilityMap = new HashMap<>();
+        compatibilityMap.put("bigint", new HashSet<>(Arrays.asList(new String[] {"timestamp"})));
+        compatibilityMap.put("blob", new HashSet<>(Arrays.asList(new String[] {"ascii", "bigint", "boolean", "date", "decimal", "double", "duration",
+                                                                               "float", "inet", "int", "smallint", "text", "time", "timestamp",
+                                                                               "timeuuid", "tinyint", "uuid", "varchar", "varint"})));
+        compatibilityMap.put("date", new HashSet<>(Arrays.asList(new String[] {"int"})));
+        compatibilityMap.put("time", new HashSet<>(Arrays.asList(new String[] {"bigint"})));
+        compatibilityMap.put("text", new HashSet<>(Arrays.asList(new String[] {"ascii", "varchar"})));
+        compatibilityMap.put("timestamp", new HashSet<>(Arrays.asList(new String[] {"bigint"})));
+        compatibilityMap.put("varchar", new HashSet<>(Arrays.asList(new String[] {"ascii", "text"})));
+        compatibilityMap.put("varint", new HashSet<>(Arrays.asList(new String[] {"bigint", "int", "timestamp"})));
+        compatibilityMap.put("uuid", new HashSet<>(Arrays.asList(new String[] {"timeuuid"})));
+
+        for (String sourceTypeString: primitiveTypes)
+        {
+            AbstractType sourceType = CQLTypeParser.parse("KEYSPACE", sourceTypeString, Types.none());
+            for (String destinationTypeString: primitiveTypes)
+            {
+                AbstractType destinationType = CQLTypeParser.parse("KEYSPACE", destinationTypeString, Types.none());
+
+                if (compatibilityMap.get(destinationTypeString) != null &&
+                    compatibilityMap.get(destinationTypeString).contains(sourceTypeString) ||
+                    sourceTypeString.equals(destinationTypeString))
+                {
+                    assertTrue(sourceTypeString + " should be compatible with " + destinationTypeString,
+                               destinationType.isValueCompatibleWith(sourceType));
+                }
+                else
+                {
+                    assertFalse(sourceTypeString + " should not be compatible with " + destinationTypeString,
+                                destinationType.isValueCompatibleWith(sourceType));
+                }
+            }
+        }
+    }
+
+    @Test
+    public void clusteringColumnTypeCompatibilityTest() throws Throwable
+    {
+        Map<String, Set<String>> compatibilityMap = new HashMap<>();
+        compatibilityMap.put("blob", new HashSet<>(Arrays.asList(new String[] {"ascii", "text", "varchar"})));
+        compatibilityMap.put("text", new HashSet<>(Arrays.asList(new String[] {"ascii", "varchar"})));
+        compatibilityMap.put("varchar", new HashSet<>(Arrays.asList(new String[] {"ascii", "text" })));
+
+        for (String sourceTypeString: primitiveTypes)
+        {
+            AbstractType sourceType = CQLTypeParser.parse("KEYSPACE", sourceTypeString, Types.none());
+            for (String destinationTypeString: primitiveTypes)
+            {
+                AbstractType destinationType = CQLTypeParser.parse("KEYSPACE", destinationTypeString, Types.none());
+
+                if (compatibilityMap.get(destinationTypeString) != null &&
+                    compatibilityMap.get(destinationTypeString).contains(sourceTypeString) ||
+                    sourceTypeString.equals(destinationTypeString))
+                {
+                    assertTrue(sourceTypeString + " should be compatible with " + destinationTypeString,
+                               destinationType.isCompatibleWith(sourceType));
+                }
+                else
+                {
+                    assertFalse(sourceTypeString + " should not be compatible with " + destinationTypeString,
+                                destinationType.isCompatibleWith(sourceType));
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
index 2c1a8d2..44bd58c 100644
--- a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
+++ b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
@@ -234,10 +234,10 @@ public class ActiveRepairServiceTest
         UUID prsId = UUID.randomUUID();
         ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), null, true, 0, false);
         ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(prsId);
-        prs.markSSTablesRepairing(store.metadata.cfId, prsId);
+        prs.markSSTablesRepairing(store.metadata.id, prsId);
 
         //retrieve all sstable references from parent repair sessions
-        Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId);
+        Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.id, prsId);
         Set<SSTableReader> retrieved = Sets.newHashSet(refs.iterator());
         assertEquals(original, retrieved);
         refs.release();
@@ -256,7 +256,7 @@ public class ActiveRepairServiceTest
         }, OperationType.COMPACTION, null);
 
         //retrieve sstable references from parent repair session again - removed sstable must not be present
-        refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId);
+        refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.id, prsId);
         retrieved = Sets.newHashSet(refs.iterator());
         assertEquals(newLiveSet, retrieved);
         assertFalse(retrieved.contains(removed));
@@ -272,8 +272,8 @@ public class ActiveRepairServiceTest
         ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), null, true, System.currentTimeMillis(), true);
 
         ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(prsId);
-        prs.markSSTablesRepairing(store.metadata.cfId, prsId);
-        try (Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId))
+        prs.markSSTablesRepairing(store.metadata.id, prsId);
+        try (Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.id, prsId))
         {
             Set<SSTableReader> retrieved = Sets.newHashSet(refs.iterator());
             assertEquals(original, retrieved);
@@ -284,7 +284,7 @@ public class ActiveRepairServiceTest
         {
             UUID newPrsId = UUID.randomUUID();
             ActiveRepairService.instance.registerParentRepairSession(newPrsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), null, true, System.currentTimeMillis(), true);
-            ActiveRepairService.instance.getParentRepairSession(newPrsId).markSSTablesRepairing(store.metadata.cfId, newPrsId);
+            ActiveRepairService.instance.getParentRepairSession(newPrsId).markSSTablesRepairing(store.metadata.id, newPrsId);
         }
         catch (Throwable t)
         {
@@ -292,7 +292,7 @@ public class ActiveRepairServiceTest
         }
         assertTrue(exception);
 
-        try (Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId))
+        try (Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.id, prsId))
         {
             Set<SSTableReader> retrieved = Sets.newHashSet(refs.iterator());
             assertEquals(original, retrieved);
@@ -306,19 +306,19 @@ public class ActiveRepairServiceTest
         UUID prsId = UUID.randomUUID();
         Set<SSTableReader> original = Sets.newHashSet(store.select(View.select(SSTableSet.CANONICAL, (s) -> !s.isRepaired())).sstables);
         ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), Collections.singleton(new Range<>(store.getPartitioner().getMinimumToken(), store.getPartitioner().getMinimumToken())), true, System.currentTimeMillis(), true);
-        ActiveRepairService.instance.getParentRepairSession(prsId).maybeSnapshot(store.metadata.cfId, prsId);
+        ActiveRepairService.instance.getParentRepairSession(prsId).maybeSnapshot(store.metadata.id, prsId);
 
         UUID prsId2 = UUID.randomUUID();
         ActiveRepairService.instance.registerParentRepairSession(prsId2, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), Collections.singleton(new Range<>(store.getPartitioner().getMinimumToken(), store.getPartitioner().getMinimumToken())), true, System.currentTimeMillis(), true);
         createSSTables(store, 2);
-        ActiveRepairService.instance.getParentRepairSession(prsId).maybeSnapshot(store.metadata.cfId, prsId);
-        try (Refs<SSTableReader> refs = ActiveRepairService.instance.getParentRepairSession(prsId).getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId))
+        ActiveRepairService.instance.getParentRepairSession(prsId).maybeSnapshot(store.metadata.id, prsId);
+        try (Refs<SSTableReader> refs = ActiveRepairService.instance.getParentRepairSession(prsId).getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.id, prsId))
         {
             assertEquals(original, Sets.newHashSet(refs.iterator()));
         }
         store.forceMajorCompaction();
         // after a major compaction the original sstables will be gone and we will have no sstables to anticompact:
-        try (Refs<SSTableReader> refs = ActiveRepairService.instance.getParentRepairSession(prsId).getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId))
+        try (Refs<SSTableReader> refs = ActiveRepairService.instance.getParentRepairSession(prsId).getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.id, prsId))
         {
             assertEquals(0, refs.size());
         }
@@ -331,21 +331,21 @@ public class ActiveRepairServiceTest
         Set<SSTableReader> original = Sets.newHashSet(store.select(View.select(SSTableSet.CANONICAL, (s) -> !s.isRepaired())).sstables);
         UUID prsId = UUID.randomUUID();
         ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), Collections.singleton(new Range<>(store.getPartitioner().getMinimumToken(), store.getPartitioner().getMinimumToken())), true, System.currentTimeMillis(), true);
-        ActiveRepairService.instance.getParentRepairSession(prsId).maybeSnapshot(store.metadata.cfId, prsId);
+        ActiveRepairService.instance.getParentRepairSession(prsId).maybeSnapshot(store.metadata.id, prsId);
 
         UUID prsId2 = UUID.randomUUID();
         ActiveRepairService.instance.registerParentRepairSession(prsId2, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), Collections.singleton(new Range<>(store.getPartitioner().getMinimumToken(), store.getPartitioner().getMinimumToken())), true, System.currentTimeMillis(), true);
         boolean exception = false;
         try
         {
-            ActiveRepairService.instance.getParentRepairSession(prsId2).maybeSnapshot(store.metadata.cfId, prsId2);
+            ActiveRepairService.instance.getParentRepairSession(prsId2).maybeSnapshot(store.metadata.id, prsId2);
         }
         catch (Throwable t)
         {
             exception = true;
         }
         assertTrue(exception);
-        try (Refs<SSTableReader> refs = ActiveRepairService.instance.getParentRepairSession(prsId).getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId))
+        try (Refs<SSTableReader> refs = ActiveRepairService.instance.getParentRepairSession(prsId).getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.id, prsId))
         {
             assertEquals(original, Sets.newHashSet(refs.iterator()));
         }
@@ -368,7 +368,7 @@ public class ActiveRepairServiceTest
         {
             for (int j = 0; j < 10; j++)
             {
-                new RowUpdateBuilder(cfs.metadata, timestamp, Integer.toString(j))
+                new RowUpdateBuilder(cfs.metadata(), timestamp, Integer.toString(j))
                 .clustering("c")
                 .add("val", "val")
                 .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/service/DataResolverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/DataResolverTest.java b/test/unit/org/apache/cassandra/service/DataResolverTest.java
index dba3e95..413f032 100644
--- a/test/unit/org/apache/cassandra/service/DataResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/DataResolverTest.java
@@ -29,8 +29,8 @@ import org.junit.*;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
@@ -69,9 +69,9 @@ public class DataResolverTest
     private Keyspace ks;
     private ColumnFamilyStore cfs;
     private ColumnFamilyStore cfs2;
-    private CFMetaData cfm;
-    private CFMetaData cfm2;
-    private ColumnDefinition m;
+    private TableMetadata cfm;
+    private TableMetadata cfm2;
+    private ColumnMetadata m;
     private int nowInSec;
     private ReadCommand command;
     private MessageRecorder messageRecorder;
@@ -81,23 +81,23 @@ public class DataResolverTest
     public static void defineSchema() throws ConfigurationException
     {
         DatabaseDescriptor.daemonInitialization();
-        CFMetaData cfMetadata = CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD)
-                                                  .addPartitionKey("key", BytesType.instance)
-                                                  .addClusteringColumn("col1", AsciiType.instance)
-                                                  .addRegularColumn("c1", AsciiType.instance)
-                                                  .addRegularColumn("c2", AsciiType.instance)
-                                                  .addRegularColumn("one", AsciiType.instance)
-                                                  .addRegularColumn("two", AsciiType.instance)
-                                                  .build();
-
-        CFMetaData cfMetaData2 = CFMetaData.Builder.create(KEYSPACE1, CF_COLLECTION)
-                                                   .addPartitionKey("k", ByteType.instance)
-                                                   .addRegularColumn("m", MapType.getInstance(IntegerType.instance, IntegerType.instance, true))
-                                                   .build();
+
+        TableMetadata.Builder builder1 =
+            TableMetadata.builder(KEYSPACE1, CF_STANDARD)
+                         .addPartitionKeyColumn("key", BytesType.instance)
+                         .addClusteringColumn("col1", AsciiType.instance)
+                         .addRegularColumn("c1", AsciiType.instance)
+                         .addRegularColumn("c2", AsciiType.instance)
+                         .addRegularColumn("one", AsciiType.instance)
+                         .addRegularColumn("two", AsciiType.instance);
+
+        TableMetadata.Builder builder2 =
+            TableMetadata.builder(KEYSPACE1, CF_COLLECTION)
+                         .addPartitionKeyColumn("k", ByteType.instance)
+                         .addRegularColumn("m", MapType.getInstance(IntegerType.instance, IntegerType.instance, true));
+
         SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    cfMetadata, cfMetaData2);
+        SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1), builder1, builder2);
     }
 
     @Before
@@ -106,10 +106,10 @@ public class DataResolverTest
         dk = Util.dk("key1");
         ks = Keyspace.open(KEYSPACE1);
         cfs = ks.getColumnFamilyStore(CF_STANDARD);
-        cfm = cfs.metadata;
+        cfm = cfs.metadata();
         cfs2 = ks.getColumnFamilyStore(CF_COLLECTION);
-        cfm2 = cfs2.metadata;
-        m = cfm2.getColumnDefinition(new ColumnIdentifier("m", false));
+        cfm2 = cfs2.metadata();
+        m = cfm2.getColumn(new ColumnIdentifier("m", false));
 
         nowInSec = FBUtilities.nowInSeconds();
         command = Util.cmd(cfs, dk).withNowInSeconds(nowInSec).build();
@@ -631,7 +631,7 @@ public class DataResolverTest
 
         MessageOut<Mutation> msg;
         msg = getSentMessage(peer1);
-        Iterator<Row> rowIter = msg.payload.getPartitionUpdate(cfm2.cfId).iterator();
+        Iterator<Row> rowIter = msg.payload.getPartitionUpdate(cfm2).iterator();
         assertTrue(rowIter.hasNext());
         Row row = rowIter.next();
         assertFalse(rowIter.hasNext());
@@ -676,7 +676,7 @@ public class DataResolverTest
 
         MessageOut<Mutation> msg;
         msg = getSentMessage(peer1);
-        Iterator<Row> rowIter = msg.payload.getPartitionUpdate(cfm2.cfId).iterator();
+        Iterator<Row> rowIter = msg.payload.getPartitionUpdate(cfm2).iterator();
         assertTrue(rowIter.hasNext());
         Row row = rowIter.next();
         assertFalse(rowIter.hasNext());
@@ -728,7 +728,7 @@ public class DataResolverTest
 
         MessageOut<Mutation> msg;
         msg = getSentMessage(peer2);
-        Iterator<Row> rowIter = msg.payload.getPartitionUpdate(cfm2.cfId).iterator();
+        Iterator<Row> rowIter = msg.payload.getPartitionUpdate(cfm2).iterator();
         assertTrue(rowIter.hasNext());
         Row row = rowIter.next();
         assertFalse(rowIter.hasNext());
@@ -779,7 +779,7 @@ public class DataResolverTest
 
         MessageOut<Mutation> msg;
         msg = getSentMessage(peer1);
-        Row row = Iterators.getOnlyElement(msg.payload.getPartitionUpdate(cfm2.cfId).iterator());
+        Row row = Iterators.getOnlyElement(msg.payload.getPartitionUpdate(cfm2).iterator());
 
         ComplexColumnData cd = row.getComplexColumnData(m);
 
@@ -857,8 +857,8 @@ public class DataResolverTest
     {
         assertEquals(MessagingService.Verb.READ_REPAIR, message.verb);
         PartitionUpdate update = ((Mutation)message.payload).getPartitionUpdates().iterator().next();
-        assertEquals(update.metadata().ksName, cfm.ksName);
-        assertEquals(update.metadata().cfName, cfm.cfName);
+        assertEquals(update.metadata().keyspace, cfm.keyspace);
+        assertEquals(update.metadata().name, cfm.name);
     }
 
 
@@ -891,7 +891,7 @@ public class DataResolverTest
         return new RangeTombstone(Slice.make(startBound, endBound), new DeletionTime(markedForDeleteAt, localDeletionTime));
     }
 
-    private UnfilteredPartitionIterator fullPartitionDelete(CFMetaData cfm, DecoratedKey dk, long timestamp, int nowInSec)
+    private UnfilteredPartitionIterator fullPartitionDelete(TableMetadata cfm, DecoratedKey dk, long timestamp, int nowInSec)
     {
         return new SingletonUnfilteredPartitionIterator(PartitionUpdate.fullPartitionDelete(cfm, dk, timestamp, nowInSec).unfilteredIterator());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/service/JoinTokenRingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/JoinTokenRingTest.java b/test/unit/org/apache/cassandra/service/JoinTokenRingTest.java
index 866910e..c2aeb56 100644
--- a/test/unit/org/apache/cassandra/service/JoinTokenRingTest.java
+++ b/test/unit/org/apache/cassandra/service/JoinTokenRingTest.java
@@ -48,7 +48,7 @@ public class JoinTokenRingTest
         ss.joinRing();
 
         SecondaryIndexManager indexManager = ColumnFamilyStore.getIfExists("JoinTokenRingTestKeyspace7", "Indexed1").indexManager;
-        StubIndex stub = (StubIndex) indexManager.getIndexByName("value_index");
+        StubIndex stub = (StubIndex) indexManager.getIndexByName("Indexed1_value_index");
         Assert.assertTrue(stub.preJoinInvocation);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
index 754def9..aaa2594 100644
--- a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
+++ b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
@@ -34,7 +34,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.Util.PartitionerSwitcher;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
@@ -715,7 +715,7 @@ public class LeaveAndBootstrapTest
 
     private AbstractReplicationStrategy getStrategy(String keyspaceName, TokenMetadata tmd)
     {
-        KeyspaceMetadata ksmd = Schema.instance.getKSMetaData(keyspaceName);
+        KeyspaceMetadata ksmd = Schema.instance.getKeyspaceMetadata(keyspaceName);
         return AbstractReplicationStrategy.createReplicationStrategy(
                 keyspaceName,
                 ksmd.params.replication.klass,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/service/MoveTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java b/test/unit/org/apache/cassandra/service/MoveTest.java
index 90d546c..df4e294 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -30,12 +30,13 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
@@ -141,14 +142,17 @@ public class MoveTest
         });
 
         final TokenMetadata tmd = StorageService.instance.getTokenMetadata();
-                tmd.clearUnsafe();
-                tmd.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.0.0.1"));
-                tmd.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.0.0.2"));
+
+        tmd.clearUnsafe();
+        tmd.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.0.0.1"));
+        tmd.updateHostId(UUID.randomUUID(), InetAddress.getByName("127.0.0.2"));
 
         KeyspaceMetadata keyspace =  KeyspaceMetadata.create(keyspaceName,
                                                              KeyspaceParams.nts(configOptions(replicas)),
-                                                             Tables.of(CFMetaData.Builder.create(keyspaceName, "CF1")
-                                                                                         .addPartitionKey("key", BytesType.instance).build()));
+                                                             Tables.of(TableMetadata.builder(keyspaceName, "CF1")
+                                                                                    .addPartitionKeyColumn("key", BytesType.instance)
+                                                                                    .build()));
+
         MigrationManager.announceNewKeyspace(keyspace);
     }
 
@@ -1007,7 +1011,7 @@ public class MoveTest
 
     private AbstractReplicationStrategy getStrategy(String keyspaceName, TokenMetadata tmd)
     {
-        KeyspaceMetadata ksmd = Schema.instance.getKSMetaData(keyspaceName);
+        KeyspaceMetadata ksmd = Schema.instance.getKeyspaceMetadata(keyspaceName);
         return AbstractReplicationStrategy.createReplicationStrategy(
                 keyspaceName,
                 ksmd.params.replication.klass,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/service/PaxosStateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/PaxosStateTest.java b/test/unit/org/apache/cassandra/service/PaxosStateTest.java
index 8054c61..6c12001 100644
--- a/test/unit/org/apache/cassandra/service/PaxosStateTest.java
+++ b/test/unit/org/apache/cassandra/service/PaxosStateTest.java
@@ -59,7 +59,7 @@ public class PaxosStateTest
         ColumnFamilyStore cfs = Keyspace.open("PaxosStateTestKeyspace1").getColumnFamilyStore("Standard1");
         String key = "key" + System.nanoTime();
         ByteBuffer value = ByteBufferUtil.bytes(0);
-        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros(), key);
+        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), FBUtilities.timestampMicros(), key);
         builder.clustering("a").add("val", value);
         PartitionUpdate update = Iterables.getOnlyElement(builder.build().getPartitionUpdates());
 
@@ -78,7 +78,7 @@ public class PaxosStateTest
         assertNoDataPresent(cfs, Util.dk(key));
 
         // Now try again with a ballot created after the truncation
-        long timestamp = SystemKeyspace.getTruncatedAt(update.metadata().cfId) + 1;
+        long timestamp = SystemKeyspace.getTruncatedAt(update.metadata().id) + 1;
         Commit afterTruncate = newProposal(timestamp, update);
         PaxosState.commit(afterTruncate);
         assertDataPresent(cfs, Util.dk(key), "val", value);
@@ -93,7 +93,7 @@ public class PaxosStateTest
     {
         Row row = Util.getOnlyRowUnfiltered(Util.cmd(cfs, key).build());
         assertEquals(0, ByteBufferUtil.compareUnsigned(value,
-                row.getCell(cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes(name))).value()));
+                row.getCell(cfs.metadata().getColumn(ByteBufferUtil.bytes(name))).value()));
     }
 
     private void assertNoDataPresent(ColumnFamilyStore cfs, DecoratedKey key)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/service/QueryPagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/QueryPagerTest.java b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
index 2104d14..d0b7704 100644
--- a/test/unit/org/apache/cassandra/service/QueryPagerTest.java
+++ b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
@@ -27,7 +27,8 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 
 import org.apache.cassandra.*;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.RowIterator;
@@ -59,16 +60,18 @@ public class QueryPagerTest
     public static void defineSchema() throws ConfigurationException
     {
         SchemaLoader.prepareServer();
+
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     KeyspaceParams.simple(1),
                                     SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+
         SchemaLoader.createKeyspace(KEYSPACE_CQL,
                                     KeyspaceParams.simple(1),
-                                    CFMetaData.compile("CREATE TABLE " + CF_CQL + " ("
-                                            + "k text,"
-                                            + "c text,"
-                                            + "v text,"
-                                            + "PRIMARY KEY (k, c))", KEYSPACE_CQL));
+                                    CreateTableStatement.parse("CREATE TABLE " + CF_CQL + " ("
+                                                               + "k text,"
+                                                               + "c text,"
+                                                               + "v text,"
+                                                               + "PRIMARY KEY (k, c))", KEYSPACE_CQL));
         addData();
     }
 
@@ -101,7 +104,7 @@ public class QueryPagerTest
         {
             for (int j = 0; j < nbCols; j++)
             {
-                RowUpdateBuilder builder = new RowUpdateBuilder(cfs().metadata, FBUtilities.timestampMicros(), "k" + i);
+                RowUpdateBuilder builder = new RowUpdateBuilder(cfs().metadata(), FBUtilities.timestampMicros(), "k" + i);
                 builder.clustering("c" + j).add("val", "").build().applyUnsafe();
             }
         }
@@ -156,12 +159,12 @@ public class QueryPagerTest
     private static SinglePartitionReadCommand sliceQuery(String key, String start, String end, boolean reversed, int count)
     {
         ClusteringComparator cmp = cfs().getComparator();
-        CFMetaData metadata = cfs().metadata;
+        TableMetadata metadata = cfs().metadata();
 
         Slice slice = Slice.make(cmp.make(start), cmp.make(end));
         ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(Slices.with(cmp, slice), reversed);
 
-        return SinglePartitionReadCommand.create(cfs().metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, Util.dk(key), filter);
+        return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, Util.dk(key), filter);
     }
 
     private static ReadCommand rangeNamesQuery(String keyStart, String keyEnd, int count, String... names)
@@ -427,7 +430,7 @@ public class QueryPagerTest
         for (int i = 0; i < 5; i++)
             executeInternal(String.format("INSERT INTO %s.%s (k, c, v) VALUES ('k%d', 'c%d', null)", keyspace, table, 0, i));
 
-        ReadCommand command = SinglePartitionReadCommand.create(cfs.metadata, nowInSec, Util.dk("k0"), Slice.ALL);
+        ReadCommand command = SinglePartitionReadCommand.create(cfs.metadata(), nowInSec, Util.dk("k0"), Slice.ALL);
 
         QueryPager pager = command.getPager(null, ProtocolVersion.CURRENT);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/service/StartupChecksTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StartupChecksTest.java b/test/unit/org/apache/cassandra/service/StartupChecksTest.java
index 224f3d9..e2c3e6b 100644
--- a/test/unit/org/apache/cassandra/service/StartupChecksTest.java
+++ b/test/unit/org/apache/cassandra/service/StartupChecksTest.java
@@ -27,7 +27,7 @@ import org.junit.*;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.exceptions.StartupException;
 import org.apache.cassandra.io.util.FileUtils;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 297d19d..3ef7bbb 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -35,9 +35,9 @@ import org.junit.runner.RunWith;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.WindowsFailedSnapshotTracker;
 import org.apache.cassandra.dht.Murmur3Partitioner;
@@ -206,7 +206,7 @@ public class StorageServiceServerTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, configOptions));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name,
                                                                                                             InetAddress.getByName("127.0.0.1"));
@@ -249,7 +249,7 @@ public class StorageServiceServerTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, configOptions));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
         assert primaryRanges.size() == 1;
@@ -286,7 +286,7 @@ public class StorageServiceServerTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, configOptions));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         // endpoints in DC1 should not have primary range
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
@@ -325,7 +325,7 @@ public class StorageServiceServerTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, configOptions));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         // endpoints in DC1 should not have primary range
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.1"));
@@ -377,7 +377,7 @@ public class StorageServiceServerTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, configOptions));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         // endpoints in DC1 should not have primary range
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
@@ -444,7 +444,7 @@ public class StorageServiceServerTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, configOptions));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         // endpoints in DC1 should have primary ranges which also cover DC2
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.1"));
@@ -503,7 +503,7 @@ public class StorageServiceServerTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.simpleTransient(2));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
         assert primaryRanges.size() == 1;
@@ -534,7 +534,7 @@ public class StorageServiceServerTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.simpleTransient(2));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangeForEndpointWithinDC(meta.name, InetAddress.getByName("127.0.0.1"));
         assert primaryRanges.size() == 1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java b/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
index 80ef29d..ae18be9 100644
--- a/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
+++ b/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
@@ -25,6 +25,7 @@ import java.util.UUID;
 
 import org.junit.Test;
 
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.FBUtilities;
 
 public class SessionInfoTest
@@ -35,17 +36,17 @@ public class SessionInfoTest
     @Test
     public void testTotals()
     {
-        UUID cfId = UUID.randomUUID();
+        TableId tableId = TableId.generate();
         InetAddress local = FBUtilities.getLocalAddress();
 
         Collection<StreamSummary> summaries = new ArrayList<>();
         for (int i = 0; i < 10; i++)
         {
-            StreamSummary summary = new StreamSummary(cfId, i, (i + 1) * 10);
+            StreamSummary summary = new StreamSummary(tableId, i, (i + 1) * 10);
             summaries.add(summary);
         }
 
-        StreamSummary sending = new StreamSummary(cfId, 10, 100);
+        StreamSummary sending = new StreamSummary(tableId, 10, 100);
         SessionInfo info = new SessionInfo(local, 0, local, summaries, Collections.singleton(sending), StreamSession.State.PREPARING);
 
         assert info.getTotalFilesToReceive() == 45;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
index 04be91a..edd9d9c 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
@@ -41,9 +41,11 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.messages.OutgoingFileMessage;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.Ref;
+import org.hsqldb.Table;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
@@ -84,7 +86,7 @@ public class StreamTransferTaskTest
         }
 
         // create streaming task that streams those two sstables
-        StreamTransferTask task = new StreamTransferTask(session, cfs.metadata.cfId);
+        StreamTransferTask task = new StreamTransferTask(session, cfs.metadata.id);
         for (SSTableReader sstable : cfs.getLiveSSTables())
         {
             List<Range<Token>> ranges = new ArrayList<>();
@@ -133,7 +135,7 @@ public class StreamTransferTaskTest
         }
 
         // create streaming task that streams those two sstables
-        StreamTransferTask task = new StreamTransferTask(session, cfs.metadata.cfId);
+        StreamTransferTask task = new StreamTransferTask(session, cfs.metadata.id);
         List<Ref<SSTableReader>> refs = new ArrayList<>(cfs.getLiveSSTables().size());
         for (SSTableReader sstable : cfs.getLiveSSTables())
         {
@@ -146,7 +148,7 @@ public class StreamTransferTaskTest
         assertEquals(2, task.getTotalNumberOfFiles());
 
         //add task to stream session, so it is aborted when stream session fails
-        session.transfers.put(UUID.randomUUID(), task);
+        session.transfers.put(TableId.generate(), task);
 
         //make a copy of outgoing file messages, since task is cleared when it's aborted
         Collection<OutgoingFileMessage> files = new LinkedList<>(task.files.values());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index 8f3061a..9219e18 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -35,7 +35,7 @@ import junit.framework.Assert;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -54,6 +54,9 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.Refs;
 
+import static org.apache.cassandra.SchemaLoader.compositeIndexCFMD;
+import static org.apache.cassandra.SchemaLoader.createKeyspace;
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
@@ -83,25 +86,26 @@ public class StreamingTransferTest
     {
         SchemaLoader.prepareServer();
         StorageService.instance.initServer();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
-                                    CFMetaData.Builder.create(KEYSPACE1, CF_COUNTER, false, true, true)
-                                                      .addPartitionKey("key", BytesType.instance)
-                                                      .build(),
-                                    CFMetaData.Builder.create(KEYSPACE1, CF_STANDARDINT)
-                                                      .addPartitionKey("key", AsciiType.instance)
-                                                      .addClusteringColumn("cols", Int32Type.instance)
-                                                      .addRegularColumn("val", BytesType.instance)
-                                                      .build(),
-                                    SchemaLoader.compositeIndexCFMD(KEYSPACE1, CF_INDEX, true));
-        SchemaLoader.createKeyspace(KEYSPACE2,
-                                    KeyspaceParams.simple(1));
-        SchemaLoader.createKeyspace(KEYSPACE_CACHEKEY,
-                                    KeyspaceParams.simple(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD),
-                                    SchemaLoader.standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD2),
-                                    SchemaLoader.standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD3));
+
+        createKeyspace(KEYSPACE1,
+                       KeyspaceParams.simple(1),
+                       standardCFMD(KEYSPACE1, CF_STANDARD),
+                       TableMetadata.builder(KEYSPACE1, CF_COUNTER)
+                                    .isCounter(true)
+                                    .addPartitionKeyColumn("key", BytesType.instance),
+                       TableMetadata.builder(KEYSPACE1, CF_STANDARDINT)
+                                    .addPartitionKeyColumn("key", AsciiType.instance)
+                                    .addClusteringColumn("cols", Int32Type.instance)
+                                    .addRegularColumn("val", BytesType.instance),
+                       compositeIndexCFMD(KEYSPACE1, CF_INDEX, true));
+
+        createKeyspace(KEYSPACE2, KeyspaceParams.simple(1));
+
+        createKeyspace(KEYSPACE_CACHEKEY,
+                       KeyspaceParams.simple(1),
+                       standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD),
+                       standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD2),
+                       standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD3));
     }
 
     /**
@@ -311,7 +315,7 @@ public class StreamingTransferTest
             {
                 long val = key.hashCode();
 
-                RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, timestamp, key);
+                RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), timestamp, key);
                 builder.clustering(col).add("birthdate", ByteBufferUtil.bytes(val));
                 builder.build().applyUnsafe();
             }
@@ -324,7 +328,7 @@ public class StreamingTransferTest
 
             // test we can search:
             UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".\"%s\" WHERE birthdate = %d",
-                    cfs.metadata.ksName, cfs.metadata.cfName, val));
+                                                                                   cfs.metadata.keyspace, cfs.metadata.name, val));
             assertEquals(1, result.size());
 
             assert result.iterator().next().getBytes("key").equals(ByteBufferUtil.bytes(key));
@@ -346,7 +350,7 @@ public class StreamingTransferTest
         String key = "key1";
 
 
-        RowUpdateBuilder updates = new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros(), key);
+        RowUpdateBuilder updates = new RowUpdateBuilder(cfs.metadata(), FBUtilities.timestampMicros(), key);
 
         // add columns of size slightly less than column_index_size to force insert column index
         updates.clustering(1)
@@ -354,7 +358,7 @@ public class StreamingTransferTest
                 .build()
                 .apply();
 
-        updates = new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros(), key);
+        updates = new RowUpdateBuilder(cfs.metadata(), FBUtilities.timestampMicros(), key);
         updates.clustering(6)
                 .add("val", ByteBuffer.wrap(new byte[DatabaseDescriptor.getColumnIndexSize()]))
                 .build()
@@ -367,7 +371,7 @@ public class StreamingTransferTest
         //        .apply();
 
 
-        updates = new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros() + 1, key);
+        updates = new RowUpdateBuilder(cfs.metadata(), FBUtilities.timestampMicros() + 1, key);
         updates.addRangeTombstone(5, 7)
                 .build()
                 .apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/tools/ToolsTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/ToolsTester.java b/test/unit/org/apache/cassandra/tools/ToolsTester.java
index ead4e31..8dd7314 100644
--- a/test/unit/org/apache/cassandra/tools/ToolsTester.java
+++ b/test/unit/org/apache/cassandra/tools/ToolsTester.java
@@ -114,12 +114,12 @@ public abstract class ToolsTester
 
     public void assertSchemaNotLoaded()
     {
-        assertClassNotLoaded("org.apache.cassandra.config.Schema");
+        assertClassNotLoaded("org.apache.cassandra.schema.Schema");
     }
 
     public void assertSchemaLoaded()
     {
-        assertClassLoaded("org.apache.cassandra.config.Schema");
+        assertClassLoaded("org.apache.cassandra.schema.Schema");
     }
 
     public void assertKeyspaceNotLoaded()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java b/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
index 90b4cdf..a796daf 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
@@ -23,7 +23,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
@@ -33,6 +33,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.schema.TriggerMetadata;
+import org.apache.cassandra.schema.Triggers;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
@@ -51,7 +52,7 @@ public class TriggerExecutorTest
     @Test
     public void sameKeySameCfColumnFamilies() throws ConfigurationException, InvalidRequestException
     {
-        CFMetaData metadata = makeCfMetaData("ks1", "cf1", TriggerMetadata.create("test", SameKeySameCfTrigger.class.getName()));
+        TableMetadata metadata = makeTableMetadata("ks1", "cf1", TriggerMetadata.create("test", SameKeySameCfTrigger.class.getName()));
         PartitionUpdate mutated = TriggerExecutor.instance.execute(makeCf(metadata, "k1", "v1", null));
 
         try (RowIterator rowIterator = UnfilteredRowIterators.filter(mutated.unfilteredIterator(),
@@ -67,21 +68,21 @@ public class TriggerExecutorTest
     @Test(expected = InvalidRequestException.class)
     public void sameKeyDifferentCfColumnFamilies() throws ConfigurationException, InvalidRequestException
     {
-        CFMetaData metadata = makeCfMetaData("ks1", "cf1", TriggerMetadata.create("test", SameKeyDifferentCfTrigger.class.getName()));
+        TableMetadata metadata = makeTableMetadata("ks1", "cf1", TriggerMetadata.create("test", SameKeyDifferentCfTrigger.class.getName()));
         TriggerExecutor.instance.execute(makeCf(metadata, "k1", "v1", null));
     }
 
     @Test(expected = InvalidRequestException.class)
     public void differentKeyColumnFamilies() throws ConfigurationException, InvalidRequestException
     {
-        CFMetaData metadata = makeCfMetaData("ks1", "cf1", TriggerMetadata.create("test", DifferentKeyTrigger.class.getName()));
+        TableMetadata metadata = makeTableMetadata("ks1", "cf1", TriggerMetadata.create("test", DifferentKeyTrigger.class.getName()));
         TriggerExecutor.instance.execute(makeCf(metadata, "k1", "v1", null));
     }
 
     @Test
     public void noTriggerMutations() throws ConfigurationException, InvalidRequestException
     {
-        CFMetaData metadata = makeCfMetaData("ks1", "cf1", TriggerMetadata.create("test", NoOpTrigger.class.getName()));
+        TableMetadata metadata = makeTableMetadata("ks1", "cf1", TriggerMetadata.create("test", NoOpTrigger.class.getName()));
         Mutation rm = new Mutation(makeCf(metadata, "k1", "v1", null));
         assertNull(TriggerExecutor.instance.execute(Collections.singletonList(rm)));
     }
@@ -89,7 +90,7 @@ public class TriggerExecutorTest
     @Test
     public void sameKeySameCfRowMutations() throws ConfigurationException, InvalidRequestException
     {
-        CFMetaData metadata = makeCfMetaData("ks1", "cf1", TriggerMetadata.create("test", SameKeySameCfTrigger.class.getName()));
+        TableMetadata metadata = makeTableMetadata("ks1", "cf1", TriggerMetadata.create("test", SameKeySameCfTrigger.class.getName()));
         PartitionUpdate cf1 = makeCf(metadata, "k1", "k1v1", null);
         PartitionUpdate cf2 = makeCf(metadata, "k2", "k2v1", null);
         Mutation rm1 = new Mutation("ks1", cf1.partitionKey()).add(cf1);
@@ -102,20 +103,20 @@ public class TriggerExecutorTest
         List<PartitionUpdate> mutatedCFs = new ArrayList<>(tmutations.get(0).getPartitionUpdates());
         assertEquals(1, mutatedCFs.size());
         Row row = mutatedCFs.get(0).iterator().next();
-        assertEquals(bytes("k1v1"), row.getCell(metadata.getColumnDefinition(bytes("c1"))).value());
-        assertEquals(bytes("trigger"), row.getCell(metadata.getColumnDefinition(bytes("c2"))).value());
+        assertEquals(bytes("k1v1"), row.getCell(metadata.getColumn(bytes("c1"))).value());
+        assertEquals(bytes("trigger"), row.getCell(metadata.getColumn(bytes("c2"))).value());
 
         mutatedCFs = new ArrayList<>(tmutations.get(1).getPartitionUpdates());
         assertEquals(1, mutatedCFs.size());
         row = mutatedCFs.get(0).iterator().next();
-        assertEquals(bytes("k2v1"), row.getCell(metadata.getColumnDefinition(bytes("c1"))).value());
-        assertEquals(bytes("trigger"), row.getCell(metadata.getColumnDefinition(bytes("c2"))).value());
+        assertEquals(bytes("k2v1"), row.getCell(metadata.getColumn(bytes("c1"))).value());
+        assertEquals(bytes("trigger"), row.getCell(metadata.getColumn(bytes("c2"))).value());
     }
 
     @Test
     public void sameKeySameCfPartialRowMutations() throws ConfigurationException, InvalidRequestException
     {
-        CFMetaData metadata = makeCfMetaData("ks1", "cf1", TriggerMetadata.create("test", SameKeySameCfPartialTrigger.class.getName()));
+        TableMetadata metadata = makeTableMetadata("ks1", "cf1", TriggerMetadata.create("test", SameKeySameCfPartialTrigger.class.getName()));
         PartitionUpdate cf1 = makeCf(metadata, "k1", "k1v1", null);
         PartitionUpdate cf2 = makeCf(metadata, "k2", "k2v1", null);
         Mutation rm1 = new Mutation("ks1", cf1.partitionKey()).add(cf1);
@@ -128,20 +129,20 @@ public class TriggerExecutorTest
         List<PartitionUpdate> mutatedCFs = new ArrayList<>(tmutations.get(0).getPartitionUpdates());
         assertEquals(1, mutatedCFs.size());
         Row row = mutatedCFs.get(0).iterator().next();
-        assertEquals(bytes("k1v1"), row.getCell(metadata.getColumnDefinition(bytes("c1"))).value());
-        assertNull(row.getCell(metadata.getColumnDefinition(bytes("c2"))));
+        assertEquals(bytes("k1v1"), row.getCell(metadata.getColumn(bytes("c1"))).value());
+        assertNull(row.getCell(metadata.getColumn(bytes("c2"))));
 
         mutatedCFs = new ArrayList<>(tmutations.get(1).getPartitionUpdates());
         assertEquals(1, mutatedCFs.size());
         row = mutatedCFs.get(0).iterator().next();
-        assertEquals(bytes("k2v1"), row.getCell(metadata.getColumnDefinition(bytes("c1"))).value());
-        assertEquals(bytes("trigger"), row.getCell(metadata.getColumnDefinition(bytes("c2"))).value());
+        assertEquals(bytes("k2v1"), row.getCell(metadata.getColumn(bytes("c1"))).value());
+        assertEquals(bytes("trigger"), row.getCell(metadata.getColumn(bytes("c2"))).value());
     }
 
     @Test
     public void sameKeyDifferentCfRowMutations() throws ConfigurationException, InvalidRequestException
     {
-        CFMetaData metadata = makeCfMetaData("ks1", "cf1", TriggerMetadata.create("test", SameKeyDifferentCfTrigger.class.getName()));
+        TableMetadata metadata = makeTableMetadata("ks1", "cf1", TriggerMetadata.create("test", SameKeyDifferentCfTrigger.class.getName()));
         PartitionUpdate cf1 = makeCf(metadata, "k1", "k1v1", null);
         PartitionUpdate cf2 = makeCf(metadata, "k2", "k2v1", null);
         Mutation rm1 = new Mutation("ks1", cf1.partitionKey()).add(cf1);
@@ -155,17 +156,17 @@ public class TriggerExecutorTest
         assertEquals(2, mutatedCFs.size());
         for (PartitionUpdate update : mutatedCFs)
         {
-            if (update.metadata().cfName.equals("cf1"))
+            if (update.metadata().name.equals("cf1"))
             {
                 Row row = update.iterator().next();
-                assertEquals(bytes("k1v1"), row.getCell(metadata.getColumnDefinition(bytes("c1"))).value());
-                assertNull(row.getCell(metadata.getColumnDefinition(bytes("c2"))));
+                assertEquals(bytes("k1v1"), row.getCell(metadata.getColumn(bytes("c1"))).value());
+                assertNull(row.getCell(metadata.getColumn(bytes("c2"))));
             }
             else
             {
                 Row row = update.iterator().next();
-                assertNull(row.getCell(metadata.getColumnDefinition(bytes("c1"))));
-                assertEquals(bytes("trigger"), row.getCell(metadata.getColumnDefinition(bytes("c2"))).value());
+                assertNull(row.getCell(metadata.getColumn(bytes("c1"))));
+                assertEquals(bytes("trigger"), row.getCell(metadata.getColumn(bytes("c2"))).value());
             }
         }
 
@@ -174,17 +175,17 @@ public class TriggerExecutorTest
 
         for (PartitionUpdate update : mutatedCFs)
         {
-            if (update.metadata().cfName.equals("cf1"))
+            if (update.metadata().name.equals("cf1"))
             {
                 Row row = update.iterator().next();
-                assertEquals(bytes("k2v1"), row.getCell(metadata.getColumnDefinition(bytes("c1"))).value());
-                assertNull(row.getCell(metadata.getColumnDefinition(bytes("c2"))));
+                assertEquals(bytes("k2v1"), row.getCell(metadata.getColumn(bytes("c1"))).value());
+                assertNull(row.getCell(metadata.getColumn(bytes("c2"))));
             }
             else
             {
                 Row row = update.iterator().next();
-                assertNull(row.getCell(metadata.getColumnDefinition(bytes("c1"))));
-                assertEquals(bytes("trigger"), row.getCell(metadata.getColumnDefinition(bytes("c2"))).value());
+                assertNull(row.getCell(metadata.getColumn(bytes("c1"))));
+                assertEquals(bytes("trigger"), row.getCell(metadata.getColumn(bytes("c2"))).value());
             }
         }
     }
@@ -192,7 +193,7 @@ public class TriggerExecutorTest
     @Test
     public void sameKeyDifferentKsRowMutations() throws ConfigurationException, InvalidRequestException
     {
-        CFMetaData metadata = makeCfMetaData("ks1", "cf1", TriggerMetadata.create("test", SameKeyDifferentKsTrigger.class.getName()));
+        TableMetadata metadata = makeTableMetadata("ks1", "cf1", TriggerMetadata.create("test", SameKeyDifferentKsTrigger.class.getName()));
         PartitionUpdate cf1 = makeCf(metadata, "k1", "k1v1", null);
         PartitionUpdate cf2 = makeCf(metadata, "k2", "k2v1", null);
         Mutation rm1 = new Mutation("ks1", cf1.partitionKey()).add(cf1);
@@ -205,33 +206,33 @@ public class TriggerExecutorTest
         List<PartitionUpdate> mutatedCFs = new ArrayList<>(tmutations.get(0).getPartitionUpdates());
         assertEquals(1, mutatedCFs.size());
         Row row = mutatedCFs.get(0).iterator().next();
-        assertEquals(bytes("k1v1"), row.getCell(metadata.getColumnDefinition(bytes("c1"))).value());
-        assertNull(row.getCell(metadata.getColumnDefinition(bytes("c2"))));
+        assertEquals(bytes("k1v1"), row.getCell(metadata.getColumn(bytes("c1"))).value());
+        assertNull(row.getCell(metadata.getColumn(bytes("c2"))));
 
         mutatedCFs = new ArrayList<>(tmutations.get(1).getPartitionUpdates());
         assertEquals(1, mutatedCFs.size());
         row = mutatedCFs.get(0).iterator().next();
-        assertEquals(bytes("k2v1"), row.getCell(metadata.getColumnDefinition(bytes("c1"))).value());
-        assertNull(row.getCell(metadata.getColumnDefinition(bytes("c2"))));
+        assertEquals(bytes("k2v1"), row.getCell(metadata.getColumn(bytes("c1"))).value());
+        assertNull(row.getCell(metadata.getColumn(bytes("c2"))));
 
         mutatedCFs = new ArrayList<>(tmutations.get(2).getPartitionUpdates());
         assertEquals(1, mutatedCFs.size());
         row = mutatedCFs.get(0).iterator().next();
-        assertNull(row.getCell(metadata.getColumnDefinition(bytes("c1"))));
-        assertEquals(bytes("trigger"), row.getCell(metadata.getColumnDefinition(bytes("c2"))).value());
+        assertNull(row.getCell(metadata.getColumn(bytes("c1"))));
+        assertEquals(bytes("trigger"), row.getCell(metadata.getColumn(bytes("c2"))).value());
 
         mutatedCFs = new ArrayList<>(tmutations.get(3).getPartitionUpdates());
         assertEquals(1, mutatedCFs.size());
         row = mutatedCFs.get(0).iterator().next();
-        assertNull(row.getCell(metadata.getColumnDefinition(bytes("c1"))));
-        assertEquals(bytes("trigger"), row.getCell(metadata.getColumnDefinition(bytes("c2"))).value());
+        assertNull(row.getCell(metadata.getColumn(bytes("c1"))));
+        assertEquals(bytes("trigger"), row.getCell(metadata.getColumn(bytes("c2"))).value());
     }
 
     @Test
     public void differentKeyRowMutations() throws ConfigurationException, InvalidRequestException
     {
 
-        CFMetaData metadata = makeCfMetaData("ks1", "cf1", TriggerMetadata.create("test", DifferentKeyTrigger.class.getName()));
+        TableMetadata metadata = makeTableMetadata("ks1", "cf1", TriggerMetadata.create("test", DifferentKeyTrigger.class.getName()));
         PartitionUpdate cf1 = makeCf(metadata, "k1", "v1", null);
         Mutation rm = new Mutation("ks1", cf1.partitionKey()).add(cf1);
 
@@ -245,46 +246,39 @@ public class TriggerExecutorTest
         List<PartitionUpdate> mutatedCFs = new ArrayList<>(tmutations.get(0).getPartitionUpdates());
         assertEquals(1, mutatedCFs.size());
         Row row = mutatedCFs.get(0).iterator().next();
-        assertEquals(bytes("v1"), row.getCell(metadata.getColumnDefinition(bytes("c1"))).value());
-        assertNull(row.getCell(metadata.getColumnDefinition(bytes("c2"))));
+        assertEquals(bytes("v1"), row.getCell(metadata.getColumn(bytes("c1"))).value());
+        assertNull(row.getCell(metadata.getColumn(bytes("c2"))));
 
         mutatedCFs = new ArrayList<>(tmutations.get(1).getPartitionUpdates());
         assertEquals(1, mutatedCFs.size());
         row = mutatedCFs.get(0).iterator().next();
-        assertEquals(bytes("trigger"), row.getCell(metadata.getColumnDefinition(bytes("c2"))).value());
-        assertNull(row.getCell(metadata.getColumnDefinition(bytes("c1"))));
+        assertEquals(bytes("trigger"), row.getCell(metadata.getColumn(bytes("c2"))).value());
+        assertNull(row.getCell(metadata.getColumn(bytes("c1"))));
     }
 
-    private static CFMetaData makeCfMetaData(String ks, String cf, TriggerMetadata trigger)
+    private static TableMetadata makeTableMetadata(String ks, String cf, TriggerMetadata trigger)
     {
-        CFMetaData metadata = CFMetaData.Builder.create(ks, cf)
-                .addPartitionKey("pkey", UTF8Type.instance)
-                .addRegularColumn("c1", UTF8Type.instance)
-                .addRegularColumn("c2", UTF8Type.instance)
-                .build();
+        TableMetadata.Builder builder =
+            TableMetadata.builder(ks, cf)
+                         .addPartitionKeyColumn("pkey", UTF8Type.instance)
+                         .addRegularColumn("c1", UTF8Type.instance)
+                         .addRegularColumn("c2", UTF8Type.instance);
 
-        try
-        {
-            if (trigger != null)
-                metadata.triggers(metadata.getTriggers().with(trigger));
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new AssertionError(e);
-        }
+        if (trigger != null)
+            builder.triggers(Triggers.of(trigger));
 
-        return metadata;
+        return builder.build();
     }
 
-    private static PartitionUpdate makeCf(CFMetaData metadata, String key, String columnValue1, String columnValue2)
+    private static PartitionUpdate makeCf(TableMetadata metadata, String key, String columnValue1, String columnValue2)
     {
         Row.Builder builder = BTreeRow.unsortedBuilder(FBUtilities.nowInSeconds());
         builder.newRow(Clustering.EMPTY);
         long ts = FBUtilities.timestampMicros();
         if (columnValue1 != null)
-            builder.addCell(BufferCell.live(metadata.getColumnDefinition(bytes("c1")), ts, bytes(columnValue1)));
+            builder.addCell(BufferCell.live(metadata.getColumn(bytes("c1")), ts, bytes(columnValue1)));
         if (columnValue2 != null)
-            builder.addCell(BufferCell.live(metadata.getColumnDefinition(bytes("c2")), ts, bytes(columnValue2)));
+            builder.addCell(BufferCell.live(metadata.getColumn(bytes("c2")), ts, bytes(columnValue2)));
 
         return PartitionUpdate.singleRowUpdate(metadata, Util.dk(key), builder.build());
     }
@@ -324,7 +318,7 @@ public class TriggerExecutorTest
     {
         public Collection<Mutation> augment(Partition partition)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(makeCfMetaData(partition.metadata().ksName, "otherCf", null), FBUtilities.timestampMicros(), partition.partitionKey().getKey());
+            RowUpdateBuilder builder = new RowUpdateBuilder(makeTableMetadata(partition.metadata().keyspace, "otherCf", null), FBUtilities.timestampMicros(), partition.partitionKey().getKey());
             builder.add("c2", bytes("trigger"));
             return Collections.singletonList(builder.build());
         }
@@ -334,7 +328,7 @@ public class TriggerExecutorTest
     {
         public Collection<Mutation> augment(Partition partition)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(makeCfMetaData("otherKs", "otherCf", null), FBUtilities.timestampMicros(), partition.partitionKey().getKey());
+            RowUpdateBuilder builder = new RowUpdateBuilder(makeTableMetadata("otherKs", "otherCf", null), FBUtilities.timestampMicros(), partition.partitionKey().getKey());
             builder.add("c2", bytes("trigger"));
             return Collections.singletonList(builder.build());
         }
@@ -344,7 +338,7 @@ public class TriggerExecutorTest
     {
         public Collection<Mutation> augment(Partition partition)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(makeCfMetaData("otherKs", "otherCf", null), FBUtilities.timestampMicros(), "otherKey");
+            RowUpdateBuilder builder = new RowUpdateBuilder(makeTableMetadata("otherKs", "otherCf", null), FBUtilities.timestampMicros(), "otherKey");
             builder.add("c2", bytes("trigger"));
             return Collections.singletonList(builder.build());
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java b/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
index b6549bb..88f74a2 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
@@ -21,14 +21,16 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.Tables;
 import org.apache.cassandra.schema.TriggerMetadata;
-import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.schema.Triggers;
+import org.apache.cassandra.schema.MigrationManager;
 
 import static org.junit.Assert.*;
 
@@ -49,15 +51,18 @@ public class TriggersSchemaTest
     public void newKsContainsCfWithTrigger() throws Exception
     {
         TriggerMetadata td = TriggerMetadata.create(triggerName, triggerClass);
-        CFMetaData cfm1 = CFMetaData.compile(String.format("CREATE TABLE %s (k int PRIMARY KEY, v int)", cfName), ksName);
-        cfm1.triggers(cfm1.getTriggers().with(td));
-        KeyspaceMetadata ksm = KeyspaceMetadata.create(ksName, KeyspaceParams.simple(1), Tables.of(cfm1));
+        TableMetadata tm =
+            CreateTableStatement.parse(String.format("CREATE TABLE %s (k int PRIMARY KEY, v int)", cfName), ksName)
+                                .triggers(Triggers.of(td))
+                                .build();
+
+        KeyspaceMetadata ksm = KeyspaceMetadata.create(ksName, KeyspaceParams.simple(1), Tables.of(tm));
         MigrationManager.announceNewKeyspace(ksm);
 
-        CFMetaData cfm2 = Schema.instance.getCFMetaData(ksName, cfName);
-        assertFalse(cfm2.getTriggers().isEmpty());
-        assertEquals(1, cfm2.getTriggers().size());
-        assertEquals(td, cfm2.getTriggers().get(triggerName).get());
+        TableMetadata tm2 = Schema.instance.getTableMetadata(ksName, cfName);
+        assertFalse(tm2.triggers.isEmpty());
+        assertEquals(1, tm2.triggers.size());
+        assertEquals(td, tm2.triggers.get(triggerName).get());
     }
 
     @Test
@@ -66,50 +71,62 @@ public class TriggersSchemaTest
         KeyspaceMetadata ksm = KeyspaceMetadata.create(ksName, KeyspaceParams.simple(1));
         MigrationManager.announceNewKeyspace(ksm);
 
-        CFMetaData cfm1 = CFMetaData.compile(String.format("CREATE TABLE %s (k int PRIMARY KEY, v int)", cfName), ksName);
-        TriggerMetadata td = TriggerMetadata.create(triggerName, triggerClass);
-        cfm1.triggers(cfm1.getTriggers().with(td));
+        TableMetadata metadata =
+            CreateTableStatement.parse(String.format("CREATE TABLE %s (k int PRIMARY KEY, v int)", cfName), ksName)
+                                .triggers(Triggers.of(TriggerMetadata.create(triggerName, triggerClass)))
+                                .build();
 
-        MigrationManager.announceNewColumnFamily(cfm1);
+        MigrationManager.announceNewTable(metadata);
 
-        CFMetaData cfm2 = Schema.instance.getCFMetaData(ksName, cfName);
-        assertFalse(cfm2.getTriggers().isEmpty());
-        assertEquals(1, cfm2.getTriggers().size());
-        assertEquals(td, cfm2.getTriggers().get(triggerName).get());
+        metadata = Schema.instance.getTableMetadata(ksName, cfName);
+        assertFalse(metadata.triggers.isEmpty());
+        assertEquals(1, metadata.triggers.size());
+        assertEquals(TriggerMetadata.create(triggerName, triggerClass), metadata.triggers.get(triggerName).get());
     }
 
     @Test
     public void addTriggerToCf() throws Exception
     {
-        CFMetaData cfm1 = CFMetaData.compile(String.format("CREATE TABLE %s (k int PRIMARY KEY, v int)", cfName), ksName);
-        KeyspaceMetadata ksm = KeyspaceMetadata.create(ksName, KeyspaceParams.simple(1), Tables.of(cfm1));
+        TableMetadata tm1 =
+            CreateTableStatement.parse(String.format("CREATE TABLE %s (k int PRIMARY KEY, v int)", cfName), ksName)
+                                .build();
+        KeyspaceMetadata ksm = KeyspaceMetadata.create(ksName, KeyspaceParams.simple(1), Tables.of(tm1));
         MigrationManager.announceNewKeyspace(ksm);
 
-        CFMetaData cfm2 = Schema.instance.getCFMetaData(ksName, cfName).copy();
         TriggerMetadata td = TriggerMetadata.create(triggerName, triggerClass);
-        cfm2.triggers(cfm2.getTriggers().with(td));
-        MigrationManager.announceColumnFamilyUpdate(cfm2);
-
-        CFMetaData cfm3 = Schema.instance.getCFMetaData(ksName, cfName);
-        assertFalse(cfm3.getTriggers().isEmpty());
-        assertEquals(1, cfm3.getTriggers().size());
-        assertEquals(td, cfm3.getTriggers().get(triggerName).get());
+        TableMetadata tm2 =
+            Schema.instance
+                  .getTableMetadata(ksName, cfName)
+                  .unbuild()
+                  .triggers(Triggers.of(td))
+                  .build();
+        MigrationManager.announceTableUpdate(tm2);
+
+        TableMetadata tm3 = Schema.instance.getTableMetadata(ksName, cfName);
+        assertFalse(tm3.triggers.isEmpty());
+        assertEquals(1, tm3.triggers.size());
+        assertEquals(td, tm3.triggers.get(triggerName).get());
     }
 
     @Test
     public void removeTriggerFromCf() throws Exception
     {
         TriggerMetadata td = TriggerMetadata.create(triggerName, triggerClass);
-        CFMetaData cfm1 = CFMetaData.compile(String.format("CREATE TABLE %s (k int PRIMARY KEY, v int)", cfName), ksName);
-        cfm1.triggers(cfm1.getTriggers().with(td));
-        KeyspaceMetadata ksm = KeyspaceMetadata.create(ksName, KeyspaceParams.simple(1), Tables.of(cfm1));
+        TableMetadata tm =
+            CreateTableStatement.parse(String.format("CREATE TABLE %s (k int PRIMARY KEY, v int)", cfName), ksName)
+                                .triggers(Triggers.of(td))
+                                .build();
+        KeyspaceMetadata ksm = KeyspaceMetadata.create(ksName, KeyspaceParams.simple(1), Tables.of(tm));
         MigrationManager.announceNewKeyspace(ksm);
 
-        CFMetaData cfm2 = Schema.instance.getCFMetaData(ksName, cfName).copy();
-        cfm2.triggers(cfm2.getTriggers().without(triggerName));
-        MigrationManager.announceColumnFamilyUpdate(cfm2);
+        TableMetadata tm1 = Schema.instance.getTableMetadata(ksName, cfName);
+        TableMetadata tm2 =
+            tm1.unbuild()
+               .triggers(tm1.triggers.without(triggerName))
+               .build();
+        MigrationManager.announceTableUpdate(tm2);
 
-        CFMetaData cfm3 = Schema.instance.getCFMetaData(ksName, cfName).copy();
-        assertTrue(cfm3.getTriggers().isEmpty());
+        TableMetadata tm3 = Schema.instance.getTableMetadata(ksName, cfName);
+        assertTrue(tm3.triggers.isEmpty());
     }
 }


[23/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Cell.java b/src/java/org/apache/cassandra/db/rows/Cell.java
index 19d1f30..4636022 100644
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@ -25,6 +25,7 @@ import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
@@ -54,7 +55,7 @@ public abstract class Cell extends ColumnData
 
     public static final Serializer serializer = new BufferCell.Serializer();
 
-    protected Cell(ColumnDefinition column)
+    protected Cell(ColumnMetadata column)
     {
         super(column);
     }
@@ -130,7 +131,7 @@ public abstract class Cell extends ColumnData
      */
     public abstract CellPath path();
 
-    public abstract Cell withUpdatedColumn(ColumnDefinition newColumn);
+    public abstract Cell withUpdatedColumn(ColumnMetadata newColumn);
 
     public abstract Cell withUpdatedValue(ByteBuffer newValue);
 
@@ -171,7 +172,7 @@ public abstract class Cell extends ColumnData
         private final static int USE_ROW_TIMESTAMP_MASK      = 0x08; // Wether the cell has the same timestamp than the row this is a cell of.
         private final static int USE_ROW_TTL_MASK            = 0x10; // Wether the cell has the same ttl than the row this is a cell of.
 
-        public void serialize(Cell cell, ColumnDefinition column, DataOutputPlus out, LivenessInfo rowLiveness, SerializationHeader header) throws IOException
+        public void serialize(Cell cell, ColumnMetadata column, DataOutputPlus out, LivenessInfo rowLiveness, SerializationHeader header) throws IOException
         {
             assert cell != null;
             boolean hasValue = cell.value().hasRemaining();
@@ -210,7 +211,7 @@ public abstract class Cell extends ColumnData
                 header.getType(column).writeValue(cell.value(), out);
         }
 
-        public Cell deserialize(DataInputPlus in, LivenessInfo rowLiveness, ColumnDefinition column, SerializationHeader header, SerializationHelper helper) throws IOException
+        public Cell deserialize(DataInputPlus in, LivenessInfo rowLiveness, ColumnMetadata column, SerializationHeader header, SerializationHelper helper) throws IOException
         {
             int flags = in.readUnsignedByte();
             boolean hasValue = (flags & HAS_EMPTY_VALUE_MASK) == 0;
@@ -251,7 +252,7 @@ public abstract class Cell extends ColumnData
             return new BufferCell(column, timestamp, ttl, localDeletionTime, value, path);
         }
 
-        public long serializedSize(Cell cell, ColumnDefinition column, LivenessInfo rowLiveness, SerializationHeader header)
+        public long serializedSize(Cell cell, ColumnMetadata column, LivenessInfo rowLiveness, SerializationHeader header)
         {
             long size = 1; // flags
             boolean hasValue = cell.value().hasRemaining();
@@ -278,7 +279,7 @@ public abstract class Cell extends ColumnData
         }
 
         // Returns if the skipped cell was an actual cell (i.e. it had its presence flag).
-        public boolean skip(DataInputPlus in, ColumnDefinition column, SerializationHeader header) throws IOException
+        public boolean skip(DataInputPlus in, ColumnMetadata column, SerializationHeader header) throws IOException
         {
             int flags = in.readUnsignedByte();
             boolean hasValue = (flags & HAS_EMPTY_VALUE_MASK) == 0;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/Cells.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Cells.java b/src/java/org/apache/cassandra/db/rows/Cells.java
index 38bde16..7f2772c 100644
--- a/src/java/org/apache/cassandra/db/rows/Cells.java
+++ b/src/java/org/apache/cassandra/db/rows/Cells.java
@@ -21,7 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.Comparator;
 import java.util.Iterator;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.Conflicts;
 import org.apache.cassandra.db.DeletionTime;
 import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
@@ -197,7 +197,7 @@ public abstract class Cells
      * of cells from {@code existing} and {@code update} having the same cell path is empty, this
      * returns {@code Long.MAX_VALUE}.
      */
-    public static long reconcileComplex(ColumnDefinition column,
+    public static long reconcileComplex(ColumnMetadata column,
                                         Iterator<Cell> existing,
                                         Iterator<Cell> update,
                                         DeletionTime deletion,
@@ -282,7 +282,7 @@ public abstract class Cells
      * because deleted cells always have precedence on timestamp equality and deciding if a
      * cell is a live or not depends on the current time due to expiring cells).
      */
-    public static void addNonShadowedComplex(ColumnDefinition column,
+    public static void addNonShadowedComplex(ColumnMetadata column,
                                              Iterator<Cell> existing,
                                              Iterator<Cell> update,
                                              DeletionTime deletion,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/ColumnData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ColumnData.java b/src/java/org/apache/cassandra/db/rows/ColumnData.java
index 933da6a..166a805 100644
--- a/src/java/org/apache/cassandra/db/rows/ColumnData.java
+++ b/src/java/org/apache/cassandra/db/rows/ColumnData.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.db.rows;
 import java.security.MessageDigest;
 import java.util.Comparator;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.DeletionPurger;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.serializers.MarshalException;
@@ -35,8 +35,8 @@ public abstract class ColumnData
 {
     public static final Comparator<ColumnData> comparator = (cd1, cd2) -> cd1.column().compareTo(cd2.column());
 
-    protected final ColumnDefinition column;
-    protected ColumnData(ColumnDefinition column)
+    protected final ColumnMetadata column;
+    protected ColumnData(ColumnMetadata column)
     {
         this.column = column;
     }
@@ -46,7 +46,7 @@ public abstract class ColumnData
      *
      * @return the column this is a data for.
      */
-    public final ColumnDefinition column() { return column; }
+    public final ColumnMetadata column() { return column; }
 
     /**
      * The size of the data hold by this {@code ColumnData}.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
index b5195c5..8f9b63f 100644
--- a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
+++ b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
@@ -24,14 +24,14 @@ import java.util.Objects;
 
 import com.google.common.base.Function;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.DeletionPurger;
 import org.apache.cassandra.db.DeletionTime;
 import org.apache.cassandra.db.LivenessInfo;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.ByteType;
 import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.DroppedColumn;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.btree.BTree;
 
@@ -43,7 +43,7 @@ public class ComplexColumnData extends ColumnData implements Iterable<Cell>
 {
     static final Cell[] NO_CELLS = new Cell[0];
 
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new ComplexColumnData(ColumnDefinition.regularDef("", "", "", SetType.getInstance(ByteType.instance, true)), NO_CELLS, new DeletionTime(0, 0)));
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new ComplexColumnData(ColumnMetadata.regularColumn("", "", "", SetType.getInstance(ByteType.instance, true)), NO_CELLS, new DeletionTime(0, 0)));
 
     // The cells for 'column' sorted by cell path.
     private final Object[] cells;
@@ -51,7 +51,7 @@ public class ComplexColumnData extends ColumnData implements Iterable<Cell>
     private final DeletionTime complexDeletion;
 
     // Only ArrayBackedRow should call this.
-    ComplexColumnData(ColumnDefinition column, Object[] cells, DeletionTime complexDeletion)
+    ComplexColumnData(ColumnMetadata column, Object[] cells, DeletionTime complexDeletion)
     {
         super(column);
         assert column.isComplex();
@@ -143,7 +143,7 @@ public class ComplexColumnData extends ColumnData implements Iterable<Cell>
         return transformAndFilter(complexDeletion, Cell::markCounterLocalToBeCleared);
     }
 
-    public ComplexColumnData filter(ColumnFilter filter, DeletionTime activeDeletion, CFMetaData.DroppedColumn dropped, LivenessInfo rowLiveness)
+    public ComplexColumnData filter(ColumnFilter filter, DeletionTime activeDeletion, DroppedColumn dropped, LivenessInfo rowLiveness)
     {
         ColumnFilter.Tester cellTester = filter.newTester(column);
         if (cellTester == null && activeDeletion.isLive() && dropped == null)
@@ -235,10 +235,10 @@ public class ComplexColumnData extends ColumnData implements Iterable<Cell>
     public static class Builder
     {
         private DeletionTime complexDeletion;
-        private ColumnDefinition column;
+        private ColumnMetadata column;
         private BTree.Builder<Cell> builder;
 
-        public void newColumn(ColumnDefinition column)
+        public void newColumn(ColumnMetadata column)
         {
             this.column = column;
             this.complexDeletion = DeletionTime.LIVE; // default if writeComplexDeletion is not called

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java
index 504d60e..d8bd36f 100644
--- a/src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java
@@ -17,9 +17,9 @@
  */
 package org.apache.cassandra.db.rows;
 
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.AbstractIterator;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 
 /**
@@ -53,13 +53,13 @@ public abstract class LazilyInitializedUnfilteredRowIterator extends AbstractIte
         return iterator != null;
     }
 
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
         maybeInit();
         return iterator.metadata();
     }
 
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
         maybeInit();
         return iterator.columns();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/NativeCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/NativeCell.java b/src/java/org/apache/cassandra/db/rows/NativeCell.java
index 5930332..10ccf88 100644
--- a/src/java/org/apache/cassandra/db/rows/NativeCell.java
+++ b/src/java/org/apache/cassandra/db/rows/NativeCell.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.db.rows;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.memory.MemoryUtil;
@@ -61,7 +61,7 @@ public class NativeCell extends AbstractCell
 
     public NativeCell(NativeAllocator allocator,
                       OpOrder.Group writeOp,
-                      ColumnDefinition column,
+                      ColumnMetadata column,
                       long timestamp,
                       int ttl,
                       int localDeletionTime,
@@ -143,7 +143,7 @@ public class NativeCell extends AbstractCell
         throw new UnsupportedOperationException();
     }
 
-    public Cell withUpdatedColumn(ColumnDefinition column)
+    public Cell withUpdatedColumn(ColumnMetadata column)
     {
         return new BufferCell(column, timestamp(), ttl(), localDeletionTime(), value(), path());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundMarker.java b/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundMarker.java
index 45e594b..71b2315 100644
--- a/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundMarker.java
+++ b/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundMarker.java
@@ -21,7 +21,7 @@ import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.util.Objects;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
@@ -124,7 +124,7 @@ public class RangeTombstoneBoundMarker extends AbstractRangeTombstoneMarker<Clus
         deletion.digest(digest);
     }
 
-    public String toString(CFMetaData metadata)
+    public String toString(TableMetadata metadata)
     {
         return "Marker " + bound.toString(metadata) + '@' + deletion.markedForDeleteAt();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundaryMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundaryMarker.java b/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundaryMarker.java
index fd41bea..b6ecd9d 100644
--- a/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundaryMarker.java
+++ b/src/java/org/apache/cassandra/db/rows/RangeTombstoneBoundaryMarker.java
@@ -21,7 +21,7 @@ import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.util.Objects;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
@@ -146,7 +146,7 @@ public class RangeTombstoneBoundaryMarker extends AbstractRangeTombstoneMarker<C
         startDeletion.digest(digest);
     }
 
-    public String toString(CFMetaData metadata)
+    public String toString(TableMetadata metadata)
     {
         return String.format("Marker %s@%d-%d", bound.toString(metadata), endDeletion.markedForDeleteAt(), startDeletion.markedForDeleteAt());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/Row.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Row.java b/src/java/org/apache/cassandra/db/rows/Row.java
index c19d55e..9e7e50d 100644
--- a/src/java/org/apache/cassandra/db/rows/Row.java
+++ b/src/java/org/apache/cassandra/db/rows/Row.java
@@ -23,10 +23,10 @@ import java.util.function.Consumer;
 
 import com.google.common.base.Predicate;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MergeIterator;
@@ -60,7 +60,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      * An in-natural-order collection of the columns for which data (incl. simple tombstones)
      * is present in this row.
      */
-    public Collection<ColumnDefinition> columns();
+    public Collection<ColumnMetadata> columns();
 
     /**
      * The row deletion.
@@ -115,7 +115,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      * @param c the simple column for which to fetch the cell.
      * @return the corresponding cell or {@code null} if the row has no such cell.
      */
-    public Cell getCell(ColumnDefinition c);
+    public Cell getCell(ColumnMetadata c);
 
     /**
      * Return a cell for a given complex column and cell path.
@@ -124,7 +124,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      * @param path the cell path for which to fetch the cell.
      * @return the corresponding cell or {@code null} if the row has no such cell.
      */
-    public Cell getCell(ColumnDefinition c, CellPath path);
+    public Cell getCell(ColumnMetadata c, CellPath path);
 
     /**
      * The data for a complex column.
@@ -134,7 +134,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      * @param c the complex column for which to return the complex data.
      * @return the data for {@code c} or {@code null} is the row has no data for this column.
      */
-    public ComplexColumnData getComplexColumnData(ColumnDefinition c);
+    public ComplexColumnData getComplexColumnData(ColumnMetadata c);
 
     /**
      * An iterable over the cells of this row.
@@ -156,7 +156,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      * @param reversed if cells should returned in reverse order.
      * @return an iterable over the cells of this row in "legacy order".
      */
-    public Iterable<Cell> cellsInLegacyOrder(CFMetaData metadata, boolean reversed);
+    public Iterable<Cell> cellsInLegacyOrder(TableMetadata metadata, boolean reversed);
 
     /**
      * Whether the row stores any (non-live) complex deletion for any complex column.
@@ -180,14 +180,14 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      *
      * @return a search iterator for the cells of this row.
      */
-    public SearchIterator<ColumnDefinition, ColumnData> searchIterator();
+    public SearchIterator<ColumnMetadata, ColumnData> searchIterator();
 
     /**
      * Returns a copy of this row that:
      *   1) only includes the data for the column included by {@code filter}.
      *   2) doesn't include any data that belongs to a dropped column (recorded in {@code metadata}).
      */
-    public Row filter(ColumnFilter filter, CFMetaData metadata);
+    public Row filter(ColumnFilter filter, TableMetadata metadata);
 
     /**
      * Returns a copy of this row that:
@@ -196,7 +196,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      *   3) doesn't include any data that is shadowed/deleted by {@code activeDeletion}.
      *   4) uses {@code activeDeletion} as row deletion iff {@code setActiveDeletionToRow} and {@code activeDeletion} supersedes the row deletion.
      */
-    public Row filter(ColumnFilter filter, DeletionTime activeDeletion, boolean setActiveDeletionToRow, CFMetaData metadata);
+    public Row filter(ColumnFilter filter, DeletionTime activeDeletion, boolean setActiveDeletionToRow, TableMetadata metadata);
 
     /**
      * Returns a copy of this row without any deletion info that should be purged according to {@code purger}.
@@ -249,7 +249,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
 
     public long unsharedHeapSizeExcludingData();
 
-    public String toString(CFMetaData metadata, boolean fullDetails);
+    public String toString(TableMetadata metadata, boolean fullDetails);
 
     /**
      * Apply a function to every column in a row
@@ -475,7 +475,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
          * @param column the column for which to add the {@code complexDeletion}.
          * @param complexDeletion the complex deletion time to add.
          */
-        public void addComplexDeletion(ColumnDefinition column, DeletionTime complexDeletion);
+        public void addComplexDeletion(ColumnMetadata column, DeletionTime complexDeletion);
 
         /**
          * Builds and return built row.
@@ -690,7 +690,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
         {
             private final int nowInSec;
 
-            private ColumnDefinition column;
+            private ColumnMetadata column;
             private final List<ColumnData> versions;
 
             private DeletionTime activeDeletion;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java b/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
index 5d7d8fe..e08b370 100644
--- a/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/RowAndDeletionMergeIterator.java
@@ -20,7 +20,8 @@ package org.apache.cassandra.db.rows;
 import java.util.Comparator;
 import java.util.Iterator;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 
@@ -51,7 +52,7 @@ public class RowAndDeletionMergeIterator extends AbstractUnfilteredRowIterator
     // The currently open tombstone. Note that unless this is null, there is no point in checking nextRange.
     private RangeTombstone openRange;
 
-    public RowAndDeletionMergeIterator(CFMetaData metadata,
+    public RowAndDeletionMergeIterator(TableMetadata metadata,
                                        DecoratedKey partitionKey,
                                        DeletionTime partitionLevelDeletion,
                                        ColumnFilter selection,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowDiffListener.java b/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
index 0c7e32b..88a9bae 100644
--- a/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
+++ b/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.db.rows;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.*;
 
 /**
@@ -63,7 +63,7 @@ public interface RowDiffListener
      * @param original the complex deletion of input {@code i} for column {@code column}. May be {@code null} if input {@code i}
      * had no complex deletion but the merged row has.
      */
-    public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original);
+    public void onComplexDeletion(int i, Clustering clustering, ColumnMetadata column, DeletionTime merged, DeletionTime original);
 
     /**
      * Called for any cell that is either in the merged row or in input {@code i}.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/RowIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowIterators.java b/src/java/org/apache/cassandra/db/rows/RowIterators.java
index bce6a7d..b155d68 100644
--- a/src/java/org/apache/cassandra/db/rows/RowIterators.java
+++ b/src/java/org/apache/cassandra/db/rows/RowIterators.java
@@ -22,9 +22,9 @@ import java.security.MessageDigest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.transform.Transformation;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -76,12 +76,12 @@ public abstract class RowIterators
      */
     public static RowIterator loggingIterator(RowIterator iterator, final String id)
     {
-        CFMetaData metadata = iterator.metadata();
+        TableMetadata metadata = iterator.metadata();
         logger.info("[{}] Logging iterator on {}.{}, partition key={}, reversed={}",
                     id,
-                    metadata.ksName,
-                    metadata.cfName,
-                    metadata.getKeyValidator().getString(iterator.partitionKey().getKey()),
+                    metadata.keyspace,
+                    metadata.name,
+                    metadata.partitionKeyType.getString(iterator.partitionKey().getKey()),
                     iterator.isReverseOrder());
 
         class Log extends Transformation

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/Rows.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Rows.java b/src/java/org/apache/cassandra/db/rows/Rows.java
index a92bdac..8c7143a 100644
--- a/src/java/org/apache/cassandra/db/rows/Rows.java
+++ b/src/java/org/apache/cassandra/db/rows/Rows.java
@@ -22,11 +22,10 @@ import java.util.*;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.PeekingIterator;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
-import org.apache.cassandra.db.rows.Row.Deletion;
 import org.apache.cassandra.utils.MergeIterator;
 import org.apache.cassandra.utils.WrappedInt;
 
@@ -71,7 +70,7 @@ public abstract class Rows
      * only argument.
      * @return a newly created builder.
      */
-    public static Row.SimpleBuilder simpleBuilder(CFMetaData metadata, Object... clusteringValues)
+    public static Row.SimpleBuilder simpleBuilder(TableMetadata metadata, Object... clusteringValues)
     {
         return new SimpleBuilders.RowBuilder(metadata, clusteringValues);
     }
@@ -173,7 +172,7 @@ public abstract class Rows
                     ColumnData input = inputDatas[i];
                     if (mergedData != null || input != null)
                     {
-                        ColumnDefinition column = (mergedData != null ? mergedData : input).column;
+                        ColumnMetadata column = (mergedData != null ? mergedData : input).column;
                         if (column.isSimple())
                         {
                             diffListener.onCell(i, clustering, (Cell) mergedData, (Cell) input);
@@ -298,7 +297,7 @@ public abstract class Rows
             int comparison = nexta == null ? 1 : nextb == null ? -1 : nexta.column.compareTo(nextb.column);
             ColumnData cura = comparison <= 0 ? nexta : null;
             ColumnData curb = comparison >= 0 ? nextb : null;
-            ColumnDefinition column = (cura != null ? cura : curb).column;
+            ColumnMetadata column = (cura != null ? cura : curb).column;
             if (column.isSimple())
             {
                 timeDelta = Math.min(timeDelta, Cells.reconcile((Cell) cura, (Cell) curb, deletion, builder, nowInSec));
@@ -367,7 +366,7 @@ public abstract class Rows
             if (comparison <= 0)
             {
                 ColumnData cura = nexta;
-                ColumnDefinition column = cura.column;
+                ColumnMetadata column = cura.column;
                 ColumnData curb = comparison == 0 ? nextb : null;
                 if (column.isSimple())
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/SerializationHelper.java b/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
index e40a1e1..c7fb8e4 100644
--- a/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
+++ b/src/java/org/apache/cassandra/db/rows/SerializationHelper.java
@@ -20,11 +20,12 @@ package org.apache.cassandra.db.rows;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.schema.DroppedColumn;
 
 public class SerializationHelper
 {
@@ -50,24 +51,24 @@ public class SerializationHelper
     private final ColumnFilter columnsToFetch;
     private ColumnFilter.Tester tester;
 
-    private final Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns;
-    private CFMetaData.DroppedColumn currentDroppedComplex;
+    private final Map<ByteBuffer, DroppedColumn> droppedColumns;
+    private DroppedColumn currentDroppedComplex;
 
 
-    public SerializationHelper(CFMetaData metadata, int version, Flag flag, ColumnFilter columnsToFetch)
+    public SerializationHelper(TableMetadata metadata, int version, Flag flag, ColumnFilter columnsToFetch)
     {
         this.flag = flag;
         this.version = version;
         this.columnsToFetch = columnsToFetch;
-        this.droppedColumns = metadata.getDroppedColumns();
+        this.droppedColumns = metadata.droppedColumns;
     }
 
-    public SerializationHelper(CFMetaData metadata, int version, Flag flag)
+    public SerializationHelper(TableMetadata metadata, int version, Flag flag)
     {
         this(metadata, version, flag, null);
     }
 
-    public boolean includes(ColumnDefinition column)
+    public boolean includes(ColumnMetadata column)
     {
         return columnsToFetch == null || columnsToFetch.fetches(column);
     }
@@ -83,7 +84,7 @@ public class SerializationHelper
         // actually requested by the user however (canSkipValue), we can skip the full cell if the cell
         // timestamp is lower than the row one, because in that case, the row timestamp is enough proof
         // of the liveness of the row. Otherwise, we'll only be able to skip the values of those cells.
-        ColumnDefinition column = cell.column();
+        ColumnMetadata column = cell.column();
         if (column.isComplex())
         {
             if (!includes(cell.path()))
@@ -102,7 +103,7 @@ public class SerializationHelper
         return path == null || tester == null || tester.fetches(path);
     }
 
-    public boolean canSkipValue(ColumnDefinition column)
+    public boolean canSkipValue(ColumnMetadata column)
     {
         return columnsToFetch != null && !columnsToFetch.fetchedColumnIsQueried(column);
     }
@@ -112,7 +113,7 @@ public class SerializationHelper
         return path != null && tester != null && !tester.fetchedCellIsQueried(path);
     }
 
-    public void startOfComplexColumn(ColumnDefinition column)
+    public void startOfComplexColumn(ColumnMetadata column)
     {
         this.tester = columnsToFetch == null ? null : columnsToFetch.newTester(column);
         this.currentDroppedComplex = droppedColumns.get(column.name.bytes);
@@ -125,7 +126,7 @@ public class SerializationHelper
 
     public boolean isDropped(Cell cell, boolean isComplex)
     {
-        CFMetaData.DroppedColumn dropped = isComplex ? currentDroppedComplex : droppedColumns.get(cell.column().name.bytes);
+        DroppedColumn dropped = isComplex ? currentDroppedComplex : droppedColumns.get(cell.column().name.bytes);
         return dropped != null && cell.timestamp() <= dropped.droppedTime;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/Unfiltered.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Unfiltered.java b/src/java/org/apache/cassandra/db/rows/Unfiltered.java
index 37ad447..39d9e75 100644
--- a/src/java/org/apache/cassandra/db/rows/Unfiltered.java
+++ b/src/java/org/apache/cassandra/db/rows/Unfiltered.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.db.rows;
 
 import java.security.MessageDigest;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.Clusterable;
 
 /**
@@ -53,11 +53,11 @@ public interface Unfiltered extends Clusterable
      * invalid (some value is invalid for its column type, or some field
      * is nonsensical).
      */
-    public void validateData(CFMetaData metadata);
+    public void validateData(TableMetadata metadata);
 
-    public String toString(CFMetaData metadata);
-    public String toString(CFMetaData metadata, boolean fullDetails);
-    public String toString(CFMetaData metadata, boolean includeClusterKeys, boolean fullDetails);
+    public String toString(TableMetadata metadata);
+    public String toString(TableMetadata metadata, boolean fullDetails);
+    public String toString(TableMetadata metadata, boolean includeClusterKeys, boolean fullDetails);
 
     default boolean isRow()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
index 45c026f..7cac5e6 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorSerializer.java
@@ -23,11 +23,11 @@ import java.io.IOError;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
@@ -177,14 +177,14 @@ public class UnfilteredRowIteratorSerializer
         return size;
     }
 
-    public Header deserializeHeader(CFMetaData metadata, ColumnFilter selection, DataInputPlus in, int version, SerializationHelper.Flag flag) throws IOException
+    public Header deserializeHeader(TableMetadata metadata, ColumnFilter selection, DataInputPlus in, int version, SerializationHelper.Flag flag) throws IOException
     {
-        DecoratedKey key = metadata.decorateKey(ByteBufferUtil.readWithVIntLength(in));
+        DecoratedKey key = metadata.partitioner.decorateKey(ByteBufferUtil.readWithVIntLength(in));
         int flags = in.readUnsignedByte();
         boolean isReversed = (flags & IS_REVERSED) != 0;
         if ((flags & IS_EMPTY) != 0)
         {
-            SerializationHeader sh = new SerializationHeader(false, metadata, PartitionColumns.NONE, EncodingStats.NO_STATS);
+            SerializationHeader sh = new SerializationHeader(false, metadata, RegularAndStaticColumns.NONE, EncodingStats.NO_STATS);
             return new Header(sh, key, isReversed, true, null, null, 0);
         }
 
@@ -204,7 +204,7 @@ public class UnfilteredRowIteratorSerializer
         return new Header(header, key, isReversed, false, partitionDeletion, staticRow, rowEstimate);
     }
 
-    public UnfilteredRowIterator deserialize(DataInputPlus in, int version, CFMetaData metadata, SerializationHelper.Flag flag, Header header) throws IOException
+    public UnfilteredRowIterator deserialize(DataInputPlus in, int version, TableMetadata metadata, SerializationHelper.Flag flag, Header header) throws IOException
     {
         if (header.isEmpty)
             return EmptyIterators.unfilteredRow(metadata, header.key, header.isReversed);
@@ -230,7 +230,7 @@ public class UnfilteredRowIteratorSerializer
         };
     }
 
-    public UnfilteredRowIterator deserialize(DataInputPlus in, int version, CFMetaData metadata, ColumnFilter selection, SerializationHelper.Flag flag) throws IOException
+    public UnfilteredRowIterator deserialize(DataInputPlus in, int version, TableMetadata metadata, ColumnFilter selection, SerializationHelper.Flag flag) throws IOException
     {
         return deserialize(in, version, metadata, flag, deserializeHeader(metadata, selection, in, version, flag));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java
index 8dbc606..74ade1e 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIteratorWithLowerBound.java
@@ -25,7 +25,7 @@ import java.nio.ByteBuffer;
 import java.util.Comparator;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
 import org.apache.cassandra.db.filter.ColumnFilter;
@@ -108,8 +108,8 @@ public class UnfilteredRowIteratorWithLowerBound extends LazilyInitializedUnfilt
         if (lowerBound != null && ret != null)
             assert comparator().compare(lowerBound, ret.clustering()) <= 0
                 : String.format("Lower bound [%s ]is bigger than first returned value [%s] for sstable %s",
-                                lowerBound.toString(sstable.metadata),
-                                ret.toString(sstable.metadata),
+                                lowerBound.toString(metadata()),
+                                ret.toString(metadata()),
                                 sstable.getFilename());
 
         return ret;
@@ -117,13 +117,13 @@ public class UnfilteredRowIteratorWithLowerBound extends LazilyInitializedUnfilt
 
     private Comparator<Clusterable> comparator()
     {
-        return filter.isReversed() ? sstable.metadata.comparator.reversed() : sstable.metadata.comparator;
+        return filter.isReversed() ? metadata().comparator.reversed() : metadata().comparator;
     }
 
     @Override
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
-        return sstable.metadata;
+        return sstable.metadata();
     }
 
     @Override
@@ -133,7 +133,7 @@ public class UnfilteredRowIteratorWithLowerBound extends LazilyInitializedUnfilt
     }
 
     @Override
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
         return selectedColumns.fetchedColumns();
     }
@@ -184,10 +184,10 @@ public class UnfilteredRowIteratorWithLowerBound extends LazilyInitializedUnfilt
         {
             IndexInfo column = onHeapRetriever.columnsIndex(filter.isReversed() ? rowIndexEntry.columnsIndexCount() - 1 : 0);
             ClusteringPrefix lowerBoundPrefix = filter.isReversed() ? column.lastName : column.firstName;
-            assert lowerBoundPrefix.getRawValues().length <= sstable.metadata.comparator.size() :
+            assert lowerBoundPrefix.getRawValues().length <= metadata().comparator.size() :
             String.format("Unexpected number of clustering values %d, expected %d or fewer for %s",
                           lowerBoundPrefix.getRawValues().length,
-                          sstable.metadata.comparator.size(),
+                          metadata().comparator.size(),
                           sstable.getFilename());
             return ClusteringBound.inclusiveOpen(filter.isReversed(), lowerBoundPrefix.getRawValues());
         }
@@ -217,10 +217,10 @@ public class UnfilteredRowIteratorWithLowerBound extends LazilyInitializedUnfilt
 
         final StatsMetadata m = sstable.getSSTableMetadata();
         List<ByteBuffer> vals = filter.isReversed() ? m.maxClusteringValues : m.minClusteringValues;
-        assert vals.size() <= sstable.metadata.comparator.size() :
+        assert vals.size() <= metadata().comparator.size() :
         String.format("Unexpected number of clustering values %d, expected %d or fewer for %s",
                       vals.size(),
-                      sstable.metadata.comparator.size(),
+                      metadata().comparator.size(),
                       sstable.getFilename());
         return  ClusteringBound.inclusiveOpen(filter.isReversed(), vals.toArray(new ByteBuffer[vals.size()]));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
index 004783e..b03799a 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
@@ -23,16 +23,15 @@ import java.security.MessageDigest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.transform.FilteredRows;
 import org.apache.cassandra.db.transform.MoreRows;
 import org.apache.cassandra.db.transform.Transformation;
-import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.IMergeIterator;
 import org.apache.cassandra.utils.MergeIterator;
@@ -135,9 +134,9 @@ public abstract class UnfilteredRowIterators
     /**
      * Returns an empty unfiltered iterator for a given partition.
      */
-    public static UnfilteredRowIterator noRowsIterator(final CFMetaData cfm, final DecoratedKey partitionKey, final Row staticRow, final DeletionTime partitionDeletion, final boolean isReverseOrder)
+    public static UnfilteredRowIterator noRowsIterator(final TableMetadata metadata, final DecoratedKey partitionKey, final Row staticRow, final DeletionTime partitionDeletion, final boolean isReverseOrder)
     {
-        return EmptyIterators.unfilteredRow(cfm, partitionKey, isReverseOrder, staticRow, partitionDeletion);
+        return EmptyIterators.unfilteredRow(metadata, partitionKey, isReverseOrder, staticRow, partitionDeletion);
     }
 
     /**
@@ -199,7 +198,7 @@ public abstract class UnfilteredRowIterators
      */
     public static UnfilteredRowIterator concat(final UnfilteredRowIterator iter1, final UnfilteredRowIterator iter2)
     {
-        assert iter1.metadata().cfId.equals(iter2.metadata().cfId)
+        assert iter1.metadata().id.equals(iter2.metadata().id)
             && iter1.partitionKey().equals(iter2.partitionKey())
             && iter1.partitionLevelDeletion().equals(iter2.partitionLevelDeletion())
             && iter1.isReverseOrder() == iter2.isReverseOrder()
@@ -310,12 +309,12 @@ public abstract class UnfilteredRowIterators
      */
     public static UnfilteredRowIterator loggingIterator(UnfilteredRowIterator iterator, final String id, final boolean fullDetails)
     {
-        CFMetaData metadata = iterator.metadata();
+        TableMetadata metadata = iterator.metadata();
         logger.info("[{}] Logging iterator on {}.{}, partition key={}, reversed={}, deletion={}",
                     id,
-                    metadata.ksName,
-                    metadata.cfName,
-                    metadata.getKeyValidator().getString(iterator.partitionKey().getKey()),
+                    metadata.keyspace,
+                    metadata.name,
+                    metadata.partitionKeyType.getString(iterator.partitionKey().getKey()),
                     iterator.isReverseOrder(),
                     iterator.partitionLevelDeletion().markedForDeleteAt());
 
@@ -354,9 +353,9 @@ public abstract class UnfilteredRowIterators
         private final IMergeIterator<Unfiltered, Unfiltered> mergeIterator;
         private final MergeListener listener;
 
-        private UnfilteredRowMergeIterator(CFMetaData metadata,
+        private UnfilteredRowMergeIterator(TableMetadata metadata,
                                            List<UnfilteredRowIterator> iterators,
-                                           PartitionColumns columns,
+                                           RegularAndStaticColumns columns,
                                            DeletionTime partitionDeletion,
                                            int nowInSec,
                                            boolean reversed,
@@ -413,7 +412,7 @@ public abstract class UnfilteredRowIterators
             for (int i = 1; i < iterators.size(); i++)
             {
                 UnfilteredRowIterator iter = iterators.get(i);
-                assert first.metadata().cfId.equals(iter.metadata().cfId);
+                assert first.metadata().id.equals(iter.metadata().id);
                 assert first.partitionKey().equals(iter.partitionKey());
                 assert first.isReverseOrder() == iter.isReverseOrder();
             }
@@ -463,20 +462,20 @@ public abstract class UnfilteredRowIterators
             return merged;
         }
 
-        private static PartitionColumns collectColumns(List<UnfilteredRowIterator> iterators)
+        private static RegularAndStaticColumns collectColumns(List<UnfilteredRowIterator> iterators)
         {
-            PartitionColumns first = iterators.get(0).columns();
+            RegularAndStaticColumns first = iterators.get(0).columns();
             Columns statics = first.statics;
             Columns regulars = first.regulars;
             for (int i = 1; i < iterators.size(); i++)
             {
-                PartitionColumns cols = iterators.get(i).columns();
+                RegularAndStaticColumns cols = iterators.get(i).columns();
                 statics = statics.mergeTo(cols.statics);
                 regulars = regulars.mergeTo(cols.regulars);
             }
             return statics == first.statics && regulars == first.regulars
                  ? first
-                 : new PartitionColumns(statics, regulars);
+                 : new RegularAndStaticColumns(statics, regulars);
         }
 
         private static EncodingStats mergeStats(List<UnfilteredRowIterator> iterators)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
index c79cc8c..ecfd1c0 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import com.google.common.collect.Collections2;
 
 import net.nicoulaj.compilecommand.annotations.Inline;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.Row.Deletion;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -224,7 +224,7 @@ public class UnfilteredSerializer
         if ((flags & HAS_ALL_COLUMNS) == 0)
             Columns.serializer.serializeSubset(Collections2.transform(row, ColumnData::column), headerColumns, out);
 
-        SearchIterator<ColumnDefinition, ColumnDefinition> si = headerColumns.iterator();
+        SearchIterator<ColumnMetadata, ColumnMetadata> si = headerColumns.iterator();
 
         try
         {
@@ -232,9 +232,9 @@ public class UnfilteredSerializer
                 // We can obtain the column for data directly from data.column(). However, if the cell/complex data
                 // originates from a sstable, the column we'll get will have the type used when the sstable was serialized,
                 // and if that type have been recently altered, that may not be the type we want to serialize the column
-                // with. So we use the ColumnDefinition from the "header" which is "current". Also see #11810 for what
+                // with. So we use the ColumnMetadata from the "header" which is "current". Also see #11810 for what
                 // happens if we don't do that.
-                ColumnDefinition column = si.next(cd.column());
+                ColumnMetadata column = si.next(cd.column());
                 assert column != null : cd.column.toString();
 
                 try
@@ -259,7 +259,7 @@ public class UnfilteredSerializer
         }
     }
 
-    private void writeComplexColumn(ComplexColumnData data, ColumnDefinition column, boolean hasComplexDeletion, LivenessInfo rowLiveness, SerializationHeader header, DataOutputPlus out)
+    private void writeComplexColumn(ComplexColumnData data, ColumnMetadata column, boolean hasComplexDeletion, LivenessInfo rowLiveness, SerializationHeader header, DataOutputPlus out)
     throws IOException
     {
         if (hasComplexDeletion)
@@ -347,10 +347,10 @@ public class UnfilteredSerializer
         if (!hasAllColumns)
             size += Columns.serializer.serializedSubsetSize(Collections2.transform(row, ColumnData::column), header.columns(isStatic));
 
-        SearchIterator<ColumnDefinition, ColumnDefinition> si = headerColumns.iterator();
+        SearchIterator<ColumnMetadata, ColumnMetadata> si = headerColumns.iterator();
         for (ColumnData data : row)
         {
-            ColumnDefinition column = si.next(data.column());
+            ColumnMetadata column = si.next(data.column());
             assert column != null;
 
             if (data.column.isSimple())
@@ -362,7 +362,7 @@ public class UnfilteredSerializer
         return size;
     }
 
-    private long sizeOfComplexColumn(ComplexColumnData data, ColumnDefinition column, boolean hasComplexDeletion, LivenessInfo rowLiveness, SerializationHeader header)
+    private long sizeOfComplexColumn(ComplexColumnData data, ColumnMetadata column, boolean hasComplexDeletion, LivenessInfo rowLiveness, SerializationHeader header)
     {
         long size = 0;
 
@@ -602,7 +602,7 @@ public class UnfilteredSerializer
         }
     }
 
-    private void readSimpleColumn(ColumnDefinition column, DataInputPlus in, SerializationHeader header, SerializationHelper helper, Row.Builder builder, LivenessInfo rowLiveness)
+    private void readSimpleColumn(ColumnMetadata column, DataInputPlus in, SerializationHeader header, SerializationHelper helper, Row.Builder builder, LivenessInfo rowLiveness)
     throws IOException
     {
         if (helper.includes(column))
@@ -617,7 +617,7 @@ public class UnfilteredSerializer
         }
     }
 
-    private void readComplexColumn(ColumnDefinition column, DataInputPlus in, SerializationHeader header, SerializationHelper helper, boolean hasComplexDeletion, Row.Builder builder, LivenessInfo rowLiveness)
+    private void readComplexColumn(ColumnMetadata column, DataInputPlus in, SerializationHeader header, SerializationHelper helper, boolean hasComplexDeletion, Row.Builder builder, LivenessInfo rowLiveness)
     throws IOException
     {
         if (helper.includes(column))
@@ -667,7 +667,7 @@ public class UnfilteredSerializer
         in.skipBytesFully(markerSize);
     }
 
-    private void skipComplexColumn(DataInputPlus in, ColumnDefinition column, SerializationHeader header, boolean hasComplexDeletion)
+    private void skipComplexColumn(DataInputPlus in, ColumnMetadata column, SerializationHeader header, boolean hasComplexDeletion)
     throws IOException
     {
         if (hasComplexDeletion)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/WithOnlyQueriedData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/WithOnlyQueriedData.java b/src/java/org/apache/cassandra/db/rows/WithOnlyQueriedData.java
index dcf0891..ecae172 100644
--- a/src/java/org/apache/cassandra/db/rows/WithOnlyQueriedData.java
+++ b/src/java/org/apache/cassandra/db/rows/WithOnlyQueriedData.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.db.rows;
 
-import org.apache.cassandra.db.PartitionColumns;
+import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.transform.Transformation;
 
@@ -36,7 +36,7 @@ public class WithOnlyQueriedData<I extends BaseRowIterator<?>> extends Transform
     }
 
     @Override
-    protected PartitionColumns applyToPartitionColumns(PartitionColumns columns)
+    protected RegularAndStaticColumns applyToPartitionColumns(RegularAndStaticColumns columns)
     {
         return filter.queriedColumns();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/WrappingUnfilteredRowIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/WrappingUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/rows/WrappingUnfilteredRowIterator.java
index 411950e..e38be09 100644
--- a/src/java/org/apache/cassandra/db/rows/WrappingUnfilteredRowIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/WrappingUnfilteredRowIterator.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.db.rows;
 
 import com.google.common.collect.UnmodifiableIterator;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 
 /**
@@ -40,12 +40,12 @@ public abstract class WrappingUnfilteredRowIterator extends UnmodifiableIterator
         this.wrapped = wrapped;
     }
 
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
         return wrapped.metadata();
     }
 
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
         return wrapped.columns();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/transform/BaseRows.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/transform/BaseRows.java b/src/java/org/apache/cassandra/db/transform/BaseRows.java
index 9bf878f..0f03bbb 100644
--- a/src/java/org/apache/cassandra/db/transform/BaseRows.java
+++ b/src/java/org/apache/cassandra/db/transform/BaseRows.java
@@ -20,9 +20,9 @@
  */
 package org.apache.cassandra.db.transform;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.PartitionColumns;
+import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.rows.*;
 
 import static org.apache.cassandra.utils.Throwables.merge;
@@ -50,7 +50,7 @@ implements BaseRowIterator<R>
         partitionKey = copyFrom.partitionKey();
     }
 
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
         return input.metadata();
     }
@@ -60,7 +60,7 @@ implements BaseRowIterator<R>
         return input.isReverseOrder();
     }
 
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
         return input.columns();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/transform/Transformation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/transform/Transformation.java b/src/java/org/apache/cassandra/db/transform/Transformation.java
index 33c1fe7..811932c 100644
--- a/src/java/org/apache/cassandra/db/transform/Transformation.java
+++ b/src/java/org/apache/cassandra/db/transform/Transformation.java
@@ -22,7 +22,7 @@ package org.apache.cassandra.db.transform;
 
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.DeletionTime;
-import org.apache.cassandra.db.PartitionColumns;
+import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.partitions.PartitionIterator;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.db.rows.*;
@@ -116,7 +116,7 @@ public abstract class Transformation<I extends BaseRowIterator<?>>
      * NOTE: same remark than for applyToDeletion: it is only applied to the first iterator in a sequence of iterators
      * filled by MoreContents.
      */
-    protected PartitionColumns applyToPartitionColumns(PartitionColumns columns)
+    protected RegularAndStaticColumns applyToPartitionColumns(RegularAndStaticColumns columns)
     {
         return columns;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/transform/UnfilteredPartitions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/transform/UnfilteredPartitions.java b/src/java/org/apache/cassandra/db/transform/UnfilteredPartitions.java
index de0a51b..f0f295f 100644
--- a/src/java/org/apache/cassandra/db/transform/UnfilteredPartitions.java
+++ b/src/java/org/apache/cassandra/db/transform/UnfilteredPartitions.java
@@ -20,7 +20,7 @@
  */
 package org.apache.cassandra.db.transform;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 
@@ -32,7 +32,7 @@ final class UnfilteredPartitions extends BasePartitions<UnfilteredRowIterator, U
         super(input);
     }
 
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
         return input.metadata();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/transform/UnfilteredRows.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/transform/UnfilteredRows.java b/src/java/org/apache/cassandra/db/transform/UnfilteredRows.java
index ba86066..a3dc96e 100644
--- a/src/java/org/apache/cassandra/db/transform/UnfilteredRows.java
+++ b/src/java/org/apache/cassandra/db/transform/UnfilteredRows.java
@@ -21,20 +21,20 @@
 package org.apache.cassandra.db.transform;
 
 import org.apache.cassandra.db.DeletionTime;
-import org.apache.cassandra.db.PartitionColumns;
+import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.rows.EncodingStats;
 import org.apache.cassandra.db.rows.Unfiltered;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 
 final class UnfilteredRows extends BaseRows<Unfiltered, UnfilteredRowIterator> implements UnfilteredRowIterator
 {
-    private PartitionColumns partitionColumns;
+    private RegularAndStaticColumns regularAndStaticColumns;
     private DeletionTime partitionLevelDeletion;
 
     public UnfilteredRows(UnfilteredRowIterator input)
     {
         super(input);
-        partitionColumns = input.columns();
+        regularAndStaticColumns = input.columns();
         partitionLevelDeletion = input.partitionLevelDeletion();
     }
 
@@ -42,14 +42,14 @@ final class UnfilteredRows extends BaseRows<Unfiltered, UnfilteredRowIterator> i
     void add(Transformation add)
     {
         super.add(add);
-        partitionColumns = add.applyToPartitionColumns(partitionColumns);
+        regularAndStaticColumns = add.applyToPartitionColumns(regularAndStaticColumns);
         partitionLevelDeletion = add.applyToDeletion(partitionLevelDeletion);
     }
 
     @Override
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
-        return partitionColumns;
+        return regularAndStaticColumns;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/view/TableViews.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/TableViews.java b/src/java/org/apache/cassandra/db/view/TableViews.java
index 7ee5aab..0579429 100644
--- a/src/java/org/apache/cassandra/db/view/TableViews.java
+++ b/src/java/org/apache/cassandra/db/view/TableViews.java
@@ -26,13 +26,16 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.PeekingIterator;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLogPosition;
 import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.btree.BTreeSet;
@@ -43,16 +46,16 @@ import org.apache.cassandra.utils.btree.BTreeSet;
  */
 public class TableViews extends AbstractCollection<View>
 {
-    private final CFMetaData baseTableMetadata;
+    private final TableMetadataRef baseTableMetadata;
 
     // We need this to be thread-safe, but the number of times this is changed (when a view is created in the keyspace)
     // is massively exceeded by the number of times it's read (for every mutation on the keyspace), so a copy-on-write
     // list is the best option.
     private final List<View> views = new CopyOnWriteArrayList();
 
-    public TableViews(CFMetaData baseTableMetadata)
+    public TableViews(TableId id)
     {
-        this.baseTableMetadata = baseTableMetadata;
+        baseTableMetadata = Schema.instance.getTableMetadataRef(id);
     }
 
     public int size()
@@ -79,8 +82,8 @@ public class TableViews extends AbstractCollection<View>
 
     public Iterable<ColumnFamilyStore> allViewsCfs()
     {
-        Keyspace keyspace = Keyspace.open(baseTableMetadata.ksName);
-        return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().viewName));
+        Keyspace keyspace = Keyspace.open(baseTableMetadata.keyspace);
+        return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name));
     }
 
     public void forceBlockingFlush()
@@ -119,7 +122,7 @@ public class TableViews extends AbstractCollection<View>
      */
     public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog, AtomicLong baseComplete)
     {
-        assert update.metadata().cfId.equals(baseTableMetadata.cfId);
+        assert update.metadata().id.equals(baseTableMetadata.id);
 
         Collection<View> views = updatedViews(update);
         if (views.isEmpty())
@@ -168,7 +171,7 @@ public class TableViews extends AbstractCollection<View>
                                                               int nowInSec,
                                                               boolean separateUpdates)
     {
-        assert updates.metadata().cfId.equals(baseTableMetadata.cfId);
+        assert updates.metadata().id.equals(baseTableMetadata.id);
 
         List<ViewUpdateGenerator> generators = new ArrayList<>(views.size());
         for (View view : views)
@@ -191,7 +194,7 @@ public class TableViews extends AbstractCollection<View>
 
             Row existingRow;
             Row updateRow;
-            int cmp = baseTableMetadata.comparator.compare(update, existing);
+            int cmp = baseTableMetadata.get().comparator.compare(update, existing);
             if (cmp < 0)
             {
                 // We have an update where there was nothing before
@@ -261,7 +264,7 @@ public class TableViews extends AbstractCollection<View>
 
         if (separateUpdates)
         {
-            final Collection<Mutation> firstBuild = buildMutations(baseTableMetadata, generators);
+            final Collection<Mutation> firstBuild = buildMutations(baseTableMetadata.get(), generators);
 
             return new Iterator<Collection<Mutation>>()
             {
@@ -288,7 +291,7 @@ public class TableViews extends AbstractCollection<View>
                         // If the updates have been filtered, then we won't have any mutations; we need to make sure that we
                         // only return if the mutations are empty. Otherwise, we continue to search for an update which is
                         // not filtered
-                        Collection<Mutation> mutations = buildMutations(baseTableMetadata, generators);
+                        Collection<Mutation> mutations = buildMutations(baseTableMetadata.get(), generators);
                         if (!mutations.isEmpty())
                             return mutations;
                     }
@@ -325,7 +328,7 @@ public class TableViews extends AbstractCollection<View>
                 addToViewUpdateGenerators(emptyRow(updateRow.clustering(), DeletionTime.LIVE), updateRow, generators, nowInSec);
             }
 
-            return Iterators.singletonIterator(buildMutations(baseTableMetadata, generators));
+            return Iterators.singletonIterator(buildMutations(baseTableMetadata.get(), generators));
         }
     }
 
@@ -363,7 +366,7 @@ public class TableViews extends AbstractCollection<View>
     {
         Slices.Builder sliceBuilder = null;
         DeletionInfo deletionInfo = updates.deletionInfo();
-        CFMetaData metadata = updates.metadata();
+        TableMetadata metadata = updates.metadata();
         DecoratedKey key = updates.partitionKey();
         // TODO: This is subtle: we need to gather all the slices that we have to fetch between partition del, range tombstones and rows.
         if (!deletionInfo.isLive())
@@ -484,7 +487,7 @@ public class TableViews extends AbstractCollection<View>
      * @param generators the generators from which to extract the view mutations from.
      * @return the mutations created by all the generators in {@code generators}.
      */
-    private Collection<Mutation> buildMutations(CFMetaData baseTableMetadata, List<ViewUpdateGenerator> generators)
+    private Collection<Mutation> buildMutations(TableMetadata baseTableMetadata, List<ViewUpdateGenerator> generators)
     {
         // One view is probably common enough and we can optimize a bit easily
         if (generators.size() == 1)
@@ -508,7 +511,7 @@ public class TableViews extends AbstractCollection<View>
                 Mutation mutation = mutations.get(key);
                 if (mutation == null)
                 {
-                    mutation = new Mutation(baseTableMetadata.ksName, key);
+                    mutation = new Mutation(baseTableMetadata.keyspace, key);
                     mutations.put(key, mutation);
                 }
                 mutation.add(update);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/View.java b/src/java/org/apache/cassandra/db/view/View.java
index 0b8de9e..1cde464 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -28,10 +28,13 @@ import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.schema.ViewMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.utils.FBUtilities;
 import org.slf4j.Logger;
@@ -47,11 +50,11 @@ public class View
     private static final Logger logger = LoggerFactory.getLogger(View.class);
 
     public final String name;
-    private volatile ViewDefinition definition;
+    private volatile ViewMetadata definition;
 
     private final ColumnFamilyStore baseCfs;
 
-    public volatile List<ColumnDefinition> baseNonPKColumnsInViewPK;
+    public volatile List<ColumnMetadata> baseNonPKColumnsInViewPK;
 
     private final boolean includeAllColumns;
     private ViewBuilder builder;
@@ -63,33 +66,33 @@ public class View
     private SelectStatement select;
     private ReadQuery query;
 
-    public View(ViewDefinition definition,
+    public View(ViewMetadata definition,
                 ColumnFamilyStore baseCfs)
     {
         this.baseCfs = baseCfs;
-        this.name = definition.viewName;
+        this.name = definition.name;
         this.includeAllColumns = definition.includeAllColumns;
         this.rawSelect = definition.select;
 
         updateDefinition(definition);
     }
 
-    public ViewDefinition getDefinition()
+    public ViewMetadata getDefinition()
     {
         return definition;
     }
 
     /**
-     * This updates the columns stored which are dependent on the base CFMetaData.
+     * This updates the columns stored which are dependent on the base TableMetadata.
      */
-    public void updateDefinition(ViewDefinition definition)
+    public void updateDefinition(ViewMetadata definition)
     {
         this.definition = definition;
 
-        List<ColumnDefinition> nonPKDefPartOfViewPK = new ArrayList<>();
-        for (ColumnDefinition baseColumn : baseCfs.metadata.allColumns())
+        List<ColumnMetadata> nonPKDefPartOfViewPK = new ArrayList<>();
+        for (ColumnMetadata baseColumn : baseCfs.metadata().columns())
         {
-            ColumnDefinition viewColumn = getViewColumn(baseColumn);
+            ColumnMetadata viewColumn = getViewColumn(baseColumn);
             if (viewColumn != null && !baseColumn.isPrimaryKeyColumn() && viewColumn.isPrimaryKeyColumn())
                 nonPKDefPartOfViewPK.add(baseColumn);
         }
@@ -100,18 +103,18 @@ public class View
      * The view column corresponding to the provided base column. This <b>can</b>
      * return {@code null} if the column is denormalized in the view.
      */
-    public ColumnDefinition getViewColumn(ColumnDefinition baseColumn)
+    public ColumnMetadata getViewColumn(ColumnMetadata baseColumn)
     {
-        return definition.metadata.getColumnDefinition(baseColumn.name);
+        return definition.metadata.getColumn(baseColumn.name);
     }
 
     /**
      * The base column corresponding to the provided view column. This should
      * never return {@code null} since a view can't have its "own" columns.
      */
-    public ColumnDefinition getBaseColumn(ColumnDefinition viewColumn)
+    public ColumnMetadata getBaseColumn(ColumnMetadata viewColumn)
     {
-        ColumnDefinition baseColumn = baseCfs.metadata.getColumnDefinition(viewColumn.name);
+        ColumnMetadata baseColumn = baseCfs.metadata().getColumn(viewColumn.name);
         assert baseColumn != null;
         return baseColumn;
     }
@@ -148,7 +151,7 @@ public class View
 
         for (ColumnData data : update)
         {
-            if (definition.metadata.getColumnDefinition(data.column().name) != null)
+            if (definition.metadata.getColumn(data.column().name) != null)
                 return true;
         }
         return false;
@@ -169,7 +172,7 @@ public class View
     public boolean matchesViewFilter(DecoratedKey partitionKey, Row baseRow, int nowInSec)
     {
         return getReadQuery().selectsClustering(partitionKey, baseRow.clustering())
-            && getSelectStatement().rowFilterForInternalCalls().isSatisfiedBy(baseCfs.metadata, partitionKey, baseRow, nowInSec);
+            && getSelectStatement().rowFilterForInternalCalls().isSatisfiedBy(baseCfs.metadata(), partitionKey, baseRow, nowInSec);
     }
 
     /**
@@ -214,23 +217,22 @@ public class View
     }
 
     @Nullable
-    public static CFMetaData findBaseTable(String keyspace, String viewName)
+    public static TableMetadataRef findBaseTable(String keyspace, String viewName)
     {
-        ViewDefinition view = Schema.instance.getView(keyspace, viewName);
-        return (view == null) ? null : Schema.instance.getCFMetaData(view.baseTableId);
+        ViewMetadata view = Schema.instance.getView(keyspace, viewName);
+        return (view == null) ? null : Schema.instance.getTableMetadataRef(view.baseTableId);
     }
 
-    public static Iterable<ViewDefinition> findAll(String keyspace, String baseTable)
+    public static Iterable<ViewMetadata> findAll(String keyspace, String baseTable)
     {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
-        final UUID baseId = Schema.instance.getId(keyspace, baseTable);
-        return Iterables.filter(ksm.views, view -> view.baseTableId.equals(baseId));
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
+        return Iterables.filter(ksm.views, view -> view.baseTableName.equals(baseTable));
     }
 
     /**
      * Builds the string text for a materialized view's SELECT statement.
      */
-    public static String buildSelectStatement(String cfName, Collection<ColumnDefinition> includedColumns, String whereClause)
+    public static String buildSelectStatement(String cfName, Collection<ColumnMetadata> includedColumns, String whereClause)
     {
          StringBuilder rawSelect = new StringBuilder("SELECT ");
         if (includedColumns == null || includedColumns.isEmpty())
@@ -251,7 +253,7 @@ public class View
             if (rel.isMultiColumn())
             {
                 sb.append(((MultiColumnRelation) rel).getEntities().stream()
-                        .map(ColumnDefinition.Raw::toString)
+                        .map(ColumnMetadata.Raw::toString)
                         .collect(Collectors.joining(", ", "(", ")")));
             }
             else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/view/ViewBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
index 9550e1e..6250ae7 100644
--- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@ -80,13 +80,13 @@ public class ViewBuilder extends CompactionInfo.Holder
 
         // We're rebuilding everything from what's on disk, so we read everything, consider that as new updates
         // and pretend that there is nothing pre-existing.
-        UnfilteredRowIterator empty = UnfilteredRowIterators.noRowsIterator(baseCfs.metadata, key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, false);
+        UnfilteredRowIterator empty = UnfilteredRowIterators.noRowsIterator(baseCfs.metadata(), key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, false);
 
         try (ReadExecutionController orderGroup = command.executionController();
              UnfilteredRowIterator data = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command))
         {
             Iterator<Collection<Mutation>> mutations = baseCfs.keyspace.viewManager
-                                                      .forTable(baseCfs.metadata)
+                                                      .forTable(baseCfs.metadata.id)
                                                       .generateViewUpdates(Collections.singleton(view), data, empty, nowInSec, true);
 
             AtomicLong noBase = new AtomicLong(Long.MAX_VALUE);
@@ -96,9 +96,9 @@ public class ViewBuilder extends CompactionInfo.Holder
 
     public void run()
     {
-        logger.trace("Running view builder for {}.{}", baseCfs.metadata.ksName, view.name);
+        logger.trace("Running view builder for {}.{}", baseCfs.metadata.keyspace, view.name);
         UUID localHostId = SystemKeyspace.getLocalHostId();
-        String ksname = baseCfs.metadata.ksName, viewName = view.name;
+        String ksname = baseCfs.metadata.keyspace, viewName = view.name;
 
         if (SystemKeyspace.isViewBuilt(ksname, viewName))
         {
@@ -107,7 +107,7 @@ public class ViewBuilder extends CompactionInfo.Holder
             return;
         }
 
-        Iterable<Range<Token>> ranges = StorageService.instance.getLocalRanges(baseCfs.metadata.ksName);
+        Iterable<Range<Token>> ranges = StorageService.instance.getLocalRanges(baseCfs.metadata.keyspace);
         final Pair<Integer, Token> buildStatus = SystemKeyspace.getViewBuildStatus(ksname, viewName);
         Token lastToken;
         Function<org.apache.cassandra.db.lifecycle.View, Iterable<SSTableReader>> function;
@@ -222,7 +222,7 @@ public class ViewBuilder extends CompactionInfo.Holder
             if (lastToken == null || range.contains(lastToken))
                 rangesLeft = 0;
         }
-        return new CompactionInfo(baseCfs.metadata, OperationType.VIEW_BUILD, rangesLeft, rangesTotal, "ranges", compactionId);
+        return new CompactionInfo(baseCfs.metadata(), OperationType.VIEW_BUILD, rangesLeft, rangesTotal, "ranges", compactionId);
     }
 
     public void stop()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/view/ViewManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewManager.java b/src/java/org/apache/cassandra/db/view/ViewManager.java
index b30ad2a..84738b1 100644
--- a/src/java/org/apache/cassandra/db/view/ViewManager.java
+++ b/src/java/org/apache/cassandra/db/view/ViewManager.java
@@ -26,9 +26,9 @@ import com.google.common.util.concurrent.Striped;
 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.ViewDefinition;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.ViewMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.repair.SystemDistributedKeyspace;
@@ -58,7 +58,7 @@ public class ViewManager
     private static final boolean enableCoordinatorBatchlog = Boolean.getBoolean("cassandra.mv_enable_coordinator_batchlog");
 
     private final ConcurrentMap<String, View> viewsByName = new ConcurrentHashMap<>();
-    private final ConcurrentMap<UUID, TableViews> viewsByBaseTable = new ConcurrentHashMap<>();
+    private final ConcurrentMap<TableId, TableViews> viewsByBaseTable = new ConcurrentHashMap<>();
     private final Keyspace keyspace;
 
     public ViewManager(Keyspace keyspace)
@@ -75,12 +75,12 @@ public class ViewManager
         {
             for (PartitionUpdate update : mutation.getPartitionUpdates())
             {
-                assert keyspace.getName().equals(update.metadata().ksName);
+                assert keyspace.getName().equals(update.metadata().keyspace);
 
                 if (coordinatorBatchlog && keyspace.getReplicationStrategy().getReplicationFactor() == 1)
                     continue;
 
-                if (!forTable(update.metadata()).updatedViews(update).isEmpty())
+                if (!forTable(update.metadata().id).updatedViews(update).isEmpty())
                     return true;
             }
         }
@@ -93,24 +93,12 @@ public class ViewManager
         return viewsByName.values();
     }
 
-    public void update(String viewName)
-    {
-        View view = viewsByName.get(viewName);
-        assert view != null : "When updating a view, it should already be in the ViewManager";
-        view.build();
-
-        // We provide the new definition from the base metadata
-        Optional<ViewDefinition> viewDefinition = keyspace.getMetadata().views.get(viewName);
-        assert viewDefinition.isPresent() : "When updating a view, it should still be in the Keyspaces views";
-        view.updateDefinition(viewDefinition.get());
-    }
-
     public void reload()
     {
-        Map<String, ViewDefinition> newViewsByName = new HashMap<>();
-        for (ViewDefinition definition : keyspace.getMetadata().views)
+        Map<String, ViewMetadata> newViewsByName = new HashMap<>();
+        for (ViewMetadata definition : keyspace.getMetadata().views)
         {
-            newViewsByName.put(definition.viewName, definition);
+            newViewsByName.put(definition.name, definition);
         }
 
         for (String viewName : viewsByName.keySet())
@@ -119,7 +107,7 @@ public class ViewManager
                 removeView(viewName);
         }
 
-        for (Map.Entry<String, ViewDefinition> entry : newViewsByName.entrySet())
+        for (Map.Entry<String, ViewMetadata> entry : newViewsByName.entrySet())
         {
             if (!viewsByName.containsKey(entry.getKey()))
                 addView(entry.getValue());
@@ -147,11 +135,11 @@ public class ViewManager
         }
     }
 
-    public void addView(ViewDefinition definition)
+    public void addView(ViewMetadata definition)
     {
         View view = new View(definition, keyspace.getColumnFamilyStore(definition.baseTableId));
-        forTable(view.getDefinition().baseTableMetadata()).add(view);
-        viewsByName.put(definition.viewName, view);
+        forTable(view.getDefinition().baseTableId).add(view);
+        viewsByName.put(definition.name, view);
     }
 
     public void removeView(String name)
@@ -161,7 +149,7 @@ public class ViewManager
         if (view == null)
             return;
 
-        forTable(view.getDefinition().baseTableMetadata()).removeByName(name);
+        forTable(view.getDefinition().baseTableId).removeByName(name);
         SystemKeyspace.setViewRemoved(keyspace.getName(), view.name);
         SystemDistributedKeyspace.setViewRemoved(keyspace.getName(), view.name);
     }
@@ -172,14 +160,13 @@ public class ViewManager
             view.build();
     }
 
-    public TableViews forTable(CFMetaData metadata)
+    public TableViews forTable(TableId id)
     {
-        UUID baseId = metadata.cfId;
-        TableViews views = viewsByBaseTable.get(baseId);
+        TableViews views = viewsByBaseTable.get(id);
         if (views == null)
         {
-            views = new TableViews(metadata);
-            TableViews previous = viewsByBaseTable.putIfAbsent(baseId, views);
+            views = new TableViews(id);
+            TableViews previous = viewsByBaseTable.putIfAbsent(id, views);
             if (previous != null)
                 views = previous;
         }


[22/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
index edbda1c..e276f62 100644
--- a/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
+++ b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
@@ -23,8 +23,9 @@ import java.util.*;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.PeekingIterator;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
@@ -45,11 +46,11 @@ public class ViewUpdateGenerator
     private final View view;
     private final int nowInSec;
 
-    private final CFMetaData baseMetadata;
+    private final TableMetadata baseMetadata;
     private final DecoratedKey baseDecoratedKey;
     private final ByteBuffer[] basePartitionKey;
 
-    private final CFMetaData viewMetadata;
+    private final TableMetadata viewMetadata;
 
     private final Map<DecoratedKey, PartitionUpdate> updates = new HashMap<>();
 
@@ -87,9 +88,9 @@ public class ViewUpdateGenerator
 
         this.baseMetadata = view.getDefinition().baseTableMetadata();
         this.baseDecoratedKey = basePartitionKey;
-        this.basePartitionKey = extractKeyComponents(basePartitionKey, baseMetadata.getKeyValidator());
+        this.basePartitionKey = extractKeyComponents(basePartitionKey, baseMetadata.partitionKeyType);
 
-        this.viewMetadata = view.getDefinition().metadata;
+        this.viewMetadata = Schema.instance.getTableMetadata(view.getDefinition().metadata.id);
 
         this.currentViewEntryPartitionKey = new ByteBuffer[viewMetadata.partitionKeyColumns().size()];
         this.currentViewEntryBuilder = BTreeRow.sortedBuilder();
@@ -191,7 +192,7 @@ public class ViewUpdateGenerator
                  : (mergedHasLiveData ? UpdateAction.NEW_ENTRY : UpdateAction.NONE);
         }
 
-        ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+        ColumnMetadata baseColumn = view.baseNonPKColumnsInViewPK.get(0);
         assert !baseColumn.isComplex() : "A complex column couldn't be part of the view PK";
         Cell before = existingBaseRow == null ? null : existingBaseRow.getCell(baseColumn);
         Cell after = mergedBaseRow.getCell(baseColumn);
@@ -237,7 +238,7 @@ public class ViewUpdateGenerator
 
         for (ColumnData data : baseRow)
         {
-            ColumnDefinition viewColumn = view.getViewColumn(data.column());
+            ColumnMetadata viewColumn = view.getViewColumn(data.column());
             // If that base table column is not denormalized in the view, we had nothing to do.
             // Alose, if it's part of the view PK it's already been taken into account in the clustering.
             if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
@@ -293,8 +294,8 @@ public class ViewUpdateGenerator
         PeekingIterator<ColumnData> existingIter = Iterators.peekingIterator(existingBaseRow.iterator());
         for (ColumnData mergedData : mergedBaseRow)
         {
-            ColumnDefinition baseColumn = mergedData.column();
-            ColumnDefinition viewColumn = view.getViewColumn(baseColumn);
+            ColumnMetadata baseColumn = mergedData.column();
+            ColumnMetadata viewColumn = view.getViewColumn(baseColumn);
             // If that base table column is not denormalized in the view, we had nothing to do.
             // Alose, if it's part of the view PK it's already been taken into account in the clustering.
             if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
@@ -397,9 +398,9 @@ public class ViewUpdateGenerator
     private void startNewUpdate(Row baseRow)
     {
         ByteBuffer[] clusteringValues = new ByteBuffer[viewMetadata.clusteringColumns().size()];
-        for (ColumnDefinition viewColumn : viewMetadata.primaryKeyColumns())
+        for (ColumnMetadata viewColumn : viewMetadata.primaryKeyColumns())
         {
-            ColumnDefinition baseColumn = view.getBaseColumn(viewColumn);
+            ColumnMetadata baseColumn = view.getBaseColumn(viewColumn);
             ByteBuffer value = getValueForPK(baseColumn, baseRow);
             if (viewColumn.isPartitionKey())
                 currentViewEntryPartitionKey[viewColumn.position()] = value;
@@ -457,7 +458,7 @@ public class ViewUpdateGenerator
                  : LivenessInfo.withExpirationTime(baseLiveness.timestamp(), ttl, expirationTime);
         }
 
-        ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+        ColumnMetadata baseColumn = view.baseNonPKColumnsInViewPK.get(0);
         Cell cell = baseRow.getCell(baseColumn);
         assert isLive(cell) : "We shouldn't have got there if the base row had no associated entry";
 
@@ -486,7 +487,7 @@ public class ViewUpdateGenerator
         return timestamp;
     }
 
-    private void addColumnData(ColumnDefinition viewColumn, ColumnData baseTableData)
+    private void addColumnData(ColumnMetadata viewColumn, ColumnData baseTableData)
     {
         assert viewColumn.isComplex() == baseTableData.column().isComplex();
         if (!viewColumn.isComplex())
@@ -501,7 +502,7 @@ public class ViewUpdateGenerator
             addCell(viewColumn, cell);
     }
 
-    private void addCell(ColumnDefinition viewColumn, Cell baseTableCell)
+    private void addCell(ColumnMetadata viewColumn, Cell baseTableCell)
     {
         assert !viewColumn.isPrimaryKeyColumn();
         currentViewEntryBuilder.addCell(baseTableCell.withUpdatedColumn(viewColumn));
@@ -525,7 +526,7 @@ public class ViewUpdateGenerator
         {
             // We can't really know which columns of the view will be updated nor how many row will be updated for this key
             // so we rely on hopefully sane defaults.
-            update = new PartitionUpdate(viewMetadata, partitionKey, viewMetadata.partitionColumns(), 4);
+            update = new PartitionUpdate(viewMetadata, partitionKey, viewMetadata.regularAndStaticColumns(), 4);
             updates.put(partitionKey, update);
         }
         update.add(row);
@@ -537,10 +538,10 @@ public class ViewUpdateGenerator
                           ? currentViewEntryPartitionKey[0]
                           : CompositeType.build(currentViewEntryPartitionKey);
 
-        return viewMetadata.decorateKey(rawKey);
+        return viewMetadata.partitioner.decorateKey(rawKey);
     }
 
-    private ByteBuffer getValueForPK(ColumnDefinition column, Row row)
+    private ByteBuffer getValueForPK(ColumnMetadata column, Row row)
     {
         switch (column.kind)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/dht/BootStrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index 15e75fe..9235844 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -27,7 +27,7 @@ import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.dht.tokenallocator.TokenAllocation;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
index 1271a5a..ff7f2f7 100644
--- a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
@@ -17,8 +17,8 @@
  */
 package org.apache.cassandra.dht;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -287,12 +287,12 @@ public class ByteOrderedPartitioner implements IPartitioner
 
         for (String ks : Schema.instance.getKeyspaces())
         {
-            for (CFMetaData cfmd : Schema.instance.getTablesAndViews(ks))
+            for (TableMetadata cfmd : Schema.instance.getTablesAndViews(ks))
             {
                 for (Range<Token> r : sortedRanges)
                 {
                     // Looping over every KS:CF:Range, get the splits size and add it to the count
-                    allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.cfName, r, 1).size());
+                    allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.name, r, 1).size());
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
index 954b0af..16c5db1 100644
--- a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
@@ -23,13 +23,14 @@ import java.nio.charset.CharacterCodingException;
 import java.util.*;
 import java.util.concurrent.ThreadLocalRandom;
 
-import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.CachedHashDecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.VersionedValue;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -226,12 +227,12 @@ public class OrderPreservingPartitioner implements IPartitioner
 
         for (String ks : Schema.instance.getKeyspaces())
         {
-            for (CFMetaData cfmd : Schema.instance.getTablesAndViews(ks))
+            for (TableMetadata cfmd : Schema.instance.getTablesAndViews(ks))
             {
                 for (Range<Token> r : sortedRanges)
                 {
                     // Looping over every KS:CF:Range, get the splits size and add it to the count
-                    allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.cfName, r, cfmd.params.minIndexInterval).size());
+                    allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.name, r, cfmd.params.minIndexInterval).size());
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/exceptions/UnknownIndexException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/UnknownIndexException.java b/src/java/org/apache/cassandra/exceptions/UnknownIndexException.java
new file mode 100644
index 0000000..fdc6840
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/UnknownIndexException.java
@@ -0,0 +1,39 @@
+/*
+ * 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.exceptions;
+
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.cassandra.schema.TableMetadata;
+
+/**
+ * Exception thrown when we read an index id from a serialized ReadCommand and no corresponding IndexMetadata
+ * can be found in the TableMetadata#indexes collection. Note that this is an internal exception and is not meant
+ * to be user facing, the node reading the ReadCommand should proceed as if no index id were present.
+ */
+public final class UnknownIndexException extends IOException
+{
+    public final UUID indexId;
+    public UnknownIndexException(TableMetadata metadata, UUID id)
+    {
+        super(String.format("Unknown index %s for table %s", id.toString(), metadata.toString()));
+        indexId = id;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/exceptions/UnknownTableException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/exceptions/UnknownTableException.java b/src/java/org/apache/cassandra/exceptions/UnknownTableException.java
new file mode 100644
index 0000000..2cd7aab
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/UnknownTableException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.exceptions;
+
+import java.io.IOException;
+
+import org.apache.cassandra.schema.TableId;
+
+public class UnknownTableException extends IOException
+{
+    public final TableId id;
+
+    public UnknownTableException(String msg, TableId id)
+    {
+        super(msg);
+        this.id = id;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
index fd9ed00..0f44e0c 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
@@ -28,8 +28,8 @@ import java.util.concurrent.*;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Config;
+
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Murmur3Partitioner;
@@ -39,6 +39,7 @@ import org.apache.cassandra.hadoop.HadoopCompat;
 import org.apache.cassandra.io.sstable.CQLSSTableWriter;
 import org.apache.cassandra.io.sstable.SSTableLoader;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.streaming.StreamState;
 import org.apache.cassandra.utils.NativeSSTableLoaderClient;
 import org.apache.cassandra.utils.OutputHandler;
@@ -171,7 +172,7 @@ public class CqlBulkRecordWriter extends RecordWriter<Object, List<ByteBuffer>>
         if (loader == null)
         {
             ExternalClient externalClient = new ExternalClient(conf);
-            externalClient.setTableMetadata(CFMetaData.compile(schema, keyspace));
+            externalClient.setTableMetadata(TableMetadataRef.forOfflineTools(CreateTableStatement.parse(schema, keyspace).build()));
 
             loader = new SSTableLoader(outputDir, externalClient, new NullOutputHandler())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
index 45a227b..9259042 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
@@ -29,7 +29,7 @@ import com.datastax.driver.core.Row;
 import com.datastax.driver.core.Session;
 import com.datastax.driver.core.TokenRange;
 
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/hints/Hint.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/Hint.java b/src/java/org/apache/cassandra/hints/Hint.java
index 17fbf5d..4e8f139 100644
--- a/src/java/org/apache/cassandra/hints/Hint.java
+++ b/src/java/org/apache/cassandra/hints/Hint.java
@@ -18,9 +18,7 @@
 package org.apache.cassandra.hints;
 
 import java.io.IOException;
-import java.util.*;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Throwables;
@@ -29,6 +27,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableId;
 
 import static org.apache.cassandra.db.TypeSizes.sizeof;
 import static org.apache.cassandra.db.TypeSizes.sizeofUnsignedVInt;
@@ -89,9 +88,9 @@ public final class Hint
     {
         if (isLive())
         {
-            // filter out partition update for table that have been truncated since hint's creation
+            // filter out partition update for tables that have been truncated since hint's creation
             Mutation filtered = mutation;
-            for (UUID id : mutation.getColumnFamilyIds())
+            for (TableId id : mutation.getTableIds())
                 if (creationTime <= SystemKeyspace.getTruncatedAt(id))
                     filtered = filtered.without(id);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/hints/HintMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintMessage.java b/src/java/org/apache/cassandra/hints/HintMessage.java
index 723ab6d..683b894 100644
--- a/src/java/org/apache/cassandra/hints/HintMessage.java
+++ b/src/java/org/apache/cassandra/hints/HintMessage.java
@@ -27,13 +27,14 @@ import javax.annotation.Nullable;
 import com.google.common.primitives.Ints;
 
 import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.db.UnknownColumnFamilyException;
+import org.apache.cassandra.exceptions.UnknownTableException;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.io.util.TrackedDataInputPlus;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 /**
@@ -58,7 +59,7 @@ public final class HintMessage
     final Hint hint;
 
     @Nullable // will usually be null, unless a hint deserialization fails due to an unknown table id
-    final UUID unknownTableID;
+    final TableId unknownTableID;
 
     HintMessage(UUID hostId, Hint hint)
     {
@@ -67,7 +68,7 @@ public final class HintMessage
         this.unknownTableID = null;
     }
 
-    HintMessage(UUID hostId, UUID unknownTableID)
+    HintMessage(UUID hostId, TableId unknownTableID)
     {
         this.hostId = hostId;
         this.hint = null;
@@ -122,10 +123,10 @@ public final class HintMessage
             {
                 return new HintMessage(hostId, Hint.serializer.deserialize(countingIn, version));
             }
-            catch (UnknownColumnFamilyException e)
+            catch (UnknownTableException e)
             {
                 in.skipBytes(Ints.checkedCast(hintSize - countingIn.getBytesRead()));
-                return new HintMessage(hostId, e.cfId);
+                return new HintMessage(hostId, e.id);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/hints/HintsReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintsReader.java b/src/java/org/apache/cassandra/hints/HintsReader.java
index e0a73c1..6ede205 100644
--- a/src/java/org/apache/cassandra/hints/HintsReader.java
+++ b/src/java/org/apache/cassandra/hints/HintsReader.java
@@ -30,11 +30,10 @@ import com.google.common.util.concurrent.RateLimiter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.UnknownColumnFamilyException;
+import org.apache.cassandra.exceptions.UnknownTableException;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.utils.AbstractIterator;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.CLibrary;
 
 /**
  * A paged non-compressed hints reader that provides two iterators:
@@ -225,11 +224,11 @@ class HintsReader implements AutoCloseable, Iterable<HintsReader.Page>
                 hint = Hint.serializer.deserialize(input, descriptor.messagingVersion());
                 input.checkLimit(0);
             }
-            catch (UnknownColumnFamilyException e)
+            catch (UnknownTableException e)
             {
                 logger.warn("Failed to read a hint for {} - table with id {} is unknown in file {}",
                             descriptor.hostId,
-                            e.cfId,
+                            e.id,
                             descriptor.fileName());
                 input.skipBytes(Ints.checkedCast(size - input.bytesPastLimit()));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/Index.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/Index.java b/src/java/org/apache/cassandra/index/Index.java
index e254555..1b4573d 100644
--- a/src/java/org/apache/cassandra/index/Index.java
+++ b/src/java/org/apache/cassandra/index/Index.java
@@ -26,7 +26,7 @@ import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.function.BiFunction;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.OperationType;
@@ -123,7 +123,7 @@ import org.apache.cassandra.utils.concurrent.OpOrder;
  *
  * The input is the map of index options supplied in the WITH clause of a CREATE INDEX statement.
  *
- * <pre>{@code public static Map<String, String> validateOptions(Map<String, String> options, CFMetaData cfm);}</pre>
+ * <pre>{@code public static Map<String, String> validateOptions(Map<String, String> options, TableMetadata metadata);}</pre>
  *
  * In this version, the base table's metadata is also supplied as an argument.
  * If both overloaded methods are provided, only the one including the base table's metadata will be invoked.
@@ -303,7 +303,7 @@ public interface Index
      * @return true if the index depends on the supplied column being present; false if the column may be
      *              safely dropped or modified without adversely affecting the index
      */
-    public boolean dependsOn(ColumnDefinition column);
+    public boolean dependsOn(ColumnMetadata column);
 
     /**
      * Called to determine whether this index can provide a searcher to execute a query on the
@@ -313,7 +313,7 @@ public interface Index
      * @param operator the operator of a search query predicate
      * @return true if this index is capable of supporting such expressions, false otherwise
      */
-    public boolean supportsExpression(ColumnDefinition column, Operator operator);
+    public boolean supportsExpression(ColumnMetadata column, Operator operator);
 
     /**
      * If the index supports custom search expressions using the
@@ -385,7 +385,7 @@ public interface Index
      * that type of transaction, ...).
      */
     public Indexer indexerFor(DecoratedKey key,
-                              PartitionColumns columns,
+                              RegularAndStaticColumns columns,
                               int nowInSec,
                               OpOrder.Group opGroup,
                               IndexTransaction.Type transactionType);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
index 08b4f8b..f7b7d13 100644
--- a/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/index/SecondaryIndexManager.java
@@ -40,7 +40,6 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.statements.IndexTarget;
 import org.apache.cassandra.db.*;
@@ -54,12 +53,12 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.index.internal.CassandraIndex;
 import org.apache.cassandra.index.transactions.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.schema.Indexes;
 import org.apache.cassandra.service.pager.SinglePartitionPager;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.ProtocolVersion;
-import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.Refs;
@@ -140,7 +139,7 @@ public class SecondaryIndexManager implements IndexRegistry
     public void reload()
     {
         // figure out what needs to be added and dropped.
-        Indexes tableIndexes = baseCfs.metadata.getIndexes();
+        Indexes tableIndexes = baseCfs.metadata().indexes;
         indexes.keySet()
                .stream()
                .filter(indexName -> !tableIndexes.has(indexName))
@@ -214,7 +213,7 @@ public class SecondaryIndexManager implements IndexRegistry
     }
 
 
-    public Set<IndexMetadata> getDependentIndexes(ColumnDefinition column)
+    public Set<IndexMetadata> getDependentIndexes(ColumnMetadata column)
     {
         if (indexes.isEmpty())
             return Collections.emptySet();
@@ -547,11 +546,11 @@ public class SecondaryIndexManager implements IndexRegistry
     public void indexPartition(DecoratedKey key, Set<Index> indexes, int pageSize)
     {
         if (logger.isTraceEnabled())
-            logger.trace("Indexing partition {}", baseCfs.metadata.getKeyValidator().getString(key.getKey()));
+            logger.trace("Indexing partition {}", baseCfs.metadata().partitionKeyType.getString(key.getKey()));
 
         if (!indexes.isEmpty())
         {
-            SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(baseCfs.metadata,
+            SinglePartitionReadCommand cmd = SinglePartitionReadCommand.fullPartitionRead(baseCfs.metadata(),
                                                                                           FBUtilities.nowInSeconds(),
                                                                                           key);
             int nowInSec = cmd.nowInSec();
@@ -562,7 +561,7 @@ public class SecondaryIndexManager implements IndexRegistry
             {
                 try (ReadExecutionController controller = cmd.executionController();
                      OpOrder.Group writeGroup = Keyspace.writeOrder.start();
-                     UnfilteredPartitionIterator page = pager.fetchPageUnfiltered(baseCfs.metadata, pageSize, controller))
+                     UnfilteredPartitionIterator page = pager.fetchPageUnfiltered(baseCfs.metadata(), pageSize, controller))
                 {
                     if (!page.hasNext())
                         break;
@@ -642,7 +641,7 @@ public class SecondaryIndexManager implements IndexRegistry
         if (meanCellsPerPartition <= 0)
             return DEFAULT_PAGE_SIZE;
 
-        int columnsPerRow = baseCfs.metadata.partitionColumns().regulars.size();
+        int columnsPerRow = baseCfs.metadata().regularColumns().size();
         if (columnsPerRow <= 0)
             return DEFAULT_PAGE_SIZE;
 
@@ -653,8 +652,8 @@ public class SecondaryIndexManager implements IndexRegistry
 
         logger.trace("Calculated page size {} for indexing {}.{} ({}/{}/{}/{})",
                      pageSize,
-                     baseCfs.metadata.ksName,
-                     baseCfs.metadata.cfName,
+                     baseCfs.metadata.keyspace,
+                     baseCfs.metadata.name,
                      meanPartitionSize,
                      meanCellsPerPartition,
                      meanRowsPerPartition,
@@ -855,25 +854,25 @@ public class SecondaryIndexManager implements IndexRegistry
      * Transaction for use when merging rows during compaction
      */
     public CompactionTransaction newCompactionTransaction(DecoratedKey key,
-                                                          PartitionColumns partitionColumns,
+                                                          RegularAndStaticColumns regularAndStaticColumns,
                                                           int versions,
                                                           int nowInSec)
     {
         // the check for whether there are any registered indexes is already done in CompactionIterator
-        return new IndexGCTransaction(key, partitionColumns, versions, nowInSec, listIndexes());
+        return new IndexGCTransaction(key, regularAndStaticColumns, versions, nowInSec, listIndexes());
     }
 
     /**
      * Transaction for use when removing partitions during cleanup
      */
     public CleanupTransaction newCleanupTransaction(DecoratedKey key,
-                                                    PartitionColumns partitionColumns,
+                                                    RegularAndStaticColumns regularAndStaticColumns,
                                                     int nowInSec)
     {
         if (!hasIndexes())
             return CleanupTransaction.NO_OP;
 
-        return new CleanupGCTransaction(key, partitionColumns, nowInSec, listIndexes());
+        return new CleanupGCTransaction(key, regularAndStaticColumns, nowInSec, listIndexes());
     }
 
     /**
@@ -935,7 +934,7 @@ public class SecondaryIndexManager implements IndexRegistry
                 {
                 }
 
-                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
+                public void onComplexDeletion(int i, Clustering clustering, ColumnMetadata column, DeletionTime merged, DeletionTime original)
                 {
                 }
 
@@ -986,7 +985,7 @@ public class SecondaryIndexManager implements IndexRegistry
     private static final class IndexGCTransaction implements CompactionTransaction
     {
         private final DecoratedKey key;
-        private final PartitionColumns columns;
+        private final RegularAndStaticColumns columns;
         private final int versions;
         private final int nowInSec;
         private final Collection<Index> indexes;
@@ -994,7 +993,7 @@ public class SecondaryIndexManager implements IndexRegistry
         private Row[] rows;
 
         private IndexGCTransaction(DecoratedKey key,
-                                   PartitionColumns columns,
+                                   RegularAndStaticColumns columns,
                                    int versions,
                                    int nowInSec,
                                    Collection<Index> indexes)
@@ -1029,7 +1028,7 @@ public class SecondaryIndexManager implements IndexRegistry
                 {
                 }
 
-                public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
+                public void onComplexDeletion(int i, Clustering clustering, ColumnMetadata column, DeletionTime merged, DeletionTime original)
                 {
                 }
 
@@ -1089,7 +1088,7 @@ public class SecondaryIndexManager implements IndexRegistry
     private static final class CleanupGCTransaction implements CleanupTransaction
     {
         private final DecoratedKey key;
-        private final PartitionColumns columns;
+        private final RegularAndStaticColumns columns;
         private final int nowInSec;
         private final Collection<Index> indexes;
 
@@ -1097,7 +1096,7 @@ public class SecondaryIndexManager implements IndexRegistry
         private DeletionTime partitionDelete;
 
         private CleanupGCTransaction(DecoratedKey key,
-                                     PartitionColumns columns,
+                                     RegularAndStaticColumns columns,
                                      int nowInSec,
                                      Collection<Index> indexes)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/TargetParser.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/TargetParser.java b/src/java/org/apache/cassandra/index/TargetParser.java
index 849ad16..ec25259 100644
--- a/src/java/org/apache/cassandra/index/TargetParser.java
+++ b/src/java/org/apache/cassandra/index/TargetParser.java
@@ -22,8 +22,8 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.lang3.StringUtils;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.statements.IndexTarget;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -36,17 +36,17 @@ public class TargetParser
     private static final Pattern TWO_QUOTES = Pattern.compile("\"\"");
     private static final String QUOTE = "\"";
 
-    public static Pair<ColumnDefinition, IndexTarget.Type> parse(CFMetaData cfm, IndexMetadata indexDef)
+    public static Pair<ColumnMetadata, IndexTarget.Type> parse(TableMetadata metadata, IndexMetadata indexDef)
     {
         String target = indexDef.options.get("target");
         assert target != null : String.format("No target definition found for index %s", indexDef.name);
-        Pair<ColumnDefinition, IndexTarget.Type> result = parse(cfm, target);
+        Pair<ColumnMetadata, IndexTarget.Type> result = parse(metadata, target);
         if (result == null)
             throw new ConfigurationException(String.format("Unable to parse targets for index %s (%s)", indexDef.name, target));
         return result;
     }
 
-    public static Pair<ColumnDefinition, IndexTarget.Type> parse(CFMetaData cfm, String target)
+    public static Pair<ColumnMetadata, IndexTarget.Type> parse(TableMetadata metadata, String target)
     {
         // if the regex matches then the target is in the form "keys(foo)", "entries(bar)" etc
         // if not, then it must be a simple column name and implictly its type is VALUES
@@ -77,11 +77,11 @@ public class TargetParser
         }
 
         // if it's not a CQL table, we can't assume that the column name is utf8, so
-        // in that case we have to do a linear scan of the cfm's columns to get the matching one
-        if (cfm.isCQLTable())
-            return Pair.create(cfm.getColumnDefinition(new ColumnIdentifier(columnName, true)), targetType);
+        // in that case we have to do a linear scan of the table's columns to get the matching one
+        if (metadata.isCQLTable())
+            return Pair.create(metadata.getColumn(new ColumnIdentifier(columnName, true)), targetType);
         else
-            for (ColumnDefinition column : cfm.allColumns())
+            for (ColumnMetadata column : metadata.columns())
                 if (column.name.toString().equals(columnName))
                     return Pair.create(column, targetType);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
index 70aaf0d..af75906 100644
--- a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
@@ -33,8 +33,9 @@ import com.google.common.collect.ImmutableSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.cql3.statements.IndexTarget;
 import org.apache.cassandra.db.*;
@@ -76,7 +77,7 @@ public abstract class CassandraIndex implements Index
     public final ColumnFamilyStore baseCfs;
     protected IndexMetadata metadata;
     protected ColumnFamilyStore indexCfs;
-    protected ColumnDefinition indexedColumn;
+    protected ColumnMetadata indexedColumn;
     protected CassandraIndexFunctions functions;
 
     protected CassandraIndex(ColumnFamilyStore baseCfs, IndexMetadata indexDef)
@@ -91,7 +92,7 @@ public abstract class CassandraIndex implements Index
      * @param operator
      * @return
      */
-    protected boolean supportsOperator(ColumnDefinition indexedColumn, Operator operator)
+    protected boolean supportsOperator(ColumnMetadata indexedColumn, Operator operator)
     {
         return operator == Operator.EQ;
     }
@@ -145,14 +146,14 @@ public abstract class CassandraIndex implements Index
                                                   CellPath path,
                                                   ByteBuffer cellValue);
 
-    public ColumnDefinition getIndexedColumn()
+    public ColumnMetadata getIndexedColumn()
     {
         return indexedColumn;
     }
 
     public ClusteringComparator getIndexComparator()
     {
-        return indexCfs.metadata.comparator;
+        return indexCfs.metadata().comparator;
     }
 
     public ColumnFamilyStore getIndexCfs()
@@ -201,7 +202,6 @@ public abstract class CassandraIndex implements Index
     public Callable<?> getMetadataReloadTask(IndexMetadata indexDef)
     {
         return () -> {
-            indexCfs.metadata.reloadIndexMetadataProperties(baseCfs.metadata);
             indexCfs.reload();
             return null;
         };
@@ -223,12 +223,12 @@ public abstract class CassandraIndex implements Index
     private void setMetadata(IndexMetadata indexDef)
     {
         metadata = indexDef;
-        Pair<ColumnDefinition, IndexTarget.Type> target = TargetParser.parse(baseCfs.metadata, indexDef);
+        Pair<ColumnMetadata, IndexTarget.Type> target = TargetParser.parse(baseCfs.metadata(), indexDef);
         functions = getFunctions(indexDef, target);
-        CFMetaData cfm = indexCfsMetadata(baseCfs.metadata, indexDef);
+        TableMetadataRef tableRef = TableMetadataRef.forOfflineTools(indexCfsMetadata(baseCfs.metadata(), indexDef));
         indexCfs = ColumnFamilyStore.createColumnFamilyStore(baseCfs.keyspace,
-                                                             cfm.cfName,
-                                                             cfm,
+                                                             tableRef.name,
+                                                             tableRef,
                                                              baseCfs.getTracker().loadsstables);
         indexedColumn = target.left;
     }
@@ -247,12 +247,12 @@ public abstract class CassandraIndex implements Index
         return true;
     }
 
-    public boolean dependsOn(ColumnDefinition column)
+    public boolean dependsOn(ColumnMetadata column)
     {
         return indexedColumn.name.equals(column.name);
     }
 
-    public boolean supportsExpression(ColumnDefinition column, Operator operator)
+    public boolean supportsExpression(ColumnMetadata column, Operator operator)
     {
         return indexedColumn.name.equals(column.name)
                && supportsOperator(indexedColumn, operator);
@@ -338,7 +338,7 @@ public abstract class CassandraIndex implements Index
     }
 
     public Indexer indexerFor(final DecoratedKey key,
-                              final PartitionColumns columns,
+                              final RegularAndStaticColumns columns,
                               final int nowInSec,
                               final OpOrder.Group opGroup,
                               final IndexTransaction.Type transactionType)
@@ -618,11 +618,10 @@ public abstract class CassandraIndex implements Index
     {
         if (value != null && value.remaining() >= FBUtilities.MAX_UNSIGNED_SHORT)
             throw new InvalidRequestException(String.format(
-                                                           "Cannot index value of size %d for index %s on %s.%s(%s) (maximum allowed size=%d)",
+                                                           "Cannot index value of size %d for index %s on %s(%s) (maximum allowed size=%d)",
                                                            value.remaining(),
                                                            metadata.name,
-                                                           baseCfs.metadata.ksName,
-                                                           baseCfs.metadata.cfName,
+                                                           baseCfs.metadata,
                                                            indexedColumn.name.toString(),
                                                            FBUtilities.MAX_UNSIGNED_SHORT));
     }
@@ -654,7 +653,7 @@ public abstract class CassandraIndex implements Index
 
     private PartitionUpdate partitionUpdate(DecoratedKey valueKey, Row row)
     {
-        return PartitionUpdate.singleRowUpdate(indexCfs.metadata, valueKey, row);
+        return PartitionUpdate.singleRowUpdate(indexCfs.metadata(), valueKey, row);
     }
 
     private void invalidate()
@@ -697,8 +696,8 @@ public abstract class CassandraIndex implements Index
             if (sstables.isEmpty())
             {
                 logger.info("No SSTable data for {}.{} to build index {} from, marking empty index as built",
-                            baseCfs.metadata.ksName,
-                            baseCfs.metadata.cfName,
+                            baseCfs.metadata.keyspace,
+                            baseCfs.metadata.name,
                             metadata.name);
                 baseCfs.indexManager.markIndexBuilt(metadata.name);
                 return;
@@ -727,31 +726,27 @@ public abstract class CassandraIndex implements Index
     }
 
     /**
-     * Construct the CFMetadata for an index table, the clustering columns in the index table
+     * Construct the TableMetadata for an index table, the clustering columns in the index table
      * vary dependent on the kind of the indexed value.
      * @param baseCfsMetadata
      * @param indexMetadata
      * @return
      */
-    public static final CFMetaData indexCfsMetadata(CFMetaData baseCfsMetadata, IndexMetadata indexMetadata)
+    public static TableMetadata indexCfsMetadata(TableMetadata baseCfsMetadata, IndexMetadata indexMetadata)
     {
-        Pair<ColumnDefinition, IndexTarget.Type> target = TargetParser.parse(baseCfsMetadata, indexMetadata);
+        Pair<ColumnMetadata, IndexTarget.Type> target = TargetParser.parse(baseCfsMetadata, indexMetadata);
         CassandraIndexFunctions utils = getFunctions(indexMetadata, target);
-        ColumnDefinition indexedColumn = target.left;
+        ColumnMetadata indexedColumn = target.left;
         AbstractType<?> indexedValueType = utils.getIndexedValueType(indexedColumn);
 
-        // Tables for legacy KEYS indexes are non-compound and dense
-        CFMetaData.Builder builder = indexMetadata.isKeys()
-                                     ? CFMetaData.Builder.create(baseCfsMetadata.ksName,
-                                                                 baseCfsMetadata.indexColumnFamilyName(indexMetadata),
-                                                                 true, false, false)
-                                     : CFMetaData.Builder.create(baseCfsMetadata.ksName,
-                                                                 baseCfsMetadata.indexColumnFamilyName(indexMetadata));
-
-        builder =  builder.withId(baseCfsMetadata.cfId)
-                          .withPartitioner(new LocalPartitioner(indexedValueType))
-                          .addPartitionKey(indexedColumn.name, indexedColumn.type)
-                          .addClusteringColumn("partition_key", baseCfsMetadata.partitioner.partitionOrdering());
+        TableMetadata.Builder builder =
+            TableMetadata.builder(baseCfsMetadata.keyspace, baseCfsMetadata.indexTableName(indexMetadata), baseCfsMetadata.id)
+                         // tables for legacy KEYS indexes are non-compound and dense
+                         .isDense(indexMetadata.isKeys())
+                         .isCompound(!indexMetadata.isKeys())
+                         .partitioner(new LocalPartitioner(indexedValueType))
+                         .addPartitionKeyColumn(indexedColumn.name, indexedColumn.type)
+                         .addClusteringColumn("partition_key", baseCfsMetadata.partitioner.partitionOrdering());
 
         if (indexMetadata.isKeys())
         {
@@ -759,16 +754,16 @@ public abstract class CassandraIndex implements Index
             // value column defined, even though it is never used
             CompactTables.DefaultNames names =
                 CompactTables.defaultNameGenerator(ImmutableSet.of(indexedColumn.name.toString(), "partition_key"));
-            builder = builder.addRegularColumn(names.defaultCompactValueName(), EmptyType.instance);
+            builder.addRegularColumn(names.defaultCompactValueName(), EmptyType.instance);
         }
         else
         {
             // The clustering columns for a table backing a COMPOSITES index are dependent
             // on the specific type of index (there are specializations for indexes on collections)
-            builder = utils.addIndexClusteringColumns(builder, baseCfsMetadata, indexedColumn);
+            utils.addIndexClusteringColumns(builder, baseCfsMetadata, indexedColumn);
         }
 
-        return builder.build().reloadIndexMetadataProperties(baseCfsMetadata);
+        return builder.build().updateIndexTableMetadata(baseCfsMetadata.params);
     }
 
     /**
@@ -779,16 +774,16 @@ public abstract class CassandraIndex implements Index
      */
     public static CassandraIndex newIndex(ColumnFamilyStore baseCfs, IndexMetadata indexMetadata)
     {
-        return getFunctions(indexMetadata, TargetParser.parse(baseCfs.metadata, indexMetadata)).newIndexInstance(baseCfs, indexMetadata);
+        return getFunctions(indexMetadata, TargetParser.parse(baseCfs.metadata(), indexMetadata)).newIndexInstance(baseCfs, indexMetadata);
     }
 
     static CassandraIndexFunctions getFunctions(IndexMetadata indexDef,
-                                                Pair<ColumnDefinition, IndexTarget.Type> target)
+                                                Pair<ColumnMetadata, IndexTarget.Type> target)
     {
         if (indexDef.isKeys())
             return CassandraIndexFunctions.KEYS_INDEX_FUNCTIONS;
 
-        ColumnDefinition indexedColumn = target.left;
+        ColumnMetadata indexedColumn = target.left;
         if (indexedColumn.type.isCollection() && indexedColumn.type.isMultiCell())
         {
             switch (((CollectionType)indexedColumn.type).kind)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/CassandraIndexFunctions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndexFunctions.java b/src/java/org/apache/cassandra/index/internal/CassandraIndexFunctions.java
index 8047e1d..3d500a1 100644
--- a/src/java/org/apache/cassandra/index/internal/CassandraIndexFunctions.java
+++ b/src/java/org/apache/cassandra/index/internal/CassandraIndexFunctions.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.index.internal;
 
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
@@ -46,14 +46,14 @@ public interface CassandraIndexFunctions
      * @param indexedColumn
      * @return
      */
-    default AbstractType<?> getIndexedValueType(ColumnDefinition indexedColumn)
+    default AbstractType<?> getIndexedValueType(ColumnMetadata indexedColumn)
     {
         return indexedColumn.type;
     }
 
     /**
-     * Add the clustering columns for a specific type of index table to the a CFMetaData.Builder (which is being
-     * used to construct the index table's CFMetadata. In the default implementation, the clustering columns of the
+     * Add the clustering columns for a specific type of index table to the a TableMetadata.Builder (which is being
+     * used to construct the index table's TableMetadata. In the default implementation, the clustering columns of the
      * index table hold the partition key and clustering columns of the base table. This is overridden in several cases:
      * * When the indexed value is itself a clustering column, in which case, we only need store the base table's
      *   *other* clustering values in the index - the indexed value being the index table's partition key
@@ -68,11 +68,11 @@ public interface CassandraIndexFunctions
      * @param cfDef
      * @return
      */
-    default CFMetaData.Builder addIndexClusteringColumns(CFMetaData.Builder builder,
-                                                         CFMetaData baseMetadata,
-                                                         ColumnDefinition cfDef)
+    default TableMetadata.Builder addIndexClusteringColumns(TableMetadata.Builder builder,
+                                                            TableMetadata baseMetadata,
+                                                            ColumnMetadata cfDef)
     {
-        for (ColumnDefinition def : baseMetadata.clusteringColumns())
+        for (ColumnMetadata def : baseMetadata.clusteringColumns())
             builder.addClusteringColumn(def.name, def.type);
         return builder;
     }
@@ -104,21 +104,22 @@ public interface CassandraIndexFunctions
             return new ClusteringColumnIndex(baseCfs, indexMetadata);
         }
 
-        public CFMetaData.Builder addIndexClusteringColumns(CFMetaData.Builder builder,
-                                                            CFMetaData baseMetadata,
-                                                            ColumnDefinition columnDef)
+        public TableMetadata.Builder addIndexClusteringColumns(TableMetadata.Builder builder,
+                                                               TableMetadata baseMetadata,
+                                                               ColumnMetadata columnDef)
         {
-            List<ColumnDefinition> cks = baseMetadata.clusteringColumns();
+            List<ColumnMetadata> cks = baseMetadata.clusteringColumns();
             for (int i = 0; i < columnDef.position(); i++)
             {
-                ColumnDefinition def = cks.get(i);
+                ColumnMetadata def = cks.get(i);
                 builder.addClusteringColumn(def.name, def.type);
             }
             for (int i = columnDef.position() + 1; i < cks.size(); i++)
             {
-                ColumnDefinition def = cks.get(i);
+                ColumnMetadata def = cks.get(i);
                 builder.addClusteringColumn(def.name, def.type);
             }
+
             return builder;
         }
     };
@@ -138,7 +139,7 @@ public interface CassandraIndexFunctions
             return new CollectionKeyIndex(baseCfs, indexMetadata);
         }
 
-        public AbstractType<?> getIndexedValueType(ColumnDefinition indexedColumn)
+        public AbstractType<?> getIndexedValueType(ColumnMetadata indexedColumn)
         {
             return ((CollectionType) indexedColumn.type).nameComparator();
         }
@@ -152,16 +153,16 @@ public interface CassandraIndexFunctions
             return new CollectionValueIndex(baseCfs, indexMetadata);
         }
 
-        public AbstractType<?> getIndexedValueType(ColumnDefinition indexedColumn)
+        public AbstractType<?> getIndexedValueType(ColumnMetadata indexedColumn)
         {
             return ((CollectionType)indexedColumn.type).valueComparator();
         }
 
-        public CFMetaData.Builder addIndexClusteringColumns(CFMetaData.Builder builder,
-                                                            CFMetaData baseMetadata,
-                                                            ColumnDefinition columnDef)
+        public TableMetadata.Builder addIndexClusteringColumns(TableMetadata.Builder builder,
+                                                               TableMetadata baseMetadata,
+                                                               ColumnMetadata columnDef)
         {
-            for (ColumnDefinition def : baseMetadata.clusteringColumns())
+            for (ColumnMetadata def : baseMetadata.clusteringColumns())
                 builder.addClusteringColumn(def.name, def.type);
 
             // collection key
@@ -177,7 +178,7 @@ public interface CassandraIndexFunctions
             return new CollectionEntryIndex(baseCfs, indexMetadata);
         }
 
-        public AbstractType<?> getIndexedValueType(ColumnDefinition indexedColumn)
+        public AbstractType<?> getIndexedValueType(ColumnMetadata indexedColumn)
         {
             CollectionType colType = (CollectionType)indexedColumn.type;
             return CompositeType.getInstance(colType.nameComparator(), colType.valueComparator());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java b/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java
index 7b622e3..005e5b9 100644
--- a/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/CassandraIndexSearcher.java
@@ -26,7 +26,8 @@ import java.util.NavigableSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
@@ -76,8 +77,8 @@ public abstract class CassandraIndexSearcher implements Index.Searcher
     {
         ClusteringIndexFilter filter = makeIndexFilter(command);
         ColumnFamilyStore indexCfs = index.getBackingTable().get();
-        CFMetaData indexCfm = indexCfs.metadata;
-        return SinglePartitionReadCommand.create(indexCfm, command.nowInSec(), indexKey, ColumnFilter.all(indexCfm), filter)
+        TableMetadata indexMetadata = indexCfs.metadata();
+        return SinglePartitionReadCommand.create(indexMetadata, command.nowInSec(), indexKey, ColumnFilter.all(indexMetadata), filter)
                                          .queryMemtableAndDisk(indexCfs, executionController.indexReadController());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/CollatedViewIndexBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/CollatedViewIndexBuilder.java b/src/java/org/apache/cassandra/index/internal/CollatedViewIndexBuilder.java
index 811d857..103a1ee 100644
--- a/src/java/org/apache/cassandra/index/internal/CollatedViewIndexBuilder.java
+++ b/src/java/org/apache/cassandra/index/internal/CollatedViewIndexBuilder.java
@@ -51,7 +51,7 @@ public class CollatedViewIndexBuilder extends SecondaryIndexBuilder
 
     public CompactionInfo getCompactionInfo()
     {
-        return new CompactionInfo(cfs.metadata,
+        return new CompactionInfo(cfs.metadata(),
                 OperationType.INDEX_BUILD,
                 iter.getBytesRead(),
                 iter.getTotalBytes(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/composites/ClusteringColumnIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/composites/ClusteringColumnIndex.java b/src/java/org/apache/cassandra/index/internal/composites/ClusteringColumnIndex.java
index f207e9b..ab05a4e 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/ClusteringColumnIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/ClusteringColumnIndex.java
@@ -76,7 +76,7 @@ public class ClusteringColumnIndex extends CassandraIndex
     public IndexEntry decodeEntry(DecoratedKey indexedValue,
                                   Row indexEntry)
     {
-        int ckCount = baseCfs.metadata.clusteringColumns().size();
+        int ckCount = baseCfs.metadata().clusteringColumns().size();
 
         Clustering clustering = indexEntry.clustering();
         CBuilder builder = CBuilder.create(baseCfs.getComparator());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/composites/CollectionEntryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/composites/CollectionEntryIndex.java b/src/java/org/apache/cassandra/index/internal/composites/CollectionEntryIndex.java
index 1113600..efe84b6 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/CollectionEntryIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CollectionEntryIndex.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.index.internal.composites;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -57,7 +57,7 @@ public class CollectionEntryIndex extends CollectionKeyIndexBase
         ByteBuffer mapKey = components[0];
         ByteBuffer mapValue = components[1];
 
-        ColumnDefinition columnDef = indexedColumn;
+        ColumnMetadata columnDef = indexedColumn;
         Cell cell = data.getCell(columnDef, CellPath.create(mapKey));
         if (cell == null || !cell.isLive(nowInSec))
             return true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndex.java b/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndex.java
index 42c45e5..4fc20ae 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndex.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.index.internal.composites;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -56,7 +56,7 @@ public class CollectionKeyIndex extends CollectionKeyIndexBase
         return cell == null || !cell.isLive(nowInSec);
     }
 
-    public boolean supportsOperator(ColumnDefinition indexedColumn, Operator operator)
+    public boolean supportsOperator(ColumnMetadata indexedColumn, Operator operator)
     {
         return operator == Operator.CONTAINS_KEY ||
                operator == Operator.CONTAINS && indexedColumn.type instanceof SetType;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndexBase.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndexBase.java b/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndexBase.java
index ef76870..fccf522 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndexBase.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CollectionKeyIndexBase.java
@@ -73,7 +73,7 @@ public abstract class CollectionKeyIndexBase extends CassandraIndex
             indexedEntryClustering = Clustering.STATIC_CLUSTERING;
         else
         {
-            int count = 1 + baseCfs.metadata.clusteringColumns().size();
+            int count = 1 + baseCfs.metadata().clusteringColumns().size();
             CBuilder builder = CBuilder.create(baseCfs.getComparator());
             for (int i = 0; i < count - 1; i++)
                 builder.add(clustering.get(i + 1));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/composites/CollectionValueIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/composites/CollectionValueIndex.java b/src/java/org/apache/cassandra/index/internal/composites/CollectionValueIndex.java
index 5929e69..4f0f2df 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/CollectionValueIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CollectionValueIndex.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.index.internal.composites;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.CollectionType;
@@ -67,7 +67,7 @@ public class CollectionValueIndex extends CassandraIndex
         // partition key is needed at query time.
         // In the non-static case, cell will be present during indexing but
         // not when searching (CASSANDRA-7525).
-        if (prefix.size() == baseCfs.metadata.clusteringColumns().size() && path != null)
+        if (prefix.size() == baseCfs.metadata().clusteringColumns().size() && path != null)
             builder.add(path.get(0));
 
         return builder;
@@ -94,14 +94,14 @@ public class CollectionValueIndex extends CassandraIndex
                                 indexedEntryClustering);
     }
 
-    public boolean supportsOperator(ColumnDefinition indexedColumn, Operator operator)
+    public boolean supportsOperator(ColumnMetadata indexedColumn, Operator operator)
     {
         return operator == Operator.CONTAINS && !(indexedColumn.type instanceof SetType);
     }
 
     public boolean isStale(Row data, ByteBuffer indexValue, int nowInSec)
     {
-        ColumnDefinition columnDef = indexedColumn;
+        ColumnMetadata columnDef = indexedColumn;
         ComplexColumnData complexData = data.getComplexColumnData(columnDef);
         if (complexData == null)
             return true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
index b5e4a78..7ee3bb5 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/CompositesSearcher.java
@@ -21,7 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ClusteringIndexNamesFilter;
 import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter;
@@ -69,7 +69,7 @@ public class CompositesSearcher extends CassandraIndexSearcher
 
             private UnfilteredRowIterator next;
 
-            public CFMetaData metadata()
+            public TableMetadata metadata()
             {
                 return command.metadata();
             }
@@ -111,7 +111,7 @@ public class CompositesSearcher extends CassandraIndexSearcher
                     {
                         // If the index is on a static column, we just need to do a full read on the partition.
                         // Note that we want to re-use the command.columnFilter() in case of future change.
-                        dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata,
+                        dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata(),
                                                                     command.nowInSec(),
                                                                     command.columnFilter(),
                                                                     RowFilter.NONE,
@@ -148,7 +148,7 @@ public class CompositesSearcher extends CassandraIndexSearcher
 
                         // Query the gathered index hits. We still need to filter stale hits from the resulting query.
                         ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(clusterings.build(), false);
-                        dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata,
+                        dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata(),
                                                                     command.nowInSec(),
                                                                     command.columnFilter(),
                                                                     command.rowFilter(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/composites/PartitionKeyIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/composites/PartitionKeyIndex.java b/src/java/org/apache/cassandra/index/internal/composites/PartitionKeyIndex.java
index 2c0b5aa..810571c 100644
--- a/src/java/org/apache/cassandra/index/internal/composites/PartitionKeyIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/composites/PartitionKeyIndex.java
@@ -57,7 +57,7 @@ public class PartitionKeyIndex extends CassandraIndex
                                       CellPath path,
                                       ByteBuffer cellValue)
     {
-        CompositeType keyComparator = (CompositeType)baseCfs.metadata.getKeyValidator();
+        CompositeType keyComparator = (CompositeType)baseCfs.metadata().partitionKeyType;
         ByteBuffer[] components = keyComparator.split(partitionKey);
         return components[indexedColumn.position()];
     }
@@ -75,7 +75,7 @@ public class PartitionKeyIndex extends CassandraIndex
 
     public IndexEntry decodeEntry(DecoratedKey indexedValue, Row indexEntry)
     {
-        int ckCount = baseCfs.metadata.clusteringColumns().size();
+        int ckCount = baseCfs.metadata().clusteringColumns().size();
         Clustering clustering = indexEntry.clustering();
         CBuilder builder = CBuilder.create(baseCfs.getComparator());
         for (int i = 0; i < ckCount; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/keys/KeysIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/keys/KeysIndex.java b/src/java/org/apache/cassandra/index/internal/keys/KeysIndex.java
index d680253..20a1915 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysIndex.java
@@ -22,8 +22,9 @@ package org.apache.cassandra.index.internal.keys;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.CellPath;
@@ -39,17 +40,17 @@ public class KeysIndex extends CassandraIndex
         super(baseCfs, indexDef);
     }
 
-    public CFMetaData.Builder addIndexClusteringColumns(CFMetaData.Builder builder,
-                                                        CFMetaData baseMetadata,
-                                                        ColumnDefinition cfDef)
+    public TableMetadata.Builder addIndexClusteringColumns(TableMetadata.Builder builder,
+                                                           TableMetadataRef baseMetadata,
+                                                           ColumnMetadata cfDef)
     {
         // no additional clustering columns required
         return builder;
     }
 
     protected CBuilder buildIndexClusteringPrefix(ByteBuffer partitionKey,
-                                               ClusteringPrefix prefix,
-                                               CellPath path)
+                                                  ClusteringPrefix prefix,
+                                                  CellPath path)
     {
         CBuilder builder = CBuilder.create(getIndexComparator());
         builder.add(partitionKey);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
index febb09f..2ab5345 100644
--- a/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/index/internal/keys/KeysSearcher.java
@@ -22,7 +22,6 @@ import java.nio.ByteBuffer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.filter.DataLimits;
@@ -31,6 +30,7 @@ import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.index.internal.CassandraIndex;
 import org.apache.cassandra.index.internal.CassandraIndexSearcher;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
 public class KeysSearcher extends CassandraIndexSearcher
@@ -55,7 +55,7 @@ public class KeysSearcher extends CassandraIndexSearcher
         {
             private UnfilteredRowIterator next;
 
-            public CFMetaData metadata()
+            public TableMetadata metadata()
             {
                 return command.metadata();
             }
@@ -85,7 +85,7 @@ public class KeysSearcher extends CassandraIndexSearcher
                         continue;
 
                     ColumnFilter extendedFilter = getExtendedFilter(command.columnFilter());
-                    SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata,
+                    SinglePartitionReadCommand dataCmd = SinglePartitionReadCommand.create(index.baseCfs.metadata(),
                                                                                            command.nowInSec(),
                                                                                            extendedFilter,
                                                                                            command.rowFilter(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
index 4375964..5257cb7 100644
--- a/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
+++ b/src/java/org/apache/cassandra/index/sasi/SASIIndex.java
@@ -52,7 +52,10 @@ import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.format.SSTableFlushObserver;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.notifications.*;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.IndexMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.concurrent.OpOrder;
@@ -65,7 +68,7 @@ public class SASIIndex implements Index, INotificationConsumer
                                                        Set<Index> indexes,
                                                        Collection<SSTableReader> sstablesToRebuild)
         {
-            NavigableMap<SSTableReader, Map<ColumnDefinition, ColumnIndex>> sstables = new TreeMap<>((a, b) -> {
+            NavigableMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables = new TreeMap<>((a, b) -> {
                 return Integer.compare(a.descriptor.generation, b.descriptor.generation);
             });
 
@@ -77,7 +80,7 @@ public class SASIIndex implements Index, INotificationConsumer
                        sstablesToRebuild.stream()
                                         .filter((sstable) -> !sasi.index.hasSSTable(sstable))
                                         .forEach((sstable) -> {
-                                            Map<ColumnDefinition, ColumnIndex> toBuild = sstables.get(sstable);
+                                            Map<ColumnMetadata, ColumnIndex> toBuild = sstables.get(sstable);
                                             if (toBuild == null)
                                                 sstables.put(sstable, (toBuild = new HashMap<>()));
 
@@ -100,18 +103,18 @@ public class SASIIndex implements Index, INotificationConsumer
         this.baseCfs = baseCfs;
         this.config = config;
 
-        ColumnDefinition column = TargetParser.parse(baseCfs.metadata, config).left;
-        this.index = new ColumnIndex(baseCfs.metadata.getKeyValidator(), column, config);
+        ColumnMetadata column = TargetParser.parse(baseCfs.metadata(), config).left;
+        this.index = new ColumnIndex(baseCfs.metadata().partitionKeyType, column, config);
 
         Tracker tracker = baseCfs.getTracker();
         tracker.subscribe(this);
 
-        SortedMap<SSTableReader, Map<ColumnDefinition, ColumnIndex>> toRebuild = new TreeMap<>((a, b)
+        SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> toRebuild = new TreeMap<>((a, b)
                                                 -> Integer.compare(a.descriptor.generation, b.descriptor.generation));
 
         for (SSTableReader sstable : index.init(tracker.getView().liveSSTables()))
         {
-            Map<ColumnDefinition, ColumnIndex> perSSTable = toRebuild.get(sstable);
+            Map<ColumnMetadata, ColumnIndex> perSSTable = toRebuild.get(sstable);
             if (perSSTable == null)
                 toRebuild.put(sstable, (perSSTable = new HashMap<>()));
 
@@ -121,16 +124,16 @@ public class SASIIndex implements Index, INotificationConsumer
         CompactionManager.instance.submitIndexBuild(new SASIIndexBuilder(baseCfs, toRebuild));
     }
 
-    public static Map<String, String> validateOptions(Map<String, String> options, CFMetaData cfm)
+    public static Map<String, String> validateOptions(Map<String, String> options, TableMetadata metadata)
     {
-        if (!(cfm.partitioner instanceof Murmur3Partitioner))
+        if (!(metadata.partitioner instanceof Murmur3Partitioner))
             throw new ConfigurationException("SASI only supports Murmur3Partitioner.");
 
         String targetColumn = options.get("target");
         if (targetColumn == null)
             throw new ConfigurationException("unknown target column");
 
-        Pair<ColumnDefinition, IndexTarget.Type> target = TargetParser.parse(cfm, targetColumn);
+        Pair<ColumnMetadata, IndexTarget.Type> target = TargetParser.parse(metadata, targetColumn);
         if (target == null)
             throw new ConfigurationException("failed to retrieve target column for: " + targetColumn);
 
@@ -200,17 +203,17 @@ public class SASIIndex implements Index, INotificationConsumer
         return Optional.empty();
     }
 
-    public boolean indexes(PartitionColumns columns)
+    public boolean indexes(RegularAndStaticColumns columns)
     {
         return columns.contains(index.getDefinition());
     }
 
-    public boolean dependsOn(ColumnDefinition column)
+    public boolean dependsOn(ColumnMetadata column)
     {
         return index.getDefinition().compareTo(column) == 0;
     }
 
-    public boolean supportsExpression(ColumnDefinition column, Operator operator)
+    public boolean supportsExpression(ColumnMetadata column, Operator operator)
     {
         return dependsOn(column) && index.supports(operator);
     }
@@ -236,7 +239,7 @@ public class SASIIndex implements Index, INotificationConsumer
     public void validate(PartitionUpdate update) throws InvalidRequestException
     {}
 
-    public Indexer indexerFor(DecoratedKey key, PartitionColumns columns, int nowInSec, OpOrder.Group opGroup, IndexTransaction.Type transactionType)
+    public Indexer indexerFor(DecoratedKey key, RegularAndStaticColumns columns, int nowInSec, OpOrder.Group opGroup, IndexTransaction.Type transactionType)
     {
         return new Indexer()
         {
@@ -282,14 +285,14 @@ public class SASIIndex implements Index, INotificationConsumer
 
     public Searcher searcherFor(ReadCommand command) throws InvalidRequestException
     {
-        CFMetaData config = command.metadata();
-        ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(config.cfId);
+        TableMetadata config = command.metadata();
+        ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(config.id);
         return controller -> new QueryPlan(cfs, command, DatabaseDescriptor.getRangeRpcTimeout()).execute(controller);
     }
 
     public SSTableFlushObserver getFlushObserver(Descriptor descriptor, OperationType opType)
     {
-        return newWriter(baseCfs.metadata.getKeyValidator(), descriptor, Collections.singletonMap(index.getDefinition(), index), opType);
+        return newWriter(baseCfs.metadata().partitionKeyType, descriptor, Collections.singletonMap(index.getDefinition(), index), opType);
     }
 
     public BiFunction<PartitionIterator, ReadCommand, PartitionIterator> postProcessorFor(ReadCommand command)
@@ -336,7 +339,7 @@ public class SASIIndex implements Index, INotificationConsumer
 
     protected static PerSSTableIndexWriter newWriter(AbstractType<?> keyValidator,
                                                      Descriptor descriptor,
-                                                     Map<ColumnDefinition, ColumnIndex> indexes,
+                                                     Map<ColumnMetadata, ColumnIndex> indexes,
                                                      OperationType opType)
     {
         return new PerSSTableIndexWriter(keyValidator, descriptor, opType, indexes);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java b/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java
index d50875a..a01e45b 100644
--- a/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java
+++ b/src/java/org/apache/cassandra/index/sasi/SASIIndexBuilder.java
@@ -24,7 +24,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.*;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowIndexEntry;
@@ -49,12 +49,12 @@ class SASIIndexBuilder extends SecondaryIndexBuilder
     private final ColumnFamilyStore cfs;
     private final UUID compactionId = UUIDGen.getTimeUUID();
 
-    private final SortedMap<SSTableReader, Map<ColumnDefinition, ColumnIndex>> sstables;
+    private final SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables;
 
     private long bytesProcessed = 0;
     private final long totalSizeInBytes;
 
-    public SASIIndexBuilder(ColumnFamilyStore cfs, SortedMap<SSTableReader, Map<ColumnDefinition, ColumnIndex>> sstables)
+    public SASIIndexBuilder(ColumnFamilyStore cfs, SortedMap<SSTableReader, Map<ColumnMetadata, ColumnIndex>> sstables)
     {
         long totalIndexBytes = 0;
         for (SSTableReader sstable : sstables.keySet())
@@ -67,18 +67,18 @@ class SASIIndexBuilder extends SecondaryIndexBuilder
 
     public void build()
     {
-        AbstractType<?> keyValidator = cfs.metadata.getKeyValidator();
-        for (Map.Entry<SSTableReader, Map<ColumnDefinition, ColumnIndex>> e : sstables.entrySet())
+        AbstractType<?> keyValidator = cfs.metadata().partitionKeyType;
+        for (Map.Entry<SSTableReader, Map<ColumnMetadata, ColumnIndex>> e : sstables.entrySet())
         {
             SSTableReader sstable = e.getKey();
-            Map<ColumnDefinition, ColumnIndex> indexes = e.getValue();
+            Map<ColumnMetadata, ColumnIndex> indexes = e.getValue();
 
             try (RandomAccessReader dataFile = sstable.openDataReader())
             {
                 PerSSTableIndexWriter indexWriter = SASIIndex.newWriter(keyValidator, sstable.descriptor, indexes, OperationType.COMPACTION);
 
                 long previousKeyPosition = 0;
-                try (KeyIterator keys = new KeyIterator(sstable.descriptor, cfs.metadata))
+                try (KeyIterator keys = new KeyIterator(sstable.descriptor, cfs.metadata()))
                 {
                     while (keys.hasNext())
                     {
@@ -99,7 +99,7 @@ class SASIIndexBuilder extends SecondaryIndexBuilder
                             try (SSTableIdentityIterator partition = SSTableIdentityIterator.create(sstable, dataFile, key))
                             {
                                 // if the row has statics attached, it has to be indexed separately
-                                if (cfs.metadata.hasStaticColumns())
+                                if (cfs.metadata().hasStaticColumns())
                                     indexWriter.nextUnfilteredCluster(partition.staticRow());
 
                                 while (partition.hasNext())
@@ -123,7 +123,7 @@ class SASIIndexBuilder extends SecondaryIndexBuilder
 
     public CompactionInfo getCompactionInfo()
     {
-        return new CompactionInfo(cfs.metadata,
+        return new CompactionInfo(cfs.metadata(),
                                   OperationType.INDEX_BUILD,
                                   bytesProcessed,
                                   totalSizeInBytes,


[31/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index ecabd2f..7560e2f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -27,21 +27,20 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CFName;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.IndexName;
-import org.apache.cassandra.cql3.Validation;
 import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.schema.Indexes;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -75,20 +74,20 @@ public class CreateIndexStatement extends SchemaAlteringStatement
 
     public void validate(ClientState state) throws RequestValidationException
     {
-        CFMetaData cfm = Validation.validateColumnFamily(keyspace(), columnFamily());
+        TableMetadata table = Schema.instance.validateTable(keyspace(), columnFamily());
 
-        if (cfm.isCounter())
+        if (table.isCounter())
             throw new InvalidRequestException("Secondary indexes are not supported on counter tables");
 
-        if (cfm.isView())
+        if (table.isView())
             throw new InvalidRequestException("Secondary indexes are not supported on materialized views");
 
-        if (cfm.isCompactTable() && !cfm.isStaticCompactTable())
+        if (table.isCompactTable() && !table.isStaticCompactTable())
             throw new InvalidRequestException("Secondary indexes are not supported on COMPACT STORAGE tables that have clustering columns");
 
         List<IndexTarget> targets = new ArrayList<>(rawTargets.size());
         for (IndexTarget.Raw rawTarget : rawTargets)
-            targets.add(rawTarget.prepare(cfm));
+            targets.add(rawTarget.prepare(table));
 
         if (targets.isEmpty() && !properties.isCustom)
             throw new InvalidRequestException("Only CUSTOM indexes can be created without specifying a target column");
@@ -98,16 +97,16 @@ public class CreateIndexStatement extends SchemaAlteringStatement
 
         for (IndexTarget target : targets)
         {
-            ColumnDefinition cd = cfm.getColumnDefinition(target.column);
+            ColumnMetadata cd = table.getColumn(target.column);
 
             if (cd == null)
                 throw new InvalidRequestException("No column definition found for column " + target.column);
 
             // TODO: we could lift that limitation
-            if (cfm.isCompactTable() && cd.isPrimaryKeyColumn())
+            if (table.isCompactTable() && cd.isPrimaryKeyColumn())
                 throw new InvalidRequestException("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables");
 
-            if (cd.kind == ColumnDefinition.Kind.PARTITION_KEY && cfm.getKeyValidatorAsClusteringComparator().size() == 1)
+            if (cd.kind == ColumnMetadata.Kind.PARTITION_KEY && table.partitionKeyColumns().size() == 1)
                 throw new InvalidRequestException(String.format("Cannot create secondary index on partition key column %s", target.column));
 
             boolean isMap = cd.type instanceof MapType;
@@ -126,7 +125,7 @@ public class CreateIndexStatement extends SchemaAlteringStatement
 
         if (!Strings.isNullOrEmpty(indexName))
         {
-            if (Schema.instance.getKSMetaData(keyspace()).existingIndexNames(null).contains(indexName))
+            if (Schema.instance.getKeyspaceMetadata(keyspace()).existingIndexNames(null).contains(indexName))
             {
                 if (ifNotExists)
                     return;
@@ -152,7 +151,7 @@ public class CreateIndexStatement extends SchemaAlteringStatement
             throw new InvalidRequestException("full() indexes can only be created on frozen collections");
     }
 
-    private void validateIsSimpleIndexIfTargetColumnNotCollection(ColumnDefinition cd, IndexTarget target) throws InvalidRequestException
+    private void validateIsSimpleIndexIfTargetColumnNotCollection(ColumnMetadata cd, IndexTarget target) throws InvalidRequestException
     {
         if (!cd.type.isCollection() && target.type != IndexTarget.Type.SIMPLE)
             throw new InvalidRequestException(String.format("Cannot create %s() index on %s. " +
@@ -183,10 +182,10 @@ public class CreateIndexStatement extends SchemaAlteringStatement
 
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
     {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).copy();
+        TableMetadata current = Schema.instance.getTableMetadata(keyspace(), columnFamily());
         List<IndexTarget> targets = new ArrayList<>(rawTargets.size());
         for (IndexTarget.Raw rawTarget : rawTargets)
-            targets.add(rawTarget.prepare(cfm));
+            targets.add(rawTarget.prepare(current));
 
         String acceptedName = indexName;
         if (Strings.isNullOrEmpty(acceptedName))
@@ -196,7 +195,7 @@ public class CreateIndexStatement extends SchemaAlteringStatement
                                                          targets.size() == 1 ? targets.get(0).column.toString() : null);
         }
 
-        if (Schema.instance.getKSMetaData(keyspace()).existingIndexNames(null).contains(acceptedName))
+        if (Schema.instance.getKeyspaceMetadata(keyspace()).existingIndexNames(null).contains(acceptedName))
         {
             if (ifNotExists)
                 return null;
@@ -214,13 +213,13 @@ public class CreateIndexStatement extends SchemaAlteringStatement
         else
         {
             indexOptions = Collections.emptyMap();
-            kind = cfm.isCompound() ? IndexMetadata.Kind.COMPOSITES : IndexMetadata.Kind.KEYS;
+            kind = current.isCompound() ? IndexMetadata.Kind.COMPOSITES : IndexMetadata.Kind.KEYS;
         }
 
-        IndexMetadata index = IndexMetadata.fromIndexTargets(cfm, targets, acceptedName, kind, indexOptions);
+        IndexMetadata index = IndexMetadata.fromIndexTargets(targets, acceptedName, kind, indexOptions);
 
         // check to disallow creation of an index which duplicates an existing one in all but name
-        Optional<IndexMetadata> existingIndex = Iterables.tryFind(cfm.getIndexes(), existing -> existing.equalsWithoutName(index));
+        Optional<IndexMetadata> existingIndex = Iterables.tryFind(current.indexes, existing -> existing.equalsWithoutName(index));
         if (existingIndex.isPresent())
         {
             if (ifNotExists)
@@ -231,10 +230,14 @@ public class CreateIndexStatement extends SchemaAlteringStatement
                                                                 existingIndex.get().name));
         }
 
+        TableMetadata updated =
+            current.unbuild()
+                   .indexes(current.indexes.with(index))
+                   .build();
+
         logger.trace("Updating index definition for {}", indexName);
-        cfm.indexes(cfm.getIndexes().with(index));
 
-        MigrationManager.announceColumnFamilyUpdate(cfm, isLocalOnly);
+        MigrationManager.announceTableUpdate(updated, isLocalOnly);
 
         // Creating an index is akin to updating the CF
         return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
index b1e63fb..f859ff7 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
@@ -20,13 +20,14 @@ package org.apache.cassandra.cql3.statements;
 import java.util.regex.Pattern;
 
 import org.apache.cassandra.auth.*;
-import org.apache.cassandra.cql3.Validation;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.locator.LocalStrategy;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.service.*;
 import org.apache.cassandra.transport.Event;
 
@@ -74,7 +75,7 @@ public class CreateKeyspaceStatement extends SchemaAlteringStatement
      */
     public void validate(ClientState state) throws RequestValidationException
     {
-        Validation.validateKeyspaceNotSystem(name);
+        Schema.validateKeyspaceNotSystem(name);
 
         // keyspace name
         if (!PATTERN_WORD_CHARS.matcher(name).matches())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 aabd221..1b09c0b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@ -30,11 +30,8 @@ import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.TableParams;
-import org.apache.cassandra.schema.Types;
+import org.apache.cassandra.schema.*;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -61,9 +58,9 @@ public class CreateTableStatement extends SchemaAlteringStatement
     private final Set<ColumnIdentifier> staticColumns;
     private final TableParams params;
     private final boolean ifNotExists;
-    private final UUID id;
+    private final TableId id;
 
-    public CreateTableStatement(CFName name, TableParams params, boolean ifNotExists, Set<ColumnIdentifier> staticColumns, UUID id)
+    public CreateTableStatement(CFName name, TableParams params, boolean ifNotExists, Set<ColumnIdentifier> staticColumns, TableId id)
     {
         super(name);
         this.params = params;
@@ -86,7 +83,7 @@ public class CreateTableStatement extends SchemaAlteringStatement
     {
         try
         {
-            MigrationManager.announceNewColumnFamily(getCFMetaData(), isLocalOnly);
+            MigrationManager.announceNewTable(toTableMetadata(), isLocalOnly);
             return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
         }
         catch (AlreadyExistsException e)
@@ -113,12 +110,33 @@ public class CreateTableStatement extends SchemaAlteringStatement
         }
     }
 
-    public CFMetaData.Builder metadataBuilder()
+    /**
+     */
+    public static TableMetadata.Builder parse(String cql, String keyspace)
+    {
+        CreateTableStatement.RawStatement raw = CQLFragmentParser.parseAny(CqlParser::createTableStatement, cql, "CREATE TABLE");
+        raw.prepareKeyspace(keyspace);
+        CreateTableStatement prepared = (CreateTableStatement) raw.prepare(Types.none()).statement;
+        return prepared.builder();
+    }
+
+    public TableMetadata.Builder builder()
     {
-        CFMetaData.Builder builder = CFMetaData.Builder.create(keyspace(), columnFamily(), isDense, isCompound, hasCounters);
-        builder.withId(id);
+        TableMetadata.Builder builder = TableMetadata.builder(keyspace(), columnFamily());
+
+        if (id != null)
+            builder.id(id);
+
+        builder.isDense(isDense)
+               .isCompound(isCompound)
+               .isCounter(hasCounters)
+               .isSuper(false)
+               .isView(false)
+               .params(params);
+
         for (int i = 0; i < keyAliases.size(); i++)
-            builder.addPartitionKey(keyAliases.get(i), keyTypes.get(i));
+            builder.addPartitionKeyColumn(keyAliases.get(i), keyTypes.get(i));
+
         for (int i = 0; i < columnAliases.size(); i++)
             builder.addClusteringColumn(columnAliases.get(i), clusteringTypes.get(i));
 
@@ -136,14 +154,14 @@ public class CreateTableStatement extends SchemaAlteringStatement
         boolean isCompactTable = isDense || !isCompound;
         if (isCompactTable)
         {
-            CompactTables.DefaultNames names = CompactTables.defaultNameGenerator(builder.usedColumnNames());
+            CompactTables.DefaultNames names = CompactTables.defaultNameGenerator(builder.columnNames());
             // Compact tables always have a clustering and a single regular value.
             if (isStaticCompact)
             {
                 builder.addClusteringColumn(names.defaultClusteringName(), UTF8Type.instance);
                 builder.addRegularColumn(names.defaultCompactValueName(), hasCounters ? CounterColumnType.instance : BytesType.instance);
             }
-            else if (isDense && !builder.hasRegulars())
+            else if (isDense && !builder.hasRegularColumns())
             {
                 // Even for dense, we might not have our regular column if it wasn't part of the declaration. If
                 // that's the case, add it but with a specific EmptyType so we can recognize that case later
@@ -155,20 +173,14 @@ public class CreateTableStatement extends SchemaAlteringStatement
     }
 
     /**
-     * Returns a CFMetaData instance based on the parameters parsed from this
+     * Returns a TableMetadata instance based on the parameters parsed from this
      * {@code CREATE} statement, or defaults where applicable.
      *
-     * @return a CFMetaData instance corresponding to the values parsed from this statement
-     * @throws InvalidRequestException on failure to validate parsed parameters
+     * @return a TableMetadata instance corresponding to the values parsed from this statement
      */
-    public CFMetaData getCFMetaData()
-    {
-        return metadataBuilder().build().params(params);
-    }
-
-    public TableParams params()
+    public TableMetadata toTableMetadata()
     {
-        return params;
+        return builder().build();
     }
 
     public static class RawStatement extends CFStatement
@@ -195,7 +207,7 @@ public class CreateTableStatement extends SchemaAlteringStatement
          */
         public ParsedStatement.Prepared prepare() throws RequestValidationException
         {
-            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace());
+            KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace());
             if (ksm == null)
                 throw new ConfigurationException(String.format("Keyspace %s doesn't exist", keyspace()));
             return prepare(ksm.types);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
index 8f84e4a..0f12b9c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
@@ -20,18 +20,17 @@ package org.apache.cassandra.cql3.statements;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.cql3.Validation;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TriggerMetadata;
 import org.apache.cassandra.schema.Triggers;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.triggers.TriggerExecutor;
@@ -59,8 +58,8 @@ public class CreateTriggerStatement extends SchemaAlteringStatement
 
     public void validate(ClientState state) throws RequestValidationException
     {
-        CFMetaData cfm = Validation.validateColumnFamily(keyspace(), columnFamily());
-        if (cfm.isView())
+        TableMetadata metadata = Schema.instance.validateTable(keyspace(), columnFamily());
+        if (metadata.isView())
             throw new InvalidRequestException("Cannot CREATE TRIGGER against a materialized view");
 
         try
@@ -75,8 +74,8 @@ public class CreateTriggerStatement extends SchemaAlteringStatement
 
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws ConfigurationException, InvalidRequestException
     {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).copy();
-        Triggers triggers = cfm.getTriggers();
+        TableMetadata current = Schema.instance.getTableMetadata(keyspace(), columnFamily());
+        Triggers triggers = current.triggers;
 
         if (triggers.get(triggerName).isPresent())
         {
@@ -86,9 +85,14 @@ public class CreateTriggerStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException(String.format("Trigger %s already exists", triggerName));
         }
 
-        cfm.triggers(triggers.with(TriggerMetadata.create(triggerName, triggerClass)));
+        TableMetadata updated =
+            current.unbuild()
+                   .triggers(triggers.with(TriggerMetadata.create(triggerName, triggerClass)))
+                   .build();
+
         logger.info("Adding trigger with name {} and class {}", triggerName, triggerClass);
-        MigrationManager.announceColumnFamilyUpdate(cfm, isLocalOnly);
+
+        MigrationManager.announceTableUpdate(updated, isLocalOnly);
         return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
index ff9af75..4462bc2 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTypeStatement.java
@@ -21,15 +21,15 @@ import java.util.*;
 import java.util.stream.Collectors;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.schema.Types;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -67,7 +67,7 @@ public class CreateTypeStatement extends SchemaAlteringStatement
 
     public void validate(ClientState state) throws RequestValidationException
     {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(name.getKeyspace());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name.getKeyspace());
         if (ksm == null)
             throw new InvalidRequestException(String.format("Cannot add type in unknown keyspace %s", name.getKeyspace()));
 
@@ -120,7 +120,7 @@ public class CreateTypeStatement extends SchemaAlteringStatement
 
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
     {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(name.getKeyspace());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name.getKeyspace());
         assert ksm != null; // should haven't validate otherwise
 
         // Can happen with ifNotExists

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index cf5be1e..8e92534 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@ -24,10 +24,6 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 import org.apache.cassandra.cql3.selection.RawSelector;
@@ -40,9 +36,13 @@ import org.apache.cassandra.exceptions.AlreadyExistsException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TableParams;
+import org.apache.cassandra.schema.ViewMetadata;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -51,8 +51,8 @@ public class CreateViewStatement extends SchemaAlteringStatement
     private final CFName baseName;
     private final List<RawSelector> selectClause;
     private final WhereClause whereClause;
-    private final List<ColumnDefinition.Raw> partitionKeys;
-    private final List<ColumnDefinition.Raw> clusteringKeys;
+    private final List<ColumnMetadata.Raw> partitionKeys;
+    private final List<ColumnMetadata.Raw> clusteringKeys;
     public final CFProperties properties = new CFProperties();
     private final boolean ifNotExists;
 
@@ -60,8 +60,8 @@ public class CreateViewStatement extends SchemaAlteringStatement
                                CFName baseName,
                                List<RawSelector> selectClause,
                                WhereClause whereClause,
-                               List<ColumnDefinition.Raw> partitionKeys,
-                               List<ColumnDefinition.Raw> clusteringKeys,
+                               List<ColumnMetadata.Raw> partitionKeys,
+                               List<ColumnMetadata.Raw> clusteringKeys,
                                boolean ifNotExists)
     {
         super(viewName);
@@ -91,11 +91,11 @@ public class CreateViewStatement extends SchemaAlteringStatement
         void add(ColumnIdentifier identifier, AbstractType<?> type);
     }
 
-    private void add(CFMetaData baseCfm, Iterable<ColumnIdentifier> columns, AddColumn adder)
+    private void add(TableMetadata baseCfm, Iterable<ColumnIdentifier> columns, AddColumn adder)
     {
         for (ColumnIdentifier column : columns)
         {
-            AbstractType<?> type = baseCfm.getColumnDefinition(column).type;
+            AbstractType<?> type = baseCfm.getColumn(column).type;
             if (properties.definedOrdering.containsKey(column))
             {
                 boolean desc = properties.definedOrdering.get(column);
@@ -133,14 +133,14 @@ public class CreateViewStatement extends SchemaAlteringStatement
         if (!baseName.getKeyspace().equals(keyspace()))
             throw new InvalidRequestException("Cannot create a materialized view on a table in a separate keyspace");
 
-        CFMetaData cfm = Validation.validateColumnFamily(baseName.getKeyspace(), baseName.getColumnFamily());
+        TableMetadata metadata = Schema.instance.validateTable(baseName.getKeyspace(), baseName.getColumnFamily());
 
-        if (cfm.isCounter())
+        if (metadata.isCounter())
             throw new InvalidRequestException("Materialized views are not supported on counter tables");
-        if (cfm.isView())
+        if (metadata.isView())
             throw new InvalidRequestException("Materialized views cannot be created against other materialized views");
 
-        if (cfm.params.gcGraceSeconds == 0)
+        if (metadata.params.gcGraceSeconds == 0)
         {
             throw new InvalidRequestException(String.format("Cannot create materialized view '%s' for base table " +
                                                             "'%s' with gc_grace_seconds of 0, since this value is " +
@@ -163,21 +163,21 @@ public class CreateViewStatement extends SchemaAlteringStatement
             if (selector.alias != null)
                 throw new InvalidRequestException("Cannot use alias when defining a materialized view");
 
-            Selectable s = selectable.prepare(cfm);
+            Selectable s = selectable.prepare(metadata);
             if (s instanceof Term.Raw)
                 throw new InvalidRequestException("Cannot use terms in selection when defining a materialized view");
 
-            ColumnDefinition cdef = (ColumnDefinition)s;
+            ColumnMetadata cdef = (ColumnMetadata)s;
             included.add(cdef.name);
         }
 
-        Set<ColumnDefinition.Raw> targetPrimaryKeys = new HashSet<>();
-        for (ColumnDefinition.Raw identifier : Iterables.concat(partitionKeys, clusteringKeys))
+        Set<ColumnMetadata.Raw> targetPrimaryKeys = new HashSet<>();
+        for (ColumnMetadata.Raw identifier : Iterables.concat(partitionKeys, clusteringKeys))
         {
             if (!targetPrimaryKeys.add(identifier))
                 throw new InvalidRequestException("Duplicate entry found in PRIMARY KEY: "+identifier);
 
-            ColumnDefinition cdef = identifier.prepare(cfm);
+            ColumnMetadata cdef = identifier.prepare(metadata);
 
             if (cdef.type.isMultiCell())
                 throw new InvalidRequestException(String.format("Cannot use MultiCell column '%s' in PRIMARY KEY of materialized view", identifier));
@@ -190,8 +190,8 @@ public class CreateViewStatement extends SchemaAlteringStatement
         }
 
         // build the select statement
-        Map<ColumnDefinition.Raw, Boolean> orderings = Collections.emptyMap();
-        List<ColumnDefinition.Raw> groups = Collections.emptyList();
+        Map<ColumnMetadata.Raw, Boolean> orderings = Collections.emptyMap();
+        List<ColumnMetadata.Raw> groups = Collections.emptyList();
         SelectStatement.Parameters parameters = new SelectStatement.Parameters(orderings, groups, false, true, false);
 
         SelectStatement.RawStatement rawSelect = new SelectStatement.RawStatement(baseName, parameters, selectClause, whereClause, null, null);
@@ -212,7 +212,7 @@ public class CreateViewStatement extends SchemaAlteringStatement
         String whereClauseText = View.relationsToWhereClause(whereClause.relations);
 
         Set<ColumnIdentifier> basePrimaryKeyCols = new HashSet<>();
-        for (ColumnDefinition definition : Iterables.concat(cfm.partitionKeyColumns(), cfm.clusteringColumns()))
+        for (ColumnMetadata definition : Iterables.concat(metadata.partitionKeyColumns(), metadata.clusteringColumns()))
             basePrimaryKeyCols.add(definition.name);
 
         List<ColumnIdentifier> targetClusteringColumns = new ArrayList<>();
@@ -220,11 +220,11 @@ public class CreateViewStatement extends SchemaAlteringStatement
 
         // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
         boolean hasNonPKColumn = false;
-        for (ColumnDefinition.Raw raw : partitionKeys)
-            hasNonPKColumn |= getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
+        for (ColumnMetadata.Raw raw : partitionKeys)
+            hasNonPKColumn |= getColumnIdentifier(metadata, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
 
-        for (ColumnDefinition.Raw raw : clusteringKeys)
-            hasNonPKColumn |= getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
+        for (ColumnMetadata.Raw raw : clusteringKeys)
+            hasNonPKColumn |= getColumnIdentifier(metadata, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
 
         // We need to include all of the primary key columns from the base table in order to make sure that we do not
         // overwrite values in the view. We cannot support "collapsing" the base table into a smaller number of rows in
@@ -234,7 +234,7 @@ public class CreateViewStatement extends SchemaAlteringStatement
         boolean missingClusteringColumns = false;
         StringBuilder columnNames = new StringBuilder();
         List<ColumnIdentifier> includedColumns = new ArrayList<>();
-        for (ColumnDefinition def : cfm.allColumns())
+        for (ColumnMetadata def : metadata.columns())
         {
             ColumnIdentifier identifier = def.name;
             boolean includeDef = included.isEmpty() || included.contains(identifier);
@@ -281,21 +281,23 @@ public class CreateViewStatement extends SchemaAlteringStatement
                                               "the corresponding data in the parent table.");
         }
 
-        CFMetaData.Builder cfmBuilder = CFMetaData.Builder.createView(keyspace(), columnFamily());
-        add(cfm, targetPartitionKeys, cfmBuilder::addPartitionKey);
-        add(cfm, targetClusteringColumns, cfmBuilder::addClusteringColumn);
-        add(cfm, includedColumns, cfmBuilder::addRegularColumn);
-        cfmBuilder.withId(properties.properties.getId());
-
-        CFMetaData viewCfm = cfmBuilder.build().params(params);
-        ViewDefinition definition = new ViewDefinition(keyspace(),
-                                                       columnFamily(),
-                                                       Schema.instance.getId(keyspace(), baseName.getColumnFamily()),
-                                                       baseName.getColumnFamily(),
-                                                       included.isEmpty(),
-                                                       rawSelect,
-                                                       whereClauseText,
-                                                       viewCfm);
+        TableMetadata.Builder builder =
+            TableMetadata.builder(keyspace(), columnFamily(), properties.properties.getId())
+                         .isView(true)
+                         .params(params);
+
+        add(metadata, targetPartitionKeys, builder::addPartitionKeyColumn);
+        add(metadata, targetClusteringColumns, builder::addClusteringColumn);
+        add(metadata, includedColumns, builder::addRegularColumn);
+
+        ViewMetadata definition = new ViewMetadata(keyspace(),
+                                                   columnFamily(),
+                                                   metadata.id,
+                                                   metadata.name,
+                                                   included.isEmpty(),
+                                                   rawSelect,
+                                                   whereClauseText,
+                                                   builder.build());
 
         try
         {
@@ -310,14 +312,14 @@ public class CreateViewStatement extends SchemaAlteringStatement
         }
     }
 
-    private static boolean getColumnIdentifier(CFMetaData cfm,
+    private static boolean getColumnIdentifier(TableMetadata cfm,
                                                Set<ColumnIdentifier> basePK,
                                                boolean hasNonPKColumn,
-                                               ColumnDefinition.Raw raw,
+                                               ColumnMetadata.Raw raw,
                                                List<ColumnIdentifier> columns,
                                                StatementRestrictions restrictions)
     {
-        ColumnDefinition def = raw.prepare(cfm);
+        ColumnMetadata def = raw.prepare(cfm);
 
         boolean isPk = basePK.contains(def.name);
         if (!isPk && hasNonPKColumn)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index 10f3d6f..e880bf8 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -19,8 +19,6 @@ package org.apache.cassandra.cql3.statements;
 
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.conditions.ColumnCondition;
 import org.apache.cassandra.cql3.conditions.Conditions;
@@ -29,6 +27,8 @@ import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.Slice;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.Pair;
 
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
@@ -40,7 +40,7 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
 public class DeleteStatement extends ModificationStatement
 {
     private DeleteStatement(int boundTerms,
-                            CFMetaData cfm,
+                            TableMetadata cfm,
                             Operations operations,
                             StatementRestrictions restrictions,
                             Conditions conditions,
@@ -53,6 +53,8 @@ public class DeleteStatement extends ModificationStatement
     public void addUpdateForKey(PartitionUpdate update, Clustering clustering, UpdateParameters params)
     throws InvalidRequestException
     {
+        TableMetadata metadata = metadata();
+
         List<Operation> regularDeletions = getRegularOperations();
         List<Operation> staticDeletions = getStaticOperations();
 
@@ -64,7 +66,7 @@ public class DeleteStatement extends ModificationStatement
                 update.addPartitionDeletion(params.deletionTime());
             }
             // ... or a row deletion ...
-            else if (clustering.size() == cfm.clusteringColumns().size())
+            else if (clustering.size() == metadata.clusteringColumns().size())
             {
                 params.newRow(clustering);
                 params.addRowDeletion();
@@ -73,7 +75,7 @@ public class DeleteStatement extends ModificationStatement
             // ... or a range of rows deletion.
             else
             {
-                update.add(params.makeRangeTombstone(cfm.comparator, clustering));
+                update.add(params.makeRangeTombstone(metadata.comparator, clustering));
             }
         }
         else
@@ -83,7 +85,7 @@ public class DeleteStatement extends ModificationStatement
                 // if the clustering size is zero but there are some clustering columns, it means that it's a
                 // range deletion (the full partition) in which case we need to throw an error as range deletion
                 // do not support specific columns
-                checkFalse(clustering.size() == 0 && cfm.clusteringColumns().size() != 0,
+                checkFalse(clustering.size() == 0 && metadata.clusteringColumns().size() != 0,
                            "Range deletions are not supported for specific columns");
 
                 params.newRow(clustering);
@@ -124,7 +126,7 @@ public class DeleteStatement extends ModificationStatement
                       Attributes.Raw attrs,
                       List<Operation.RawDeletion> deletions,
                       WhereClause whereClause,
-                      List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions,
+                      List<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>> conditions,
                       boolean ifExists)
         {
             super(name, StatementType.DELETE, attrs, conditions, false, ifExists);
@@ -134,7 +136,7 @@ public class DeleteStatement extends ModificationStatement
 
 
         @Override
-        protected ModificationStatement prepareInternal(CFMetaData cfm,
+        protected ModificationStatement prepareInternal(TableMetadata metadata,
                                                         VariableSpecifications boundNames,
                                                         Conditions conditions,
                                                         Attributes attrs)
@@ -143,25 +145,25 @@ public class DeleteStatement extends ModificationStatement
 
             for (Operation.RawDeletion deletion : deletions)
             {
-                ColumnDefinition def = getColumnDefinition(cfm, deletion.affectedColumn());
+                ColumnMetadata def = getColumnDefinition(metadata, deletion.affectedColumn());
 
                 // For compact, we only have one value except the key, so the only form of DELETE that make sense is without a column
                 // list. However, we support having the value name for coherence with the static/sparse case
                 checkFalse(def.isPrimaryKeyColumn(), "Invalid identifier %s for deletion (should not be a PRIMARY KEY part)", def.name);
 
-                Operation op = deletion.prepare(cfm.ksName, def, cfm);
+                Operation op = deletion.prepare(metadata.keyspace, def, metadata);
                 op.collectMarkerSpecification(boundNames);
                 operations.add(op);
             }
 
-            StatementRestrictions restrictions = newRestrictions(cfm,
+            StatementRestrictions restrictions = newRestrictions(metadata,
                                                                  boundNames,
                                                                  operations,
                                                                  whereClause,
                                                                  conditions);
 
             DeleteStatement stmt = new DeleteStatement(boundNames.size(),
-                                                       cfm,
+                                                       metadata,
                                                        operations,
                                                        restrictions,
                                                        conditions,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java
index 6f41af5..1a02903 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java
@@ -22,16 +22,15 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.cql3.Validation;
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -64,7 +63,7 @@ public final class DropAggregateStatement extends SchemaAlteringStatement
         if (!functionName.hasKeyspace())
             throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
 
-        Validation.validateKeyspaceNotSystem(functionName.keyspace);
+        Schema.validateKeyspaceNotSystem(functionName.keyspace);
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
@@ -92,7 +91,7 @@ public final class DropAggregateStatement extends SchemaAlteringStatement
         Function old = null;
         if (argsPresent)
         {
-            if (Schema.instance.getKSMetaData(functionName.keyspace) != null)
+            if (Schema.instance.getKeyspaceMetadata(functionName.keyspace) != null)
             {
                 List<AbstractType<?>> argTypes = new ArrayList<>(argRawTypes.size());
                 for (CQL3Type.Raw rawType : argRawTypes)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
index e098d0a..87bfff8 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
@@ -25,17 +25,16 @@ import com.google.common.base.Joiner;
 
 import org.apache.cassandra.auth.FunctionResource;
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.cql3.Validation;
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -65,7 +64,7 @@ public final class DropFunctionStatement extends SchemaAlteringStatement
     @Override
     public Prepared prepare() throws InvalidRequestException
     {
-        if (Schema.instance.getKSMetaData(functionName.keyspace) != null)
+        if (Schema.instance.getKeyspaceMetadata(functionName.keyspace) != null)
         {
             argTypes = new ArrayList<>(argRawTypes.size());
             for (CQL3Type.Raw rawType : argRawTypes)
@@ -94,7 +93,7 @@ public final class DropFunctionStatement extends SchemaAlteringStatement
         if (!functionName.hasKeyspace())
             throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
 
-        Validation.validateKeyspaceNotSystem(functionName.keyspace);
+        Schema.validateKeyspaceNotSystem(functionName.keyspace);
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
@@ -139,7 +138,7 @@ public final class DropFunctionStatement extends SchemaAlteringStatement
                 throw new InvalidRequestException(getMissingFunctionError());
         }
 
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(old.name().keyspace);
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(old.name().keyspace);
         Collection<UDAggregate> referrers = ksm.functions.aggregatesUsingFunction(old);
         if (!referrers.isEmpty())
             throw new InvalidRequestException(String.format("Function '%s' still referenced by %s", old, referrers));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
index fcd06d4..2420f3b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
@@ -18,15 +18,15 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.IndexName;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.db.KeyspaceNotDefinedException;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.transport.messages.ResultMessage;
@@ -45,17 +45,17 @@ public class DropIndexStatement extends SchemaAlteringStatement
 
     public String columnFamily()
     {
-        CFMetaData cfm = lookupIndexedTable();
-        return cfm == null ? null : cfm.cfName;
+        TableMetadata metadata = lookupIndexedTable();
+        return metadata == null ? null : metadata.name;
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
     {
-        CFMetaData cfm = lookupIndexedTable();
-        if (cfm == null)
+        TableMetadata metadata = lookupIndexedTable();
+        if (metadata == null)
             return;
 
-        state.hasColumnFamilyAccess(cfm.ksName, cfm.cfName, Permission.ALTER);
+        state.hasColumnFamilyAccess(metadata.keyspace, metadata.name, Permission.ALTER);
     }
 
     public void validate(ClientState state)
@@ -72,17 +72,20 @@ public class DropIndexStatement extends SchemaAlteringStatement
 
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
     {
-        CFMetaData cfm = lookupIndexedTable();
-        if (cfm == null)
+        TableMetadata current = lookupIndexedTable();
+        if (current == null)
             return null;
 
-        CFMetaData updatedCfm = cfm.copy();
-        updatedCfm.indexes(updatedCfm.getIndexes().without(indexName));
-        MigrationManager.announceColumnFamilyUpdate(updatedCfm, isLocalOnly);
+        TableMetadata updated =
+            current.unbuild()
+                   .indexes(current.indexes.without(indexName))
+                   .build();
+
+        MigrationManager.announceTableUpdate(updated, isLocalOnly);
         // Dropping an index is akin to updating the CF
         // Note that we shouldn't call columnFamily() at this point because the index has been dropped and the call to lookupIndexedTable()
         // in that method would now throw.
-        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, cfm.ksName, cfm.cfName);
+        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, current.keyspace, current.name);
     }
 
     /**
@@ -94,9 +97,9 @@ public class DropIndexStatement extends SchemaAlteringStatement
      * @throws InvalidRequestException if the index cannot be found and "IF EXISTS" is not
      * set on the statement.
      */
-    private CFMetaData lookupIndexedTable()
+    private TableMetadata lookupIndexedTable()
     {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace());
         if (ksm == null)
             throw new KeyspaceNotDefinedException("Keyspace " + keyspace() + " does not exist");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
index 6930891..21040f9 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropKeyspaceStatement.java
@@ -18,13 +18,13 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.Validation;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -47,7 +47,7 @@ public class DropKeyspaceStatement extends SchemaAlteringStatement
 
     public void validate(ClientState state) throws RequestValidationException
     {
-        Validation.validateKeyspaceNotSystem(keyspace);
+        Schema.validateKeyspaceNotSystem(keyspace);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
index 5641185..83b787e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
@@ -18,16 +18,16 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ViewDefinition;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CFName;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.ViewMetadata;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -63,25 +63,25 @@ public class DropTableStatement extends SchemaAlteringStatement
     {
         try
         {
-            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace());
+            KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace());
             if (ksm == null)
                 throw new ConfigurationException(String.format("Cannot drop table in unknown keyspace '%s'", keyspace()));
-            CFMetaData cfm = ksm.getTableOrViewNullable(columnFamily());
-            if (cfm != null)
+            TableMetadata metadata = ksm.getTableOrViewNullable(columnFamily());
+            if (metadata != null)
             {
-                if (cfm.isView())
+                if (metadata.isView())
                     throw new InvalidRequestException("Cannot use DROP TABLE on Materialized View");
 
                 boolean rejectDrop = false;
                 StringBuilder messageBuilder = new StringBuilder();
-                for (ViewDefinition def : ksm.views)
+                for (ViewMetadata def : ksm.views)
                 {
-                    if (def.baseTableId.equals(cfm.cfId))
+                    if (def.baseTableId.equals(metadata.id))
                     {
                         if (rejectDrop)
                             messageBuilder.append(',');
                         rejectDrop = true;
-                        messageBuilder.append(def.viewName);
+                        messageBuilder.append(def.name);
                     }
                 }
                 if (rejectDrop)
@@ -91,7 +91,7 @@ public class DropTableStatement extends SchemaAlteringStatement
                                                                     messageBuilder.toString()));
                 }
             }
-            MigrationManager.announceColumnFamilyDrop(keyspace(), columnFamily(), isLocalOnly);
+            MigrationManager.announceTableDrop(keyspace(), columnFamily(), isLocalOnly);
             return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
         }
         catch (ConfigurationException e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java
index 9342824..b9536fb 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java
@@ -20,17 +20,16 @@ package org.apache.cassandra.cql3.statements;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.cql3.Validation;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Triggers;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -56,13 +55,13 @@ public class DropTriggerStatement extends SchemaAlteringStatement
 
     public void validate(ClientState state) throws RequestValidationException
     {
-        Validation.validateColumnFamily(keyspace(), columnFamily());
+        Schema.instance.validateTable(keyspace(), columnFamily());
     }
 
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws ConfigurationException, InvalidRequestException
     {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).copy();
-        Triggers triggers = cfm.getTriggers();
+        TableMetadata current = Schema.instance.getTableMetadata(keyspace(), columnFamily());
+        Triggers triggers = current.triggers;
 
         if (!triggers.get(triggerName).isPresent())
         {
@@ -73,8 +72,14 @@ public class DropTriggerStatement extends SchemaAlteringStatement
         }
 
         logger.info("Dropping trigger with name {}", triggerName);
-        cfm.triggers(triggers.without(triggerName));
-        MigrationManager.announceColumnFamilyUpdate(cfm, isLocalOnly);
+
+        TableMetadata updated =
+            current.unbuild()
+                   .triggers(triggers.without(triggerName))
+                   .build();
+
+        MigrationManager.announceTableUpdate(updated, isLocalOnly);
+
         return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
index cd6daae..5909888 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
@@ -18,14 +18,16 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -54,7 +56,7 @@ public class DropTypeStatement extends SchemaAlteringStatement
 
     public void validate(ClientState state) throws RequestValidationException
     {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(name.getKeyspace());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name.getKeyspace());
         if (ksm == null)
         {
             if (ifExists)
@@ -92,10 +94,10 @@ public class DropTypeStatement extends SchemaAlteringStatement
             if (!ut.name.equals(name.getUserTypeName()) && ut.referencesUserType(name.getStringTypeName()))
                 throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by user type %s", name, ut.getNameAsString()));
 
-        for (CFMetaData cfm : ksm.tablesAndViews())
-            for (ColumnDefinition def : cfm.allColumns())
+        for (TableMetadata table : ksm.tablesAndViews())
+            for (ColumnMetadata def : table.columns())
                 if (def.type.referencesUserType(name.getStringTypeName()))
-                    throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by table %s.%s", name, cfm.ksName, cfm.cfName));
+                    throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by table %s", name, table.toString()));
     }
 
     @Override
@@ -106,7 +108,7 @@ public class DropTypeStatement extends SchemaAlteringStatement
 
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
     {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(name.getKeyspace());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name.getKeyspace());
         if (ksm == null)
             return null; // do not assert (otherwise IF EXISTS case fails)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
index 2f393d3..1909e89 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
@@ -19,14 +19,14 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.CFName;
 import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -42,9 +42,9 @@ public class DropViewStatement extends SchemaAlteringStatement
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
     {
-        CFMetaData baseTable = View.findBaseTable(keyspace(), columnFamily());
+        TableMetadataRef baseTable = View.findBaseTable(keyspace(), columnFamily());
         if (baseTable != null)
-            state.hasColumnFamilyAccess(keyspace(), baseTable.cfName, Permission.ALTER);
+            state.hasColumnFamilyAccess(keyspace(), baseTable.name, Permission.ALTER);
     }
 
     public void validate(ClientState state)
@@ -56,24 +56,6 @@ public class DropViewStatement extends SchemaAlteringStatement
     {
         try
         {
-//            ViewDefinition view = Schema.instance.getViewDefinition(keyspace(), columnFamily());
-//            if (view == null)
-//            {
-//                if (Schema.instance.getCFMetaData(keyspace(), columnFamily()) != null)
-//                    throw new ConfigurationException(String.format("Cannot drop table '%s' in keyspace '%s'.", columnFamily(), keyspace()));
-//
-//                throw new ConfigurationException(String.format("Cannot drop non existing materialized view '%s' in keyspace '%s'.", columnFamily(), keyspace()));
-//            }
-//
-//            CFMetaData baseCfm = Schema.instance.getCFMetaData(view.baseTableId);
-//            if (baseCfm == null)
-//            {
-//                if (ifExists)
-//                    throw new ConfigurationException(String.format("Cannot drop materialized view '%s' in keyspace '%s' without base CF.", columnFamily(), keyspace()));
-//                else
-//                    throw new InvalidRequestException(String.format("View '%s' could not be found in any of the tables of keyspace '%s'", cfName, keyspace()));
-//            }
-
             MigrationManager.announceViewDrop(keyspace(), columnFamily(), isLocalOnly);
             return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java b/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
index 84af273..12d8022 100644
--- a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
+++ b/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
@@ -19,8 +19,11 @@ package org.apache.cassandra.cql3.statements;
 
 import java.util.regex.Pattern;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 
 public class IndexTarget
@@ -28,93 +31,72 @@ public class IndexTarget
     public static final String TARGET_OPTION_NAME = "target";
     public static final String CUSTOM_INDEX_OPTION_NAME = "class_name";
 
-    /**
-     * The name of the option used to specify that the index is on the collection keys.
-     */
-    public static final String INDEX_KEYS_OPTION_NAME = "index_keys";
-
-    /**
-     * The name of the option used to specify that the index is on the collection (map) entries.
-     */
-    public static final String INDEX_ENTRIES_OPTION_NAME = "index_keys_and_values";
-
-    /**
-     * Regex for *unquoted* column names, anything which does not match this pattern must be a quoted name
-     */
-    private static final Pattern COLUMN_IDENTIFIER_PATTERN = Pattern.compile("[a-z_0-9]+");
-
     public final ColumnIdentifier column;
-    public final boolean quoteName;
     public final Type type;
 
     public IndexTarget(ColumnIdentifier column, Type type)
     {
         this.column = column;
         this.type = type;
-
-        // if the column name contains anything other than lower case alphanumerics
-        // or underscores, then it must be quoted when included in the target string
-        quoteName = !COLUMN_IDENTIFIER_PATTERN.matcher(column.toString()).matches();
     }
 
-    public String asCqlString(CFMetaData cfm)
+    public String asCqlString()
     {
-        if (!cfm.getColumnDefinition(column).type.isCollection())
-            return column.toCQLString();
-
-        return String.format("%s(%s)", type.toString(), column.toCQLString());
+        return type == Type.SIMPLE
+               ? column.toCQLString()
+               : String.format("%s(%s)", type.toString(), column.toCQLString());
     }
 
     public static class Raw
     {
-        private final ColumnDefinition.Raw column;
+        private final ColumnMetadata.Raw column;
         private final Type type;
 
-        private Raw(ColumnDefinition.Raw column, Type type)
+        private Raw(ColumnMetadata.Raw column, Type type)
         {
             this.column = column;
             this.type = type;
         }
 
-        public static Raw simpleIndexOn(ColumnDefinition.Raw c)
+        public static Raw simpleIndexOn(ColumnMetadata.Raw c)
         {
             return new Raw(c, Type.SIMPLE);
         }
 
-        public static Raw valuesOf(ColumnDefinition.Raw c)
+        public static Raw valuesOf(ColumnMetadata.Raw c)
         {
             return new Raw(c, Type.VALUES);
         }
 
-        public static Raw keysOf(ColumnDefinition.Raw c)
+        public static Raw keysOf(ColumnMetadata.Raw c)
         {
             return new Raw(c, Type.KEYS);
         }
 
-        public static Raw keysAndValuesOf(ColumnDefinition.Raw c)
+        public static Raw keysAndValuesOf(ColumnMetadata.Raw c)
         {
             return new Raw(c, Type.KEYS_AND_VALUES);
         }
 
-        public static Raw fullCollection(ColumnDefinition.Raw c)
+        public static Raw fullCollection(ColumnMetadata.Raw c)
         {
             return new Raw(c, Type.FULL);
         }
 
-        public IndexTarget prepare(CFMetaData cfm)
+        public IndexTarget prepare(TableMetadata table)
         {
             // Until we've prepared the target column, we can't be certain about the target type
             // because (for backwards compatibility) an index on a collection's values uses the
             // same syntax as an index on a regular column (i.e. the 'values' in
             // 'CREATE INDEX on table(values(collection));' is optional). So we correct the target type
             // when the target column is a collection & the target type is SIMPLE.
-            ColumnDefinition columnDef = column.prepare(cfm);
+            ColumnMetadata columnDef = column.prepare(table);
             Type actualType = (type == Type.SIMPLE && columnDef.type.isCollection()) ? Type.VALUES : type;
             return new IndexTarget(columnDef.name, actualType);
         }
     }
 
-    public static enum Type
+    public enum Type
     {
         VALUES, KEYS, KEYS_AND_VALUES, FULL, SIMPLE;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/KeyspaceAttributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/KeyspaceAttributes.java b/src/java/org/apache/cassandra/cql3/statements/KeyspaceAttributes.java
index db6b0d6..d2280ce 100644
--- a/src/java/org/apache/cassandra/cql3/statements/KeyspaceAttributes.java
+++ b/src/java/org/apache/cassandra/cql3/statements/KeyspaceAttributes.java
@@ -21,6 +21,7 @@ import java.util.*;
 
 import com.google.common.collect.ImmutableSet;
 
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.KeyspaceParams.Option;
 import org.apache.cassandra.schema.ReplicationParams;
@@ -42,6 +43,10 @@ public final class KeyspaceAttributes extends PropertyDefinitions
     public void validate()
     {
         validate(validKeywords, obsoleteKeywords);
+
+        Map<String, String> replicationOptions = getAllReplicationOptions();
+        if (!replicationOptions.isEmpty() && !replicationOptions.containsKey(ReplicationParams.CLASS))
+            throw new ConfigurationException("Missing replication strategy class");
     }
 
     public String getReplicationStrategyClass()
@@ -78,4 +83,9 @@ public final class KeyspaceAttributes extends PropertyDefinitions
                                       : ReplicationParams.fromMap(getAllReplicationOptions());
         return new KeyspaceParams(durableWrites, replication);
     }
+
+    public boolean hasOption(Option option)
+    {
+        return hasProperty(option.toString());
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
index b8f2f92..be7fb5d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
@@ -21,7 +21,7 @@ import java.util.*;
 
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.InvalidRequestException;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
index 3fee57a..0c0822c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
@@ -26,7 +26,7 @@ import com.google.common.collect.Lists;
 
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.marshal.BooleanType;
 import org.apache.cassandra.db.marshal.MapType;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
index 0101363..9641333 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
@@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableList;
 
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.ResultSet;


[07/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
index a037d90..8341e30 100644
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexTest.java
@@ -30,7 +30,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.cql3.statements.IndexTarget;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -40,6 +40,8 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -81,10 +83,10 @@ public class SecondaryIndexTest
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(WITH_COMPOSITE_INDEX);
 
-        new RowUpdateBuilder(cfs.metadata, 0, "k1").clustering("c").add("birthdate", 1L).add("notbirthdate", 1L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "k2").clustering("c").add("birthdate", 2L).add("notbirthdate", 2L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "k3").clustering("c").add("birthdate", 1L).add("notbirthdate", 2L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "k4").clustering("c").add("birthdate", 3L).add("notbirthdate", 2L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "k1").clustering("c").add("birthdate", 1L).add("notbirthdate", 1L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "k2").clustering("c").add("birthdate", 2L).add("notbirthdate", 2L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "k3").clustering("c").add("birthdate", 1L).add("notbirthdate", 2L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "k4").clustering("c").add("birthdate", 3L).add("notbirthdate", 2L).build().applyUnsafe();
 
         // basic single-expression query
         List<FilteredPartition> partitions = Util.getAll(Util.cmd(cfs).fromKeyExcl("k1").toKeyIncl("k3").columns("birthdate").build());
@@ -157,7 +159,7 @@ public class SecondaryIndexTest
 
         for (int i = 0; i < 100; i++)
         {
-            new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros(), "key" + i)
+            new RowUpdateBuilder(cfs.metadata(), FBUtilities.timestampMicros(), "key" + i)
                     .clustering("c")
                     .add("birthdate", 34L)
                     .add("notbirthdate", ByteBufferUtil.bytes((long) (i % 2)))
@@ -189,15 +191,15 @@ public class SecondaryIndexTest
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(WITH_COMPOSITE_INDEX);
         ByteBuffer bBB = ByteBufferUtil.bytes("birthdate");
-        ColumnDefinition bDef = cfs.metadata.getColumnDefinition(bBB);
+        ColumnMetadata bDef = cfs.metadata().getColumn(bBB);
         ByteBuffer col = ByteBufferUtil.bytes("birthdate");
 
         // Confirm addition works
-        new RowUpdateBuilder(cfs.metadata, 0, "k1").clustering("c").add("birthdate", 1L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "k1").clustering("c").add("birthdate", 1L).build().applyUnsafe();
         assertIndexedOne(cfs, col, 1L);
 
         // delete the column directly
-        RowUpdateBuilder.deleteRow(cfs.metadata, 1, "k1", "c").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), 1, "k1", "c").applyUnsafe();
         assertIndexedNone(cfs, col, 1L);
 
         // verify that it's not being indexed under any other value either
@@ -205,26 +207,26 @@ public class SecondaryIndexTest
         assertNull(cfs.indexManager.getBestIndexFor(rc));
 
         // resurrect w/ a newer timestamp
-        new RowUpdateBuilder(cfs.metadata, 2, "k1").clustering("c").add("birthdate", 1L).build().apply();;
+        new RowUpdateBuilder(cfs.metadata(), 2, "k1").clustering("c").add("birthdate", 1L).build().apply();;
         assertIndexedOne(cfs, col, 1L);
 
         // verify that row and delete w/ older timestamp does nothing
-        RowUpdateBuilder.deleteRow(cfs.metadata, 1, "k1", "c").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), 1, "k1", "c").applyUnsafe();
         assertIndexedOne(cfs, col, 1L);
 
         // similarly, column delete w/ older timestamp should do nothing
-        new RowUpdateBuilder(cfs.metadata, 1, "k1").clustering("c").delete(bDef).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, "k1").clustering("c").delete(bDef).build().applyUnsafe();
         assertIndexedOne(cfs, col, 1L);
 
         // delete the entire row (w/ newer timestamp this time)
         // todo - checking the # of index searchers for the command is probably not the best thing to test here
-        RowUpdateBuilder.deleteRow(cfs.metadata, 3, "k1", "c").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), 3, "k1", "c").applyUnsafe();
         rc = Util.cmd(cfs).build();
         assertNull(cfs.indexManager.getBestIndexFor(rc));
 
         // make sure obsolete mutations don't generate an index entry
         // todo - checking the # of index searchers for the command is probably not the best thing to test here
-        new RowUpdateBuilder(cfs.metadata, 3, "k1").clustering("c").add("birthdate", 1L).build().apply();;
+        new RowUpdateBuilder(cfs.metadata(), 3, "k1").clustering("c").add("birthdate", 1L).build().apply();;
         rc = Util.cmd(cfs).build();
         assertNull(cfs.indexManager.getBestIndexFor(rc));
     }
@@ -237,8 +239,8 @@ public class SecondaryIndexTest
         ByteBuffer col = ByteBufferUtil.bytes("birthdate");
 
         // create a row and update the birthdate value, test that the index query fetches the new version
-        new RowUpdateBuilder(cfs.metadata, 1, "testIndexUpdate").clustering("c").add("birthdate", 100L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 2, "testIndexUpdate").clustering("c").add("birthdate", 200L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, "testIndexUpdate").clustering("c").add("birthdate", 100L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 2, "testIndexUpdate").clustering("c").add("birthdate", 200L).build().applyUnsafe();
 
         // Confirm old version fetch fails
         assertIndexedNone(cfs, col, 100L);
@@ -259,23 +261,23 @@ public class SecondaryIndexTest
         ByteBuffer col = ByteBufferUtil.bytes("birthdate");
 
         // create a row and update the birthdate value with an expiring column
-        new RowUpdateBuilder(cfs.metadata, 1L, 500, "K100").clustering("c").add("birthdate", 100L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1L, 500, "K100").clustering("c").add("birthdate", 100L).build().applyUnsafe();
         assertIndexedOne(cfs, col, 100L);
 
         // requires a 1s sleep because we calculate local expiry time as (now() / 1000) + ttl
         TimeUnit.SECONDS.sleep(1);
 
         // now overwrite with the same name/value/ttl, but the local expiry time will be different
-        new RowUpdateBuilder(cfs.metadata, 1L, 500, "K100").clustering("c").add("birthdate", 100L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1L, 500, "K100").clustering("c").add("birthdate", 100L).build().applyUnsafe();
         assertIndexedOne(cfs, col, 100L);
 
         // check that modifying the indexed value using the same timestamp behaves as expected
-        new RowUpdateBuilder(cfs.metadata, 1L, 500, "K101").clustering("c").add("birthdate", 101L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1L, 500, "K101").clustering("c").add("birthdate", 101L).build().applyUnsafe();
         assertIndexedOne(cfs, col, 101L);
 
         TimeUnit.SECONDS.sleep(1);
 
-        new RowUpdateBuilder(cfs.metadata, 1L, 500, "K101").clustering("c").add("birthdate", 102L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1L, 500, "K101").clustering("c").add("birthdate", 102L).build().applyUnsafe();
         // Confirm 101 is gone
         assertIndexedNone(cfs, col, 101L);
 
@@ -292,13 +294,13 @@ public class SecondaryIndexTest
         ByteBuffer col = ByteBufferUtil.bytes("birthdate");
 
         // create a row and update the "birthdate" value
-        new RowUpdateBuilder(cfs.metadata, 1, "k1").noRowMarker().add("birthdate", 1L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, "k1").noRowMarker().add("birthdate", 1L).build().applyUnsafe();
 
         // force a flush, so our index isn't being read from a memtable
         keyspace.getColumnFamilyStore(WITH_KEYS_INDEX).forceBlockingFlush();
 
         // now apply another update, but force the index update to be skipped
-        keyspace.apply(new RowUpdateBuilder(cfs.metadata, 2, "k1").noRowMarker().add("birthdate", 2L).build(),
+        keyspace.apply(new RowUpdateBuilder(cfs.metadata(), 2, "k1").noRowMarker().add("birthdate", 2L).build(),
                        true,
                        false);
 
@@ -311,7 +313,7 @@ public class SecondaryIndexTest
 
         // now, reset back to the original value, still skipping the index update, to
         // make sure the value was expunged from the index when it was discovered to be inconsistent
-        keyspace.apply(new RowUpdateBuilder(cfs.metadata, 3, "k1").noRowMarker().add("birthdate", 1L).build(),
+        keyspace.apply(new RowUpdateBuilder(cfs.metadata(), 3, "k1").noRowMarker().add("birthdate", 1L).build(),
                        true,
                        false);
         assertIndexedNone(cfs, col, 1L);
@@ -342,7 +344,7 @@ public class SecondaryIndexTest
         ByteBuffer col = ByteBufferUtil.bytes(colName);
 
         // create a row and update the author value
-        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 0, "k1");
+        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 0, "k1");
         if (!isStatic)
             builder = builder.clustering("c");
         builder.add(colName, 10l).build().applyUnsafe();
@@ -355,7 +357,7 @@ public class SecondaryIndexTest
         assertIndexedOne(cfs, col, 10l);
 
         // now apply another update, but force the index update to be skipped
-        builder = new RowUpdateBuilder(cfs.metadata, 0, "k1");
+        builder = new RowUpdateBuilder(cfs.metadata(), 0, "k1");
         if (!isStatic)
             builder = builder.clustering("c");
         builder.add(colName, 20l);
@@ -371,7 +373,7 @@ public class SecondaryIndexTest
         // now, reset back to the original value, still skipping the index update, to
         // make sure the value was expunged from the index when it was discovered to be inconsistent
         // TODO: Figure out why this is re-inserting
-        builder = new RowUpdateBuilder(cfs.metadata, 2, "k1");
+        builder = new RowUpdateBuilder(cfs.metadata(), 2, "k1");
         if (!isStatic)
             builder = builder.clustering("c");
         builder.add(colName, 10L);
@@ -391,10 +393,10 @@ public class SecondaryIndexTest
         ByteBuffer colName = ByteBufferUtil.bytes("birthdate");
 
         // Insert indexed value.
-        new RowUpdateBuilder(cfs.metadata, 1, "k1").clustering("c").add("birthdate", 10l).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, "k1").clustering("c").add("birthdate", 10l).build().applyUnsafe();
 
         // Now delete the value
-        RowUpdateBuilder.deleteRow(cfs.metadata, 2, "k1", "c").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), 2, "k1", "c").applyUnsafe();
 
         // We want the data to be gcable, but even if gcGrace == 0, we still need to wait 1 second
         // since we won't gc on a tie.
@@ -414,10 +416,10 @@ public class SecondaryIndexTest
         ByteBuffer colName = ByteBufferUtil.bytes("birthdate");
 
         // Insert indexed value.
-        new RowUpdateBuilder(cfs.metadata, 1, "k1").add("birthdate", 10l).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 1, "k1").add("birthdate", 10l).build().applyUnsafe();
 
         // Now delete the value
-        RowUpdateBuilder.deleteRow(cfs.metadata, 2, "k1").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), 2, "k1").applyUnsafe();
 
         // We want the data to be gcable, but even if gcGrace == 0, we still need to wait 1 second
         // since we won't gc on a tie.
@@ -436,14 +438,14 @@ public class SecondaryIndexTest
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(WITH_COMPOSITE_INDEX);
         Mutation rm;
 
-        new RowUpdateBuilder(cfs.metadata, 0, "kk1").clustering("c").add("birthdate", 1L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "kk1").clustering("c").add("notbirthdate", 1L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "kk2").clustering("c").add("birthdate", 1L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "kk2").clustering("c").add("notbirthdate", 2L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "kk3").clustering("c").add("birthdate", 1L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "kk3").clustering("c").add("notbirthdate", 2L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "kk4").clustering("c").add("birthdate", 1L).build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "kk4").clustering("c").add("notbirthdate", 2L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "kk1").clustering("c").add("birthdate", 1L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "kk1").clustering("c").add("notbirthdate", 1L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "kk2").clustering("c").add("birthdate", 1L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "kk2").clustering("c").add("notbirthdate", 2L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "kk3").clustering("c").add("birthdate", 1L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "kk3").clustering("c").add("notbirthdate", 2L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "kk4").clustering("c").add("birthdate", 1L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "kk4").clustering("c").add("notbirthdate", 2L).build().applyUnsafe();
 
         // basic single-expression query, limit 1
         Util.getOnlyRow(Util.cmd(cfs)
@@ -460,28 +462,39 @@ public class SecondaryIndexTest
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COMPOSITE_INDEX_TO_BE_ADDED);
 
         // create a row and update the birthdate value, test that the index query fetches the new version
-        new RowUpdateBuilder(cfs.metadata, 0, "k1").clustering("c").add("birthdate", 1L).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "k1").clustering("c").add("birthdate", 1L).build().applyUnsafe();
 
         String indexName = "birthdate_index";
-        ColumnDefinition old = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("birthdate"));
+        ColumnMetadata old = cfs.metadata().getColumn(ByteBufferUtil.bytes("birthdate"));
         IndexMetadata indexDef =
-            IndexMetadata.fromIndexTargets(cfs.metadata,
-                                           Collections.singletonList(new IndexTarget(old.name, IndexTarget.Type.VALUES)),
+            IndexMetadata.fromIndexTargets(
+            Collections.singletonList(new IndexTarget(old.name, IndexTarget.Type.VALUES)),
                                            indexName,
                                            IndexMetadata.Kind.COMPOSITES,
                                            Collections.EMPTY_MAP);
-        cfs.metadata.indexes(cfs.metadata.getIndexes().with(indexDef));
-        Future<?> future = cfs.indexManager.addIndex(indexDef);
-        future.get();
+
+        TableMetadata current = cfs.metadata();
+
+        TableMetadata updated =
+            current.unbuild()
+                   .indexes(current.indexes.with(indexDef))
+                   .build();
+        MigrationManager.announceTableUpdate(updated, true);
+
+        // fait for the index to be built
+        Index index = cfs.indexManager.getIndex(indexDef);
+        do
+        {
+            TimeUnit.MILLISECONDS.sleep(100);
+        }
+        while (!cfs.indexManager.isIndexQueryable(index));
 
         // we had a bug (CASSANDRA-2244) where index would get created but not flushed -- check for that
         // the way we find the index cfs is a bit convoluted at the moment
-        boolean flushed = false;
         ColumnFamilyStore indexCfs = cfs.indexManager.getIndex(indexDef)
                                                      .getBackingTable()
                                                      .orElseThrow(throwAssert("Index not found"));
-        flushed = !indexCfs.getLiveSSTables().isEmpty();
-        assertTrue(flushed);
+        assertFalse(indexCfs.getLiveSSTables().isEmpty());
         assertIndexedOne(cfs, ByteBufferUtil.bytes("birthdate"), 1L);
 
         // validate that drop clears it out & rebuild works (CASSANDRA-2320)
@@ -490,7 +503,7 @@ public class SecondaryIndexTest
         assertFalse(cfs.getBuiltIndexes().contains(indexName));
 
         // rebuild & re-query
-        future = cfs.indexManager.addIndex(indexDef);
+        Future future = cfs.indexManager.addIndex(indexDef);
         future.get();
         assertIndexedOne(cfs, ByteBufferUtil.bytes("birthdate"), 1L);
     }
@@ -503,7 +516,7 @@ public class SecondaryIndexTest
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(WITH_KEYS_INDEX);
 
         for (int i = 0; i < 10; i++)
-            new RowUpdateBuilder(cfs.metadata, 0, "k" + i).noRowMarker().add("birthdate", 1l).build().applyUnsafe();
+            new RowUpdateBuilder(cfs.metadata(), 0, "k" + i).noRowMarker().add("birthdate", 1l).build().applyUnsafe();
 
         assertIndexedCount(cfs, ByteBufferUtil.bytes("birthdate"), 1l, 10);
         cfs.forceBlockingFlush();
@@ -520,7 +533,7 @@ public class SecondaryIndexTest
     }
     private void assertIndexedCount(ColumnFamilyStore cfs, ByteBuffer col, Object val, int count)
     {
-        ColumnDefinition cdef = cfs.metadata.getColumnDefinition(col);
+        ColumnMetadata cdef = cfs.metadata().getColumn(col);
 
         ReadCommand rc = Util.cmd(cfs).filterOn(cdef.name.toString(), Operator.EQ, ((AbstractType) cdef.cellValueType()).decompose(val)).build();
         Index.Searcher searcher = cfs.indexManager.getBestIndexFor(rc).searcherFor(rc);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
index f89aa8d..1d4bdb6 100644
--- a/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
+++ b/test/unit/org/apache/cassandra/db/SinglePartitionSliceCommandTest.java
@@ -21,8 +21,6 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
 import java.util.Iterator;
 
 import org.junit.Assert;
@@ -34,13 +32,12 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.filter.DataLimits;
@@ -67,27 +64,27 @@ public class SinglePartitionSliceCommandTest
     private static final String KEYSPACE = "ks";
     private static final String TABLE = "tbl";
 
-    private static CFMetaData cfm;
-    private static ColumnDefinition v;
-    private static ColumnDefinition s;
+    private static TableMetadata metadata;
+    private static ColumnMetadata v;
+    private static ColumnMetadata s;
 
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
         DatabaseDescriptor.daemonInitialization();
 
-        cfm = CFMetaData.Builder.create(KEYSPACE, TABLE)
-                                .addPartitionKey("k", UTF8Type.instance)
-                                .addStaticColumn("s", UTF8Type.instance)
-                                .addClusteringColumn("i", IntegerType.instance)
-                                .addRegularColumn("v", UTF8Type.instance)
-                                .build();
+        metadata =
+            TableMetadata.builder(KEYSPACE, TABLE)
+                         .addPartitionKeyColumn("k", UTF8Type.instance)
+                         .addStaticColumn("s", UTF8Type.instance)
+                         .addClusteringColumn("i", IntegerType.instance)
+                         .addRegularColumn("v", UTF8Type.instance)
+                         .build();
 
         SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), cfm);
-        cfm = Schema.instance.getCFMetaData(KEYSPACE, TABLE);
-        v = cfm.getColumnDefinition(new ColumnIdentifier("v", true));
-        s = cfm.getColumnDefinition(new ColumnIdentifier("s", true));
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), metadata);
+        v = metadata.getColumn(new ColumnIdentifier("v", true));
+        s = metadata.getColumn(new ColumnIdentifier("s", true));
     }
 
     @Before
@@ -103,7 +100,7 @@ public class SinglePartitionSliceCommandTest
         Assert.assertTrue(ri.columns().contains(s));
         Row staticRow = ri.staticRow();
         Iterator<Cell> cellIterator = staticRow.cells().iterator();
-        Assert.assertTrue(staticRow.toString(cfm, true), cellIterator.hasNext());
+        Assert.assertTrue(staticRow.toString(metadata, true), cellIterator.hasNext());
         Cell cell = cellIterator.next();
         Assert.assertEquals(s, cell.column());
         Assert.assertEquals(ByteBufferUtil.bytesToHex(cell.value()), ByteBufferUtil.bytes("s"), cell.value());
@@ -113,14 +110,14 @@ public class SinglePartitionSliceCommandTest
     @Test
     public void staticColumnsAreReturned() throws IOException
     {
-        DecoratedKey key = cfm.decorateKey(ByteBufferUtil.bytes("k1"));
+        DecoratedKey key = metadata.partitioner.decorateKey(ByteBufferUtil.bytes("k1"));
 
         QueryProcessor.executeInternal("INSERT INTO ks.tbl (k, s) VALUES ('k1', 's')");
         Assert.assertFalse(QueryProcessor.executeInternal("SELECT s FROM ks.tbl WHERE k='k1'").isEmpty());
 
-        ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
+        ColumnFilter columnFilter = ColumnFilter.selection(RegularAndStaticColumns.of(s));
         ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.NONE, false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, cfm,
+        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, metadata,
                                                          FBUtilities.nowInSeconds(),
                                                          columnFilter,
                                                          RowFilter.NONE,
@@ -155,7 +152,7 @@ public class SinglePartitionSliceCommandTest
         }
 
         // check (de)serialized iterator for sstable static cell
-        Schema.instance.getColumnFamilyStoreInstance(cfm.cfId).forceBlockingFlush();
+        Schema.instance.getColumnFamilyStoreInstance(metadata.id).forceBlockingFlush();
         try (ReadExecutionController executionController = cmd.executionController(); UnfilteredPartitionIterator pi = cmd.executeLocally(executionController))
         {
             response = ReadResponse.createDataResponse(pi, cmd);
@@ -173,12 +170,12 @@ public class SinglePartitionSliceCommandTest
     @Test
     public void toCQLStringIsSafeToCall() throws IOException
     {
-        DecoratedKey key = cfm.decorateKey(ByteBufferUtil.bytes("k1"));
+        DecoratedKey key = metadata.partitioner.decorateKey(ByteBufferUtil.bytes("k1"));
 
-        ColumnFilter columnFilter = ColumnFilter.selection(PartitionColumns.of(s));
+        ColumnFilter columnFilter = ColumnFilter.selection(RegularAndStaticColumns.of(s));
         Slice slice = Slice.make(ClusteringBound.BOTTOM, ClusteringBound.inclusiveEndOf(ByteBufferUtil.bytes("i1")));
-        ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(cfm.comparator, slice), false);
-        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, cfm,
+        ClusteringIndexSliceFilter sliceFilter = new ClusteringIndexSliceFilter(Slices.with(metadata.comparator, slice), false);
+        ReadCommand cmd = new SinglePartitionReadCommand(false, MessagingService.VERSION_30, metadata,
                                                          FBUtilities.nowInSeconds(),
                                                          columnFilter,
                                                          RowFilter.NONE,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index d188821..feac1e9 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -30,7 +30,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/TransformerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TransformerTest.java b/test/unit/org/apache/cassandra/db/TransformerTest.java
index fe87af8..eda3a52 100644
--- a/test/unit/org/apache/cassandra/db/TransformerTest.java
+++ b/test/unit/org/apache/cassandra/db/TransformerTest.java
@@ -25,7 +25,7 @@ import org.junit.Test;
 
 import junit.framework.Assert;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.Int32Type;
@@ -44,18 +44,18 @@ public class TransformerTest
         DatabaseDescriptor.daemonInitialization();
     }
 
-    static final CFMetaData metadata = metadata();
+    static final TableMetadata metadata = metadata();
     static final DecoratedKey partitionKey = new BufferDecoratedKey(new Murmur3Partitioner.LongToken(0L), ByteBufferUtil.EMPTY_BYTE_BUFFER);
-    static final Row staticRow = BTreeRow.singleCellRow(Clustering.STATIC_CLUSTERING, new BufferCell(metadata.partitionColumns().columns(true).getSimple(0), 0L, 0, 0, ByteBufferUtil.bytes(-1), null));
+    static final Row staticRow = BTreeRow.singleCellRow(Clustering.STATIC_CLUSTERING, new BufferCell(metadata.regularAndStaticColumns().columns(true).getSimple(0), 0L, 0, 0, ByteBufferUtil.bytes(-1), null));
 
-    static CFMetaData metadata()
+    static TableMetadata metadata()
     {
-        CFMetaData.Builder builder = CFMetaData.Builder.create("", "");
-        builder.addPartitionKey("pk", BytesType.instance);
-        builder.addClusteringColumn("c", Int32Type.instance);
-        builder.addStaticColumn("s", Int32Type.instance);
-        builder.addRegularColumn("v", Int32Type.instance);
-        return builder.build();
+        return TableMetadata.builder("", "")
+                            .addPartitionKeyColumn("pk", BytesType.instance)
+                            .addClusteringColumn("c", Int32Type.instance)
+                            .addStaticColumn("s", Int32Type.instance)
+                            .addRegularColumn("v", Int32Type.instance)
+                            .build();
     }
 
     // Mock Data
@@ -78,7 +78,7 @@ public class TransformerTest
             return (U) row(i);
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
             return metadata;
         }
@@ -88,9 +88,9 @@ public class TransformerTest
             return false;
         }
 
-        public PartitionColumns columns()
+        public RegularAndStaticColumns columns()
         {
-            return metadata.partitionColumns();
+            return metadata.regularAndStaticColumns();
         }
 
         public DecoratedKey partitionKey()
@@ -150,7 +150,7 @@ public class TransformerTest
     private static Row row(int i)
     {
         return BTreeRow.singleCellRow(Util.clustering(metadata.comparator, i),
-                                      new BufferCell(metadata.partitionColumns().columns(false).getSimple(0), 1L, BufferCell.NO_TTL, BufferCell.NO_DELETION_TIME, ByteBufferUtil.bytes(i), null));
+                                      new BufferCell(metadata.regularAndStaticColumns().columns(false).getSimple(0), 1L, BufferCell.NO_TTL, BufferCell.NO_DELETION_TIME, ByteBufferUtil.bytes(i), null));
     }
 
     // Transformations that check mock data ranges

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/VerifyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/VerifyTest.java b/test/unit/org/apache/cassandra/db/VerifyTest.java
index 77096b9..b77d56d 100644
--- a/test/unit/org/apache/cassandra/db/VerifyTest.java
+++ b/test/unit/org/apache/cassandra/db/VerifyTest.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.db;
 import com.google.common.base.Charsets;
 
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.ChunkCache;
 import org.apache.cassandra.UpdateBuilder;
@@ -48,6 +47,10 @@ import java.nio.file.Files;
 import java.util.zip.CRC32;
 import java.util.zip.CheckedInputStream;
 
+import static org.apache.cassandra.SchemaLoader.counterCFMD;
+import static org.apache.cassandra.SchemaLoader.createKeyspace;
+import static org.apache.cassandra.SchemaLoader.loadSchema;
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
 import static org.junit.Assert.fail;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
@@ -74,22 +77,22 @@ public class VerifyTest
     {
         CompressionParams compressionParameters = CompressionParams.snappy(32768);
 
-        SchemaLoader.loadSchema();
-        SchemaLoader.createKeyspace(KEYSPACE,
-                                    KeyspaceParams.simple(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CF).compression(compressionParameters),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CF2).compression(compressionParameters),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CF3),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CF4),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CORRUPT_CF),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CORRUPT_CF2),
-                                    SchemaLoader.counterCFMD(KEYSPACE, COUNTER_CF).compression(compressionParameters),
-                                    SchemaLoader.counterCFMD(KEYSPACE, COUNTER_CF2).compression(compressionParameters),
-                                    SchemaLoader.counterCFMD(KEYSPACE, COUNTER_CF3),
-                                    SchemaLoader.counterCFMD(KEYSPACE, COUNTER_CF4),
-                                    SchemaLoader.counterCFMD(KEYSPACE, CORRUPTCOUNTER_CF),
-                                    SchemaLoader.counterCFMD(KEYSPACE, CORRUPTCOUNTER_CF2),
-                                    SchemaLoader.standardCFMD(KEYSPACE, CF_UUID, 0, UUIDType.instance));
+        loadSchema();
+        createKeyspace(KEYSPACE,
+                       KeyspaceParams.simple(1),
+                       standardCFMD(KEYSPACE, CF).compression(compressionParameters),
+                       standardCFMD(KEYSPACE, CF2).compression(compressionParameters),
+                       standardCFMD(KEYSPACE, CF3),
+                       standardCFMD(KEYSPACE, CF4),
+                       standardCFMD(KEYSPACE, CORRUPT_CF),
+                       standardCFMD(KEYSPACE, CORRUPT_CF2),
+                       counterCFMD(KEYSPACE, COUNTER_CF).compression(compressionParameters),
+                       counterCFMD(KEYSPACE, COUNTER_CF2).compression(compressionParameters),
+                       counterCFMD(KEYSPACE, COUNTER_CF3),
+                       counterCFMD(KEYSPACE, COUNTER_CF4),
+                       counterCFMD(KEYSPACE, CORRUPTCOUNTER_CF),
+                       counterCFMD(KEYSPACE, CORRUPTCOUNTER_CF2),
+                       standardCFMD(KEYSPACE, CF_UUID, 0, UUIDType.instance));
     }
 
 
@@ -352,7 +355,7 @@ public class VerifyTest
     {
         for (int i = 0; i < partitionsPerSSTable; i++)
         {
-            UpdateBuilder.create(cfs.metadata, String.valueOf(i))
+            UpdateBuilder.create(cfs.metadata(), String.valueOf(i))
                          .newRow("c1").add("val", "1")
                          .newRow("c2").add("val", "2")
                          .apply();
@@ -365,7 +368,7 @@ public class VerifyTest
     {
         for (int i = 0; i < partitionsPerSSTable; i++)
         {
-            UpdateBuilder.create(cfs.metadata, String.valueOf(i))
+            UpdateBuilder.create(cfs.metadata(), String.valueOf(i))
                          .newRow("c1").add("val", 100L)
                          .apply();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java
index edff3b7..6eaf2c8 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogReaderTest.java
@@ -27,8 +27,8 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLTester;
@@ -158,14 +158,14 @@ public class CommitLogReaderTest extends CQLTester
     }
 
     /**
-     * Since we have both cfm and non mixed into the CL, we ignore updates that aren't for the cfm the test handler
+     * Since we have both table and non mixed into the CL, we ignore updates that aren't for the table the test handler
      * is configured to check.
      * @param handler
      * @param offset integer offset of count we expect to see in record
      */
     private void confirmReadOrder(TestCLRHandler handler, int offset)
     {
-        ColumnDefinition cd = currentTableMetadata().getColumnDefinition(new ColumnIdentifier("data", false));
+        ColumnMetadata cd = currentTableMetadata().getColumn(new ColumnIdentifier("data", false));
         int i = 0;
         int j = 0;
         while (i + j < handler.seenMutationCount())
@@ -208,17 +208,17 @@ public class CommitLogReaderTest extends CQLTester
         public List<Mutation> seenMutations = new ArrayList<Mutation>();
         public boolean sawStopOnErrorCheck = false;
 
-        private final CFMetaData cfm;
+        private final TableMetadata metadata;
 
         // Accept all
         public TestCLRHandler()
         {
-            this.cfm = null;
+            this.metadata = null;
         }
 
-        public TestCLRHandler(CFMetaData cfm)
+        public TestCLRHandler(TableMetadata metadata)
         {
-            this.cfm = cfm;
+            this.metadata = metadata;
         }
 
         public boolean shouldSkipSegmentOnError(CommitLogReadException exception) throws IOException
@@ -234,7 +234,7 @@ public class CommitLogReaderTest extends CQLTester
 
         public void handleMutation(Mutation m, int size, int entryLocation, CommitLogDescriptor desc)
         {
-            if ((cfm == null) || (cfm != null && m.get(cfm) != null)) {
+            if ((metadata == null) || (metadata != null && m.get(metadata) != null)) {
                 seenMutations.add(m);
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
index 68ce57d..3ae1ae4 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDCTest.java
@@ -29,7 +29,7 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.Keyspace;
@@ -61,7 +61,7 @@ public class CommitLogSegmentManagerCDCTest extends CQLTester
     {
         createTable("CREATE TABLE %s (idx int, data text, primary key(idx)) WITH cdc=true;");
         CommitLogSegmentManagerCDC cdcMgr = (CommitLogSegmentManagerCDC)CommitLog.instance.segmentManager;
-        CFMetaData cfm = currentTableMetadata();
+        TableMetadata cfm = currentTableMetadata();
 
         // Confirm that logic to check for whether or not we can allocate new CDC segments works
         Integer originalCDCSize = DatabaseDescriptor.getCDCSpaceInMB();
@@ -159,7 +159,7 @@ public class CommitLogSegmentManagerCDCTest extends CQLTester
         try
         {
             DatabaseDescriptor.setCDCSpaceInMB(16);
-            CFMetaData ccfm = Keyspace.open(keyspace()).getColumnFamilyStore(ct).metadata;
+            TableMetadata ccfm = Keyspace.open(keyspace()).getColumnFamilyStore(ct).metadata();
             // Spin until we hit CDC capacity and make sure we get a WriteTimeout
             try
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
index e22e86f..06513a5 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerTest.java
@@ -92,7 +92,7 @@ public class CommitLogSegmentManagerTest
 
         ColumnFamilyStore cfs1 = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
 
-        final Mutation m = new RowUpdateBuilder(cfs1.metadata, 0, "k").clustering("bytes")
+        final Mutation m = new RowUpdateBuilder(cfs1.metadata(), 0, "k").clustering("bytes")
                                                                       .add("val", ByteBuffer.wrap(entropy))
                                                                       .build();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 19305ac..f98dd6b 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -37,7 +37,8 @@ import org.junit.runners.Parameterized.Parameters;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.config.Config.DiskFailurePolicy;
@@ -229,7 +230,7 @@ public class CommitLogTest
         ColumnFamilyStore cfs2 = ks.getColumnFamilyStore(STANDARD2);
 
         // Roughly 32 MB mutation
-        Mutation m = new RowUpdateBuilder(cfs1.metadata, 0, "k")
+        Mutation m = new RowUpdateBuilder(cfs1.metadata(), 0, "k")
                      .clustering("bytes")
                      .add("val", ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize() / 4))
                      .build();
@@ -242,7 +243,7 @@ public class CommitLogTest
         CommitLog.instance.add(m);
 
         // Adding new mutation on another CF
-        Mutation m2 = new RowUpdateBuilder(cfs2.metadata, 0, "k")
+        Mutation m2 = new RowUpdateBuilder(cfs2.metadata(), 0, "k")
                       .clustering("bytes")
                       .add("val", ByteBuffer.allocate(4))
                       .build();
@@ -250,8 +251,8 @@ public class CommitLogTest
 
         assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size());
 
-        UUID cfid2 = m2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
+        TableId id2 = m2.getTableIds().iterator().next();
+        CommitLog.instance.discardCompletedSegments(id2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
 
         // Assert we still have both our segments
         assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size());
@@ -265,7 +266,7 @@ public class CommitLogTest
         ColumnFamilyStore cfs2 = ks.getColumnFamilyStore(STANDARD2);
 
         // Roughly 32 MB mutation
-         Mutation rm = new RowUpdateBuilder(cfs1.metadata, 0, "k")
+         Mutation rm = new RowUpdateBuilder(cfs1.metadata(), 0, "k")
                   .clustering("bytes")
                   .add("val", ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/4) - 1))
                   .build();
@@ -277,14 +278,14 @@ public class CommitLogTest
         assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
 
         // "Flush": this won't delete anything
-        UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
+        TableId id1 = rm.getTableIds().iterator().next();
         CommitLog.instance.sync();
-        CommitLog.instance.discardCompletedSegments(cfid1, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
+        CommitLog.instance.discardCompletedSegments(id1, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
 
         assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
 
         // Adding new mutation on another CF, large enough (including CL entry overhead) that a new segment is created
-        Mutation rm2 = new RowUpdateBuilder(cfs2.metadata, 0, "k")
+        Mutation rm2 = new RowUpdateBuilder(cfs2.metadata(), 0, "k")
                        .clustering("bytes")
                        .add("val", ByteBuffer.allocate(DatabaseDescriptor.getMaxMutationSize() - 200))
                        .build();
@@ -302,8 +303,8 @@ public class CommitLogTest
         // "Flush" second cf: The first segment should be deleted since we
         // didn't write anything on cf1 since last flush (and we flush cf2)
 
-        UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
+        TableId id2 = rm2.getTableIds().iterator().next();
+        CommitLog.instance.discardCompletedSegments(id2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
 
         segments = CommitLog.instance.segmentManager.getActiveSegments();
 
@@ -315,9 +316,9 @@ public class CommitLogTest
 
     private String getDirtyCFIds(Collection<CommitLogSegment> segments)
     {
-        return "Dirty cfIds: <"
+        return "Dirty tableIds: <"
                + String.join(", ", segments.stream()
-                                           .map(CommitLogSegment::getDirtyCFIDs)
+                                           .map(CommitLogSegment::getDirtyTableIds)
                                            .flatMap(uuids -> uuids.stream())
                                            .distinct()
                                            .map(uuid -> uuid.toString()).collect(Collectors.toList()))
@@ -330,7 +331,7 @@ public class CommitLogTest
         // We don't want to allocate a size of 0 as this is optimized under the hood and our computation would
         // break testEqualRecordLimit
         int allocSize = 1;
-        Mutation rm = new RowUpdateBuilder(cfs.metadata, 0, key)
+        Mutation rm = new RowUpdateBuilder(cfs.metadata(), 0, key)
                       .clustering(colName)
                       .add("val", ByteBuffer.allocate(allocSize)).build();
 
@@ -358,7 +359,7 @@ public class CommitLogTest
     public void testEqualRecordLimit() throws Exception
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
-        Mutation rm = new RowUpdateBuilder(cfs.metadata, 0, "k")
+        Mutation rm = new RowUpdateBuilder(cfs.metadata(), 0, "k")
                       .clustering("bytes")
                       .add("val", ByteBuffer.allocate(getMaxRecordDataSize()))
                       .build();
@@ -370,7 +371,7 @@ public class CommitLogTest
     {
         Keyspace ks = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = ks.getColumnFamilyStore(STANDARD1);
-        Mutation rm = new RowUpdateBuilder(cfs.metadata, 0, "k")
+        Mutation rm = new RowUpdateBuilder(cfs.metadata(), 0, "k")
                       .clustering("bytes")
                       .add("val", ByteBuffer.allocate(1 + getMaxRecordDataSize()))
                       .build();
@@ -540,10 +541,10 @@ public class CommitLogTest
             ColumnFamilyStore cfs1 = ks.getColumnFamilyStore(STANDARD1);
             ColumnFamilyStore cfs2 = ks.getColumnFamilyStore(STANDARD2);
 
-            new RowUpdateBuilder(cfs1.metadata, 0, "k").clustering("bytes").add("val", ByteBuffer.allocate(100)).build().applyUnsafe();
+            new RowUpdateBuilder(cfs1.metadata(), 0, "k").clustering("bytes").add("val", ByteBuffer.allocate(100)).build().applyUnsafe();
             cfs1.truncateBlocking();
             DatabaseDescriptor.setAutoSnapshot(prev);
-            Mutation m2 = new RowUpdateBuilder(cfs2.metadata, 0, "k")
+            Mutation m2 = new RowUpdateBuilder(cfs2.metadata(), 0, "k")
                           .clustering("bytes")
                           .add("val", ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize() / 4))
                           .build();
@@ -555,8 +556,8 @@ public class CommitLogTest
             CommitLogPosition position = CommitLog.instance.getCurrentPosition();
             for (Keyspace keyspace : Keyspace.system())
                 for (ColumnFamilyStore syscfs : keyspace.getColumnFamilyStores())
-                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, CommitLogPosition.NONE, position);
-            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, CommitLogPosition.NONE, position);
+                    CommitLog.instance.discardCompletedSegments(syscfs.metadata().id, CommitLogPosition.NONE, position);
+            CommitLog.instance.discardCompletedSegments(cfs2.metadata().id, CommitLogPosition.NONE, position);
             assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
         }
         finally
@@ -576,7 +577,7 @@ public class CommitLogTest
             Assert.assertFalse(notDurableKs.getMetadata().params.durableWrites);
 
             ColumnFamilyStore cfs = notDurableKs.getColumnFamilyStore("Standard1");
-            new RowUpdateBuilder(cfs.metadata, 0, "key1")
+            new RowUpdateBuilder(cfs.metadata(), 0, "key1")
             .clustering("bytes").add("val", bytes("abcd"))
             .build()
             .applyUnsafe();
@@ -599,14 +600,14 @@ public class CommitLogTest
     {
         int cellCount = 0;
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
-        final Mutation rm1 = new RowUpdateBuilder(cfs.metadata, 0, "k1")
+        final Mutation rm1 = new RowUpdateBuilder(cfs.metadata(), 0, "k1")
                              .clustering("bytes")
                              .add("val", bytes("this is a string"))
                              .build();
         cellCount += 1;
         CommitLog.instance.add(rm1);
 
-        final Mutation rm2 = new RowUpdateBuilder(cfs.metadata, 0, "k2")
+        final Mutation rm2 = new RowUpdateBuilder(cfs.metadata(), 0, "k2")
                              .clustering("bytes")
                              .add("val", bytes("this is a string"))
                              .build();
@@ -615,7 +616,7 @@ public class CommitLogTest
 
         CommitLog.instance.sync();
 
-        SimpleCountingReplayer replayer = new SimpleCountingReplayer(CommitLog.instance, CommitLogPosition.NONE, cfs.metadata);
+        SimpleCountingReplayer replayer = new SimpleCountingReplayer(CommitLog.instance, CommitLogPosition.NONE, cfs.metadata());
         List<String> activeSegments = CommitLog.instance.getActiveSegmentNames();
         Assert.assertFalse(activeSegments.isEmpty());
 
@@ -636,7 +637,7 @@ public class CommitLogTest
 
         for (int i = 0; i < max; i++)
         {
-            final Mutation rm1 = new RowUpdateBuilder(cfs.metadata, 0, "k" + 1)
+            final Mutation rm1 = new RowUpdateBuilder(cfs.metadata(), 0, "k" + 1)
                                  .clustering("bytes")
                                  .add("val", bytes("this is a string"))
                                  .build();
@@ -652,7 +653,7 @@ public class CommitLogTest
 
         CommitLog.instance.sync();
 
-        SimpleCountingReplayer replayer = new SimpleCountingReplayer(CommitLog.instance, commitLogPosition, cfs.metadata);
+        SimpleCountingReplayer replayer = new SimpleCountingReplayer(CommitLog.instance, commitLogPosition, cfs.metadata());
         List<String> activeSegments = CommitLog.instance.getActiveSegmentNames();
         Assert.assertFalse(activeSegments.isEmpty());
 
@@ -665,15 +666,15 @@ public class CommitLogTest
     class SimpleCountingReplayer extends CommitLogReplayer
     {
         private final CommitLogPosition filterPosition;
-        private final CFMetaData metadata;
+        private final TableMetadata metadata;
         int cells;
         int skipped;
 
-        SimpleCountingReplayer(CommitLog commitLog, CommitLogPosition filterPosition, CFMetaData cfm)
+        SimpleCountingReplayer(CommitLog commitLog, CommitLogPosition filterPosition, TableMetadata metadata)
         {
             super(commitLog, filterPosition, Collections.emptyMap(), ReplayFilter.create());
             this.filterPosition = filterPosition;
-            this.metadata = cfm;
+            this.metadata = metadata;
         }
 
         @SuppressWarnings("resource")
@@ -694,7 +695,7 @@ public class CommitLogTest
             {
                 // Only process mutations for the CF's we're testing against, since we can't deterministically predict
                 // whether or not system keyspaces will be mutated during a test.
-                if (partitionUpdate.metadata().cfName.equals(metadata.cfName))
+                if (partitionUpdate.metadata().name.equals(metadata.name))
                 {
                     for (Row row : partitionUpdate)
                         cells += Iterables.size(row.cells());
@@ -716,7 +717,7 @@ public class CommitLogTest
 
             for (int i = 0 ; i < 5 ; i++)
             {
-                new RowUpdateBuilder(cfs.metadata, 0, "k")
+                new RowUpdateBuilder(cfs.metadata(), 0, "k")
                     .clustering("c" + i).add("val", ByteBuffer.allocate(100))
                     .build()
                     .apply();
@@ -759,7 +760,7 @@ public class CommitLogTest
 
         for (int i = 0 ; i < 5 ; i++)
         {
-            new RowUpdateBuilder(cfs.metadata, 0, "k")
+            new RowUpdateBuilder(cfs.metadata(), 0, "k")
                 .clustering("c" + i).add("val", ByteBuffer.allocate(100))
                 .build()
                 .apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
index e25d2f1..1ca9a80 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
@@ -24,7 +24,6 @@ package org.apache.cassandra.db.commitlog;
 import java.io.*;
 import java.nio.ByteBuffer;
 import java.util.Properties;
-import java.util.UUID;
 
 import junit.framework.Assert;
 
@@ -36,9 +35,11 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.Row;
@@ -46,6 +47,7 @@ import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Tables;
 import org.apache.cassandra.security.EncryptionContextGenerator;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.KillerForTests;
@@ -76,12 +78,15 @@ public class CommitLogUpgradeTest
     private KillerForTests killerForTests;
     private boolean shouldBeKilled = false;
 
-    static CFMetaData metadata = CFMetaData.Builder.createDense(KEYSPACE, TABLE, false, false)
-                                                   .addPartitionKey("key", AsciiType.instance)
-                                                   .addClusteringColumn("col", AsciiType.instance)
-                                                   .addRegularColumn("val", BytesType.instance)
-                                                   .build()
-                                                   .compression(SchemaLoader.getCompressionParameters());
+    static TableMetadata metadata =
+        TableMetadata.builder(KEYSPACE, TABLE)
+                     .isCompound(false)
+                     .isDense(true)
+                     .addPartitionKeyColumn("key", AsciiType.instance)
+                     .addClusteringColumn("col", AsciiType.instance)
+                     .addRegularColumn("val", BytesType.instance)
+                     .compression(SchemaLoader.getCompressionParameters())
+                     .build();
 
     @Before
     public void prepareToBeKilled()
@@ -107,9 +112,7 @@ public class CommitLogUpgradeTest
     public static void initialize()
     {
         SchemaLoader.loadSchema();
-        SchemaLoader.createKeyspace(KEYSPACE,
-                                    KeyspaceParams.simple(1),
-                                    metadata);
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), metadata);
         DatabaseDescriptor.setEncryptionContext(EncryptionContextGenerator.createContext(true));
     }
 
@@ -123,13 +126,9 @@ public class CommitLogUpgradeTest
         String cfidString = prop.getProperty(CFID_PROPERTY);
         if (cfidString != null)
         {
-            UUID cfid = UUID.fromString(cfidString);
-            if (Schema.instance.getCF(cfid) == null)
-            {
-                CFMetaData cfm = Schema.instance.getCFMetaData(KEYSPACE, TABLE);
-                Schema.instance.unload(cfm);
-                Schema.instance.load(cfm.copy(cfid));
-            }
+            TableId tableId = TableId.fromString(cfidString);
+            if (Schema.instance.getTableMetadata(tableId) == null)
+                Schema.instance.load(KeyspaceMetadata.create(KEYSPACE, KeyspaceParams.simple(1), Tables.of(metadata.unbuild().id(tableId).build())));
         }
 
         Hasher hasher = new Hasher();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java
index 5a03f9f..d2ad42f 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java
@@ -38,7 +38,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.UpdateBuilder;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.FileUtils;
@@ -92,9 +92,7 @@ public class CommitLogUpgradeTestMaker
         }
 
         SchemaLoader.loadSchema();
-        SchemaLoader.createKeyspace(KEYSPACE,
-                                    KeyspaceParams.simple(1),
-                                    metadata);
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), metadata);
     }
 
     public void makeLog() throws IOException, InterruptedException
@@ -134,7 +132,7 @@ public class CommitLogUpgradeTestMaker
             FileUtils.createHardLink(f, new File(dataDir, f.getName()));
 
         Properties prop = new Properties();
-        prop.setProperty(CFID_PROPERTY, Schema.instance.getId(KEYSPACE, TABLE).toString());
+        prop.setProperty(CFID_PROPERTY, Schema.instance.getTableMetadata(KEYSPACE, TABLE).id.toString());
         prop.setProperty(CELLS_PROPERTY, Integer.toString(cells));
         prop.setProperty(HASH_PROPERTY, Integer.toString(hash));
         prop.store(new FileOutputStream(new File(dataDir, PROPERTIES_FILE)),
@@ -236,7 +234,7 @@ public class CommitLogUpgradeTestMaker
                     rl.acquire();
                 ByteBuffer key = randomBytes(16, tlr);
 
-                UpdateBuilder builder = UpdateBuilder.create(Schema.instance.getCFMetaData(KEYSPACE, TABLE), Util.dk(key));
+                UpdateBuilder builder = UpdateBuilder.create(Schema.instance.getTableMetadata(KEYSPACE, TABLE), Util.dk(key));
 
                 for (int ii = 0; ii < numCells; ii++)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
index 413e716..b3dc070 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
@@ -25,7 +25,7 @@ import static org.junit.Assert.*;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -90,7 +90,7 @@ public class SnapshotDeletingTest
 
     private void populate(int rowCount) {
         long timestamp = System.currentTimeMillis();
-        CFMetaData cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata;
+        TableMetadata cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata();
         for (int i = 0; i <= rowCount; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
index 399a935..5a7bfed 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -27,12 +27,11 @@ import java.util.UUID;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
-import com.google.common.util.concurrent.RateLimiter;
 import org.junit.BeforeClass;
 import org.junit.After;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.rows.EncodingStats;
@@ -63,16 +62,14 @@ public class AntiCompactionTest
 {
     private static final String KEYSPACE1 = "AntiCompactionTest";
     private static final String CF = "AntiCompactionTest";
-    private static CFMetaData cfm;
+    private static TableMetadata metadata;
 
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
         SchemaLoader.prepareServer();
-        cfm = SchemaLoader.standardCFMD(KEYSPACE1, CF);
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        metadata = SchemaLoader.standardCFMD(KEYSPACE1, CF).build();
+        SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1), metadata);
     }
 
     @After
@@ -169,11 +166,11 @@ public class AntiCompactionTest
         File dir = cfs.getDirectories().getDirectoryForNewSSTables();
         Descriptor desc = cfs.newSSTableDescriptor(dir);
 
-        try (SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, desc, 0, 0, new SerializationHeader(true, cfm, cfm.partitionColumns(), EncodingStats.NO_STATS)))
+        try (SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, desc, 0, 0, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS)))
         {
             for (int i = 0; i < count; i++)
             {
-                UpdateBuilder builder = UpdateBuilder.create(cfm, ByteBufferUtil.bytes(i));
+                UpdateBuilder builder = UpdateBuilder.create(metadata, ByteBufferUtil.bytes(i));
                 for (int j = 0; j < count * 5; j++)
                     builder.newRow("c" + j).add("val", "value1");
                 writer.append(builder.build().unfilteredIterator());
@@ -191,7 +188,7 @@ public class AntiCompactionTest
         for (int i = 0; i < 10; i++)
         {
             String localSuffix = Integer.toString(i);
-            new RowUpdateBuilder(cfm, System.currentTimeMillis(), localSuffix + "-" + Suffix)
+            new RowUpdateBuilder(metadata, System.currentTimeMillis(), localSuffix + "-" + Suffix)
                     .clustering("c")
                     .add("val", "val" + localSuffix)
                     .build()
@@ -328,7 +325,7 @@ public class AntiCompactionTest
         store.disableAutoCompaction();
         for (int i = 0; i < 10; i++)
         {
-            new RowUpdateBuilder(cfm, System.currentTimeMillis(), Integer.toString(i))
+            new RowUpdateBuilder(metadata, System.currentTimeMillis(), Integer.toString(i))
                 .clustering("c")
                 .add("val", "val")
                 .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
index f08ea97..e0f24f2 100644
--- a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
@@ -91,7 +91,7 @@ public class BlacklistingCompactionsTest
     /**
      * Return a table metadata, we use types with fixed size to increase the chance of detecting corrupt data
      */
-    private static CFMetaData makeTable(String tableName)
+    private static TableMetadata.Builder makeTable(String tableName)
     {
         return SchemaLoader.standardCFMD(KEYSPACE1, tableName, 1, LongType.instance, LongType.instance, LongType.instance);
     }
@@ -131,7 +131,7 @@ public class BlacklistingCompactionsTest
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(tableName);
 
         final int ROWS_PER_SSTABLE = 10;
-        final int SSTABLES = cfs.metadata.params.minIndexInterval * 2 / ROWS_PER_SSTABLE;
+        final int SSTABLES = cfs.metadata().params.minIndexInterval * 2 / ROWS_PER_SSTABLE;
         final int SSTABLES_TO_CORRUPT = 8;
 
         assertTrue(String.format("Not enough sstables (%d), expected at least %d sstables to corrupt", SSTABLES, SSTABLES_TO_CORRUPT),
@@ -150,7 +150,7 @@ public class BlacklistingCompactionsTest
             {
                 DecoratedKey key = Util.dk(String.valueOf(i));
                 long timestamp = j * ROWS_PER_SSTABLE + i;
-                new RowUpdateBuilder(cfs.metadata, timestamp, key.getKey())
+                new RowUpdateBuilder(cfs.metadata(), timestamp, key.getKey())
                         .clustering(Long.valueOf(i))
                         .add("val", Long.valueOf(i))
                         .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java
index 1b400e8..14dc3be 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java
@@ -28,7 +28,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
@@ -59,16 +59,18 @@ public class CompactionControllerTest extends SchemaLoader
         SchemaLoader.prepareServer();
         SchemaLoader.createKeyspace(KEYSPACE,
                                     KeyspaceParams.simple(1),
-                                    CFMetaData.Builder.create(KEYSPACE, CF1, true, false, false)
-                                                      .addPartitionKey("pk", AsciiType.instance)
-                                                      .addClusteringColumn("ck", AsciiType.instance)
-                                                      .addRegularColumn("val", AsciiType.instance)
-                                                      .build(),
-                                    CFMetaData.Builder.create(KEYSPACE, CF2, true, false, false)
-                                                      .addPartitionKey("pk", AsciiType.instance)
-                                                      .addClusteringColumn("ck", AsciiType.instance)
-                                                      .addRegularColumn("val", AsciiType.instance)
-                                                      .build());
+                                    TableMetadata.builder(KEYSPACE, CF1)
+                                                 .isCompound(false)
+                                                 .isDense(true)
+                                                 .addPartitionKeyColumn("pk", AsciiType.instance)
+                                                 .addClusteringColumn("ck", AsciiType.instance)
+                                                 .addRegularColumn("val", AsciiType.instance),
+                                    TableMetadata.builder(KEYSPACE, CF2)
+                                                 .isCompound(false)
+                                                 .isDense(true)
+                                                 .addPartitionKeyColumn("pk", AsciiType.instance)
+                                                 .addClusteringColumn("ck", AsciiType.instance)
+                                                 .addRegularColumn("val", AsciiType.instance));
     }
 
     @Test
@@ -85,7 +87,7 @@ public class CompactionControllerTest extends SchemaLoader
         long timestamp3 = timestamp2 - 5; // oldest timestamp
 
         // add to first memtable
-        applyMutation(cfs.metadata, key, timestamp1);
+        applyMutation(cfs.metadata(), key, timestamp1);
 
         // check max purgeable timestamp without any sstables
         try(CompactionController controller = new CompactionController(cfs, null, 0))
@@ -99,7 +101,7 @@ public class CompactionControllerTest extends SchemaLoader
         Set<SSTableReader> compacting = Sets.newHashSet(cfs.getLiveSSTables()); // first sstable is compacting
 
         // create another sstable
-        applyMutation(cfs.metadata, key, timestamp2);
+        applyMutation(cfs.metadata(), key, timestamp2);
         cfs.forceBlockingFlush();
 
         // check max purgeable timestamp when compacting the first sstable with and without a memtable
@@ -107,7 +109,7 @@ public class CompactionControllerTest extends SchemaLoader
         {
             assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp2);
 
-            applyMutation(cfs.metadata, key, timestamp3);
+            applyMutation(cfs.metadata(), key, timestamp3);
 
             assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp3); //second sstable and second memtable
         }
@@ -118,9 +120,9 @@ public class CompactionControllerTest extends SchemaLoader
         //newest to oldest
         try (CompactionController controller = new CompactionController(cfs, null, 0))
         {
-            applyMutation(cfs.metadata, key, timestamp1);
-            applyMutation(cfs.metadata, key, timestamp2);
-            applyMutation(cfs.metadata, key, timestamp3);
+            applyMutation(cfs.metadata(), key, timestamp1);
+            applyMutation(cfs.metadata(), key, timestamp2);
+            applyMutation(cfs.metadata(), key, timestamp3);
 
             assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp3); //memtable only
         }
@@ -130,9 +132,9 @@ public class CompactionControllerTest extends SchemaLoader
         //oldest to newest
         try (CompactionController controller = new CompactionController(cfs, null, 0))
         {
-            applyMutation(cfs.metadata, key, timestamp3);
-            applyMutation(cfs.metadata, key, timestamp2);
-            applyMutation(cfs.metadata, key, timestamp1);
+            applyMutation(cfs.metadata(), key, timestamp3);
+            applyMutation(cfs.metadata(), key, timestamp2);
+            applyMutation(cfs.metadata(), key, timestamp1);
 
             assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp3);
         }
@@ -152,14 +154,14 @@ public class CompactionControllerTest extends SchemaLoader
         long timestamp3 = timestamp2 - 5; // oldest timestamp
 
         // create sstable with tombstone that should be expired in no older timestamps
-        applyDeleteMutation(cfs.metadata, key, timestamp2);
+        applyDeleteMutation(cfs.metadata(), key, timestamp2);
         cfs.forceBlockingFlush();
 
         // first sstable with tombstone is compacting
         Set<SSTableReader> compacting = Sets.newHashSet(cfs.getLiveSSTables());
 
         // create another sstable with more recent timestamp
-        applyMutation(cfs.metadata, key, timestamp1);
+        applyMutation(cfs.metadata(), key, timestamp1);
         cfs.forceBlockingFlush();
 
         // second sstable is overlapping
@@ -173,13 +175,13 @@ public class CompactionControllerTest extends SchemaLoader
         assertEquals(compacting.iterator().next(), expired.iterator().next());
 
         // however if we add an older mutation to the memtable then the sstable should not be expired
-        applyMutation(cfs.metadata, key, timestamp3);
+        applyMutation(cfs.metadata(), key, timestamp3);
         expired = CompactionController.getFullyExpiredSSTables(cfs, compacting, overlapping, gcBefore);
         assertNotNull(expired);
         assertEquals(0, expired.size());
     }
 
-    private void applyMutation(CFMetaData cfm, DecoratedKey key, long timestamp)
+    private void applyMutation(TableMetadata cfm, DecoratedKey key, long timestamp)
     {
         ByteBuffer val = ByteBufferUtil.bytes(1L);
 
@@ -190,7 +192,7 @@ public class CompactionControllerTest extends SchemaLoader
         .applyUnsafe();
     }
 
-    private void applyDeleteMutation(CFMetaData cfm, DecoratedKey key, long timestamp)
+    private void applyDeleteMutation(TableMetadata cfm, DecoratedKey key, long timestamp)
     {
         new Mutation(PartitionUpdate.fullPartitionDelete(cfm, key, timestamp, FBUtilities.nowInSeconds()))
         .applyUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/CompactionIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionIteratorTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionIteratorTest.java
index 3c8c1b0..99df52f 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionIteratorTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionIteratorTest.java
@@ -29,7 +29,6 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -40,8 +39,8 @@ import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.partitions.AbstractUnfilteredPartitionIterator;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadata;
 
 public class CompactionIteratorTest
 {
@@ -52,7 +51,7 @@ public class CompactionIteratorTest
     private static final String CFNAME = "Integer1";
 
     static final DecoratedKey kk;
-    static final CFMetaData metadata;
+    static final TableMetadata metadata;
     private static final int RANGE = 1000;
     private static final int COUNT = 100;
 
@@ -71,7 +70,7 @@ public class CompactionIteratorTest
                                                                          1,
                                                                          UTF8Type.instance,
                                                                          Int32Type.instance,
-                                                                         Int32Type.instance));
+                                                                         Int32Type.instance).build());
     }
 
     // See org.apache.cassandra.db.rows.UnfilteredRowsGenerator.parse for the syntax used in these tests.
@@ -339,7 +338,7 @@ public class CompactionIteratorTest
         }
 
         @Override
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
             return metadata;
         }


[24/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/filter/RowFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java
index 14903ba..bf65e96 100644
--- a/src/java/org/apache/cassandra/db/filter/RowFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java
@@ -25,25 +25,22 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.base.Objects;
-import com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.*;
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.db.partitions.ImmutableBTreePartition;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.transform.Transformation;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.IndexMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -83,21 +80,21 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         return new CQLFilter(new ArrayList<>(capacity));
     }
 
-    public SimpleExpression add(ColumnDefinition def, Operator op, ByteBuffer value)
+    public SimpleExpression add(ColumnMetadata def, Operator op, ByteBuffer value)
     {
         SimpleExpression expression = new SimpleExpression(def, op, value);
         add(expression);
         return expression;
     }
 
-    public void addMapEquality(ColumnDefinition def, ByteBuffer key, Operator op, ByteBuffer value)
+    public void addMapEquality(ColumnMetadata def, ByteBuffer key, Operator op, ByteBuffer value)
     {
         add(new MapEqualityExpression(def, key, op, value));
     }
 
-    public void addCustomIndexExpression(CFMetaData cfm, IndexMetadata targetIndex, ByteBuffer value)
+    public void addCustomIndexExpression(TableMetadata metadata, IndexMetadata targetIndex, ByteBuffer value)
     {
-        add(new CustomExpression(cfm, targetIndex, value));
+        add(new CustomExpression(metadata, targetIndex, value));
     }
 
     private void add(Expression expression)
@@ -135,7 +132,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
      * @param nowInSec the current time in seconds (to know what is live and what isn't).
      * @return {@code true} if {@code row} in partition {@code partitionKey} satisfies this row filter.
      */
-    public boolean isSatisfiedBy(CFMetaData metadata, DecoratedKey partitionKey, Row row, int nowInSec)
+    public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row, int nowInSec)
     {
         // We purge all tombstones as the expressions isSatisfiedBy methods expects it
         Row purged = row.purge(DeletionPurger.PURGE_ALL, nowInSec);
@@ -249,7 +246,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
             if (expressions.isEmpty())
                 return iter;
 
-            final CFMetaData metadata = iter.metadata();
+            final TableMetadata metadata = iter.metadata();
 
             List<Expression> partitionLevelExpressions = new ArrayList<>();
             List<Expression> rowLevelExpressions = new ArrayList<>();
@@ -323,11 +320,11 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         protected enum Kind { SIMPLE, MAP_EQUALITY, UNUSED1, CUSTOM, USER }
 
         protected abstract Kind kind();
-        protected final ColumnDefinition column;
+        protected final ColumnMetadata column;
         protected final Operator operator;
         protected final ByteBuffer value;
 
-        protected Expression(ColumnDefinition column, Operator operator, ByteBuffer value)
+        protected Expression(ColumnMetadata column, Operator operator, ByteBuffer value)
         {
             this.column = column;
             this.operator = operator;
@@ -344,7 +341,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
             return kind() == Kind.USER;
         }
 
-        public ColumnDefinition column()
+        public ColumnMetadata column()
         {
             return column;
         }
@@ -401,19 +398,21 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         /**
          * Returns whether the provided row satisfied this expression or not.
          *
+         *
+         * @param metadata
          * @param partitionKey the partition key for row to check.
          * @param row the row to check. It should *not* contain deleted cells
          * (i.e. it should come from a RowIterator).
          * @return whether the row is satisfied by this expression.
          */
-        public abstract boolean isSatisfiedBy(CFMetaData metadata, DecoratedKey partitionKey, Row row);
+        public abstract boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row);
 
-        protected ByteBuffer getValue(CFMetaData metadata, DecoratedKey partitionKey, Row row)
+        protected ByteBuffer getValue(TableMetadata metadata, DecoratedKey partitionKey, Row row)
         {
             switch (column.kind)
             {
                 case PARTITION_KEY:
-                    return metadata.getKeyValidator() instanceof CompositeType
+                    return metadata.partitionKeyType instanceof CompositeType
                          ? CompositeType.extractComponent(partitionKey.getKey(), column.position())
                          : partitionKey.getKey();
                 case CLUSTERING:
@@ -484,7 +483,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
                 }
             }
 
-            public Expression deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
+            public Expression deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException
             {
                 Kind kind = Kind.values()[in.readByte()];
 
@@ -501,7 +500,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
 
                 ByteBuffer name = ByteBufferUtil.readWithShortLength(in);
                 Operator operator = Operator.readFrom(in);
-                ColumnDefinition column = metadata.getColumnDefinition(name);
+                ColumnMetadata column = metadata.getColumn(name);
 
                 if (!metadata.isCompactTable() && column == null)
                     throw new RuntimeException("Unknown (or dropped) column " + UTF8Type.instance.getString(name) + " during deserialization");
@@ -556,12 +555,12 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
      */
     public static class SimpleExpression extends Expression
     {
-        SimpleExpression(ColumnDefinition column, Operator operator, ByteBuffer value)
+        SimpleExpression(ColumnMetadata column, Operator operator, ByteBuffer value)
         {
             super(column, operator, value);
         }
 
-        public boolean isSatisfiedBy(CFMetaData metadata, DecoratedKey partitionKey, Row row)
+        public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row)
         {
             // We support null conditions for LWT (in ColumnCondition) but not for RowFilter.
             // TODO: we should try to merge both code someday.
@@ -707,7 +706,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
     {
         private final ByteBuffer key;
 
-        public MapEqualityExpression(ColumnDefinition column, ByteBuffer key, Operator operator, ByteBuffer value)
+        public MapEqualityExpression(ColumnMetadata column, ByteBuffer key, Operator operator, ByteBuffer value)
         {
             super(column, operator, value);
             assert column.type instanceof MapType && operator == Operator.EQ;
@@ -729,7 +728,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
             return CompositeType.build(key, value);
         }
 
-        public boolean isSatisfiedBy(CFMetaData metadata, DecoratedKey partitionKey, Row row)
+        public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row)
         {
             assert key != null;
             // We support null conditions for LWT (in ColumnCondition) but not for RowFilter.
@@ -800,21 +799,21 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
     public static final class CustomExpression extends Expression
     {
         private final IndexMetadata targetIndex;
-        private final CFMetaData cfm;
+        private final TableMetadata table;
 
-        public CustomExpression(CFMetaData cfm, IndexMetadata targetIndex, ByteBuffer value)
+        public CustomExpression(TableMetadata table, IndexMetadata targetIndex, ByteBuffer value)
         {
             // The operator is not relevant, but Expression requires it so for now we just hardcode EQ
-            super(makeDefinition(cfm, targetIndex), Operator.EQ, value);
+            super(makeDefinition(table, targetIndex), Operator.EQ, value);
             this.targetIndex = targetIndex;
-            this.cfm = cfm;
+            this.table = table;
         }
 
-        private static ColumnDefinition makeDefinition(CFMetaData cfm, IndexMetadata index)
+        private static ColumnMetadata makeDefinition(TableMetadata table, IndexMetadata index)
         {
             // Similarly to how we handle non-defined columns in thift, we create a fake column definition to
             // represent the target index. This is definitely something that can be improved though.
-            return ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(index.name.getBytes()), BytesType.instance);
+            return ColumnMetadata.regularColumn(table, ByteBuffer.wrap(index.name.getBytes()), BytesType.instance);
         }
 
         public IndexMetadata getTargetIndex()
@@ -831,7 +830,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         {
             return String.format("expr(%s, %s)",
                                  targetIndex.name,
-                                 Keyspace.openAndGetStore(cfm)
+                                 Keyspace.openAndGetStore(table)
                                          .indexManager
                                          .getIndex(targetIndex)
                                          .customExpressionValueType());
@@ -843,7 +842,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         }
 
         // Filtering by custom expressions isn't supported yet, so just accept any row
-        public boolean isSatisfiedBy(CFMetaData metadata, DecoratedKey partitionKey, Row row)
+        public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row)
         {
             return true;
         }
@@ -900,7 +899,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
         {
             protected abstract UserExpression deserialize(DataInputPlus in,
                                                           int version,
-                                                          CFMetaData metadata) throws IOException;
+                                                          TableMetadata metadata) throws IOException;
         }
 
         public static void register(Class<? extends UserExpression> expressionClass, Deserializer deserializer)
@@ -908,7 +907,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
             deserializers.registerUserExpressionClass(expressionClass, deserializer);
         }
 
-        private static UserExpression deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
+        private static UserExpression deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException
         {
             int id = in.readInt();
             Deserializer deserializer = deserializers.getDeserializer(id);
@@ -929,7 +928,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
             return 4 + expression.serializedSize(version);
         }
 
-        protected UserExpression(ColumnDefinition column, Operator operator, ByteBuffer value)
+        protected UserExpression(ColumnMetadata column, Operator operator, ByteBuffer value)
         {
             super(column, operator, value);
         }
@@ -954,7 +953,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
 
         }
 
-        public RowFilter deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
+        public RowFilter deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException
         {
             in.readBoolean(); // Unused
             int size = (int)in.readUnsignedVInt();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java b/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
index 98b539e..982ee1f 100644
--- a/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
+++ b/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
@@ -20,19 +20,19 @@ package org.apache.cassandra.db.filter;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.*;
 
 public class TombstoneOverwhelmingException extends RuntimeException
 {
-    public TombstoneOverwhelmingException(int numTombstones, String query, CFMetaData metadata, DecoratedKey lastPartitionKey, ClusteringPrefix lastClustering)
+    public TombstoneOverwhelmingException(int numTombstones, String query, TableMetadata metadata, DecoratedKey lastPartitionKey, ClusteringPrefix lastClustering)
     {
         super(String.format("Scanned over %d tombstones during query '%s' (last scanned row partion key was (%s)); query aborted",
                             numTombstones, query, makePKString(metadata, lastPartitionKey.getKey(), lastClustering)));
     }
 
-    private static String makePKString(CFMetaData metadata, ByteBuffer partitionKey, ClusteringPrefix clustering)
+    private static String makePKString(TableMetadata metadata, ByteBuffer partitionKey, ClusteringPrefix clustering)
     {
         StringBuilder sb = new StringBuilder();
 
@@ -40,7 +40,7 @@ public class TombstoneOverwhelmingException extends RuntimeException
             sb.append("(");
 
         // TODO: We should probably make that a lot easier/transparent for partition keys
-        AbstractType<?> pkType = metadata.getKeyValidator();
+        AbstractType<?> pkType = metadata.partitionKeyType;
         if (pkType instanceof CompositeType)
         {
             CompositeType ct = (CompositeType)pkType;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
index f1c2c44..89b82a1 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
@@ -29,7 +29,8 @@ import com.google.common.collect.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.compaction.OperationType;
@@ -539,9 +540,9 @@ public class LifecycleTransaction extends Transactional.AbstractTransactional
         return LogTransaction.removeUnfinishedLeftovers(cfs.getDirectories().getCFDirectories());
     }
 
-    public static boolean removeUnfinishedLeftovers(CFMetaData cfMetaData)
+    public static boolean removeUnfinishedLeftovers(TableMetadata metadata)
     {
-        return LogTransaction.removeUnfinishedLeftovers(cfMetaData);
+        return LogTransaction.removeUnfinishedLeftovers(metadata);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
index 75e82f3..face9fa 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/LogTransaction.java
@@ -34,7 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.SystemKeyspace;
@@ -410,7 +410,7 @@ class LogTransaction extends Transactional.AbstractTransactional implements Tran
      * @return true if the leftovers of all transaction logs found were removed, false otherwise.
      *
      */
-    static boolean removeUnfinishedLeftovers(CFMetaData metadata)
+    static boolean removeUnfinishedLeftovers(TableMetadata metadata)
     {
         return removeUnfinishedLeftovers(new Directories(metadata, ColumnFamilyStore.getInitialDirectories()).getCFDirectories());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/marshal/CompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index dfcd772..9eadd85 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -26,6 +26,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -74,6 +75,11 @@ public class CompositeType extends AbstractCompositeType
         return getInstance(parser.getTypeParameters());
     }
 
+    public static CompositeType getInstance(Iterable<AbstractType<?>> types)
+    {
+        return getInstance(Lists.newArrayList(types));
+    }
+
     public static CompositeType getInstance(AbstractType... types)
     {
         return getInstance(Arrays.asList(types));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/marshal/UserType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index edfb5f3..f139edd 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -425,4 +425,9 @@ public class UserType extends TupleType
             sb.append(")");
         return sb.toString();
     }
+
+    public String toCQLString()
+    {
+        return String.format("%s.%s", ColumnIdentifier.maybeQuote(keyspace), ColumnIdentifier.maybeQuote(getNameAsString()));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
index 74ac033..465dec3 100644
--- a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.db.partitions;
 
 import java.util.Iterator;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.*;
@@ -32,30 +32,28 @@ import static org.apache.cassandra.utils.btree.BTree.Dir.desc;
 
 public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
 {
-    protected static final Holder EMPTY = new Holder(PartitionColumns.NONE, BTree.empty(), DeletionInfo.LIVE, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS);
+    protected static final Holder EMPTY = new Holder(RegularAndStaticColumns.NONE, BTree.empty(), DeletionInfo.LIVE, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS);
 
-    protected final CFMetaData metadata;
     protected final DecoratedKey partitionKey;
 
     protected abstract Holder holder();
     protected abstract boolean canHaveShadowedData();
 
-    protected AbstractBTreePartition(CFMetaData metadata, DecoratedKey partitionKey)
+    protected AbstractBTreePartition(DecoratedKey partitionKey)
     {
-        this.metadata = metadata;
         this.partitionKey = partitionKey;
     }
 
     protected static final class Holder
     {
-        final PartitionColumns columns;
+        final RegularAndStaticColumns columns;
         final DeletionInfo deletionInfo;
         // the btree of rows
         final Object[] tree;
         final Row staticRow;
         final EncodingStats stats;
 
-        Holder(PartitionColumns columns, Object[] tree, DeletionInfo deletionInfo, Row staticRow, EncodingStats stats)
+        Holder(RegularAndStaticColumns columns, Object[] tree, DeletionInfo deletionInfo, Row staticRow, EncodingStats stats)
         {
             this.columns = columns;
             this.tree = tree;
@@ -87,10 +85,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
         return !BTree.isEmpty(holder.tree);
     }
 
-    public CFMetaData metadata()
-    {
-        return metadata;
-    }
+    public abstract TableMetadata metadata();
 
     public DecoratedKey partitionKey()
     {
@@ -102,7 +97,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
         return holder().deletionInfo.getPartitionDeletion();
     }
 
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
         return holder().columns;
     }
@@ -126,7 +121,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
         if (columns.fetchedColumns().statics.isEmpty() || (current.staticRow.isEmpty() && partitionDeletion.isLive()))
             return Rows.EMPTY_STATIC_ROW;
 
-        Row row = current.staticRow.filter(columns, partitionDeletion, setActiveDeletionToRow, metadata);
+        Row row = current.staticRow.filter(columns, partitionDeletion, setActiveDeletionToRow, metadata());
         return row == null ? Rows.EMPTY_STATIC_ROW : row;
     }
 
@@ -136,7 +131,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
         final Holder current = holder();
         return new SearchIterator<Clustering, Row>()
         {
-            private final SearchIterator<Clustering, Row> rawIter = new BTreeSearchIterator<>(current.tree, metadata.comparator, desc(reversed));
+            private final SearchIterator<Clustering, Row> rawIter = new BTreeSearchIterator<>(current.tree, metadata().comparator, desc(reversed));
             private final DeletionTime partitionDeletion = current.deletionInfo.getPartitionDeletion();
 
             public boolean hasNext()
@@ -162,7 +157,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
                 if (row == null)
                     return activeDeletion.isLive() ? null : BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(activeDeletion));
 
-                return row.filter(columns, activeDeletion, true, metadata);
+                return row.filter(columns, activeDeletion, true, metadata());
             }
         };
     }
@@ -183,7 +178,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
         if (slices.size() == 0)
         {
             DeletionTime partitionDeletion = current.deletionInfo.getPartitionDeletion();
-            return UnfilteredRowIterators.noRowsIterator(metadata, partitionKey(), staticRow, partitionDeletion, reversed);
+            return UnfilteredRowIterators.noRowsIterator(metadata(), partitionKey(), staticRow, partitionDeletion, reversed);
         }
 
         return slices.size() == 1
@@ -195,7 +190,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
     {
         ClusteringBound start = slice.start() == ClusteringBound.BOTTOM ? null : slice.start();
         ClusteringBound end = slice.end() == ClusteringBound.TOP ? null : slice.end();
-        Iterator<Row> rowIter = BTree.slice(current.tree, metadata.comparator, start, true, end, true, desc(reversed));
+        Iterator<Row> rowIter = BTree.slice(current.tree, metadata().comparator, start, true, end, true, desc(reversed));
         Iterator<RangeTombstone> deleteIter = current.deletionInfo.rangeIterator(slice, reversed);
         return merge(rowIter, deleteIter, selection, reversed, current, staticRow);
     }
@@ -203,7 +198,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
     private RowAndDeletionMergeIterator merge(Iterator<Row> rowIter, Iterator<RangeTombstone> deleteIter,
                                               ColumnFilter selection, boolean reversed, Holder current, Row staticRow)
     {
-        return new RowAndDeletionMergeIterator(metadata, partitionKey(), current.deletionInfo.getPartitionDeletion(),
+        return new RowAndDeletionMergeIterator(metadata(), partitionKey(), current.deletionInfo.getPartitionDeletion(),
                                                selection, staticRow, reversed, current.stats,
                                                rowIter, deleteIter,
                                                canHaveShadowedData());
@@ -216,7 +211,7 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
 
         private AbstractIterator(Holder current, Row staticRow, ColumnFilter selection, boolean isReversed)
         {
-            super(AbstractBTreePartition.this.metadata,
+            super(AbstractBTreePartition.this.metadata(),
                   AbstractBTreePartition.this.partitionKey(),
                   current.deletionInfo.getPartitionDeletion(),
                   selection.fetchedColumns(), // non-selected columns will be filtered in subclasses by RowAndDeletionMergeIterator
@@ -276,8 +271,8 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
 
     protected static Holder build(UnfilteredRowIterator iterator, int initialRowCapacity, boolean ordered)
     {
-        CFMetaData metadata = iterator.metadata();
-        PartitionColumns columns = iterator.columns();
+        TableMetadata metadata = iterator.metadata();
+        RegularAndStaticColumns columns = iterator.columns();
         boolean reversed = iterator.isReverseOrder();
 
         BTree.Builder<Row> builder = BTree.builder(metadata.comparator, initialRowCapacity);
@@ -303,8 +298,8 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
     // passes a MutableDeletionInfo that it mutates later.
     protected static Holder build(RowIterator rows, DeletionInfo deletion, boolean buildEncodingStats, int initialRowCapacity)
     {
-        CFMetaData metadata = rows.metadata();
-        PartitionColumns columns = rows.columns();
+        TableMetadata metadata = rows.metadata();
+        RegularAndStaticColumns columns = rows.columns();
         boolean reversed = rows.isReverseOrder();
 
         BTree.Builder<Row> builder = BTree.builder(metadata.comparator, initialRowCapacity);
@@ -327,17 +322,16 @@ public abstract class AbstractBTreePartition implements Partition, Iterable<Row>
     {
         StringBuilder sb = new StringBuilder();
 
-        sb.append(String.format("[%s.%s] key=%s columns=%s",
-                                metadata.ksName,
-                                metadata.cfName,
-                                metadata.getKeyValidator().getString(partitionKey().getKey()),
+        sb.append(String.format("[%s] key=%s columns=%s",
+                                metadata().toString(),
+                                metadata().partitionKeyType.getString(partitionKey().getKey()),
                                 columns()));
 
         if (staticRow() != Rows.EMPTY_STATIC_ROW)
-            sb.append("\n    ").append(staticRow().toString(metadata));
+            sb.append("\n    ").append(staticRow().toString(metadata()));
 
         for (Row row : this)
-            sb.append("\n    ").append(row.toString(metadata));
+            sb.append("\n    ").append(row.toString(metadata()));
 
         return sb.toString();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
index c9c6006..ee94963 100644
--- a/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AtomicBTreePartition.java
@@ -24,7 +24,8 @@ import java.util.List;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
@@ -50,7 +51,7 @@ import org.apache.cassandra.utils.memory.MemtableAllocator;
  */
 public class AtomicBTreePartition extends AbstractBTreePartition
 {
-    public static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreePartition(CFMetaData.createFake("keyspace", "table"),
+    public static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreePartition(null,
                                                                                        DatabaseDescriptor.getPartitioner().decorateKey(ByteBuffer.allocate(1)),
                                                                                        null));
 
@@ -83,10 +84,13 @@ public class AtomicBTreePartition extends AbstractBTreePartition
     private final MemtableAllocator allocator;
     private volatile Holder ref;
 
-    public AtomicBTreePartition(CFMetaData metadata, DecoratedKey partitionKey, MemtableAllocator allocator)
+    private final TableMetadataRef metadata;
+
+    public AtomicBTreePartition(TableMetadataRef metadata, DecoratedKey partitionKey, MemtableAllocator allocator)
     {
         // involved in potential bug? partition columns may be a subset if we alter columns while it's in memtable
-        super(metadata, partitionKey);
+        super(partitionKey);
+        this.metadata = metadata;
         this.allocator = allocator;
         this.ref = EMPTY;
     }
@@ -96,6 +100,11 @@ public class AtomicBTreePartition extends AbstractBTreePartition
         return ref;
     }
 
+    public TableMetadata metadata()
+    {
+        return metadata.get();
+    }
+
     protected boolean canHaveShadowedData()
     {
         return true;
@@ -148,7 +157,7 @@ public class AtomicBTreePartition extends AbstractBTreePartition
                     deletionInfo = current.deletionInfo;
                 }
 
-                PartitionColumns columns = update.columns().mergeTo(current.columns);
+                RegularAndStaticColumns columns = update.columns().mergeTo(current.columns);
                 Row newStatic = update.staticRow();
                 Row staticRow = newStatic.isEmpty()
                               ? current.staticRow

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java
index 90d64f4..48895dc 100644
--- a/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/CachedBTreePartition.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.partitions;
 
 import java.io.IOException;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.rows.*;
@@ -27,6 +26,9 @@ import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.btree.BTree;
 
 public class CachedBTreePartition extends ImmutableBTreePartition implements CachedPartition
@@ -36,7 +38,7 @@ public class CachedBTreePartition extends ImmutableBTreePartition implements Cac
     private final int cachedLiveRows;
     private final int rowsWithNonExpiringCells;
 
-    private CachedBTreePartition(CFMetaData metadata,
+    private CachedBTreePartition(TableMetadata metadata,
                                  DecoratedKey partitionKey,
                                  Holder holder,
                                  int createdAtInSec,
@@ -150,7 +152,7 @@ public class CachedBTreePartition extends ImmutableBTreePartition implements Cac
             out.writeInt(p.createdAtInSec);
             out.writeInt(p.cachedLiveRows);
             out.writeInt(p.rowsWithNonExpiringCells);
-            CFMetaData.serializer.serialize(partition.metadata(), out, version);
+            partition.metadata().id.serialize(out);
             try (UnfilteredRowIterator iter = p.unfilteredIterator())
             {
                 UnfilteredRowIteratorSerializer.serializer.serialize(iter, null, out, version, p.rowCount());
@@ -173,7 +175,7 @@ public class CachedBTreePartition extends ImmutableBTreePartition implements Cac
             int rowsWithNonExpiringCells = in.readInt();
 
 
-            CFMetaData metadata = CFMetaData.serializer.deserialize(in, version);
+            TableMetadata metadata = Schema.instance.getExistingTableMetadata(TableId.deserialize(in));
             UnfilteredRowIteratorSerializer.Header header = UnfilteredRowIteratorSerializer.serializer.deserializeHeader(metadata, null, in, version, SerializationHelper.Flag.LOCAL);
             assert !header.isReversed && header.rowEstimate >= 0;
 
@@ -184,11 +186,11 @@ public class CachedBTreePartition extends ImmutableBTreePartition implements Cac
             }
 
             return new CachedBTreePartition(metadata,
-                                                  header.key,
-                                                  holder,
-                                                  createdAtInSec,
-                                                  cachedLiveRows,
-                                                  rowsWithNonExpiringCells);
+                                            header.key,
+                                            holder,
+                                            createdAtInSec,
+                                            cachedLiveRows,
+                                            rowsWithNonExpiringCells);
 
         }
 
@@ -204,7 +206,7 @@ public class CachedBTreePartition extends ImmutableBTreePartition implements Cac
                 return TypeSizes.sizeof(p.createdAtInSec)
                      + TypeSizes.sizeof(p.cachedLiveRows)
                      + TypeSizes.sizeof(p.rowsWithNonExpiringCells)
-                     + CFMetaData.serializer.serializedSize(partition.metadata(), version)
+                     + partition.metadata().id.serializedSize()
                      + UnfilteredRowIteratorSerializer.serializer.serializedSize(iter, null, MessagingService.current_version, p.rowCount());
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/partitions/FilteredPartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/FilteredPartition.java b/src/java/org/apache/cassandra/db/partitions/FilteredPartition.java
index 70a4678..5730076 100644
--- a/src/java/org/apache/cassandra/db/partitions/FilteredPartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/FilteredPartition.java
@@ -19,10 +19,10 @@ package org.apache.cassandra.db.partitions;
 
 import java.util.Iterator;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.DeletionInfo;
-import org.apache.cassandra.db.PartitionColumns;
+import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.rows.*;
 
 public class FilteredPartition extends ImmutableBTreePartition
@@ -48,9 +48,9 @@ public class FilteredPartition extends ImmutableBTreePartition
         final Iterator<Row> iter = iterator();
         return new RowIterator()
         {
-            public CFMetaData metadata()
+            public TableMetadata metadata()
             {
-                return metadata;
+                return FilteredPartition.this.metadata();
             }
 
             public boolean isReverseOrder()
@@ -58,7 +58,7 @@ public class FilteredPartition extends ImmutableBTreePartition
                 return false;
             }
 
-            public PartitionColumns columns()
+            public RegularAndStaticColumns columns()
             {
                 return FilteredPartition.this.columns();
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/partitions/ImmutableBTreePartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/ImmutableBTreePartition.java b/src/java/org/apache/cassandra/db/partitions/ImmutableBTreePartition.java
index 8d96f1e..5139d40 100644
--- a/src/java/org/apache/cassandra/db/partitions/ImmutableBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/ImmutableBTreePartition.java
@@ -18,34 +18,37 @@
 */
 package org.apache.cassandra.db.partitions;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.DeletionInfo;
-import org.apache.cassandra.db.PartitionColumns;
+import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.rows.*;
 
 public class ImmutableBTreePartition extends AbstractBTreePartition
 {
 
     protected final Holder holder;
+    protected final TableMetadata metadata;
 
-    public ImmutableBTreePartition(CFMetaData metadata,
-                                      DecoratedKey partitionKey,
-                                      PartitionColumns columns,
-                                      Row staticRow,
-                                      Object[] tree,
-                                      DeletionInfo deletionInfo,
-                                      EncodingStats stats)
+    public ImmutableBTreePartition(TableMetadata metadata,
+                                   DecoratedKey partitionKey,
+                                   RegularAndStaticColumns columns,
+                                   Row staticRow,
+                                   Object[] tree,
+                                   DeletionInfo deletionInfo,
+                                   EncodingStats stats)
     {
-        super(metadata, partitionKey);
+        super(partitionKey);
+        this.metadata = metadata;
         this.holder = new Holder(columns, tree, deletionInfo, staticRow, stats);
     }
 
-    protected ImmutableBTreePartition(CFMetaData metadata,
+    protected ImmutableBTreePartition(TableMetadata metadata,
                                       DecoratedKey partitionKey,
                                       Holder holder)
     {
-        super(metadata, partitionKey);
+        super(partitionKey);
+        this.metadata = metadata;
         this.holder = holder;
     }
 
@@ -111,6 +114,11 @@ public class ImmutableBTreePartition extends AbstractBTreePartition
         return new ImmutableBTreePartition(iterator.metadata(), iterator.partitionKey(), build(iterator, initialRowCapacity, ordered));
     }
 
+    public TableMetadata metadata()
+    {
+        return metadata;
+    }
+
     protected Holder holder()
     {
         return holder;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/partitions/Partition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/Partition.java b/src/java/org/apache/cassandra/db/partitions/Partition.java
index 04568e9..baeb6d5 100644
--- a/src/java/org/apache/cassandra/db/partitions/Partition.java
+++ b/src/java/org/apache/cassandra/db/partitions/Partition.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.db.partitions;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.Slices;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
@@ -34,11 +34,11 @@ import org.apache.cassandra.utils.SearchIterator;
  */
 public interface Partition
 {
-    public CFMetaData metadata();
+    public TableMetadata metadata();
     public DecoratedKey partitionKey();
     public DeletionTime partitionLevelDeletion();
 
-    public PartitionColumns columns();
+    public RegularAndStaticColumns columns();
 
     public EncodingStats stats();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
index 1ed961f..33c9408 100644
--- a/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
+++ b/src/java/org/apache/cassandra/db/partitions/PartitionUpdate.java
@@ -28,13 +28,14 @@ import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.util.*;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.btree.BTree;
 import org.apache.cassandra.utils.btree.UpdateFunction;
@@ -74,48 +75,52 @@ public class PartitionUpdate extends AbstractBTreePartition
 
     private final boolean canHaveShadowedData;
 
-    private PartitionUpdate(CFMetaData metadata,
+    private final TableMetadata metadata;
+
+    private PartitionUpdate(TableMetadata metadata,
                             DecoratedKey key,
-                            PartitionColumns columns,
+                            RegularAndStaticColumns columns,
                             MutableDeletionInfo deletionInfo,
                             int initialRowCapacity,
                             boolean canHaveShadowedData)
     {
-        super(metadata, key);
+        super(key);
+        this.metadata = metadata;
         this.deletionInfo = deletionInfo;
         this.holder = new Holder(columns, BTree.empty(), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS);
         this.canHaveShadowedData = canHaveShadowedData;
         rowBuilder = builder(initialRowCapacity);
     }
 
-    private PartitionUpdate(CFMetaData metadata,
+    private PartitionUpdate(TableMetadata metadata,
                             DecoratedKey key,
                             Holder holder,
                             MutableDeletionInfo deletionInfo,
                             boolean canHaveShadowedData)
     {
-        super(metadata, key);
+        super(key);
+        this.metadata = metadata;
         this.holder = holder;
         this.deletionInfo = deletionInfo;
         this.isBuilt = true;
         this.canHaveShadowedData = canHaveShadowedData;
     }
 
-    public PartitionUpdate(CFMetaData metadata,
+    public PartitionUpdate(TableMetadata metadata,
                            DecoratedKey key,
-                           PartitionColumns columns,
+                           RegularAndStaticColumns columns,
                            int initialRowCapacity)
     {
         this(metadata, key, columns, MutableDeletionInfo.live(), initialRowCapacity, true);
     }
 
-    public PartitionUpdate(CFMetaData metadata,
+    public PartitionUpdate(TableMetadata metadata,
                            ByteBuffer key,
-                           PartitionColumns columns,
+                           RegularAndStaticColumns columns,
                            int initialRowCapacity)
     {
         this(metadata,
-             metadata.decorateKey(key),
+             metadata.partitioner.decorateKey(key),
              columns,
              initialRowCapacity);
     }
@@ -128,10 +133,10 @@ public class PartitionUpdate extends AbstractBTreePartition
      *
      * @return the newly created empty (and immutable) update.
      */
-    public static PartitionUpdate emptyUpdate(CFMetaData metadata, DecoratedKey key)
+    public static PartitionUpdate emptyUpdate(TableMetadata metadata, DecoratedKey key)
     {
         MutableDeletionInfo deletionInfo = MutableDeletionInfo.live();
-        Holder holder = new Holder(PartitionColumns.NONE, BTree.empty(), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS);
+        Holder holder = new Holder(RegularAndStaticColumns.NONE, BTree.empty(), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS);
         return new PartitionUpdate(metadata, key, holder, deletionInfo, false);
     }
 
@@ -145,10 +150,10 @@ public class PartitionUpdate extends AbstractBTreePartition
      *
      * @return the newly created partition deletion update.
      */
-    public static PartitionUpdate fullPartitionDelete(CFMetaData metadata, DecoratedKey key, long timestamp, int nowInSec)
+    public static PartitionUpdate fullPartitionDelete(TableMetadata metadata, DecoratedKey key, long timestamp, int nowInSec)
     {
         MutableDeletionInfo deletionInfo = new MutableDeletionInfo(timestamp, nowInSec);
-        Holder holder = new Holder(PartitionColumns.NONE, BTree.empty(), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS);
+        Holder holder = new Holder(RegularAndStaticColumns.NONE, BTree.empty(), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS);
         return new PartitionUpdate(metadata, key, holder, deletionInfo, false);
     }
 
@@ -161,17 +166,17 @@ public class PartitionUpdate extends AbstractBTreePartition
      *
      * @return the newly created partition update containing only {@code row}.
      */
-    public static PartitionUpdate singleRowUpdate(CFMetaData metadata, DecoratedKey key, Row row)
+    public static PartitionUpdate singleRowUpdate(TableMetadata metadata, DecoratedKey key, Row row)
     {
         MutableDeletionInfo deletionInfo = MutableDeletionInfo.live();
         if (row.isStatic())
         {
-            Holder holder = new Holder(new PartitionColumns(Columns.from(row.columns()), Columns.NONE), BTree.empty(), deletionInfo, row, EncodingStats.NO_STATS);
+            Holder holder = new Holder(new RegularAndStaticColumns(Columns.from(row.columns()), Columns.NONE), BTree.empty(), deletionInfo, row, EncodingStats.NO_STATS);
             return new PartitionUpdate(metadata, key, holder, deletionInfo, false);
         }
         else
         {
-            Holder holder = new Holder(new PartitionColumns(Columns.NONE, Columns.from(row.columns())), BTree.singleton(row), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS);
+            Holder holder = new Holder(new RegularAndStaticColumns(Columns.NONE, Columns.from(row.columns())), BTree.singleton(row), deletionInfo, Rows.EMPTY_STATIC_ROW, EncodingStats.NO_STATS);
             return new PartitionUpdate(metadata, key, holder, deletionInfo, false);
         }
     }
@@ -185,9 +190,9 @@ public class PartitionUpdate extends AbstractBTreePartition
      *
      * @return the newly created partition update containing only {@code row}.
      */
-    public static PartitionUpdate singleRowUpdate(CFMetaData metadata, ByteBuffer key, Row row)
+    public static PartitionUpdate singleRowUpdate(TableMetadata metadata, ByteBuffer key, Row row)
     {
-        return singleRowUpdate(metadata, metadata.decorateKey(key), row);
+        return singleRowUpdate(metadata, metadata.partitioner.decorateKey(key), row);
     }
 
     /**
@@ -289,9 +294,9 @@ public class PartitionUpdate extends AbstractBTreePartition
      *
      * @return the newly created partition deletion update.
      */
-    public static PartitionUpdate fullPartitionDelete(CFMetaData metadata, ByteBuffer key, long timestamp, int nowInSec)
+    public static PartitionUpdate fullPartitionDelete(TableMetadata metadata, ByteBuffer key, long timestamp, int nowInSec)
     {
-        return fullPartitionDelete(metadata, metadata.decorateKey(key), timestamp, nowInSec);
+        return fullPartitionDelete(metadata, metadata.partitioner.decorateKey(key), timestamp, nowInSec);
     }
 
     /**
@@ -371,8 +376,13 @@ public class PartitionUpdate extends AbstractBTreePartition
         return size;
     }
 
+    public TableMetadata metadata()
+    {
+        return metadata;
+    }
+
     @Override
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
         // The superclass implementation calls holder(), but that triggers a build of the PartitionUpdate. But since
         // the columns are passed to the ctor, we know the holder always has the proper columns even if it doesn't have
@@ -415,7 +425,7 @@ public class PartitionUpdate extends AbstractBTreePartition
 
     private BTree.Builder<Row> builder(int initialCapacity)
     {
-        return BTree.<Row>builder(metadata.comparator, initialCapacity)
+        return BTree.<Row>builder(metadata().comparator, initialCapacity)
                     .setQuickResolver((a, b) ->
                                       Rows.merge(a, b, createdAtInSec));
     }
@@ -526,7 +536,7 @@ public class PartitionUpdate extends AbstractBTreePartition
     public void add(RangeTombstone range)
     {
         assertNotBuilt();
-        deletionInfo.add(range, metadata.comparator);
+        deletionInfo.add(range, metadata().comparator);
     }
 
     /**
@@ -582,7 +592,7 @@ public class PartitionUpdate extends AbstractBTreePartition
         Holder holder = this.holder;
         Object[] cur = holder.tree;
         Object[] add = rowBuilder.build();
-        Object[] merged = BTree.<Row>merge(cur, add, metadata.comparator,
+        Object[] merged = BTree.<Row>merge(cur, add, metadata().comparator,
                                            UpdateFunction.Simple.of((a, b) -> Rows.merge(a, b, createdAtInSec)));
 
         assert deletionInfo == holder.deletionInfo;
@@ -604,10 +614,9 @@ public class PartitionUpdate extends AbstractBTreePartition
         // modified.
 
         StringBuilder sb = new StringBuilder();
-        sb.append(String.format("[%s.%s] key=%s columns=%s",
-                                metadata.ksName,
-                                metadata.cfName,
-                                metadata.getKeyValidator().getString(partitionKey().getKey()),
+        sb.append(String.format("[%s] key=%s columns=%s",
+                                metadata.toString(),
+                                metadata.partitionKeyType.getString(partitionKey().getKey()),
                                 columns()));
 
         sb.append("\n    deletionInfo=").append(deletionInfo);
@@ -624,7 +633,7 @@ public class PartitionUpdate extends AbstractBTreePartition
      * Int32Type, string for UTF8Type, ...). It is also allowed to pass a single {@code DecoratedKey} value directly.
      * @return a newly created builder.
      */
-    public static SimpleBuilder simpleBuilder(CFMetaData metadata, Object... partitionKeyValues)
+    public static SimpleBuilder simpleBuilder(TableMetadata metadata, Object... partitionKeyValues)
     {
         return new SimpleBuilders.PartitionUpdateBuilder(metadata, partitionKeyValues);
     }
@@ -640,7 +649,7 @@ public class PartitionUpdate extends AbstractBTreePartition
         /**
          * The metadata of the table this is a builder on.
          */
-        public CFMetaData metadata();
+        public TableMetadata metadata();
 
         /**
          * Sets the timestamp to use for the following additions to this builder or any derived (row) builder.
@@ -777,14 +786,14 @@ public class PartitionUpdate extends AbstractBTreePartition
             {
                 assert !iter.isReverseOrder();
 
-                CFMetaData.serializer.serialize(update.metadata(), out, version);
+                update.metadata.id.serialize(out);
                 UnfilteredRowIteratorSerializer.serializer.serialize(iter, null, out, version, update.rowCount());
             }
         }
 
         public PartitionUpdate deserialize(DataInputPlus in, int version, SerializationHelper.Flag flag) throws IOException
         {
-            CFMetaData metadata = CFMetaData.serializer.deserialize(in, version);
+            TableMetadata metadata = Schema.instance.getExistingTableMetadata(TableId.deserialize(in));
             UnfilteredRowIteratorSerializer.Header header = UnfilteredRowIteratorSerializer.serializer.deserializeHeader(metadata, null, in, version, flag);
             if (header.isEmpty)
                 return emptyUpdate(metadata, header.key);
@@ -820,7 +829,7 @@ public class PartitionUpdate extends AbstractBTreePartition
         {
             try (UnfilteredRowIterator iter = update.unfilteredIterator())
             {
-                return CFMetaData.serializer.serializedSize(update.metadata(), version)
+                return update.metadata.id.serializedSize()
                      + UnfilteredRowIteratorSerializer.serializer.serializedSize(iter, null, version, update.rowCount());
             }
         }
@@ -834,10 +843,10 @@ public class PartitionUpdate extends AbstractBTreePartition
     public static class CounterMark
     {
         private final Row row;
-        private final ColumnDefinition column;
+        private final ColumnMetadata column;
         private final CellPath path;
 
-        private CounterMark(Row row, ColumnDefinition column, CellPath path)
+        private CounterMark(Row row, ColumnMetadata column, CellPath path)
         {
             this.row = row;
             this.column = column;
@@ -849,7 +858,7 @@ public class PartitionUpdate extends AbstractBTreePartition
             return row.clustering();
         }
 
-        public ColumnDefinition column()
+        public ColumnMetadata column()
         {
             return column;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java b/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
index 3b968e6..b739e8b 100644
--- a/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
+++ b/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.db.partitions;
 
 import java.util.NoSuchElementException;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 
 public class SingletonUnfilteredPartitionIterator implements UnfilteredPartitionIterator
@@ -32,7 +32,7 @@ public class SingletonUnfilteredPartitionIterator implements UnfilteredPartition
         this.iter = iter;
     }
 
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
         return iter.metadata();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterator.java b/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterator.java
index 872225f..cd8e47f 100644
--- a/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterator.java
+++ b/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterator.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.db.partitions;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 
 /**
@@ -30,5 +30,5 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator;
  */
 public interface UnfilteredPartitionIterator extends BasePartitionIterator<UnfilteredRowIterator>
 {
-    public CFMetaData metadata();
+    public TableMetadata metadata();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java b/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java
index 76420cf..40eaad1 100644
--- a/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java
+++ b/src/java/org/apache/cassandra/db/partitions/UnfilteredPartitionIterators.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.security.MessageDigest;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.*;
@@ -31,7 +30,7 @@ import org.apache.cassandra.db.transform.MorePartitions;
 import org.apache.cassandra.db.transform.Transformation;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.MergeIterator;
 
 /**
@@ -113,7 +112,7 @@ public abstract class UnfilteredPartitionIterators
         assert listener != null;
         assert !iterators.isEmpty();
 
-        final CFMetaData metadata = iterators.get(0).metadata();
+        final TableMetadata metadata = iterators.get(0).metadata();
 
         final MergeIterator<UnfilteredRowIterator, UnfilteredRowIterator> merged = MergeIterator.get(iterators, partitionComparator, new MergeIterator.Reducer<UnfilteredRowIterator, UnfilteredRowIterator>()
         {
@@ -154,7 +153,7 @@ public abstract class UnfilteredPartitionIterators
 
         return new AbstractUnfilteredPartitionIterator()
         {
-            public CFMetaData metadata()
+            public TableMetadata metadata()
             {
                 return metadata;
             }
@@ -185,7 +184,7 @@ public abstract class UnfilteredPartitionIterators
         if (iterators.size() == 1)
             return iterators.get(0);
 
-        final CFMetaData metadata = iterators.get(0).metadata();
+        final TableMetadata metadata = iterators.get(0).metadata();
 
         final MergeIterator<UnfilteredRowIterator, UnfilteredRowIterator> merged = MergeIterator.get(iterators, partitionComparator, new MergeIterator.Reducer<UnfilteredRowIterator, UnfilteredRowIterator>()
         {
@@ -215,7 +214,7 @@ public abstract class UnfilteredPartitionIterators
 
         return new AbstractUnfilteredPartitionIterator()
         {
-            public CFMetaData metadata()
+            public TableMetadata metadata()
             {
                 return metadata;
             }
@@ -304,7 +303,7 @@ public abstract class UnfilteredPartitionIterators
             out.writeBoolean(false);
         }
 
-        public UnfilteredPartitionIterator deserialize(final DataInputPlus in, final int version, final CFMetaData metadata, final ColumnFilter selection, final SerializationHelper.Flag flag) throws IOException
+        public UnfilteredPartitionIterator deserialize(final DataInputPlus in, final int version, final TableMetadata metadata, final ColumnFilter selection, final SerializationHelper.Flag flag) throws IOException
         {
             // Skip now unused isForThrift boolean
             in.readBoolean();
@@ -315,7 +314,7 @@ public abstract class UnfilteredPartitionIterators
                 private boolean hasNext;
                 private boolean nextReturned = true;
 
-                public CFMetaData metadata()
+                public TableMetadata metadata()
                 {
                     return metadata;
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/AbstractCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractCell.java b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
index ca83783..a9b058e 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractCell.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
@@ -21,7 +21,7 @@ import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.util.Objects;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.DeletionPurger;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.context.CounterContext;
@@ -39,7 +39,7 @@ import org.apache.cassandra.utils.memory.AbstractAllocator;
  */
 public abstract class AbstractCell extends Cell
 {
-    protected AbstractCell(ColumnDefinition column)
+    protected AbstractCell(ColumnMetadata column)
     {
         super(column);
     }
@@ -139,7 +139,7 @@ public abstract class AbstractCell extends Cell
             throw new MarshalException("Shoud not have a TTL without an associated local deletion time");
 
         // non-frozen UDTs require both the cell path & value to validate,
-        // so that logic is pushed down into ColumnDefinition. Tombstone
+        // so that logic is pushed down into ColumnMetadata. Tombstone
         // validation is done there too as it also involves the cell path
         // for complex columns
         column().validateCell(this);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java b/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
index 153243c..a7c48c1 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractRangeTombstoneMarker.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.db.rows;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.ClusteringBoundOrBoundary;
 
 public abstract class AbstractRangeTombstoneMarker<B extends ClusteringBoundOrBoundary> implements RangeTombstoneMarker
@@ -56,7 +56,7 @@ public abstract class AbstractRangeTombstoneMarker<B extends ClusteringBoundOrBo
         return bound.isClose(reversed);
     }
 
-    public void validateData(CFMetaData metadata)
+    public void validateData(TableMetadata metadata)
     {
         ClusteringBoundOrBoundary bound = clustering();
         for (int i = 0; i < bound.size(); i++)
@@ -67,11 +67,11 @@ public abstract class AbstractRangeTombstoneMarker<B extends ClusteringBoundOrBo
         }
     }
 
-    public String toString(CFMetaData metadata, boolean fullDetails)
+    public String toString(TableMetadata metadata, boolean fullDetails)
     {
         return toString(metadata);
     }
-    public String toString(CFMetaData metadata, boolean includeClusteringKeys, boolean fullDetails)
+    public String toString(TableMetadata metadata, boolean includeClusteringKeys, boolean fullDetails)
     {
         return toString(metadata);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/AbstractRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractRow.java b/src/java/org/apache/cassandra/db/rows/AbstractRow.java
index 356722e..c869a1a 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractRow.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractRow.java
@@ -26,7 +26,7 @@ import java.util.stream.StreamSupport;
 
 import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.UserType;
@@ -71,7 +71,7 @@ public abstract class AbstractRow extends AbstractCollection<ColumnData> impleme
             cd.digest(digest);
     }
 
-    public void validateData(CFMetaData metadata)
+    public void validateData(TableMetadata metadata)
     {
         Clustering clustering = clustering();
         for (int i = 0; i < clustering.size(); i++)
@@ -89,17 +89,17 @@ public abstract class AbstractRow extends AbstractCollection<ColumnData> impleme
             cd.validate();
     }
 
-    public String toString(CFMetaData metadata)
+    public String toString(TableMetadata metadata)
     {
         return toString(metadata, false);
     }
 
-    public String toString(CFMetaData metadata, boolean fullDetails)
+    public String toString(TableMetadata metadata, boolean fullDetails)
     {
         return toString(metadata, true, fullDetails);
     }
 
-    public String toString(CFMetaData metadata, boolean includeClusterKeys, boolean fullDetails)
+    public String toString(TableMetadata metadata, boolean includeClusterKeys, boolean fullDetails)
     {
         StringBuilder sb = new StringBuilder();
         sb.append("Row");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/AbstractUnfilteredRowIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractUnfilteredRowIterator.java b/src/java/org/apache/cassandra/db/rows/AbstractUnfilteredRowIterator.java
index f2389a7..2c3f78f 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractUnfilteredRowIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractUnfilteredRowIterator.java
@@ -17,25 +17,25 @@
  */
 package org.apache.cassandra.db.rows;
 
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.AbstractIterator;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 
 public abstract class AbstractUnfilteredRowIterator extends AbstractIterator<Unfiltered> implements UnfilteredRowIterator
 {
-    protected final CFMetaData metadata;
+    protected final TableMetadata metadata;
     protected final DecoratedKey partitionKey;
     protected final DeletionTime partitionLevelDeletion;
-    protected final PartitionColumns columns;
+    protected final RegularAndStaticColumns columns;
     protected final Row staticRow;
     protected final boolean isReverseOrder;
     protected final EncodingStats stats;
 
-    protected AbstractUnfilteredRowIterator(CFMetaData metadata,
+    protected AbstractUnfilteredRowIterator(TableMetadata metadata,
                                             DecoratedKey partitionKey,
                                             DeletionTime partitionLevelDeletion,
-                                            PartitionColumns columns,
+                                            RegularAndStaticColumns columns,
                                             Row staticRow,
                                             boolean isReverseOrder,
                                             EncodingStats stats)
@@ -49,12 +49,12 @@ public abstract class AbstractUnfilteredRowIterator extends AbstractIterator<Unf
         this.stats = stats;
     }
 
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
         return metadata;
     }
 
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
         return columns;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/BTreeRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/BTreeRow.java b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
index 54da16b..0fd6337 100644
--- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java
+++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
@@ -27,12 +27,13 @@ import com.google.common.collect.Collections2;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.DroppedColumn;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.btree.BTree;
 import org.apache.cassandra.utils.btree.BTreeSearchIterator;
@@ -191,7 +192,7 @@ public class BTreeRow extends AbstractRow
         return clustering;
     }
 
-    public Collection<ColumnDefinition> columns()
+    public Collection<ColumnMetadata> columns()
     {
         return Collections2.transform(this, ColumnData::column);
     }
@@ -213,13 +214,13 @@ public class BTreeRow extends AbstractRow
         return deletion;
     }
 
-    public Cell getCell(ColumnDefinition c)
+    public Cell getCell(ColumnMetadata c)
     {
         assert !c.isComplex();
-        return (Cell) BTree.<Object>find(btree, ColumnDefinition.asymmetricColumnDataComparator, c);
+        return (Cell) BTree.<Object>find(btree, ColumnMetadata.asymmetricColumnDataComparator, c);
     }
 
-    public Cell getCell(ColumnDefinition c, CellPath path)
+    public Cell getCell(ColumnMetadata c, CellPath path)
     {
         assert c.isComplex();
         ComplexColumnData cd = getComplexColumnData(c);
@@ -228,10 +229,10 @@ public class BTreeRow extends AbstractRow
         return cd.getCell(path);
     }
 
-    public ComplexColumnData getComplexColumnData(ColumnDefinition c)
+    public ComplexColumnData getComplexColumnData(ColumnMetadata c)
     {
         assert c.isComplex();
-        return (ComplexColumnData) BTree.<Object>find(btree, ColumnDefinition.asymmetricColumnDataComparator, c);
+        return (ComplexColumnData) BTree.<Object>find(btree, ColumnMetadata.asymmetricColumnDataComparator, c);
     }
 
     public int size()
@@ -249,19 +250,19 @@ public class BTreeRow extends AbstractRow
         return CellIterator::new;
     }
 
-    public BTreeSearchIterator<ColumnDefinition, ColumnData> searchIterator()
+    public BTreeSearchIterator<ColumnMetadata, ColumnData> searchIterator()
     {
-        return BTree.slice(btree, ColumnDefinition.asymmetricColumnDataComparator, BTree.Dir.ASC);
+        return BTree.slice(btree, ColumnMetadata.asymmetricColumnDataComparator, BTree.Dir.ASC);
     }
 
-    public Row filter(ColumnFilter filter, CFMetaData metadata)
+    public Row filter(ColumnFilter filter, TableMetadata metadata)
     {
         return filter(filter, DeletionTime.LIVE, false, metadata);
     }
 
-    public Row filter(ColumnFilter filter, DeletionTime activeDeletion, boolean setActiveDeletionToRow, CFMetaData metadata)
+    public Row filter(ColumnFilter filter, DeletionTime activeDeletion, boolean setActiveDeletionToRow, TableMetadata metadata)
     {
-        Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns = metadata.getDroppedColumns();
+        Map<ByteBuffer, DroppedColumn> droppedColumns = metadata.droppedColumns;
 
         boolean mayFilterColumns = !filter.fetchesAllColumns(isStatic());
         boolean mayHaveShadowed = activeDeletion.supersedes(deletion.time());
@@ -282,16 +283,16 @@ public class BTreeRow extends AbstractRow
         }
 
         Columns columns = filter.fetchedColumns().columns(isStatic());
-        Predicate<ColumnDefinition> inclusionTester = columns.inOrderInclusionTester();
-        Predicate<ColumnDefinition> queriedByUserTester = filter.queriedColumns().columns(isStatic()).inOrderInclusionTester();
+        Predicate<ColumnMetadata> inclusionTester = columns.inOrderInclusionTester();
+        Predicate<ColumnMetadata> queriedByUserTester = filter.queriedColumns().columns(isStatic()).inOrderInclusionTester();
         final LivenessInfo rowLiveness = newInfo;
         return transformAndFilter(newInfo, newDeletion, (cd) -> {
 
-            ColumnDefinition column = cd.column();
+            ColumnMetadata column = cd.column();
             if (!inclusionTester.test(column))
                 return null;
 
-            CFMetaData.DroppedColumn dropped = droppedColumns.get(column.name.bytes);
+            DroppedColumn dropped = droppedColumns.get(column.name.bytes);
             if (column.isComplex())
                 return ((ComplexColumnData) cd).filter(filter, mayHaveShadowed ? activeDeletion : DeletionTime.LIVE, dropped, rowLiveness);
 
@@ -313,7 +314,7 @@ public class BTreeRow extends AbstractRow
 
         return transformAndFilter(primaryKeyLivenessInfo, deletion, (cd) -> {
 
-            ColumnDefinition column = cd.column();
+            ColumnMetadata column = cd.column();
             if (column.isComplex())
                 return ((ComplexColumnData)cd).withOnlyQueriedData(filter);
 
@@ -455,16 +456,16 @@ public class BTreeRow extends AbstractRow
     // assumption that Row objects are immutable. This method should go away post-#6506 in particular.
     // This method is in particular not exposed by the Row API on purpose.
     // This method also *assumes* that the cell we're setting already exists.
-    public void setValue(ColumnDefinition column, CellPath path, ByteBuffer value)
+    public void setValue(ColumnMetadata column, CellPath path, ByteBuffer value)
     {
-        ColumnData current = (ColumnData) BTree.<Object>find(btree, ColumnDefinition.asymmetricColumnDataComparator, column);
+        ColumnData current = (ColumnData) BTree.<Object>find(btree, ColumnMetadata.asymmetricColumnDataComparator, column);
         if (column.isSimple())
             BTree.replaceInSitu(btree, ColumnData.comparator, current, ((Cell) current).withUpdatedValue(value));
         else
             ((ComplexColumnData) current).setValue(path, value);
     }
 
-    public Iterable<Cell> cellsInLegacyOrder(CFMetaData metadata, boolean reversed)
+    public Iterable<Cell> cellsInLegacyOrder(TableMetadata metadata, boolean reversed)
     {
         return () -> new CellInLegacyOrderIterator(metadata, reversed);
     }
@@ -508,9 +509,9 @@ public class BTreeRow extends AbstractRow
         private Iterator<Cell> complexCells;
         private final Object[] data;
 
-        private CellInLegacyOrderIterator(CFMetaData metadata, boolean reversed)
+        private CellInLegacyOrderIterator(TableMetadata metadata, boolean reversed)
         {
-            AbstractType<?> nameComparator = metadata.getColumnDefinitionNameComparator(isStatic() ? ColumnDefinition.Kind.STATIC : ColumnDefinition.Kind.REGULAR);
+            AbstractType<?> nameComparator = metadata.columnDefinitionNameComparator(isStatic() ? ColumnMetadata.Kind.STATIC : ColumnMetadata.Kind.REGULAR);
             this.comparator = reversed ? Collections.reverseOrder(nameComparator) : nameComparator;
             this.reversed = reversed;
 
@@ -591,7 +592,7 @@ public class BTreeRow extends AbstractRow
         // a simple marker class that will sort to the beginning of a run of complex cells to store the deletion time
         private static class ComplexColumnDeletion extends BufferCell
         {
-            public ComplexColumnDeletion(ColumnDefinition column, DeletionTime deletionTime)
+            public ComplexColumnDeletion(ColumnMetadata column, DeletionTime deletionTime)
             {
                 super(column, deletionTime.markedForDeleteAt(), 0, deletionTime.localDeletionTime(), ByteBufferUtil.EMPTY_BYTE_BUFFER, CellPath.BOTTOM);
             }
@@ -609,7 +610,7 @@ public class BTreeRow extends AbstractRow
             public ColumnData resolve(Object[] cells, int lb, int ub)
             {
                 Cell cell = (Cell) cells[lb];
-                ColumnDefinition column = cell.column;
+                ColumnMetadata column = cell.column;
                 if (cell.column.isSimple())
                 {
                     assert lb + 1 == ub || nowInSec != Integer.MIN_VALUE;
@@ -738,7 +739,7 @@ public class BTreeRow extends AbstractRow
             hasComplex |= cell.column.isComplex();
         }
 
-        public void addComplexDeletion(ColumnDefinition column, DeletionTime complexDeletion)
+        public void addComplexDeletion(ColumnMetadata column, DeletionTime complexDeletion)
         {
             getCells().add(new ComplexColumnDeletion(column, complexDeletion));
             hasComplex = true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/BaseRowIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/BaseRowIterator.java b/src/java/org/apache/cassandra/db/rows/BaseRowIterator.java
index ce37297..4033248 100644
--- a/src/java/org/apache/cassandra/db/rows/BaseRowIterator.java
+++ b/src/java/org/apache/cassandra/db/rows/BaseRowIterator.java
@@ -18,9 +18,9 @@
 */
 package org.apache.cassandra.db.rows;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.PartitionColumns;
+import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.utils.CloseableIterator;
 
 /**
@@ -32,7 +32,7 @@ public interface BaseRowIterator<U extends Unfiltered> extends CloseableIterator
     /**
      * The metadata for the table this iterator on.
      */
-    public CFMetaData metadata();
+    public TableMetadata metadata();
 
     /**
      * Whether or not the rows returned by this iterator are in reversed
@@ -44,7 +44,7 @@ public interface BaseRowIterator<U extends Unfiltered> extends CloseableIterator
      * A subset of the columns for the (static and regular) rows returned by this iterator.
      * Every row returned by this iterator must guarantee that it has only those columns.
      */
-    public PartitionColumns columns();
+    public RegularAndStaticColumns columns();
 
     /**
      * The partition key of the partition this in an iterator over.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/BufferCell.java b/src/java/org/apache/cassandra/db/rows/BufferCell.java
index 9b31c16..76c6d3e 100644
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.db.rows;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.marshal.ByteType;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
@@ -27,7 +27,7 @@ import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 public class BufferCell extends AbstractCell
 {
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferCell(ColumnDefinition.regularDef("", "", "", ByteType.instance), 0L, 0, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, null));
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferCell(ColumnMetadata.regularColumn("", "", "", ByteType.instance), 0L, 0, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, null));
 
     private final long timestamp;
     private final int ttl;
@@ -36,7 +36,7 @@ public class BufferCell extends AbstractCell
     private final ByteBuffer value;
     private final CellPath path;
 
-    public BufferCell(ColumnDefinition column, long timestamp, int ttl, int localDeletionTime, ByteBuffer value, CellPath path)
+    public BufferCell(ColumnMetadata column, long timestamp, int ttl, int localDeletionTime, ByteBuffer value, CellPath path)
     {
         super(column);
         assert !column.isPrimaryKeyColumn();
@@ -48,33 +48,33 @@ public class BufferCell extends AbstractCell
         this.path = path;
     }
 
-    public static BufferCell live(ColumnDefinition column, long timestamp, ByteBuffer value)
+    public static BufferCell live(ColumnMetadata column, long timestamp, ByteBuffer value)
     {
         return live(column, timestamp, value, null);
     }
 
-    public static BufferCell live(ColumnDefinition column, long timestamp, ByteBuffer value, CellPath path)
+    public static BufferCell live(ColumnMetadata column, long timestamp, ByteBuffer value, CellPath path)
     {
         return new BufferCell(column, timestamp, NO_TTL, NO_DELETION_TIME, value, path);
     }
 
-    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value)
+    public static BufferCell expiring(ColumnMetadata column, long timestamp, int ttl, int nowInSec, ByteBuffer value)
     {
         return expiring(column, timestamp, ttl, nowInSec, value, null);
     }
 
-    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value, CellPath path)
+    public static BufferCell expiring(ColumnMetadata column, long timestamp, int ttl, int nowInSec, ByteBuffer value, CellPath path)
     {
         assert ttl != NO_TTL;
         return new BufferCell(column, timestamp, ttl, nowInSec + ttl, value, path);
     }
 
-    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec)
+    public static BufferCell tombstone(ColumnMetadata column, long timestamp, int nowInSec)
     {
         return tombstone(column, timestamp, nowInSec, null);
     }
 
-    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec, CellPath path)
+    public static BufferCell tombstone(ColumnMetadata column, long timestamp, int nowInSec, CellPath path)
     {
         return new BufferCell(column, timestamp, NO_TTL, nowInSec, ByteBufferUtil.EMPTY_BYTE_BUFFER, path);
     }
@@ -104,7 +104,7 @@ public class BufferCell extends AbstractCell
         return path;
     }
 
-    public Cell withUpdatedColumn(ColumnDefinition newColumn)
+    public Cell withUpdatedColumn(ColumnMetadata newColumn)
     {
         return new BufferCell(newColumn, timestamp, ttl, localDeletionTime, value, path);
     }


[15/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/paxos/PaxosState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosState.java b/src/java/org/apache/cassandra/service/paxos/PaxosState.java
index ee1ba6a..cf7f3d3 100644
--- a/src/java/org/apache/cassandra/service/paxos/PaxosState.java
+++ b/src/java/org/apache/cassandra/service/paxos/PaxosState.java
@@ -27,7 +27,7 @@ import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.Striped;
 import com.google.common.util.concurrent.Uninterruptibles;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.tracing.Tracing;
@@ -41,7 +41,7 @@ public class PaxosState
     private final Commit accepted;
     private final Commit mostRecentCommit;
 
-    public PaxosState(DecoratedKey key, CFMetaData metadata)
+    public PaxosState(DecoratedKey key, TableMetadata metadata)
     {
         this(Commit.emptyCommit(key, metadata), Commit.emptyCommit(key, metadata), Commit.emptyCommit(key, metadata));
     }
@@ -92,7 +92,7 @@ public class PaxosState
         }
         finally
         {
-            Keyspace.open(toPrepare.update.metadata().ksName).getColumnFamilyStore(toPrepare.update.metadata().cfId).metric.casPrepare.addNano(System.nanoTime() - start);
+            Keyspace.open(toPrepare.update.metadata().keyspace).getColumnFamilyStore(toPrepare.update.metadata().id).metric.casPrepare.addNano(System.nanoTime() - start);
         }
 
     }
@@ -127,7 +127,7 @@ public class PaxosState
         }
         finally
         {
-            Keyspace.open(proposal.update.metadata().ksName).getColumnFamilyStore(proposal.update.metadata().cfId).metric.casPropose.addNano(System.nanoTime() - start);
+            Keyspace.open(proposal.update.metadata().keyspace).getColumnFamilyStore(proposal.update.metadata().id).metric.casPropose.addNano(System.nanoTime() - start);
         }
     }
 
@@ -143,7 +143,7 @@ public class PaxosState
             // erase the in-progress update.
             // The table may have been truncated since the proposal was initiated. In that case, we
             // don't want to perform the mutation and potentially resurrect truncated data
-            if (UUIDGen.unixTimestamp(proposal.ballot) >= SystemKeyspace.getTruncatedAt(proposal.update.metadata().cfId))
+            if (UUIDGen.unixTimestamp(proposal.ballot) >= SystemKeyspace.getTruncatedAt(proposal.update.metadata().id))
             {
                 Tracing.trace("Committing proposal {}", proposal);
                 Mutation mutation = proposal.makeMutation();
@@ -158,7 +158,7 @@ public class PaxosState
         }
         finally
         {
-            Keyspace.open(proposal.update.metadata().ksName).getColumnFamilyStore(proposal.update.metadata().cfId).metric.casCommit.addNano(System.nanoTime() - start);
+            Keyspace.open(proposal.update.metadata().keyspace).getColumnFamilyStore(proposal.update.metadata().id).metric.casCommit.addNano(System.nanoTime() - start);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
index 5915eab..381c498 100644
--- a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
+++ b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
@@ -28,12 +28,13 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.DecoratedKey;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.utils.UUIDGen;
@@ -49,7 +50,7 @@ public class PrepareCallback extends AbstractPaxosCallback<PrepareResponse>
 
     private final Map<InetAddress, Commit> commitsByReplica = new ConcurrentHashMap<InetAddress, Commit>();
 
-    public PrepareCallback(DecoratedKey key, CFMetaData metadata, int targets, ConsistencyLevel consistency, long queryStartNanoTime)
+    public PrepareCallback(DecoratedKey key, TableMetadata metadata, int targets, ConsistencyLevel consistency, long queryStartNanoTime)
     {
         super(targets, consistency, queryStartNanoTime);
         // need to inject the right key in the empty commit so comparing with empty commits in the reply works as expected
@@ -89,7 +90,7 @@ public class PrepareCallback extends AbstractPaxosCallback<PrepareResponse>
         latch.countDown();
     }
 
-    public Iterable<InetAddress> replicasMissingMostRecentCommit(CFMetaData metadata, int nowInSec)
+    public Iterable<InetAddress> replicasMissingMostRecentCommit(TableMetadata metadata, int nowInSec)
     {
         // In general, we need every replicas that have answered to the prepare (a quorum) to agree on the MRC (see
         // coment in StorageProxy.beginAndRepairPaxos(), but basically we need to make sure at least a quorum of nodes

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReader.java b/src/java/org/apache/cassandra/streaming/StreamReader.java
index fab9372..0eee6f0 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -31,8 +31,8 @@ import org.slf4j.LoggerFactory;
 
 import com.ning.compress.lzf.LZFInputStream;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
@@ -42,22 +42,19 @@ import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.RewindableDataInputStreamPlus;
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.streaming.messages.FileMessageHeader;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.io.util.TrackedInputStream;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
-import static org.apache.cassandra.utils.Throwables.extractIOExceptionCause;
-
 /**
  * StreamReader reads from stream and writes to SSTable.
  */
 public class StreamReader
 {
     private static final Logger logger = LoggerFactory.getLogger(StreamReader.class);
-    protected final UUID cfId;
+    protected final TableId tableId;
     protected final long estimatedKeys;
     protected final Collection<Pair<Long, Long>> sections;
     protected final StreamSession session;
@@ -71,7 +68,7 @@ public class StreamReader
     public StreamReader(FileMessageHeader header, StreamSession session)
     {
         this.session = session;
-        this.cfId = header.cfId;
+        this.tableId = header.tableId;
         this.estimatedKeys = header.estimatedKeys;
         this.sections = header.sections;
         this.inputVersion = header.version;
@@ -92,15 +89,11 @@ public class StreamReader
     {
         long totalSize = totalSize();
 
-        Pair<String, String> kscf = Schema.instance.getCF(cfId);
-        ColumnFamilyStore cfs = null;
-        if (kscf != null)
-            cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
-
-        if (kscf == null || cfs == null)
+        ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId);
+        if (cfs == null)
         {
             // schema was dropped during streaming
-            throw new IOException("CF " + cfId + " was dropped during streaming");
+            throw new IOException("CF " + tableId + " was dropped during streaming");
         }
 
         logger.debug("[Stream #{}] Start receiving file #{} from {}, repairedAt = {}, size = {}, ks = '{}', table = '{}'.",
@@ -108,7 +101,7 @@ public class StreamReader
                      cfs.getColumnFamilyName());
 
         TrackedInputStream in = new TrackedInputStream(new LZFInputStream(Channels.newInputStream(channel)));
-        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, getHeader(cfs.metadata),
+        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata(), in, inputVersion, getHeader(cfs.metadata()),
                                                                  totalSize, session.planId());
         SSTableMultiWriter writer = null;
         try
@@ -142,7 +135,7 @@ public class StreamReader
         }
     }
 
-    protected SerializationHeader getHeader(CFMetaData metadata)
+    protected SerializationHeader getHeader(TableMetadata metadata)
     {
         return header != null? header.toHeader(metadata) : null; //pre-3.0 sstable have no SerializationHeader
     }
@@ -153,7 +146,7 @@ public class StreamReader
         if (localDir == null)
             throw new IOException(String.format("Insufficient disk space to store %s", FBUtilities.prettyPrintMemory(totalSize)));
 
-        RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, format, sstableLevel, totalSize, session.getTransaction(cfId), getHeader(cfs.metadata));
+        RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, format, sstableLevel, totalSize, session.getTransaction(tableId), getHeader(cfs.metadata()));
         StreamHook.instance.reportIncomingFile(cfs, writer, session, fileSeqNum);
         return writer;
     }
@@ -181,7 +174,7 @@ public class StreamReader
         public static final String BUFFER_FILE_PREFIX = "buf";
         public static final String BUFFER_FILE_SUFFIX = "dat";
 
-        private final CFMetaData metadata;
+        private final TableMetadata metadata;
         private final DataInputPlus in;
         private final SerializationHeader header;
         private final SerializationHelper helper;
@@ -192,7 +185,7 @@ public class StreamReader
         private Row staticRow;
         private IOException exception;
 
-        public StreamDeserializer(CFMetaData metadata, InputStream in, Version version, SerializationHeader header,
+        public StreamDeserializer(TableMetadata metadata, InputStream in, Version version, SerializationHeader header,
                                   long totalSize, UUID sessionId) throws IOException
         {
             this.metadata = metadata;
@@ -203,22 +196,22 @@ public class StreamReader
 
         public StreamDeserializer newPartition() throws IOException
         {
-            key = metadata.decorateKey(ByteBufferUtil.readWithShortLength(in));
+            key = metadata.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(in));
             partitionLevelDeletion = DeletionTime.serializer.deserialize(in);
             iterator = SSTableSimpleIterator.create(metadata, in, header, helper, partitionLevelDeletion);
             staticRow = iterator.readStaticRow();
             return this;
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
             return metadata;
         }
 
-        public PartitionColumns columns()
+        public RegularAndStaticColumns columns()
         {
             // We don't know which columns we'll get so assume it can be all of them
-            return metadata.partitionColumns();
+            return metadata.regularAndStaticColumns();
         }
 
         public boolean isReverseOrder()
@@ -308,13 +301,13 @@ public class StreamReader
             }
         }
 
-        private static File getTempBufferFile(CFMetaData metadata, long totalSize, UUID sessionId) throws IOException
+        private static File getTempBufferFile(TableMetadata metadata, long totalSize, UUID sessionId) throws IOException
         {
-            ColumnFamilyStore cfs = Keyspace.open(metadata.ksName).getColumnFamilyStore(metadata.cfName);
+            ColumnFamilyStore cfs = Keyspace.open(metadata.keyspace).getColumnFamilyStore(metadata.name);
             if (cfs == null)
             {
                 // schema was dropped during streaming
-                throw new RuntimeException(String.format("CF %s.%s was dropped during streaming", metadata.ksName, metadata.cfName));
+                throw new RuntimeException(String.format("Table %s was dropped during streaming", metadata.toString()));
             }
 
             long maxSize = Math.min(MAX_SPILL_FILE_SIZE, totalSize);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index 6c60b74..d0c4d50 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
@@ -30,7 +29,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
@@ -45,8 +43,8 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.Throwables;
 import org.apache.cassandra.utils.concurrent.Refs;
 
@@ -75,9 +73,9 @@ public class StreamReceiveTask extends StreamTask
 
     private int remoteSSTablesReceived = 0;
 
-    public StreamReceiveTask(StreamSession session, UUID cfId, int totalFiles, long totalSize)
+    public StreamReceiveTask(StreamSession session, TableId tableId, int totalFiles, long totalSize)
     {
-        super(session, cfId);
+        super(session, tableId);
         this.totalFiles = totalFiles;
         this.totalSize = totalSize;
         // this is an "offline" transaction, as we currently manually expose the sstables once done;
@@ -102,7 +100,7 @@ public class StreamReceiveTask extends StreamTask
         }
 
         remoteSSTablesReceived++;
-        assert cfId.equals(sstable.getCfId());
+        assert tableId.equals(sstable.getTableId());
 
         Collection<SSTableReader> finished = null;
         try
@@ -136,7 +134,7 @@ public class StreamReceiveTask extends StreamTask
     public synchronized LifecycleTransaction getTransaction()
     {
         if (done)
-            throw new RuntimeException(String.format("Stream receive task %s of cf %s already finished.", session.planId(), cfId));
+            throw new RuntimeException(String.format("Stream receive task %s of cf %s already finished.", session.planId(), tableId));
         return txn;
     }
 
@@ -156,8 +154,8 @@ public class StreamReceiveTask extends StreamTask
             ColumnFamilyStore cfs = null;
             try
             {
-                Pair<String, String> kscf = Schema.instance.getCF(task.cfId);
-                if (kscf == null)
+                cfs = ColumnFamilyStore.getIfExists(task.tableId);
+                if (cfs == null)
                 {
                     // schema was dropped during streaming
                     task.sstables.clear();
@@ -165,9 +163,8 @@ public class StreamReceiveTask extends StreamTask
                     task.session.taskCompleted(task);
                     return;
                 }
-                cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
-                hasViews = !Iterables.isEmpty(View.findAll(kscf.left, kscf.right));
-                hasCDC = cfs.metadata.params.cdc;
+                hasViews = !Iterables.isEmpty(View.findAll(cfs.metadata.keyspace, cfs.getTableName()));
+                hasCDC = cfs.metadata().params.cdc;
 
                 Collection<SSTableReader> readers = task.sstables;
 
@@ -193,7 +190,7 @@ public class StreamReceiveTask extends StreamTask
                                 {
                                     try (UnfilteredRowIterator rowIterator = scanner.next())
                                     {
-                                        Mutation m = new Mutation(PartitionUpdate.fromIterator(rowIterator, ColumnFilter.all(cfs.metadata)));
+                                        Mutation m = new Mutation(PartitionUpdate.fromIterator(rowIterator, ColumnFilter.all(cfs.metadata())));
 
                                         // MV *can* be applied unsafe if there's no CDC on the CFS as we flush below
                                         // before transaction is done.
@@ -215,7 +212,7 @@ public class StreamReceiveTask extends StreamTask
                         cfs.indexManager.buildAllIndexesBlocking(readers);
 
                         //invalidate row and counter cache
-                        if (cfs.isRowCacheEnabled() || cfs.metadata.isCounter())
+                        if (cfs.isRowCacheEnabled() || cfs.metadata().isCounter())
                         {
                             List<Bounds<Token>> boundsToInvalidate = new ArrayList<>(readers.size());
                             readers.forEach(sstable -> boundsToInvalidate.add(new Bounds<Token>(sstable.first.getToken(), sstable.last.getToken())));
@@ -230,7 +227,7 @@ public class StreamReceiveTask extends StreamTask
                                                  cfs.keyspace.getName(), cfs.getTableName());
                             }
 
-                            if (cfs.metadata.isCounter())
+                            if (cfs.metadata().isCounter())
                             {
                                 int invalidatedKeys = cfs.invalidateCounterCache(nonOverlappingBounds);
                                 if (invalidatedKeys > 0)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index 736d30f..faa05d1 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -45,6 +45,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.*;
 import org.apache.cassandra.metrics.StreamingMetrics;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.streaming.messages.*;
 import org.apache.cassandra.utils.CassandraVersion;
@@ -147,9 +148,9 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     protected final Set<StreamRequest> requests = Sets.newConcurrentHashSet();
     // streaming tasks are created and managed per ColumnFamily ID
     @VisibleForTesting
-    protected final ConcurrentHashMap<UUID, StreamTransferTask> transfers = new ConcurrentHashMap<>();
+    protected final ConcurrentHashMap<TableId, StreamTransferTask> transfers = new ConcurrentHashMap<>();
     // data receivers, filled after receiving prepare message
-    private final Map<UUID, StreamReceiveTask> receivers = new ConcurrentHashMap<>();
+    private final Map<TableId, StreamReceiveTask> receivers = new ConcurrentHashMap<>();
     private final StreamingMetrics metrics;
     /* can be null when session is created in remote */
     private final StreamConnectionFactory factory;
@@ -223,10 +224,10 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     }
 
 
-    public LifecycleTransaction getTransaction(UUID cfId)
+    public LifecycleTransaction getTransaction(TableId tableId)
     {
-        assert receivers.containsKey(cfId);
-        return receivers.get(cfId).getTransaction();
+        assert receivers.containsKey(tableId);
+        return receivers.get(tableId).getTransaction();
     }
 
     private boolean isKeepAliveSupported()
@@ -424,13 +425,13 @@ public class StreamSession implements IEndpointStateChangeSubscriber
                 continue;
             }
 
-            UUID cfId = details.ref.get().metadata.cfId;
-            StreamTransferTask task = transfers.get(cfId);
+            TableId tableId = details.ref.get().metadata().id;
+            StreamTransferTask task = transfers.get(tableId);
             if (task == null)
             {
                 //guarantee atomicity
-                StreamTransferTask newTask = new StreamTransferTask(this, cfId);
-                task = transfers.putIfAbsent(cfId, newTask);
+                StreamTransferTask newTask = new StreamTransferTask(this, tableId);
+                task = transfers.putIfAbsent(tableId, newTask);
                 if (task == null)
                     task = newTask;
             }
@@ -525,7 +526,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
 
             case RECEIVED:
                 ReceivedMessage received = (ReceivedMessage) message;
-                received(received.cfId, received.sequenceNumber);
+                received(received.tableId, received.sequenceNumber);
                 break;
 
             case COMPLETE:
@@ -634,7 +635,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         StreamingMetrics.totalOutgoingBytes.inc(headerSize);
         metrics.outgoingBytes.inc(headerSize);
         // schedule timeout for receiving ACK
-        StreamTransferTask task = transfers.get(header.cfId);
+        StreamTransferTask task = transfers.get(header.tableId);
         if (task != null)
         {
             task.scheduleTimeout(header.sequenceNumber, 12, TimeUnit.HOURS);
@@ -652,8 +653,8 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         StreamingMetrics.totalIncomingBytes.inc(headerSize);
         metrics.incomingBytes.inc(headerSize);
         // send back file received message
-        handler.sendMessage(new ReceivedMessage(message.header.cfId, message.header.sequenceNumber));
-        receivers.get(message.header.cfId).received(message.sstable);
+        handler.sendMessage(new ReceivedMessage(message.header.tableId, message.header.sequenceNumber));
+        receivers.get(message.header.tableId).received(message.sstable);
     }
 
     public void progress(String filename, ProgressInfo.Direction direction, long bytes, long total)
@@ -662,9 +663,9 @@ public class StreamSession implements IEndpointStateChangeSubscriber
         streamResult.handleProgress(progress);
     }
 
-    public void received(UUID cfId, int sequenceNumber)
+    public void received(TableId tableId, int sequenceNumber)
     {
-        transfers.get(cfId).complete(sequenceNumber);
+        transfers.get(tableId).complete(sequenceNumber);
     }
 
     /**
@@ -723,13 +724,13 @@ public class StreamSession implements IEndpointStateChangeSubscriber
 
     public synchronized void taskCompleted(StreamReceiveTask completedTask)
     {
-        receivers.remove(completedTask.cfId);
+        receivers.remove(completedTask.tableId);
         maybeCompleted();
     }
 
     public synchronized void taskCompleted(StreamTransferTask completedTask)
     {
-        transfers.remove(completedTask.cfId);
+        transfers.remove(completedTask.tableId);
         maybeCompleted();
     }
 
@@ -793,7 +794,7 @@ public class StreamSession implements IEndpointStateChangeSubscriber
     {
         failIfFinished();
         if (summary.files > 0)
-            receivers.put(summary.cfId, new StreamReceiveTask(this, summary.cfId, summary.files, summary.totalSize));
+            receivers.put(summary.tableId, new StreamReceiveTask(this, summary.tableId, summary.files, summary.totalSize));
     }
 
     private void startStreamingFiles()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/StreamSummary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSummary.java b/src/java/org/apache/cassandra/streaming/StreamSummary.java
index c427283..0d94f57 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSummary.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSummary.java
@@ -28,6 +28,7 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 /**
@@ -37,7 +38,7 @@ public class StreamSummary implements Serializable
 {
     public static final IVersionedSerializer<StreamSummary> serializer = new StreamSummarySerializer();
 
-    public final UUID cfId;
+    public final TableId tableId;
 
     /**
      * Number of files to transfer. Can be 0 if nothing to transfer for some streaming request.
@@ -45,9 +46,9 @@ public class StreamSummary implements Serializable
     public final int files;
     public final long totalSize;
 
-    public StreamSummary(UUID cfId, int files, long totalSize)
+    public StreamSummary(TableId tableId, int files, long totalSize)
     {
-        this.cfId = cfId;
+        this.tableId = tableId;
         this.files = files;
         this.totalSize = totalSize;
     }
@@ -58,20 +59,20 @@ public class StreamSummary implements Serializable
         if (this == o) return true;
         if (o == null || getClass() != o.getClass()) return false;
         StreamSummary summary = (StreamSummary) o;
-        return files == summary.files && totalSize == summary.totalSize && cfId.equals(summary.cfId);
+        return files == summary.files && totalSize == summary.totalSize && tableId.equals(summary.tableId);
     }
 
     @Override
     public int hashCode()
     {
-        return Objects.hashCode(cfId, files, totalSize);
+        return Objects.hashCode(tableId, files, totalSize);
     }
 
     @Override
     public String toString()
     {
         final StringBuilder sb = new StringBuilder("StreamSummary{");
-        sb.append("path=").append(cfId);
+        sb.append("path=").append(tableId);
         sb.append(", files=").append(files);
         sb.append(", totalSize=").append(totalSize);
         sb.append('}');
@@ -80,25 +81,24 @@ public class StreamSummary implements Serializable
 
     public static class StreamSummarySerializer implements IVersionedSerializer<StreamSummary>
     {
-        // arbitrary version is fine for UUIDSerializer for now...
         public void serialize(StreamSummary summary, DataOutputPlus out, int version) throws IOException
         {
-            UUIDSerializer.serializer.serialize(summary.cfId, out, MessagingService.current_version);
+            summary.tableId.serialize(out);
             out.writeInt(summary.files);
             out.writeLong(summary.totalSize);
         }
 
         public StreamSummary deserialize(DataInputPlus in, int version) throws IOException
         {
-            UUID cfId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
+            TableId tableId = TableId.deserialize(in);
             int files = in.readInt();
             long totalSize = in.readLong();
-            return new StreamSummary(cfId, files, totalSize);
+            return new StreamSummary(tableId, files, totalSize);
         }
 
         public long serializedSize(StreamSummary summary, int version)
         {
-            long size = UUIDSerializer.serializer.serializedSize(summary.cfId, MessagingService.current_version);
+            long size = summary.tableId.serializedSize();
             size += TypeSizes.sizeof(summary.files);
             size += TypeSizes.sizeof(summary.totalSize);
             return size;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/StreamTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamTask.java b/src/java/org/apache/cassandra/streaming/StreamTask.java
index ac72cff..1e22c34 100644
--- a/src/java/org/apache/cassandra/streaming/StreamTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamTask.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.util.UUID;
+import org.apache.cassandra.schema.TableId;
 
 /**
  * StreamTask is an abstraction of the streaming task performed over specific ColumnFamily.
@@ -27,12 +27,12 @@ public abstract class StreamTask
     /** StreamSession that this task belongs */
     protected final StreamSession session;
 
-    protected final UUID cfId;
+    protected final TableId tableId;
 
-    protected StreamTask(StreamSession session, UUID cfId)
+    protected StreamTask(StreamSession session, TableId tableId)
     {
         this.session = session;
-        this.cfId = cfId;
+        this.tableId = tableId;
     }
 
     /**
@@ -56,6 +56,6 @@ public abstract class StreamTask
      */
     public StreamSummary getSummary()
     {
-        return new StreamSummary(cfId, getTotalNumberOfFiles(), getTotalSize());
+        return new StreamSummary(tableId, getTotalNumberOfFiles(), getTotalSize());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
index 4f313c3..aa3251b 100644
--- a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
@@ -27,6 +27,7 @@ import com.google.common.base.Throwables;
 
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.messages.OutgoingFileMessage;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.concurrent.Ref;
@@ -47,14 +48,14 @@ public class StreamTransferTask extends StreamTask
 
     private long totalSize;
 
-    public StreamTransferTask(StreamSession session, UUID cfId)
+    public StreamTransferTask(StreamSession session, TableId tableId)
     {
-        super(session, cfId);
+        super(session, tableId);
     }
 
     public synchronized void addTransferFile(Ref<SSTableReader> ref, long estimatedKeys, List<Pair<Long, Long>> sections, long repairedAt)
     {
-        assert ref.get() != null && cfId.equals(ref.get().metadata.cfId);
+        assert ref.get() != null && tableId.equals(ref.get().metadata().id);
         OutgoingFileMessage message = new OutgoingFileMessage(ref, sequenceNumber.getAndIncrement(), estimatedKeys, sections, repairedAt, session.keepSSTableLevel());
         message = StreamHook.instance.reportOutgoingFile(session, ref.get(), message);
         files.put(message.header.sequenceNumber, message);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
index 2044d4d..6ac607f 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -25,9 +25,7 @@ import com.google.common.base.Throwables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.util.TrackedInputStream;
@@ -66,15 +64,12 @@ public class CompressedStreamReader extends StreamReader
     {
         long totalSize = totalSize();
 
-        Pair<String, String> kscf = Schema.instance.getCF(cfId);
-        ColumnFamilyStore cfs = null;
-        if (kscf != null)
-            cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
+        ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(tableId);
 
-        if (kscf == null || cfs == null)
+        if (cfs == null)
         {
             // schema was dropped during streaming
-            throw new IOException("CF " + cfId + " was dropped during streaming");
+            throw new IOException("CF " + tableId + " was dropped during streaming");
         }
 
         logger.debug("[Stream #{}] Start receiving file #{} from {}, repairedAt = {}, size = {}, ks = '{}', table = '{}'.",
@@ -85,7 +80,7 @@ public class CompressedStreamReader extends StreamReader
                                                               ChecksumType.CRC32, cfs::getCrcCheckChance);
         TrackedInputStream in = new TrackedInputStream(cis);
 
-        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, getHeader(cfs.metadata),
+        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata(), in, inputVersion, getHeader(cfs.metadata()),
                                                                  totalSize, session.planId());
         SSTableMultiWriter writer = null;
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/management/StreamSummaryCompositeData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/management/StreamSummaryCompositeData.java b/src/java/org/apache/cassandra/streaming/management/StreamSummaryCompositeData.java
index 9ef23ab..a1f2496 100644
--- a/src/java/org/apache/cassandra/streaming/management/StreamSummaryCompositeData.java
+++ b/src/java/org/apache/cassandra/streaming/management/StreamSummaryCompositeData.java
@@ -19,18 +19,18 @@ package org.apache.cassandra.streaming.management;
 
 import java.util.HashMap;
 import java.util.Map;
-import java.util.UUID;
 import javax.management.openmbean.*;
 
 import com.google.common.base.Throwables;
 
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.StreamSummary;
 
 /**
  */
 public class StreamSummaryCompositeData
 {
-    private static final String[] ITEM_NAMES = new String[]{"cfId",
+    private static final String[] ITEM_NAMES = new String[]{"tableId",
                                                             "files",
                                                             "totalSize"};
     private static final String[] ITEM_DESCS = new String[]{"ColumnFamilu ID",
@@ -60,7 +60,7 @@ public class StreamSummaryCompositeData
     public static CompositeData toCompositeData(StreamSummary streamSummary)
     {
         Map<String, Object> valueMap = new HashMap<>();
-        valueMap.put(ITEM_NAMES[0], streamSummary.cfId.toString());
+        valueMap.put(ITEM_NAMES[0], streamSummary.tableId.toString());
         valueMap.put(ITEM_NAMES[1], streamSummary.files);
         valueMap.put(ITEM_NAMES[2], streamSummary.totalSize);
         try
@@ -76,7 +76,7 @@ public class StreamSummaryCompositeData
     public static StreamSummary fromCompositeData(CompositeData cd)
     {
         Object[] values = cd.getAll(ITEM_NAMES);
-        return new StreamSummary(UUID.fromString((String) values[0]),
+        return new StreamSummary(TableId.fromString((String) values[0]),
                                  (int) values[1],
                                  (long) values[2]);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
index b0639ea..a37420b 100644
--- a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.streaming.messages;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.UUID;
 
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.TypeSizes;
@@ -30,9 +29,9 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.compress.CompressionInfo;
 import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.UUIDSerializer;
 
 /**
  * StreamingFileHeader is appended before sending actual data to describe what it's sending.
@@ -41,7 +40,7 @@ public class FileMessageHeader
 {
     public static FileMessageHeaderSerializer serializer = new FileMessageHeaderSerializer();
 
-    public final UUID cfId;
+    public final TableId tableId;
     public final int sequenceNumber;
     /** SSTable version */
     public final Version version;
@@ -64,7 +63,7 @@ public class FileMessageHeader
     /* cached size value */
     private transient final long size;
 
-    public FileMessageHeader(UUID cfId,
+    public FileMessageHeader(TableId tableId,
                              int sequenceNumber,
                              Version version,
                              SSTableFormat.Type format,
@@ -75,7 +74,7 @@ public class FileMessageHeader
                              int sstableLevel,
                              SerializationHeader.Component header)
     {
-        this.cfId = cfId;
+        this.tableId = tableId;
         this.sequenceNumber = sequenceNumber;
         this.version = version;
         this.format = format;
@@ -89,7 +88,7 @@ public class FileMessageHeader
         this.size = calculateSize();
     }
 
-    public FileMessageHeader(UUID cfId,
+    public FileMessageHeader(TableId tableId,
                              int sequenceNumber,
                              Version version,
                              SSTableFormat.Type format,
@@ -100,7 +99,7 @@ public class FileMessageHeader
                              int sstableLevel,
                              SerializationHeader.Component header)
     {
-        this.cfId = cfId;
+        this.tableId = tableId;
         this.sequenceNumber = sequenceNumber;
         this.version = version;
         this.format = format;
@@ -152,7 +151,7 @@ public class FileMessageHeader
     public String toString()
     {
         final StringBuilder sb = new StringBuilder("Header (");
-        sb.append("cfId: ").append(cfId);
+        sb.append("tableId: ").append(tableId);
         sb.append(", #").append(sequenceNumber);
         sb.append(", version: ").append(version);
         sb.append(", format: ").append(format);
@@ -171,13 +170,13 @@ public class FileMessageHeader
         if (this == o) return true;
         if (o == null || getClass() != o.getClass()) return false;
         FileMessageHeader that = (FileMessageHeader) o;
-        return sequenceNumber == that.sequenceNumber && cfId.equals(that.cfId);
+        return sequenceNumber == that.sequenceNumber && tableId.equals(that.tableId);
     }
 
     @Override
     public int hashCode()
     {
-        int result = cfId.hashCode();
+        int result = tableId.hashCode();
         result = 31 * result + sequenceNumber;
         return result;
     }
@@ -186,7 +185,7 @@ public class FileMessageHeader
     {
         public CompressionInfo serialize(FileMessageHeader header, DataOutputPlus out, int version) throws IOException
         {
-            UUIDSerializer.serializer.serialize(header.cfId, out, version);
+            header.tableId.serialize(out);
             out.writeInt(header.sequenceNumber);
             out.writeUTF(header.version.toString());
             out.writeUTF(header.format.name);
@@ -212,7 +211,7 @@ public class FileMessageHeader
 
         public FileMessageHeader deserialize(DataInputPlus in, int version) throws IOException
         {
-            UUID cfId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
+            TableId tableId = TableId.deserialize(in);
             int sequenceNumber = in.readInt();
             Version sstableVersion = SSTableFormat.Type.current().info.getVersion(in.readUTF());
             SSTableFormat.Type format = SSTableFormat.Type.validate(in.readUTF());
@@ -227,12 +226,12 @@ public class FileMessageHeader
             int sstableLevel = in.readInt();
             SerializationHeader.Component header =  SerializationHeader.serializer.deserialize(sstableVersion, in);
 
-            return new FileMessageHeader(cfId, sequenceNumber, sstableVersion, format, estimatedKeys, sections, compressionInfo, repairedAt, sstableLevel, header);
+            return new FileMessageHeader(tableId, sequenceNumber, sstableVersion, format, estimatedKeys, sections, compressionInfo, repairedAt, sstableLevel, header);
         }
 
         public long serializedSize(FileMessageHeader header, int version)
         {
-            long size = UUIDSerializer.serializer.serializedSize(header.cfId, version);
+            long size = header.tableId.serializedSize();
             size += TypeSizes.sizeof(header.sequenceNumber);
             size += TypeSizes.sizeof(header.version.toString());
             size += TypeSizes.sizeof(header.format.name);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java b/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
index 6723d17..fba9ec4 100644
--- a/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
@@ -72,7 +72,7 @@ public class OutgoingFileMessage extends StreamMessage
 
         SSTableReader sstable = ref.get();
         filename = sstable.getFilename();
-        this.header = new FileMessageHeader(sstable.metadata.cfId,
+        this.header = new FileMessageHeader(sstable.metadata().id,
                                             sequenceNumber,
                                             sstable.descriptor.version,
                                             sstable.descriptor.formatType,
@@ -81,7 +81,7 @@ public class OutgoingFileMessage extends StreamMessage
                                             sstable.compression ? sstable.getCompressionMetadata() : null,
                                             repairedAt,
                                             keepSSTableLevel ? sstable.getSSTableLevel() : 0,
-                                            sstable.header == null ? null : sstable.header.toComponent());
+                                            sstable.header.toComponent());
     }
 
     public synchronized void serialize(DataOutputStreamPlus out, int version, StreamSession session) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
index 251b9c8..55dd7e6 100644
--- a/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
@@ -20,14 +20,12 @@ package org.apache.cassandra.streaming.messages;
 import java.io.*;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
-import java.util.UUID;
 
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.streaming.StreamSession;
-import org.apache.cassandra.utils.UUIDSerializer;
 
 public class ReceivedMessage extends StreamMessage
 {
@@ -37,23 +35,23 @@ public class ReceivedMessage extends StreamMessage
         public ReceivedMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
         {
             DataInputPlus input = new DataInputStreamPlus(Channels.newInputStream(in));
-            return new ReceivedMessage(UUIDSerializer.serializer.deserialize(input, MessagingService.current_version), input.readInt());
+            return new ReceivedMessage(TableId.deserialize(input), input.readInt());
         }
 
         public void serialize(ReceivedMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
         {
-            UUIDSerializer.serializer.serialize(message.cfId, out, MessagingService.current_version);
+            message.tableId.serialize(out);
             out.writeInt(message.sequenceNumber);
         }
     };
 
-    public final UUID cfId;
+    public final TableId tableId;
     public final int sequenceNumber;
 
-    public ReceivedMessage(UUID cfId, int sequenceNumber)
+    public ReceivedMessage(TableId tableId, int sequenceNumber)
     {
         super(Type.RECEIVED);
-        this.cfId = cfId;
+        this.tableId = tableId;
         this.sequenceNumber = sequenceNumber;
     }
 
@@ -61,7 +59,7 @@ public class ReceivedMessage extends StreamMessage
     public String toString()
     {
         final StringBuilder sb = new StringBuilder("Received (");
-        sb.append(cfId).append(", #").append(sequenceNumber).append(')');
+        sb.append(tableId).append(", #").append(sequenceNumber).append(')');
         return sb.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/tools/JsonTransformer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/JsonTransformer.java b/src/java/org/apache/cassandra/tools/JsonTransformer.java
index dde732a..1d05103 100644
--- a/src/java/org/apache/cassandra/tools/JsonTransformer.java
+++ b/src/java/org/apache/cassandra/tools/JsonTransformer.java
@@ -30,8 +30,9 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Stream;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
@@ -46,14 +47,14 @@ import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.Unfiltered;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonGenerator;
 import org.codehaus.jackson.impl.Indenter;
-import org.codehaus.jackson.util.DefaultPrettyPrinter;
 import org.codehaus.jackson.util.DefaultPrettyPrinter.NopIndenter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.codehaus.jackson.util.DefaultPrettyPrinter;
 
 public final class JsonTransformer
 {
@@ -68,7 +69,7 @@ public final class JsonTransformer
 
     private final CompactIndenter arrayIndenter = new CompactIndenter();
 
-    private final CFMetaData metadata;
+    private final TableMetadata metadata;
 
     private final ISSTableScanner currentScanner;
 
@@ -76,7 +77,7 @@ public final class JsonTransformer
 
     private long currentPosition = 0;
 
-    private JsonTransformer(JsonGenerator json, ISSTableScanner currentScanner, boolean rawTime, CFMetaData metadata)
+    private JsonTransformer(JsonGenerator json, ISSTableScanner currentScanner, boolean rawTime, TableMetadata metadata)
     {
         this.json = json;
         this.metadata = metadata;
@@ -89,7 +90,7 @@ public final class JsonTransformer
         json.setPrettyPrinter(prettyPrinter);
     }
 
-    public static void toJson(ISSTableScanner currentScanner, Stream<UnfilteredRowIterator> partitions, boolean rawTime, CFMetaData metadata, OutputStream out)
+    public static void toJson(ISSTableScanner currentScanner, Stream<UnfilteredRowIterator> partitions, boolean rawTime, TableMetadata metadata, OutputStream out)
             throws IOException
     {
         try (JsonGenerator json = jsonFactory.createJsonGenerator(new OutputStreamWriter(out, StandardCharsets.UTF_8)))
@@ -101,7 +102,7 @@ public final class JsonTransformer
         }
     }
 
-    public static void keysToJson(ISSTableScanner currentScanner, Stream<DecoratedKey> keys, boolean rawTime, CFMetaData metadata, OutputStream out) throws IOException
+    public static void keysToJson(ISSTableScanner currentScanner, Stream<DecoratedKey> keys, boolean rawTime, TableMetadata metadata, OutputStream out) throws IOException
     {
         try (JsonGenerator json = jsonFactory.createJsonGenerator(new OutputStreamWriter(out, StandardCharsets.UTF_8)))
         {
@@ -119,7 +120,7 @@ public final class JsonTransformer
 
     private void serializePartitionKey(DecoratedKey key)
     {
-        AbstractType<?> keyValidator = metadata.getKeyValidator();
+        AbstractType<?> keyValidator = metadata.partitionKeyType;
         objectIndenter.setCompact(true);
         try
         {
@@ -223,7 +224,7 @@ public final class JsonTransformer
         }
         catch (IOException e)
         {
-            String key = metadata.getKeyValidator().getString(partition.partitionKey().getKey());
+            String key = metadata.partitionKeyType.getString(partition.partitionKey().getKey());
             logger.error("Fatal error parsing partition: {}", key, e);
         }
     }
@@ -334,10 +335,10 @@ public final class JsonTransformer
             objectIndenter.setCompact(true);
             json.writeStartArray();
             arrayIndenter.setCompact(true);
-            List<ColumnDefinition> clusteringColumns = metadata.clusteringColumns();
+            List<ColumnMetadata> clusteringColumns = metadata.clusteringColumns();
             for (int i = 0; i < clusteringColumns.size(); i++)
             {
-                ColumnDefinition column = clusteringColumns.get(i);
+                ColumnMetadata column = clusteringColumns.get(i);
                 if (i >= clustering.size())
                 {
                     json.writeString("*");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java b/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java
index 1f407cb..56c57d9 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.tools;
 
-import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Collections;
 import java.util.HashSet;
@@ -27,8 +26,8 @@ import java.util.Set;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
@@ -46,7 +45,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
  */
 public class SSTableExpiredBlockers
 {
-    public static void main(String[] args) throws IOException
+    public static void main(String[] args)
     {
         PrintStream out = System.out;
         if (args.length < 2)
@@ -61,11 +60,7 @@ public class SSTableExpiredBlockers
         String columnfamily = args[args.length - 1];
         Schema.instance.loadFromDisk(false);
 
-        CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnfamily);
-        if (metadata == null)
-            throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",
-                    keyspace,
-                    columnfamily));
+        TableMetadata metadata = Schema.instance.validateTable(keyspace, columnfamily);
 
         Keyspace ks = Keyspace.openWithoutSSTables(keyspace);
         ColumnFamilyStore cfs = ks.getColumnFamilyStore(columnfamily);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 52d5ecf..913ee1f 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -27,7 +27,7 @@ import java.util.stream.StreamSupport;
 
 import org.apache.commons.cli.*;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.DecoratedKey;
@@ -43,6 +43,7 @@ import org.apache.cassandra.io.sstable.KeyIterator;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -88,10 +89,10 @@ public class SSTableExport
      * Construct table schema from info stored in SSTable's Stats.db
      *
      * @param desc SSTable's descriptor
-     * @return Restored CFMetaData
+     * @return Restored TableMetadata
      * @throws IOException when Stats.db cannot be read
      */
-    public static CFMetaData metadataFromSSTable(Descriptor desc) throws IOException
+    public static TableMetadata metadataFromSSTable(Descriptor desc) throws IOException
     {
         if (!desc.version.isCompatible())
             throw new IOException("Cannot process old and unsupported SSTable version.");
@@ -101,7 +102,7 @@ public class SSTableExport
         SerializationHeader.Component header = (SerializationHeader.Component) sstableMetadata.get(MetadataType.HEADER);
         IPartitioner partitioner = FBUtilities.newPartitioner(desc);
 
-        CFMetaData.Builder builder = CFMetaData.Builder.create("keyspace", "table").withPartitioner(partitioner);
+        TableMetadata.Builder builder = TableMetadata.builder("keyspace", "table").partitioner(partitioner);
         header.getStaticColumns().entrySet().stream()
                 .forEach(entry -> {
                     ColumnIdentifier ident = ColumnIdentifier.getInterned(UTF8Type.instance.getString(entry.getKey()), true);
@@ -112,7 +113,7 @@ public class SSTableExport
                     ColumnIdentifier ident = ColumnIdentifier.getInterned(UTF8Type.instance.getString(entry.getKey()), true);
                     builder.addRegularColumn(ident, entry.getValue());
                 });
-        builder.addPartitionKey("PartitionKey", header.getKeyType());
+        builder.addPartitionKeyColumn("PartitionKey", header.getKeyType());
         for (int i = 0; i < header.getClusteringTypes().size(); i++)
         {
             builder.addClusteringColumn("clustering" + (i > 0 ? i : ""), header.getClusteringTypes().get(i));
@@ -170,7 +171,7 @@ public class SSTableExport
         Descriptor desc = Descriptor.fromFilename(ssTableFileName);
         try
         {
-            CFMetaData metadata = metadataFromSSTable(desc);
+            TableMetadata metadata = metadataFromSSTable(desc);
             if (cmd.hasOption(ENUMERATE_KEYS_OPTION))
             {
                 try (KeyIterator iter = new KeyIterator(desc, metadata))
@@ -183,14 +184,14 @@ public class SSTableExport
             }
             else
             {
-                SSTableReader sstable = SSTableReader.openNoValidation(desc, metadata);
+                SSTableReader sstable = SSTableReader.openNoValidation(desc, TableMetadataRef.forOfflineTools(metadata));
                 IPartitioner partitioner = sstable.getPartitioner();
                 final ISSTableScanner currentScanner;
                 if ((keys != null) && (keys.length > 0))
                 {
                     List<AbstractBounds<PartitionPosition>> bounds = Arrays.stream(keys)
                             .filter(key -> !excludes.contains(key))
-                            .map(metadata.getKeyValidator()::fromString)
+                            .map(metadata.partitionKeyType::fromString)
                             .map(partitioner::decorateKey)
                             .sorted()
                             .map(DecoratedKey::getToken)
@@ -202,7 +203,7 @@ public class SSTableExport
                     currentScanner = sstable.getScanner();
                 }
                 Stream<UnfilteredRowIterator> partitions = iterToStream(currentScanner).filter(i ->
-                    excludes.isEmpty() || !excludes.contains(metadata.getKeyValidator().getString(i.partitionKey().getKey()))
+                    excludes.isEmpty() || !excludes.contains(metadata.partitionKeyType.getString(i.partitionKey().getKey()))
                 );
                 if (cmd.hasOption(DEBUG_OUTPUT_OPTION))
                 {
@@ -213,19 +214,19 @@ public class SSTableExport
 
                         if (!partition.partitionLevelDeletion().isLive())
                         {
-                            System.out.println("[" + metadata.getKeyValidator().getString(partition.partitionKey().getKey()) + "]@" +
+                            System.out.println("[" + metadata.partitionKeyType.getString(partition.partitionKey().getKey()) + "]@" +
                                                position.get() + " " + partition.partitionLevelDeletion());
                         }
                         if (!partition.staticRow().isEmpty())
                         {
-                            System.out.println("[" + metadata.getKeyValidator().getString(partition.partitionKey().getKey()) + "]@" +
+                            System.out.println("[" + metadata.partitionKeyType.getString(partition.partitionKey().getKey()) + "]@" +
                                                position.get() + " " + partition.staticRow().toString(metadata, true));
                         }
                         partition.forEachRemaining(row ->
                         {
                             System.out.println(
-                                    "[" + metadata.getKeyValidator().getString(partition.partitionKey().getKey()) + "]@"
-                                            + position.get() + " " + row.toString(metadata, false, true));
+                            "[" + metadata.partitionKeyType.getString(partition.partitionKey().getKey()) + "]@"
+                            + position.get() + " " + row.toString(metadata, false, true));
                             position.set(currentScanner.getCurrentPosition());
                         });
                     });

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 915edf1..3a66ef9 100644
--- a/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
+++ b/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
@@ -21,7 +21,7 @@ import java.io.PrintStream;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
@@ -69,7 +69,7 @@ public class SSTableLevelResetter
             String keyspaceName = args[1];
             String columnfamily = args[2];
             // validate columnfamily
-            if (Schema.instance.getCFMetaData(keyspaceName, columnfamily) == null)
+            if (Schema.instance.getTableMetadataRef(keyspaceName, columnfamily) == null)
             {
                 System.err.println("ColumnFamily not found: " + keyspaceName + "/" + columnfamily);
                 System.exit(1);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java b/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
index 9f0395b..1116575 100644
--- a/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
+++ b/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
@@ -34,7 +34,7 @@ import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.SetMultimap;
 
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Directories;
@@ -92,7 +92,7 @@ public class SSTableOfflineRelevel
         String columnfamily = args[args.length - 1];
         Schema.instance.loadFromDisk(false);
 
-        if (Schema.instance.getCFMetaData(keyspace, columnfamily) == null)
+        if (Schema.instance.getTableMetadataRef(keyspace, columnfamily) == null)
             throw new IllegalArgumentException(String.format("Unknown keyspace/columnFamily %s.%s",
                     keyspace,
                     columnfamily));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/tools/StandaloneSSTableUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneSSTableUtil.java b/src/java/org/apache/cassandra/tools/StandaloneSSTableUtil.java
index 2e8ee0b..adfe7e0 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSSTableUtil.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSSTableUtil.java
@@ -18,9 +18,8 @@
  */
 package org.apache.cassandra.tools;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
@@ -52,7 +51,7 @@ public class StandaloneSSTableUtil
             Util.initDatabaseDescriptor();
             Schema.instance.loadFromDisk(false);
 
-            CFMetaData metadata = Schema.instance.getCFMetaData(options.keyspaceName, options.cfName);
+            TableMetadata metadata = Schema.instance.getTableMetadata(options.keyspaceName, options.cfName);
             if (metadata == null)
                 throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",
                                                                  options.keyspaceName,
@@ -82,7 +81,7 @@ public class StandaloneSSTableUtil
         }
     }
 
-    private static void listFiles(Options options, CFMetaData metadata, OutputHandler handler) throws IOException
+    private static void listFiles(Options options, TableMetadata metadata, OutputHandler handler) throws IOException
     {
         Directories directories = new Directories(metadata, ColumnFamilyStore.getInitialDirectories());
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 54b340e..f7f48c8 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -28,7 +28,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import org.apache.commons.cli.*;
 
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
@@ -61,7 +61,7 @@ public class StandaloneScrubber
             // load keyspace descriptions.
             Schema.instance.loadFromDisk(false);
 
-            if (Schema.instance.getKSMetaData(options.keyspaceName) == null)
+            if (Schema.instance.getKeyspaceMetadata(options.keyspaceName) == null)
                 throw new IllegalArgumentException(String.format("Unknown keyspace %s", options.keyspaceName));
 
             // Do not load sstables since they might be broken

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 aaaa9db..c5be02e 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
@@ -22,7 +22,7 @@ import java.io.File;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 e55b3a8..ed25e42 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.cli.*;
 
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
@@ -33,6 +32,7 @@ import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
 
@@ -55,7 +55,7 @@ public class StandaloneUpgrader
             // load keyspace descriptions.
             Schema.instance.loadFromDisk(false);
 
-            if (Schema.instance.getCFMetaData(options.keyspace, options.cf) == null)
+            if (Schema.instance.getTableMetadataRef(options.keyspace, options.cf) == null)
                 throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",
                                                                  options.keyspace,
                                                                  options.cf));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/tools/StandaloneVerifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneVerifier.java b/src/java/org/apache/cassandra/tools/StandaloneVerifier.java
index ee55dd5..40dfbf7 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneVerifier.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneVerifier.java
@@ -18,7 +18,7 @@
  */
 package org.apache.cassandra.tools;
 
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
@@ -56,7 +56,7 @@ public class StandaloneVerifier
 
             boolean hasFailed = false;
 
-            if (Schema.instance.getCFMetaData(options.keyspaceName, options.cfName) == null)
+            if (Schema.instance.getTableMetadataRef(options.keyspaceName, options.cfName) == null)
                 throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",
                                                                  options.keyspaceName,
                                                                  options.cfName));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
index c964b2f..4102846 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
@@ -24,7 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import io.airlift.command.Option;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.tools.NodeProbe;
 import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/tools/nodetool/Repair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Repair.java b/src/java/org/apache/cassandra/tools/nodetool/Repair.java
index 350601a..48f929f 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Repair.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Repair.java
@@ -31,7 +31,7 @@ import java.util.Set;
 
 import com.google.common.collect.Sets;
 
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.repair.RepairParallelism;
 import org.apache.cassandra.repair.messages.RepairOption;
 import org.apache.cassandra.tools.NodeProbe;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
index ac8b4f7..20c992c 100644
--- a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
+++ b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
@@ -20,18 +20,23 @@ package org.apache.cassandra.tracing;
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Tables;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
 
+import static java.lang.String.format;
+
 public final class TraceKeyspace
 {
     private TraceKeyspace()
@@ -41,36 +46,41 @@ public final class TraceKeyspace
     public static final String SESSIONS = "sessions";
     public static final String EVENTS = "events";
 
-    private static final CFMetaData Sessions =
-        compile(SESSIONS,
-                "tracing sessions",
-                "CREATE TABLE %s ("
-                + "session_id uuid,"
-                + "command text,"
-                + "client inet,"
-                + "coordinator inet,"
-                + "duration int,"
-                + "parameters map<text, text>,"
-                + "request text,"
-                + "started_at timestamp,"
-                + "PRIMARY KEY ((session_id)))");
-
-    private static final CFMetaData Events =
-        compile(EVENTS,
-                "tracing events",
-                "CREATE TABLE %s ("
-                + "session_id uuid,"
-                + "event_id timeuuid,"
-                + "activity text,"
-                + "source inet,"
-                + "source_elapsed int,"
-                + "thread text,"
-                + "PRIMARY KEY ((session_id), event_id))");
-
-    private static CFMetaData compile(String name, String description, String schema)
+    private static final TableMetadata Sessions =
+        parse(SESSIONS,
+              "tracing sessions",
+              "CREATE TABLE %s ("
+              + "session_id uuid,"
+              + "command text,"
+              + "client inet,"
+              + "coordinator inet,"
+              + "duration int,"
+              + "parameters map<text, text>,"
+              + "request text,"
+              + "started_at timestamp,"
+              + "PRIMARY KEY ((session_id)))");
+
+    private static final TableMetadata Events =
+        parse(EVENTS,
+              "tracing events",
+              "CREATE TABLE %s ("
+              + "session_id uuid,"
+              + "event_id timeuuid,"
+              + "activity text,"
+              + "source inet,"
+              + "source_elapsed int,"
+              + "thread text,"
+              + "PRIMARY KEY ((session_id), event_id))");
+
+    private static TableMetadata parse(String table, String description, String cql)
     {
-        return CFMetaData.compile(String.format(schema, name), SchemaConstants.TRACE_KEYSPACE_NAME)
-                         .comment(description);
+        return CreateTableStatement.parse(format(cql, table), SchemaConstants.TRACE_KEYSPACE_NAME)
+                                   .id(TableId.forSystemTable(SchemaConstants.TRACE_KEYSPACE_NAME, table))
+                                   .dcLocalReadRepairChance(0.0)
+                                   .gcGraceSeconds(0)
+                                   .memtableFlushPeriod((int) TimeUnit.HOURS.toMillis(1))
+                                   .comment(description)
+                                   .build();
     }
 
     public static KeyspaceMetadata metadata()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/transport/Server.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java
index 1eeecac..f38d83d 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -49,6 +49,8 @@ import io.netty.util.internal.logging.Slf4JLoggerFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaChangeListener;
 import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.service.*;
 import org.apache.cassandra.transport.messages.EventMessage;
@@ -100,7 +102,7 @@ public class Server implements CassandraDaemon.Server
             eventExecutorGroup = builder.eventExecutorGroup;
         EventNotifier notifier = new EventNotifier(this);
         StorageService.instance.register(notifier);
-        MigrationManager.instance.register(notifier);
+        Schema.instance.registerListener(notifier);
     }
 
     public void stop()
@@ -448,7 +450,7 @@ public class Server implements CassandraDaemon.Server
         }
     }
 
-    private static class EventNotifier extends MigrationListener implements IEndpointLifecycleSubscriber
+    private static class EventNotifier extends SchemaChangeListener implements IEndpointLifecycleSubscriber
     {
         private final Server server;
 
@@ -584,12 +586,12 @@ public class Server implements CassandraDaemon.Server
             send(new Event.SchemaChange(Event.SchemaChange.Change.CREATED, ksName));
         }
 
-        public void onCreateColumnFamily(String ksName, String cfName)
+        public void onCreateTable(String ksName, String cfName)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, ksName, cfName));
         }
 
-        public void onCreateUserType(String ksName, String typeName)
+        public void onCreateType(String ksName, String typeName)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TYPE, ksName, typeName));
         }
@@ -606,28 +608,28 @@ public class Server implements CassandraDaemon.Server
                                         ksName, aggregateName, AbstractType.asCQLTypeStringList(argTypes)));
         }
 
-        public void onUpdateKeyspace(String ksName)
+        public void onAlterKeyspace(String ksName)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, ksName));
         }
 
-        public void onUpdateColumnFamily(String ksName, String cfName, boolean affectsStatements)
+        public void onAlterTable(String ksName, String cfName, boolean affectsStatements)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, ksName, cfName));
         }
 
-        public void onUpdateUserType(String ksName, String typeName)
+        public void onAlterType(String ksName, String typeName)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TYPE, ksName, typeName));
         }
 
-        public void onUpdateFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+        public void onAlterFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.FUNCTION,
                                         ksName, functionName, AbstractType.asCQLTypeStringList(argTypes)));
         }
 
-        public void onUpdateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+        public void onAlterAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.AGGREGATE,
                                         ksName, aggregateName, AbstractType.asCQLTypeStringList(argTypes)));
@@ -638,12 +640,12 @@ public class Server implements CassandraDaemon.Server
             send(new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, ksName));
         }
 
-        public void onDropColumnFamily(String ksName, String cfName)
+        public void onDropTable(String ksName, String cfName)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TABLE, ksName, cfName));
         }
 
-        public void onDropUserType(String ksName, String typeName)
+        public void onDropType(String ksName, String typeName)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TYPE, ksName, typeName));
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
index 703e69a..906b342 100644
--- a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
+++ b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
@@ -33,6 +33,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.exceptions.CassandraException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.schema.TriggerMetadata;
 import org.apache.cassandra.schema.Triggers;
 import org.apache.cassandra.utils.FBUtilities;
@@ -86,7 +87,7 @@ public class TriggerExecutor
         if (intermediate == null || intermediate.isEmpty())
             return updates;
 
-        return PartitionUpdate.merge(validateForSinglePartition(updates.metadata().cfId, updates.partitionKey(), intermediate));
+        return PartitionUpdate.merge(validateForSinglePartition(updates.metadata().id, updates.partitionKey(), intermediate));
     }
 
     /**
@@ -157,9 +158,9 @@ public class TriggerExecutor
         return merged;
     }
 
-    private List<PartitionUpdate> validateForSinglePartition(UUID cfId,
-                                                                   DecoratedKey key,
-                                                                   Collection<Mutation> tmutations)
+    private List<PartitionUpdate> validateForSinglePartition(TableId tableId,
+                                                             DecoratedKey key,
+                                                             Collection<Mutation> tmutations)
     throws InvalidRequestException
     {
         validate(tmutations);
@@ -169,7 +170,7 @@ public class TriggerExecutor
             List<PartitionUpdate> updates = Lists.newArrayList(Iterables.getOnlyElement(tmutations).getPartitionUpdates());
             if (updates.size() > 1)
                 throw new InvalidRequestException("The updates generated by triggers are not all for the same partition");
-            validateSamePartition(cfId, key, Iterables.getOnlyElement(updates));
+            validateSamePartition(tableId, key, Iterables.getOnlyElement(updates));
             return updates;
         }
 
@@ -178,20 +179,20 @@ public class TriggerExecutor
         {
             for (PartitionUpdate update : mutation.getPartitionUpdates())
             {
-                validateSamePartition(cfId, key, update);
+                validateSamePartition(tableId, key, update);
                 updates.add(update);
             }
         }
         return updates;
     }
 
-    private void validateSamePartition(UUID cfId, DecoratedKey key, PartitionUpdate update)
+    private void validateSamePartition(TableId tableId, DecoratedKey key, PartitionUpdate update)
     throws InvalidRequestException
     {
         if (!key.equals(update.partitionKey()))
             throw new InvalidRequestException("Partition key of additional mutation does not match primary update key");
 
-        if (!cfId.equals(update.metadata().cfId))
+        if (!tableId.equals(update.metadata().id))
             throw new InvalidRequestException("table of additional mutation does not match primary update table");
     }
 
@@ -211,7 +212,7 @@ public class TriggerExecutor
      */
     private List<Mutation> executeInternal(PartitionUpdate update)
     {
-        Triggers triggers = update.metadata().getTriggers();
+        Triggers triggers = update.metadata().triggers;
         if (triggers.isEmpty())
             return null;
         List<Mutation> tmutations = Lists.newLinkedList();
@@ -238,7 +239,7 @@ public class TriggerExecutor
         }
         catch (Exception ex)
         {
-            throw new RuntimeException(String.format("Exception while executing trigger on table with ID: %s", update.metadata().cfId), ex);
+            throw new RuntimeException(String.format("Exception while executing trigger on table with ID: %s", update.metadata().id), ex);
         }
         finally
         {


[21/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java b/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java
index 0958113..269fc95 100644
--- a/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/index/sasi/conf/ColumnIndex.java
@@ -28,7 +28,7 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Memtable;
@@ -57,7 +57,7 @@ public class ColumnIndex
 
     private final AbstractType<?> keyValidator;
 
-    private final ColumnDefinition column;
+    private final ColumnMetadata column;
     private final Optional<IndexMetadata> config;
 
     private final AtomicReference<IndexMemtable> memtable;
@@ -70,7 +70,7 @@ public class ColumnIndex
 
     private final boolean isTokenized;
 
-    public ColumnIndex(AbstractType<?> keyValidator, ColumnDefinition column, IndexMetadata metadata)
+    public ColumnIndex(AbstractType<?> keyValidator, ColumnMetadata column, IndexMetadata metadata)
     {
         this.keyValidator = keyValidator;
         this.column = column;
@@ -147,7 +147,7 @@ public class ColumnIndex
         tracker.update(oldSSTables, newSSTables);
     }
 
-    public ColumnDefinition getDefinition()
+    public ColumnMetadata getDefinition()
     {
         return column;
     }
@@ -229,7 +229,7 @@ public class ColumnIndex
 
     }
 
-    public static ByteBuffer getValueOf(ColumnDefinition column, Row row, int nowInSecs)
+    public static ByteBuffer getValueOf(ColumnMetadata column, Row row, int nowInSecs)
     {
         if (row == null)
             return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java b/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
index c66dd02..d319636 100644
--- a/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
+++ b/src/java/org/apache/cassandra/index/sasi/conf/IndexMode.java
@@ -22,7 +22,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.index.sasi.analyzer.AbstractAnalyzer;
 import org.apache.cassandra.index.sasi.analyzer.NoOpAnalyzer;
 import org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer;
@@ -110,12 +110,12 @@ public class IndexMode
         }
     }
 
-    public static IndexMode getMode(ColumnDefinition column, Optional<IndexMetadata> config) throws ConfigurationException
+    public static IndexMode getMode(ColumnMetadata column, Optional<IndexMetadata> config) throws ConfigurationException
     {
         return getMode(column, config.isPresent() ? config.get().options : null);
     }
 
-    public static IndexMode getMode(ColumnDefinition column, Map<String, String> indexOptions) throws ConfigurationException
+    public static IndexMode getMode(ColumnMetadata column, Map<String, String> indexOptions) throws ConfigurationException
     {
         if (indexOptions == null || indexOptions.isEmpty())
             return IndexMode.NOT_INDEXED;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
index 9fa4e87..20f6292 100644
--- a/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
+++ b/src/java/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriter.java
@@ -27,7 +27,7 @@ import java.util.concurrent.*;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.rows.Row;
@@ -79,10 +79,10 @@ public class PerSSTableIndexWriter implements SSTableFlushObserver
     private final OperationType source;
 
     private final AbstractType<?> keyValidator;
-    private final Map<ColumnDefinition, ColumnIndex> supportedIndexes;
+    private final Map<ColumnMetadata, ColumnIndex> supportedIndexes;
 
     @VisibleForTesting
-    protected final Map<ColumnDefinition, Index> indexes;
+    protected final Map<ColumnMetadata, Index> indexes;
 
     private DecoratedKey currentKey;
     private long currentKeyPosition;
@@ -91,7 +91,7 @@ public class PerSSTableIndexWriter implements SSTableFlushObserver
     public PerSSTableIndexWriter(AbstractType<?> keyValidator,
                                  Descriptor descriptor,
                                  OperationType source,
-                                 Map<ColumnDefinition, ColumnIndex> supportedIndexes)
+                                 Map<ColumnMetadata, ColumnIndex> supportedIndexes)
     {
         this.keyValidator = keyValidator;
         this.descriptor = descriptor;
@@ -155,7 +155,7 @@ public class PerSSTableIndexWriter implements SSTableFlushObserver
         }
     }
 
-    public Index getIndex(ColumnDefinition columnDef)
+    public Index getIndex(ColumnMetadata columnDef)
     {
         return indexes.get(columnDef);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java b/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java
index ca60ac5..21cb7ce 100644
--- a/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java
+++ b/src/java/org/apache/cassandra/index/sasi/memory/TrieMemIndex.java
@@ -22,8 +22,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ConcurrentSkipListSet;
 
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.index.sasi.conf.ColumnIndex;
 import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
 import org.apache.cassandra.index.sasi.disk.Token;
@@ -106,9 +106,9 @@ public class TrieMemIndex extends MemIndex
     {
         public static final SizeEstimatingNodeFactory NODE_FACTORY = new SizeEstimatingNodeFactory();
 
-        protected final ColumnDefinition definition;
+        protected final ColumnMetadata definition;
 
-        public ConcurrentTrie(ColumnDefinition column)
+        public ConcurrentTrie(ColumnMetadata column)
         {
             definition = column;
         }
@@ -162,7 +162,7 @@ public class TrieMemIndex extends MemIndex
     {
         private final ConcurrentRadixTree<ConcurrentSkipListSet<DecoratedKey>> trie;
 
-        private ConcurrentPrefixTrie(ColumnDefinition column)
+        private ConcurrentPrefixTrie(ColumnMetadata column)
         {
             super(column);
             trie = new ConcurrentRadixTree<>(NODE_FACTORY);
@@ -200,7 +200,7 @@ public class TrieMemIndex extends MemIndex
     {
         private final ConcurrentSuffixTree<ConcurrentSkipListSet<DecoratedKey>> trie;
 
-        private ConcurrentSuffixTrie(ColumnDefinition column)
+        private ConcurrentSuffixTrie(ColumnMetadata column)
         {
             super(column);
             trie = new ConcurrentSuffixTree<>(NODE_FACTORY);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/sasi/plan/Expression.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/plan/Expression.java b/src/java/org/apache/cassandra/index/sasi/plan/Expression.java
index 93f1938..4f9d6b0 100644
--- a/src/java/org/apache/cassandra/index/sasi/plan/Expression.java
+++ b/src/java/org/apache/cassandra/index/sasi/plan/Expression.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.index.sasi.analyzer.AbstractAnalyzer;
 import org.apache.cassandra.index.sasi.conf.ColumnIndex;
@@ -115,7 +115,7 @@ public class Expression
     @VisibleForTesting
     public Expression(String name, AbstractType<?> validator)
     {
-        this(null, new ColumnIndex(UTF8Type.instance, ColumnDefinition.regularDef("sasi", "internal", name, validator), null));
+        this(null, new ColumnIndex(UTF8Type.instance, ColumnMetadata.regularColumn("sasi", "internal", name, validator), null));
     }
 
     public Expression setLower(Bound newLower)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/sasi/plan/Operation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/plan/Operation.java b/src/java/org/apache/cassandra/index/sasi/plan/Operation.java
index 7c744e1..148bc1f 100644
--- a/src/java/org/apache/cassandra/index/sasi/plan/Operation.java
+++ b/src/java/org/apache/cassandra/index/sasi/plan/Operation.java
@@ -21,8 +21,8 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.ColumnDefinition.Kind;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.ColumnMetadata.Kind;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.filter.RowFilter;
 import org.apache.cassandra.db.rows.Row;
@@ -37,6 +37,7 @@ import org.apache.cassandra.index.sasi.utils.RangeUnionIterator;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
+
 import org.apache.cassandra.utils.FBUtilities;
 
 @SuppressWarnings("resource")
@@ -65,14 +66,14 @@ public class Operation extends RangeIterator<Long, Token>
     private final QueryController controller;
 
     protected final OperationType op;
-    protected final ListMultimap<ColumnDefinition, Expression> expressions;
+    protected final ListMultimap<ColumnMetadata, Expression> expressions;
     protected final RangeIterator<Long, Token> range;
 
     protected Operation left, right;
 
     private Operation(OperationType operation,
                       QueryController controller,
-                      ListMultimap<ColumnDefinition, Expression> expressions,
+                      ListMultimap<ColumnMetadata, Expression> expressions,
                       RangeIterator<Long, Token> range,
                       Operation left, Operation right)
     {
@@ -207,7 +208,7 @@ public class Operation extends RangeIterator<Long, Token>
         boolean result = false;
         int idx = 0;
 
-        for (ColumnDefinition column : expressions.keySet())
+        for (ColumnMetadata column : expressions.keySet())
         {
             if (column.kind == Kind.PARTITION_KEY)
                 continue;
@@ -262,11 +263,11 @@ public class Operation extends RangeIterator<Long, Token>
     }
 
     @VisibleForTesting
-    protected static ListMultimap<ColumnDefinition, Expression> analyzeGroup(QueryController controller,
-                                                                             OperationType op,
-                                                                             List<RowFilter.Expression> expressions)
+    protected static ListMultimap<ColumnMetadata, Expression> analyzeGroup(QueryController controller,
+                                                                           OperationType op,
+                                                                           List<RowFilter.Expression> expressions)
     {
-        ListMultimap<ColumnDefinition, Expression> analyzed = ArrayListMultimap.create();
+        ListMultimap<ColumnMetadata, Expression> analyzed = ArrayListMultimap.create();
 
         // sort all of the expressions in the operation by name and priority of the logical operator
         // this gives us an efficient way to handle inequality and combining into ranges without extra processing
@@ -429,7 +430,7 @@ public class Operation extends RangeIterator<Long, Token>
         {
             if (!expressions.isEmpty())
             {
-                ListMultimap<ColumnDefinition, Expression> analyzedExpressions = analyzeGroup(controller, op, expressions);
+                ListMultimap<ColumnMetadata, Expression> analyzedExpressions = analyzeGroup(controller, op, expressions);
                 RangeIterator.Builder<Long, Token> range = controller.getIndexes(op, analyzedExpressions.values());
 
                 Operation rightOp = null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java b/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java
index 336a740..d2f990e 100644
--- a/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java
+++ b/src/java/org/apache/cassandra/index/sasi/plan/QueryController.java
@@ -22,10 +22,15 @@ import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Sets;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionRangeReadCommand;
+import org.apache.cassandra.db.ReadExecutionController;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
 import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.filter.RowFilter;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.index.sasi.SASIIndex;
@@ -39,9 +44,9 @@ import org.apache.cassandra.index.sasi.plan.Operation.OperationType;
 import org.apache.cassandra.index.sasi.utils.RangeIntersectionIterator;
 import org.apache.cassandra.index.sasi.utils.RangeIterator;
 import org.apache.cassandra.index.sasi.utils.RangeUnionIterator;
-import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.Pair;
 
 public class QueryController
@@ -63,7 +68,7 @@ public class QueryController
         this.executionStart = System.nanoTime();
     }
 
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
         return command.metadata();
     }
@@ -80,7 +85,7 @@ public class QueryController
 
     public AbstractType<?> getKeyValidator()
     {
-        return cfs.metadata.getKeyValidator();
+        return cfs.metadata().partitionKeyType;
     }
 
     public ColumnIndex getIndex(RowFilter.Expression expression)
@@ -96,7 +101,7 @@ public class QueryController
             throw new NullPointerException();
         try
         {
-            SinglePartitionReadCommand partition = SinglePartitionReadCommand.create(cfs.metadata,
+            SinglePartitionReadCommand partition = SinglePartitionReadCommand.create(cfs.metadata(),
                                                                                      command.nowInSec(),
                                                                                      command.columnFilter(),
                                                                                      command.rowFilter().withoutExpressions(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java b/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java
index 8a25f79..5e78684 100644
--- a/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java
+++ b/src/java/org/apache/cassandra/index/sasi/plan/QueryPlan.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.index.sasi.plan;
 
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.db.rows.*;
@@ -28,6 +27,7 @@ import org.apache.cassandra.index.sasi.disk.Token;
 import org.apache.cassandra.index.sasi.plan.Operation.OperationType;
 import org.apache.cassandra.exceptions.RequestTimeoutException;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.AbstractIterator;
 
 public class QueryPlan
@@ -153,7 +153,7 @@ public class QueryPlan
             }
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
             return controller.metadata();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 7efca63..b1e15ed 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -27,13 +27,12 @@ import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.EncodingStats;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.EncodingStats;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.utils.Pair;
 
 /**
  * Base class for the sstable writers used by CQLSSTableWriter.
@@ -41,13 +40,13 @@ import org.apache.cassandra.utils.Pair;
 abstract class AbstractSSTableSimpleWriter implements Closeable
 {
     protected final File directory;
-    protected final CFMetaData metadata;
-    protected final PartitionColumns columns;
+    protected final TableMetadataRef metadata;
+    protected final RegularAndStaticColumns columns;
     protected SSTableFormat.Type formatType = SSTableFormat.Type.current();
     protected static AtomicInteger generation = new AtomicInteger(0);
     protected boolean makeRangeAware = false;
 
-    protected AbstractSSTableSimpleWriter(File directory, CFMetaData metadata, PartitionColumns columns)
+    protected AbstractSSTableSimpleWriter(File directory, TableMetadataRef metadata, RegularAndStaticColumns columns)
     {
         this.metadata = metadata;
         this.directory = directory;
@@ -67,13 +66,13 @@ abstract class AbstractSSTableSimpleWriter implements Closeable
 
     protected SSTableTxnWriter createWriter()
     {
-        SerializationHeader header = new SerializationHeader(true, metadata, columns, EncodingStats.NO_STATS);
+        SerializationHeader header = new SerializationHeader(true, metadata.get(), columns, EncodingStats.NO_STATS);
 
         if (makeRangeAware)
             return SSTableTxnWriter.createRangeAware(metadata, 0,  ActiveRepairService.UNREPAIRED_SSTABLE, formatType, 0, header);
 
         return SSTableTxnWriter.create(metadata,
-                                       createDescriptor(directory, metadata.ksName, metadata.cfName, formatType),
+                                       createDescriptor(directory, metadata.keyspace, metadata.name, formatType),
                                        0,
                                        ActiveRepairService.UNREPAIRED_SSTABLE,
                                        0,
@@ -117,7 +116,7 @@ abstract class AbstractSSTableSimpleWriter implements Closeable
 
     PartitionUpdate getUpdateFor(ByteBuffer key) throws IOException
     {
-        return getUpdateFor(metadata.decorateKey(key));
+        return getUpdateFor(metadata.get().partitioner.decorateKey(key));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 a195235..d026a03 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -31,9 +31,9 @@ import java.util.stream.Collectors;
 
 import com.datastax.driver.core.ProtocolVersion;
 import com.datastax.driver.core.TypeCodec;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
@@ -55,6 +55,7 @@ import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.schema.Types;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -248,7 +249,7 @@ public class CQLSSTableWriter implements Closeable
         long now = System.currentTimeMillis() * 1000;
         // Note that we asks indexes to not validate values (the last 'false' arg below) because that triggers a 'Keyspace.open'
         // and that forces a lot of initialization that we don't want.
-        UpdateParameters params = new UpdateParameters(insert.cfm,
+        UpdateParameters params = new UpdateParameters(insert.metadata,
                                                        insert.updatedColumns(),
                                                        options,
                                                        insert.getTimestamp(now, options),
@@ -309,7 +310,7 @@ public class CQLSSTableWriter implements Closeable
      */
     public com.datastax.driver.core.UserType getUDType(String dataType)
     {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(insert.keyspace());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(insert.keyspace());
         UserType userType = ksm.types.getNullable(ByteBufferUtil.bytes(dataType));
         return (com.datastax.driver.core.UserType) UDHelper.driverType(userType);
     }
@@ -509,16 +510,16 @@ public class CQLSSTableWriter implements Closeable
             {
                 String keyspace = schemaStatement.keyspace();
 
-                if (Schema.instance.getKSMetaData(keyspace) == null)
+                if (Schema.instance.getKeyspaceMetadata(keyspace) == null)
                     Schema.instance.load(KeyspaceMetadata.create(keyspace, KeyspaceParams.simple(1)));
 
                 createTypes(keyspace);
-                CFMetaData cfMetaData = createTable(keyspace);
+                TableMetadataRef tableMetadata = TableMetadataRef.forOfflineTools(createTable(keyspace));
                 Pair<UpdateStatement, List<ColumnSpecification>> preparedInsert = prepareInsert();
 
                 AbstractSSTableSimpleWriter writer = sorted
-                                                     ? new SSTableSimpleWriter(directory, cfMetaData, preparedInsert.left.updatedColumns())
-                                                     : new SSTableSimpleUnsortedWriter(directory, cfMetaData, preparedInsert.left.updatedColumns(), bufferSizeInMB);
+                                                     ? new SSTableSimpleWriter(directory, tableMetadata, preparedInsert.left.updatedColumns())
+                                                     : new SSTableSimpleUnsortedWriter(directory, tableMetadata, preparedInsert.left.updatedColumns(), bufferSizeInMB);
 
                 if (formatType != null)
                     writer.setSSTableFormatType(formatType);
@@ -529,39 +530,38 @@ public class CQLSSTableWriter implements Closeable
 
         private void createTypes(String keyspace)
         {
-            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
+            KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
             Types.RawBuilder builder = Types.rawBuilder(keyspace);
             for (CreateTypeStatement st : typeStatements)
                 st.addToRawBuilder(builder);
 
             ksm = ksm.withSwapped(builder.build());
-            Schema.instance.setKeyspaceMetadata(ksm);
+            Schema.instance.load(ksm);
         }
         /**
          * Creates the table according to schema statement
          *
          * @param keyspace name of the keyspace where table should be created
          */
-        private CFMetaData createTable(String keyspace)
+        private TableMetadata createTable(String keyspace)
         {
-            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
+            KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
 
-            CFMetaData cfMetaData = ksm.tables.getNullable(schemaStatement.columnFamily());
-            if (cfMetaData == null)
-            {
-                CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(ksm.types).statement;
-                statement.validate(ClientState.forInternalCalls());
+            TableMetadata tableMetadata = ksm.tables.getNullable(schemaStatement.columnFamily());
 
-                cfMetaData = statement.getCFMetaData();
+            if (tableMetadata != null)
+                return tableMetadata;
 
-                Schema.instance.load(cfMetaData);
-                Schema.instance.setKeyspaceMetadata(ksm.withSwapped(ksm.tables.with(cfMetaData)));
-            }
+            CreateTableStatement statement = (CreateTableStatement) schemaStatement.prepare(ksm.types).statement;
+            statement.validate(ClientState.forInternalCalls());
 
+            TableMetadata.Builder builder = statement.builder();
             if (partitioner != null)
-                return cfMetaData.copy(partitioner);
-            else
-                return cfMetaData;
+                builder.partitioner(partitioner);
+            TableMetadata metadata = builder.build();
+
+            Schema.instance.load(ksm.withSwapped(ksm.tables.with(metadata)));
+            return metadata;
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
index 95ade16..b8d236a 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
@@ -40,6 +40,7 @@ import org.apache.cassandra.db.lifecycle.View;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.WrappedRunnable;
 
@@ -195,10 +196,10 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
      * compacting.
      */
     @SuppressWarnings("resource")
-    private Pair<List<SSTableReader>, Map<UUID, LifecycleTransaction>> getCompactingAndNonCompactingSSTables()
+    private Pair<List<SSTableReader>, Map<TableId, LifecycleTransaction>> getCompactingAndNonCompactingSSTables()
     {
         List<SSTableReader> allCompacting = new ArrayList<>();
-        Map<UUID, LifecycleTransaction> allNonCompacting = new HashMap<>();
+        Map<TableId, LifecycleTransaction> allNonCompacting = new HashMap<>();
         for (Keyspace ks : Keyspace.all())
         {
             for (ColumnFamilyStore cfStore: ks.getColumnFamilyStores())
@@ -213,7 +214,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
                 }
                 while (null == (txn = cfStore.getTracker().tryModify(nonCompacting, OperationType.UNKNOWN)));
 
-                allNonCompacting.put(cfStore.metadata.cfId, txn);
+                allNonCompacting.put(cfStore.metadata.id, txn);
                 allCompacting.addAll(Sets.difference(allSSTables, nonCompacting));
             }
         }
@@ -222,7 +223,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
 
     public void redistributeSummaries() throws IOException
     {
-        Pair<List<SSTableReader>, Map<UUID, LifecycleTransaction>> compactingAndNonCompacting = getCompactingAndNonCompactingSSTables();
+        Pair<List<SSTableReader>, Map<TableId, LifecycleTransaction>> compactingAndNonCompacting = getCompactingAndNonCompactingSSTables();
         try
         {
             redistributeSummaries(new IndexSummaryRedistribution(compactingAndNonCompacting.left,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java
index fc326dc..47c3fca 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java
@@ -28,7 +28,6 @@ import java.util.Map;
 import java.util.UUID;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,6 +39,7 @@ import org.apache.cassandra.db.compaction.CompactionInterruptedException;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
@@ -57,12 +57,12 @@ public class IndexSummaryRedistribution extends CompactionInfo.Holder
     static final double DOWNSAMPLE_THESHOLD = 0.75;
 
     private final List<SSTableReader> compacting;
-    private final Map<UUID, LifecycleTransaction> transactions;
+    private final Map<TableId, LifecycleTransaction> transactions;
     private final long memoryPoolBytes;
     private final UUID compactionId;
     private volatile long remainingSpace;
 
-    public IndexSummaryRedistribution(List<SSTableReader> compacting, Map<UUID, LifecycleTransaction> transactions, long memoryPoolBytes)
+    public IndexSummaryRedistribution(List<SSTableReader> compacting, Map<TableId, LifecycleTransaction> transactions, long memoryPoolBytes)
     {
         this.compacting = compacting;
         this.transactions = transactions;
@@ -127,7 +127,7 @@ public class IndexSummaryRedistribution extends CompactionInfo.Holder
     }
 
     private List<SSTableReader> adjustSamplingLevels(List<SSTableReader> sstables,
-                                                     Map<UUID, LifecycleTransaction> transactions,
+                                                     Map<TableId, LifecycleTransaction> transactions,
                                                      double totalReadsPerSec, long memoryPoolCapacity) throws IOException
     {
         List<ResampleEntry> toDownsample = new ArrayList<>(sstables.size() / 4);
@@ -144,8 +144,8 @@ public class IndexSummaryRedistribution extends CompactionInfo.Holder
             if (isStopRequested())
                 throw new CompactionInterruptedException(getCompactionInfo());
 
-            int minIndexInterval = sstable.metadata.params.minIndexInterval;
-            int maxIndexInterval = sstable.metadata.params.maxIndexInterval;
+            int minIndexInterval = sstable.metadata().params.minIndexInterval;
+            int maxIndexInterval = sstable.metadata().params.maxIndexInterval;
 
             double readsPerSec = sstable.getReadMeter() == null ? 0.0 : sstable.getReadMeter().fifteenMinuteRate();
             long idealSpace = Math.round(remainingSpace * (readsPerSec / totalReadsPerSec));
@@ -217,7 +217,7 @@ public class IndexSummaryRedistribution extends CompactionInfo.Holder
                 logger.trace("SSTable {} is within thresholds of ideal sampling", sstable);
                 remainingSpace -= sstable.getIndexSummaryOffHeapSize();
                 newSSTables.add(sstable);
-                transactions.get(sstable.metadata.cfId).cancel(sstable);
+                transactions.get(sstable.metadata().id).cancel(sstable);
             }
             totalReadsPerSec -= readsPerSec;
         }
@@ -228,7 +228,7 @@ public class IndexSummaryRedistribution extends CompactionInfo.Holder
             toDownsample = result.right;
             newSSTables.addAll(result.left);
             for (SSTableReader sstable : result.left)
-                transactions.get(sstable.metadata.cfId).cancel(sstable);
+                transactions.get(sstable.metadata().id).cancel(sstable);
         }
 
         // downsample first, then upsample
@@ -244,10 +244,10 @@ public class IndexSummaryRedistribution extends CompactionInfo.Holder
             logger.trace("Re-sampling index summary for {} from {}/{} to {}/{} of the original number of entries",
                          sstable, sstable.getIndexSummarySamplingLevel(), Downsampling.BASE_SAMPLING_LEVEL,
                          entry.newSamplingLevel, Downsampling.BASE_SAMPLING_LEVEL);
-            ColumnFamilyStore cfs = Keyspace.open(sstable.metadata.ksName).getColumnFamilyStore(sstable.metadata.cfId);
+            ColumnFamilyStore cfs = Keyspace.open(sstable.metadata().keyspace).getColumnFamilyStore(sstable.metadata().id);
             SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(cfs, entry.newSamplingLevel);
             newSSTables.add(replacement);
-            transactions.get(sstable.metadata.cfId).update(replacement, true);
+            transactions.get(sstable.metadata().id).update(replacement, true);
         }
 
         return newSSTables;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/KeyIterator.java b/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
index d51e97b..091e969 100644
--- a/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/KeyIterator.java
@@ -20,14 +20,13 @@ package org.apache.cassandra.io.sstable;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.cassandra.utils.AbstractIterator;
-
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.AbstractIterator;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CloseableIterator;
 
@@ -86,7 +85,7 @@ public class KeyIterator extends AbstractIterator<DecoratedKey> implements Close
 
     private long keyPosition;
 
-    public KeyIterator(Descriptor desc, CFMetaData metadata)
+    public KeyIterator(Descriptor desc, TableMetadata metadata)
     {
         this.desc = desc;
         in = new In(new File(desc.filenameFor(Component.PRIMARY_INDEX)));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java b/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
index 6f395f8..e64d95d 100644
--- a/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
@@ -39,7 +39,7 @@ public class ReducingKeyIterator implements CloseableIterator<DecoratedKey>
     {
         iters = new ArrayList<>(sstables.size());
         for (SSTableReader sstable : sstables)
-            iters.add(new KeyIterator(sstable.descriptor, sstable.metadata));
+            iters.add(new KeyIterator(sstable.descriptor, sstable.metadata()));
     }
 
     private void maybeInit()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/SSTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index 8556cfa..57aeff6 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -30,7 +30,6 @@ import com.google.common.io.Files;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowIndexEntry;
@@ -39,9 +38,11 @@ import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.DiskOptimizationStrategy;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.memory.HeapAllocator;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 /**
  * This class is built on top of the SequenceFile. It stores
@@ -63,21 +64,20 @@ public abstract class SSTable
 
     public final Descriptor descriptor;
     protected final Set<Component> components;
-    public final CFMetaData metadata;
     public final boolean compression;
 
     public DecoratedKey first;
     public DecoratedKey last;
 
     protected final DiskOptimizationStrategy optimizationStrategy;
+    protected final TableMetadataRef metadata;
 
-    protected SSTable(Descriptor descriptor, Set<Component> components, CFMetaData metadata, DiskOptimizationStrategy optimizationStrategy)
+    protected SSTable(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata, DiskOptimizationStrategy optimizationStrategy)
     {
         // In almost all cases, metadata shouldn't be null, but allowing null allows to create a mostly functional SSTable without
         // full schema definition. SSTableLoader use that ability
         assert descriptor != null;
         assert components != null;
-        assert metadata != null;
 
         this.descriptor = descriptor;
         Set<Component> dataComponents = new HashSet<>(components);
@@ -118,9 +118,14 @@ public abstract class SSTable
         return true;
     }
 
+    public TableMetadata metadata()
+    {
+        return metadata.get();
+    }
+
     public IPartitioner getPartitioner()
     {
-        return metadata.partitioner;
+        return metadata().partitioner;
     }
 
     public DecoratedKey decorateKey(ByteBuffer key)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
index 2a79f88..3ade9ff 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.io.sstable;
 
 import java.io.*;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
@@ -54,8 +54,8 @@ public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterat
         try
         {
             DeletionTime partitionLevelDeletion = DeletionTime.serializer.deserialize(file);
-            SerializationHelper helper = new SerializationHelper(sstable.metadata, sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL);
-            SSTableSimpleIterator iterator = SSTableSimpleIterator.create(sstable.metadata, file, sstable.header, helper, partitionLevelDeletion);
+            SerializationHelper helper = new SerializationHelper(sstable.metadata(), sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL);
+            SSTableSimpleIterator iterator = SSTableSimpleIterator.create(sstable.metadata(), file, sstable.header, helper, partitionLevelDeletion);
             return new SSTableIdentityIterator(sstable, key, partitionLevelDeletion, file.getPath(), iterator);
         }
         catch (IOException e)
@@ -72,10 +72,10 @@ public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterat
             dfile.seek(indexEntry.position);
             ByteBufferUtil.skipShortLength(dfile); // Skip partition key
             DeletionTime partitionLevelDeletion = DeletionTime.serializer.deserialize(dfile);
-            SerializationHelper helper = new SerializationHelper(sstable.metadata, sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL);
+            SerializationHelper helper = new SerializationHelper(sstable.metadata(), sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL);
             SSTableSimpleIterator iterator = tombstoneOnly
-                    ? SSTableSimpleIterator.createTombstoneOnly(sstable.metadata, dfile, sstable.header, helper, partitionLevelDeletion)
-                    : SSTableSimpleIterator.create(sstable.metadata, dfile, sstable.header, helper, partitionLevelDeletion);
+                    ? SSTableSimpleIterator.createTombstoneOnly(sstable.metadata(), dfile, sstable.header, helper, partitionLevelDeletion)
+                    : SSTableSimpleIterator.create(sstable.metadata(), dfile, sstable.header, helper, partitionLevelDeletion);
             return new SSTableIdentityIterator(sstable, key, partitionLevelDeletion, dfile.getPath(), iterator);
         }
         catch (IOException e)
@@ -85,14 +85,14 @@ public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterat
         }
     }
 
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
-        return sstable.metadata;
+        return iterator.metadata;
     }
 
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
-        return metadata().partitionColumns();
+        return metadata().regularAndStaticColumns();
     }
 
     public boolean isReverseOrder()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
index e00de4a..47b37ef 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -25,7 +25,7 @@ import java.util.*;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
@@ -96,7 +96,7 @@ public class SSTableLoader implements StreamEventHandler
                                               return false;
                                           }
 
-                                          CFMetaData metadata = client.getTableMetadata(desc.cfname);
+                                          TableMetadataRef metadata = client.getTableMetadata(desc.cfname);
                                           if (metadata == null)
                                           {
                                               outputHandler.output(String.format("Skipping file %s: table %s.%s doesn't exist", name, keyspace, desc.cfname));
@@ -272,9 +272,9 @@ public class SSTableLoader implements StreamEventHandler
          * Validate that {@code keyspace} is an existing keyspace and {@code
          * cfName} one of its existing column family.
          */
-        public abstract CFMetaData getTableMetadata(String tableName);
+        public abstract TableMetadataRef getTableMetadata(String tableName);
 
-        public void setTableMetadata(CFMetaData cfm)
+        public void setTableMetadata(TableMetadataRef cfm)
         {
             throw new RuntimeException();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java
index b92bc78..1be79ab 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableMultiWriter.java
@@ -19,10 +19,10 @@
 package org.apache.cassandra.io.sstable;
 
 import java.util.Collection;
-import java.util.UUID;
 
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.Throwables;
 import org.apache.cassandra.utils.concurrent.Transactional;
 
@@ -44,7 +44,7 @@ public interface SSTableMultiWriter extends Transactional
 
     String getFilename();
     long getFilePointer();
-    UUID getCfId();
+    TableId getTableId();
 
     static void abortOrDie(SSTableMultiWriter writer)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
index ad0f3c9..c3c7472 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
@@ -21,17 +21,12 @@ import java.io.IOException;
 import java.io.IOError;
 import java.util.Iterator;
 
-import org.apache.cassandra.io.util.RewindableDataInput;
-import org.apache.cassandra.utils.AbstractIterator;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.AbstractIterator;
 
 /**
  * Utility class to handle deserializing atom from sstables.
@@ -41,23 +36,23 @@ import org.apache.cassandra.net.MessagingService;
  */
 public abstract class SSTableSimpleIterator extends AbstractIterator<Unfiltered> implements Iterator<Unfiltered>
 {
-    protected final CFMetaData metadata;
+    final TableMetadata metadata;
     protected final DataInputPlus in;
     protected final SerializationHelper helper;
 
-    private SSTableSimpleIterator(CFMetaData metadata, DataInputPlus in, SerializationHelper helper)
+    private SSTableSimpleIterator(TableMetadata metadata, DataInputPlus in, SerializationHelper helper)
     {
         this.metadata = metadata;
         this.in = in;
         this.helper = helper;
     }
 
-    public static SSTableSimpleIterator create(CFMetaData metadata, DataInputPlus in, SerializationHeader header, SerializationHelper helper, DeletionTime partitionDeletion)
+    public static SSTableSimpleIterator create(TableMetadata metadata, DataInputPlus in, SerializationHeader header, SerializationHelper helper, DeletionTime partitionDeletion)
     {
         return new CurrentFormatIterator(metadata, in, header, helper);
     }
 
-    public static SSTableSimpleIterator createTombstoneOnly(CFMetaData metadata, DataInputPlus in, SerializationHeader header, SerializationHelper helper, DeletionTime partitionDeletion)
+    public static SSTableSimpleIterator createTombstoneOnly(TableMetadata metadata, DataInputPlus in, SerializationHeader header, SerializationHelper helper, DeletionTime partitionDeletion)
     {
         return new CurrentFormatTombstoneIterator(metadata, in, header, helper);
     }
@@ -70,7 +65,7 @@ public abstract class SSTableSimpleIterator extends AbstractIterator<Unfiltered>
 
         private final Row.Builder builder;
 
-        private CurrentFormatIterator(CFMetaData metadata, DataInputPlus in, SerializationHeader header, SerializationHelper helper)
+        private CurrentFormatIterator(TableMetadata metadata, DataInputPlus in, SerializationHeader header, SerializationHelper helper)
         {
             super(metadata, in, helper);
             this.header = header;
@@ -100,7 +95,7 @@ public abstract class SSTableSimpleIterator extends AbstractIterator<Unfiltered>
     {
         private final SerializationHeader header;
 
-        private CurrentFormatTombstoneIterator(CFMetaData metadata, DataInputPlus in, SerializationHeader header, SerializationHelper helper)
+        private CurrentFormatTombstoneIterator(TableMetadata metadata, DataInputPlus in, SerializationHeader header, SerializationHelper helper)
         {
             super(metadata, in, helper);
             this.header = header;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 23e18b5..afb4461 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -28,12 +28,12 @@ import java.util.concurrent.TimeUnit;
 import com.google.common.base.Throwables;
 
 import io.netty.util.concurrent.FastThreadLocalThread;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.EncodingStats;
 import org.apache.cassandra.db.rows.UnfilteredSerializer;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 /**
@@ -60,11 +60,11 @@ class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
     private final BlockingQueue<Buffer> writeQueue = new SynchronousQueue<Buffer>();
     private final DiskWriter diskWriter = new DiskWriter();
 
-    SSTableSimpleUnsortedWriter(File directory, CFMetaData metadata, PartitionColumns columns, long bufferSizeInMB)
+    SSTableSimpleUnsortedWriter(File directory, TableMetadataRef metadata, RegularAndStaticColumns columns, long bufferSizeInMB)
     {
         super(directory, metadata, columns);
         this.bufferSize = bufferSizeInMB * 1024L * 1024L;
-        this.header = new SerializationHeader(true, metadata, columns, EncodingStats.NO_STATS);
+        this.header = new SerializationHeader(true, metadata.get(), columns, EncodingStats.NO_STATS);
         diskWriter.start();
     }
 
@@ -110,7 +110,7 @@ class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
 
     private PartitionUpdate createPartitionUpdate(DecoratedKey key)
     {
-        return new PartitionUpdate(metadata, key, columns, 4)
+        return new PartitionUpdate(metadata.get(), key, columns, 4)
         {
             @Override
             public void add(Row row)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 7fbd79d..a663051 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -22,9 +22,9 @@ import java.io.IOException;
 
 import com.google.common.base.Throwables;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.TableMetadataRef;
 
 /**
  * A SSTable writer that assumes rows are in (partitioner) sorted order.
@@ -43,7 +43,7 @@ class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
 
     private SSTableTxnWriter writer;
 
-    protected SSTableSimpleWriter(File directory, CFMetaData metadata, PartitionColumns columns)
+    protected SSTableSimpleWriter(File directory, TableMetadataRef metadata, RegularAndStaticColumns columns)
     {
         super(directory, metadata, columns);
     }
@@ -67,7 +67,7 @@ class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
             if (update != null)
                 writePartition(update);
             currentKey = key;
-            update = new PartitionUpdate(metadata, currentKey, columns, 4);
+            update = new PartitionUpdate(metadata.get(), currentKey, columns, 4);
         }
 
         assert update != null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
index 323b1bd..fcf7cd6 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableTxnWriter.java
@@ -21,7 +21,6 @@ package org.apache.cassandra.io.sstable;
 import java.io.IOException;
 import java.util.Collection;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SerializationHeader;
@@ -33,6 +32,8 @@ import org.apache.cassandra.io.sstable.format.RangeAwareSSTableWriter;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.concurrent.Transactional;
 
 /**
@@ -107,7 +108,7 @@ public class SSTableTxnWriter extends Transactional.AbstractTransactional implem
 
 
     @SuppressWarnings("resource") // log and writer closed during doPostCleanup
-    public static SSTableTxnWriter createRangeAware(CFMetaData cfm,
+    public static SSTableTxnWriter createRangeAware(TableMetadataRef metadata,
                                                     long keyCount,
                                                     long repairedAt,
                                                     SSTableFormat.Type type,
@@ -115,7 +116,7 @@ public class SSTableTxnWriter extends Transactional.AbstractTransactional implem
                                                     SerializationHeader header)
     {
 
-        ColumnFamilyStore cfs = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+        ColumnFamilyStore cfs = Keyspace.open(metadata.keyspace).getColumnFamilyStore(metadata.name);
         LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.WRITE);
         SSTableMultiWriter writer;
         try
@@ -133,7 +134,7 @@ public class SSTableTxnWriter extends Transactional.AbstractTransactional implem
     }
 
     @SuppressWarnings("resource") // log and writer closed during doPostCleanup
-    public static SSTableTxnWriter create(CFMetaData cfm,
+    public static SSTableTxnWriter create(TableMetadataRef metadata,
                                           Descriptor descriptor,
                                           long keyCount,
                                           long repairedAt,
@@ -143,8 +144,8 @@ public class SSTableTxnWriter extends Transactional.AbstractTransactional implem
     {
         // if the column family store does not exist, we create a new default SSTableMultiWriter to use:
         LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.WRITE);
-        MetadataCollector collector = new MetadataCollector(cfm.comparator).sstableLevel(sstableLevel);
-        SSTableMultiWriter writer = SimpleSSTableMultiWriter.create(descriptor, keyCount, repairedAt, cfm, collector, header, indexes, txn);
+        MetadataCollector collector = new MetadataCollector(metadata.get().comparator).sstableLevel(sstableLevel);
+        SSTableMultiWriter writer = SimpleSSTableMultiWriter.create(descriptor, keyCount, repairedAt, metadata, collector, header, indexes, txn);
         return new SSTableTxnWriter(txn, writer);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/SimpleSSTableMultiWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SimpleSSTableMultiWriter.java b/src/java/org/apache/cassandra/io/sstable/SimpleSSTableMultiWriter.java
index 2217ae2..2d7d967 100644
--- a/src/java/org/apache/cassandra/io/sstable/SimpleSSTableMultiWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SimpleSSTableMultiWriter.java
@@ -15,14 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.io.sstable;
 
 import java.util.Collection;
 import java.util.Collections;
-import java.util.UUID;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
@@ -31,6 +28,8 @@ import org.apache.cassandra.index.Index;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadataRef;
 
 public class SimpleSSTableMultiWriter implements SSTableMultiWriter
 {
@@ -80,9 +79,9 @@ public class SimpleSSTableMultiWriter implements SSTableMultiWriter
         return writer.getFilePointer();
     }
 
-    public UUID getCfId()
+    public TableId getTableId()
     {
-        return writer.metadata.cfId;
+        return writer.metadata().id;
     }
 
     public Throwable commit(Throwable accumulate)
@@ -110,13 +109,13 @@ public class SimpleSSTableMultiWriter implements SSTableMultiWriter
     public static SSTableMultiWriter create(Descriptor descriptor,
                                             long keyCount,
                                             long repairedAt,
-                                            CFMetaData cfm,
+                                            TableMetadataRef metadata,
                                             MetadataCollector metadataCollector,
                                             SerializationHeader header,
                                             Collection<Index> indexes,
                                             LifecycleTransaction txn)
     {
-        SSTableWriter writer = SSTableWriter.create(descriptor, keyCount, repairedAt, cfm, metadataCollector, header, indexes, txn);
+        SSTableWriter writer = SSTableWriter.create(descriptor, keyCount, repairedAt, metadata, metadataCollector, header, indexes, txn);
         return new SimpleSSTableMultiWriter(writer, txn);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
index 89c064b..766a930 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.UUID;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -32,6 +31,7 @@ import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -159,9 +159,9 @@ public class RangeAwareSSTableWriter implements SSTableMultiWriter
     }
 
     @Override
-    public UUID getCfId()
+    public TableId getTableId()
     {
-        return currentWriter.getCfId();
+        return currentWriter.getTableId();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java
index 29e29ef..38a7f57 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java
@@ -18,7 +18,8 @@
 package org.apache.cassandra.io.sstable.format;
 
 import com.google.common.base.CharMatcher;
-import org.apache.cassandra.config.CFMetaData;
+
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.io.sstable.format.big.BigFormat;
@@ -37,7 +38,7 @@ public interface SSTableFormat
     SSTableWriter.Factory getWriterFactory();
     SSTableReader.Factory getReaderFactory();
 
-    RowIndexEntry.IndexSerializer<?> getIndexSerializer(CFMetaData cfm, Version version, SerializationHeader header);
+    RowIndexEntry.IndexSerializer<?> getIndexSerializer(TableMetadata metadata, Version version, SerializationHeader header);
 
     public static enum Type
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index d446475..87e12eb 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -26,7 +26,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Ordering;
 import com.google.common.primitives.Longs;
@@ -44,11 +43,8 @@ import org.apache.cassandra.cache.KeyCacheKey;
 import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.EncodingStats;
@@ -56,7 +52,6 @@ import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.index.internal.CassandraIndex;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.*;
@@ -65,7 +60,11 @@ import org.apache.cassandra.io.util.*;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.schema.CachingParams;
-import org.apache.cassandra.schema.IndexMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.*;
@@ -233,8 +232,8 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     protected final AtomicLong keyCacheHit = new AtomicLong(0);
     protected final AtomicLong keyCacheRequest = new AtomicLong(0);
 
-    private final InstanceTidier tidy = new InstanceTidier(descriptor, metadata);
-    private final Ref<SSTableReader> selfRef = new Ref<>(this, tidy);
+    private final InstanceTidier tidy;
+    private final Ref<SSTableReader> selfRef;
 
     private RestorableMeter readMeter;
 
@@ -355,32 +354,28 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
     public static SSTableReader open(Descriptor descriptor) throws IOException
     {
-        CFMetaData metadata;
+        TableMetadataRef metadata;
         if (descriptor.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR))
         {
             int i = descriptor.cfname.indexOf(SECONDARY_INDEX_NAME_SEPARATOR);
-            String parentName = descriptor.cfname.substring(0, i);
             String indexName = descriptor.cfname.substring(i + 1);
-            CFMetaData parent = Schema.instance.getCFMetaData(descriptor.ksname, parentName);
-            IndexMetadata def = parent.getIndexes()
-                                      .get(indexName)
-                                      .orElseThrow(() -> new AssertionError(
-                                                                           "Could not find index metadata for index cf " + i));
-            metadata = CassandraIndex.indexCfsMetadata(parent, def);
+            metadata = Schema.instance.getIndexTableMetadataRef(descriptor.ksname, indexName);
+            if (metadata == null)
+                throw new AssertionError("Could not find index metadata for index cf " + i);
         }
         else
         {
-            metadata = Schema.instance.getCFMetaData(descriptor.ksname, descriptor.cfname);
+            metadata = Schema.instance.getTableMetadataRef(descriptor.ksname, descriptor.cfname);
         }
         return open(descriptor, metadata);
     }
 
-    public static SSTableReader open(Descriptor desc, CFMetaData metadata) throws IOException
+    public static SSTableReader open(Descriptor desc, TableMetadataRef metadata) throws IOException
     {
         return open(desc, componentsFor(desc), metadata);
     }
 
-    public static SSTableReader open(Descriptor descriptor, Set<Component> components, CFMetaData metadata) throws IOException
+    public static SSTableReader open(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata) throws IOException
     {
         return open(descriptor, components, metadata, true, true);
     }
@@ -392,7 +387,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     }
 
     // use only for offline or "Standalone" operations
-    public static SSTableReader openNoValidation(Descriptor descriptor, CFMetaData metadata) throws IOException
+    public static SSTableReader openNoValidation(Descriptor descriptor, TableMetadataRef metadata) throws IOException
     {
         return open(descriptor, componentsFor(descriptor), metadata, false, false); // do not track hotness
     }
@@ -406,7 +401,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
      * @return opened SSTableReader
      * @throws IOException
      */
-    public static SSTableReader openForBatch(Descriptor descriptor, Set<Component> components, CFMetaData metadata) throws IOException
+    public static SSTableReader openForBatch(Descriptor descriptor, Set<Component> components, TableMetadataRef metadata) throws IOException
     {
         // Minimum components without which we can't do anything
         assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor;
@@ -422,7 +417,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         // Check if sstable is created using same partitioner.
         // Partitioner can be null, which indicates older version of sstable or no stats available.
         // In that case, we skip the check.
-        String partitionerName = metadata.partitioner.getClass().getCanonicalName();
+        String partitionerName = metadata.get().partitioner.getClass().getCanonicalName();
         if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
         {
             logger.error("Cannot open {}; partitioner {} does not match system partitioner {}.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
@@ -438,7 +433,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                                              System.currentTimeMillis(),
                                              statsMetadata,
                                              OpenReason.NORMAL,
-                                             header == null? null : header.toHeader(metadata));
+                                             header.toHeader(metadata.get()));
 
         try(FileHandle.Builder ibuilder = new FileHandle.Builder(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX))
                                                      .mmapped(DatabaseDescriptor.getIndexAccessMode() == Config.DiskAccessMode.mmap)
@@ -461,10 +456,10 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     }
 
     public static SSTableReader open(Descriptor descriptor,
-                                      Set<Component> components,
-                                      CFMetaData metadata,
-                                      boolean validate,
-                                      boolean trackHotness) throws IOException
+                                     Set<Component> components,
+                                     TableMetadataRef metadata,
+                                     boolean validate,
+                                     boolean trackHotness) throws IOException
     {
         // Minimum components without which we can't do anything
         assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor;
@@ -483,7 +478,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         // Check if sstable is created using same partitioner.
         // Partitioner can be null, which indicates older version of sstable or no stats available.
         // In that case, we skip the check.
-        String partitionerName = metadata.partitioner.getClass().getCanonicalName();
+        String partitionerName = metadata.get().partitioner.getClass().getCanonicalName();
         if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
         {
             logger.error("Cannot open {}; partitioner {} does not match system partitioner {}.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
@@ -499,7 +494,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                                              System.currentTimeMillis(),
                                              statsMetadata,
                                              OpenReason.NORMAL,
-                                             header == null ? null : header.toHeader(metadata));
+                                             header.toHeader(metadata.get()));
 
         try
         {
@@ -533,7 +528,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     }
 
     public static Collection<SSTableReader> openAll(Set<Map.Entry<Descriptor, Set<Component>>> entries,
-                                                    final CFMetaData metadata)
+                                                    final TableMetadataRef metadata)
     {
         final Collection<SSTableReader> sstables = new LinkedBlockingQueue<>();
 
@@ -591,7 +586,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
      */
     public static SSTableReader internalOpen(Descriptor desc,
                                       Set<Component> components,
-                                      CFMetaData metadata,
+                                      TableMetadataRef metadata,
                                       FileHandle ifile,
                                       FileHandle dfile,
                                       IndexSummary isummary,
@@ -616,12 +611,12 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
 
     private static SSTableReader internalOpen(final Descriptor descriptor,
-                                            Set<Component> components,
-                                            CFMetaData metadata,
-                                            Long maxDataAge,
-                                            StatsMetadata sstableMetadata,
-                                            OpenReason openReason,
-                                            SerializationHeader header)
+                                              Set<Component> components,
+                                              TableMetadataRef metadata,
+                                              Long maxDataAge,
+                                              StatsMetadata sstableMetadata,
+                                              OpenReason openReason,
+                                              SerializationHeader header)
     {
         Factory readerFactory = descriptor.getFormat().getReaderFactory();
 
@@ -630,7 +625,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
     protected SSTableReader(final Descriptor desc,
                             Set<Component> components,
-                            CFMetaData metadata,
+                            TableMetadataRef metadata,
                             long maxDataAge,
                             StatsMetadata sstableMetadata,
                             OpenReason openReason,
@@ -641,7 +636,9 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         this.header = header;
         this.maxDataAge = maxDataAge;
         this.openReason = openReason;
-        this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata, desc.version, header);
+        this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata.get(), desc.version, header);
+        tidy = new InstanceTidier(descriptor, metadata.id);
+        selfRef = new Ref<>(this, tidy);
     }
 
     public static long getTotalBytes(Iterable<SSTableReader> sstables)
@@ -682,7 +679,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         // e.g. by BulkLoader, which does not initialize the cache.  As a kludge, we set up the cache
         // here when we know we're being wired into the rest of the server infrastructure.
         keyCache = CacheService.instance.keyCache;
-        final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.cfId);
+        final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata().id);
         if (cfs != null)
             setCrcCheckChance(cfs.getCrcCheckChance());
     }
@@ -694,7 +691,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
     private void load(ValidationMetadata validation) throws IOException
     {
-        if (metadata.params.bloomFilterFpChance == 1.0)
+        if (metadata().params.bloomFilterFpChance == 1.0)
         {
             // bf is disabled.
             load(false, true);
@@ -711,7 +708,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
             // bf is enabled, but filter component is missing.
             load(true, true);
         }
-        else if (validation.bloomFilterFPChance != metadata.params.bloomFilterFpChance)
+        else if (validation.bloomFilterFPChance != metadata().params.bloomFilterFpChance)
         {
             // bf fp chance in sstable metadata and it has changed since compaction.
             load(true, true);
@@ -819,9 +816,9 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                     : estimateRowsFromIndex(primaryIndex); // statistics is supposed to be optional
 
             if (recreateBloomFilter)
-                bf = FilterFactory.getFilter(estimatedKeys, metadata.params.bloomFilterFpChance, true);
+                bf = FilterFactory.getFilter(estimatedKeys, metadata().params.bloomFilterFpChance, true);
 
-            try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.params.minIndexInterval, samplingLevel))
+            try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata().params.minIndexInterval, samplingLevel))
             {
                 long indexPosition;
 
@@ -871,6 +868,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         DataInputStream iStream = null;
         try
         {
+            TableMetadata metadata = metadata();
             iStream = new DataInputStream(new FileInputStream(summariesFile));
             indexSummary = IndexSummary.serializer.deserialize(
                     iStream, getPartitioner(),
@@ -1100,8 +1098,8 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         {
             assert openReason != OpenReason.EARLY;
 
-            int minIndexInterval = metadata.params.minIndexInterval;
-            int maxIndexInterval = metadata.params.maxIndexInterval;
+            int minIndexInterval = metadata().params.minIndexInterval;
+            int maxIndexInterval = metadata().params.maxIndexInterval;
             double effectiveInterval = indexSummary.getEffectiveIndexInterval();
 
             IndexSummary newSummary;
@@ -1146,7 +1144,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         try
         {
             long indexSize = primaryIndex.length();
-            try (IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata.params.minIndexInterval, newSamplingLevel))
+            try (IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata().params.minIndexInterval, newSamplingLevel))
             {
                 long indexPosition;
                 while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
@@ -1467,28 +1465,24 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
     public KeyCacheKey getCacheKey(DecoratedKey key)
     {
-        return new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
+        return new KeyCacheKey(metadata(), descriptor, key.getKey());
     }
 
     public void cacheKey(DecoratedKey key, RowIndexEntry info)
     {
-        CachingParams caching = metadata.params.caching;
+        CachingParams caching = metadata().params.caching;
 
         if (!caching.cacheKeys() || keyCache == null || keyCache.getCapacity() == 0)
             return;
 
-        KeyCacheKey cacheKey = new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
+        KeyCacheKey cacheKey = new KeyCacheKey(metadata(), descriptor, key.getKey());
         logger.trace("Adding cache entry for {} -> {}", cacheKey, info);
         keyCache.put(cacheKey, info);
     }
 
     public RowIndexEntry getCachedPosition(DecoratedKey key, boolean updateStats)
     {
-        if (keyCacheEnabled())
-        {
-            return getCachedPosition(getCacheKey(key), updateStats);
-        }
-        return null;
+        return getCachedPosition(new KeyCacheKey(metadata(), descriptor, key.getKey()), updateStats);
     }
 
     protected RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats)
@@ -1516,7 +1510,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
     private boolean keyCacheEnabled()
     {
-        return keyCache != null && keyCache.getCapacity() > 0 && metadata.params.caching.cacheKeys();
+        return keyCache != null && keyCache.getCapacity() > 0 && metadata().params.caching.cacheKeys();
     }
 
     /**
@@ -2003,7 +1997,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
     private int compare(List<ByteBuffer> values1, List<ByteBuffer> values2)
     {
-        ClusteringComparator comparator = metadata.comparator;
+        ClusteringComparator comparator = metadata().comparator;
         for (int i = 0; i < Math.min(values1.size(), values2.size()); i++)
         {
             int cmp = comparator.subtype(i).compare(values1.get(i), values2.get(i));
@@ -2070,7 +2064,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     private static final class InstanceTidier implements Tidy
     {
         private final Descriptor descriptor;
-        private final CFMetaData metadata;
+        private final TableId tableId;
         private IFilter bf;
         private IndexSummary summary;
 
@@ -2100,10 +2094,10 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                 global.ensureReadMeter();
         }
 
-        InstanceTidier(Descriptor descriptor, CFMetaData metadata)
+        InstanceTidier(Descriptor descriptor, TableId tableId)
         {
             this.descriptor = descriptor;
-            this.metadata = metadata;
+            this.tableId = tableId;
         }
 
         public void tidy()
@@ -2115,7 +2109,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
             if (!setup)
                 return;
 
-            final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.cfId);
+            final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(tableId);
             final OpOrder.Barrier barrier;
             if (cfs != null)
             {
@@ -2285,7 +2279,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     {
         public abstract SSTableReader open(final Descriptor descriptor,
                                            Set<Component> components,
-                                           CFMetaData metadata,
+                                           TableMetadataRef metadata,
                                            Long maxDataAge,
                                            StatsMetadata sstableMetadata,
                                            OpenReason openReason,


[19/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/Indexes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Indexes.java b/src/java/org/apache/cassandra/schema/Indexes.java
index eb49d39..81d400e 100644
--- a/src/java/org/apache/cassandra/schema/Indexes.java
+++ b/src/java/org/apache/cassandra/schema/Indexes.java
@@ -15,14 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.cassandra.schema;
 
 import java.util.*;
+import java.util.stream.Stream;
 
 import com.google.common.collect.ImmutableMap;
 
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.exceptions.ConfigurationException;
+
+import static java.lang.String.format;
 
 import static com.google.common.collect.Iterables.filter;
 
@@ -35,7 +37,7 @@ import static com.google.common.collect.Iterables.filter;
  * support is added for multiple target columns per-index and for indexes with
  * TargetType.ROW
  */
-public class Indexes implements Iterable<IndexMetadata>
+public final class Indexes implements Iterable<IndexMetadata>
 {
     private final ImmutableMap<String, IndexMetadata> indexesByName;
     private final ImmutableMap<UUID, IndexMetadata> indexesById;
@@ -56,11 +58,26 @@ public class Indexes implements Iterable<IndexMetadata>
         return builder().build();
     }
 
+    public static Indexes of(IndexMetadata... indexes)
+    {
+        return builder().add(indexes).build();
+    }
+
+    public static Indexes of(Iterable<IndexMetadata> indexes)
+    {
+        return builder().add(indexes).build();
+    }
+
     public Iterator<IndexMetadata> iterator()
     {
         return indexesByName.values().iterator();
     }
 
+    public Stream<IndexMetadata> stream()
+    {
+        return indexesById.values().stream();
+    }
+
     public int size()
     {
         return indexesByName.size();
@@ -121,7 +138,7 @@ public class Indexes implements Iterable<IndexMetadata>
     public Indexes with(IndexMetadata index)
     {
         if (get(index.name).isPresent())
-            throw new IllegalStateException(String.format("Index %s already exists", index.name));
+            throw new IllegalStateException(format("Index %s already exists", index.name));
 
         return builder().add(this).add(index).build();
     }
@@ -131,7 +148,7 @@ public class Indexes implements Iterable<IndexMetadata>
      */
     public Indexes without(String name)
     {
-        IndexMetadata index = get(name).orElseThrow(() -> new IllegalStateException(String.format("Index %s doesn't exist", name)));
+        IndexMetadata index = get(name).orElseThrow(() -> new IllegalStateException(format("Index %s doesn't exist", name)));
         return builder().add(filter(this, v -> v != index)).build();
     }
 
@@ -149,6 +166,25 @@ public class Indexes implements Iterable<IndexMetadata>
         return this == o || (o instanceof Indexes && indexesByName.equals(((Indexes) o).indexesByName));
     }
 
+    public void validate(TableMetadata table)
+    {
+        /*
+         * Index name check is duplicated in Keyspaces, for the time being.
+         * The reason for this is that schema altering statements are not calling
+         * Keyspaces.validate() as of yet. TODO: remove this once they do (on CASSANDRA-9425 completion)
+         */
+        Set<String> indexNames = new HashSet<>();
+        for (IndexMetadata index : indexesByName.values())
+        {
+            if (indexNames.contains(index.name))
+                throw new ConfigurationException(format("Duplicate index name %s for table %s", index.name, table));
+
+            indexNames.add(index.name);
+        }
+
+        indexesByName.values().forEach(i -> i.validate(table));
+    }
+
     @Override
     public int hashCode()
     {
@@ -164,7 +200,7 @@ public class Indexes implements Iterable<IndexMetadata>
     public static String getAvailableIndexName(String ksName, String cfName, String indexNameRoot)
     {
 
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(ksName);
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(ksName);
         Set<String> existingNames = ksm == null ? new HashSet<>() : ksm.existingIndexNames(null);
         String baseName = IndexMetadata.getDefaultIndexName(cfName, indexNameRoot);
         String acceptedName = baseName;
@@ -196,6 +232,13 @@ public class Indexes implements Iterable<IndexMetadata>
             return this;
         }
 
+        public Builder add(IndexMetadata... indexes)
+        {
+            for (IndexMetadata index : indexes)
+                add(index);
+            return this;
+        }
+
         public Builder add(Iterable<IndexMetadata> indexes)
         {
             indexes.forEach(this::add);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java b/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
index 4fefd44..80a3869 100644
--- a/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
+++ b/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
@@ -27,11 +27,10 @@ import com.google.common.base.MoreObjects;
 import com.google.common.base.Objects;
 import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.SchemaConstants;
-import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.exceptions.ConfigurationException;
 
+import static java.lang.String.format;
+
 /**
  * An immutable representation of keyspace metadata (name, params, tables, types, and functions).
  */
@@ -94,15 +93,15 @@ public final class KeyspaceMetadata
         return new KeyspaceMetadata(name, params, tables, views, types, functions);
     }
 
-    public Iterable<CFMetaData> tablesAndViews()
+    public Iterable<TableMetadata> tablesAndViews()
     {
         return Iterables.concat(tables, views.metadatas());
     }
 
     @Nullable
-    public CFMetaData getTableOrViewNullable(String tableOrViewName)
+    public TableMetadata getTableOrViewNullable(String tableOrViewName)
     {
-        ViewDefinition view = views.getNullable(tableOrViewName);
+        ViewMetadata view = views.getNullable(tableOrViewName);
         return view == null
              ? tables.getNullable(tableOrViewName)
              : view.metadata;
@@ -111,18 +110,18 @@ public final class KeyspaceMetadata
     public Set<String> existingIndexNames(String cfToExclude)
     {
         Set<String> indexNames = new HashSet<>();
-        for (CFMetaData table : tables)
-            if (cfToExclude == null || !table.cfName.equals(cfToExclude))
-                for (IndexMetadata index : table.getIndexes())
+        for (TableMetadata table : tables)
+            if (cfToExclude == null || !table.name.equals(cfToExclude))
+                for (IndexMetadata index : table.indexes)
                     indexNames.add(index.name);
         return indexNames;
     }
 
-    public Optional<CFMetaData> findIndexedTable(String indexName)
+    public Optional<TableMetadata> findIndexedTable(String indexName)
     {
-        for (CFMetaData cfm : tablesAndViews())
-            if (cfm.getIndexes().has(indexName))
-                return Optional.of(cfm);
+        for (TableMetadata table : tablesAndViews())
+            if (table.indexes.has(indexName))
+                return Optional.of(table);
 
         return Optional.empty();
     }
@@ -167,12 +166,28 @@ public final class KeyspaceMetadata
 
     public void validate()
     {
-        if (!CFMetaData.isNameValid(name))
-            throw new ConfigurationException(String.format("Keyspace name must not be empty, more than %s characters long, "
-                                                           + "or contain non-alphanumeric-underscore characters (got \"%s\")",
-                                                           SchemaConstants.NAME_LENGTH,
-                                                           name));
+        if (!SchemaConstants.isValidName(name))
+        {
+            throw new ConfigurationException(format("Keyspace name must not be empty, more than %s characters long, "
+                                                    + "or contain non-alphanumeric-underscore characters (got \"%s\")",
+                                                    SchemaConstants.NAME_LENGTH,
+                                                    name));
+        }
+
         params.validate(name);
-        tablesAndViews().forEach(CFMetaData::validate);
+
+        tablesAndViews().forEach(TableMetadata::validate);
+
+        Set<String> indexNames = new HashSet<>();
+        for (TableMetadata table : tables)
+        {
+            for (IndexMetadata index : table.indexes)
+            {
+                if (indexNames.contains(index.name))
+                    throw new ConfigurationException(format("Duplicate index name %s in keyspace %s", index.name, name));
+
+                indexNames.add(index.name);
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/Keyspaces.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Keyspaces.java b/src/java/org/apache/cassandra/schema/Keyspaces.java
index 8c0a63e..1692f88 100644
--- a/src/java/org/apache/cassandra/schema/Keyspaces.java
+++ b/src/java/org/apache/cassandra/schema/Keyspaces.java
@@ -18,9 +18,12 @@
 package org.apache.cassandra.schema;
 
 import java.util.Iterator;
+import java.util.Set;
 import java.util.function.Predicate;
 import java.util.stream.Stream;
 
+import javax.annotation.Nullable;
+
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.MapDifference;
 import com.google.common.collect.Maps;
@@ -28,10 +31,12 @@ import com.google.common.collect.Maps;
 public final class Keyspaces implements Iterable<KeyspaceMetadata>
 {
     private final ImmutableMap<String, KeyspaceMetadata> keyspaces;
+    private final ImmutableMap<TableId, TableMetadata> tables;
 
     private Keyspaces(Builder builder)
     {
         keyspaces = builder.keyspaces.build();
+        tables = builder.tables.build();
     }
 
     public static Builder builder()
@@ -59,6 +64,23 @@ public final class Keyspaces implements Iterable<KeyspaceMetadata>
         return keyspaces.values().stream();
     }
 
+    public Set<String> names()
+    {
+        return keyspaces.keySet();
+    }
+
+    @Nullable
+    public KeyspaceMetadata getNullable(String name)
+    {
+        return keyspaces.get(name);
+    }
+
+    @Nullable
+    public TableMetadata getTableOrViewNullable(TableId id)
+    {
+        return tables.get(id);
+    }
+
     public Keyspaces filter(Predicate<KeyspaceMetadata> predicate)
     {
         Builder builder = builder();
@@ -66,6 +88,25 @@ public final class Keyspaces implements Iterable<KeyspaceMetadata>
         return builder.build();
     }
 
+    /**
+     * Creates a Keyspaces instance with the keyspace with the provided name removed
+     */
+    public Keyspaces without(String name)
+    {
+        KeyspaceMetadata keyspace = getNullable(name);
+        if (keyspace == null)
+            throw new IllegalStateException(String.format("Keyspace %s doesn't exists", name));
+
+        return builder().add(filter(k -> k != keyspace)).build();
+    }
+
+    public Keyspaces withAddedOrUpdated(KeyspaceMetadata keyspace)
+    {
+        return builder().add(filter(k -> !k.name.equals(keyspace.name)))
+                        .add(keyspace)
+                        .build();
+    }
+
     MapDifference<String, KeyspaceMetadata> diff(Keyspaces other)
     {
         return Maps.difference(keyspaces, other.keyspaces);
@@ -92,6 +133,7 @@ public final class Keyspaces implements Iterable<KeyspaceMetadata>
     public static final class Builder
     {
         private final ImmutableMap.Builder<String, KeyspaceMetadata> keyspaces = new ImmutableMap.Builder<>();
+        private final ImmutableMap.Builder<TableId, TableMetadata> tables = new ImmutableMap.Builder<>();
 
         private Builder()
         {
@@ -105,6 +147,10 @@ public final class Keyspaces implements Iterable<KeyspaceMetadata>
         public Builder add(KeyspaceMetadata keyspace)
         {
             keyspaces.put(keyspace.name, keyspace);
+
+            keyspace.tables.forEach(t -> tables.put(t.id, t));
+            keyspace.views.forEach(v -> tables.put(v.metadata.id, v.metadata));
+
             return this;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/MigrationManager.java b/src/java/org/apache/cassandra/schema/MigrationManager.java
new file mode 100644
index 0000000..d34a096
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/MigrationManager.java
@@ -0,0 +1,493 @@
+/*
+ * 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.net.InetAddress;
+import java.util.*;
+import java.util.concurrent.*;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.cql3.functions.UDFunction;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.AlreadyExistsException;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.gms.*;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.WrappedRunnable;
+
+public class MigrationManager
+{
+    private static final Logger logger = LoggerFactory.getLogger(MigrationManager.class);
+
+    public static final MigrationManager instance = new MigrationManager();
+
+    private static final RuntimeMXBean runtimeMXBean = ManagementFactory.getRuntimeMXBean();
+
+    private static final int MIGRATION_DELAY_IN_MS = 60000;
+
+    private static final int MIGRATION_TASK_WAIT_IN_SECONDS = Integer.parseInt(System.getProperty("cassandra.migration_task_wait_in_seconds", "1"));
+
+    private MigrationManager() {}
+
+    public static void scheduleSchemaPull(InetAddress endpoint, EndpointState state)
+    {
+        VersionedValue value = state.getApplicationState(ApplicationState.SCHEMA);
+
+        if (!endpoint.equals(FBUtilities.getBroadcastAddress()) && value != null)
+            maybeScheduleSchemaPull(UUID.fromString(value.value), endpoint);
+    }
+
+    /**
+     * If versions differ this node sends request with local migration list to the endpoint
+     * and expecting to receive a list of migrations to apply locally.
+     */
+    private static void maybeScheduleSchemaPull(final UUID theirVersion, final InetAddress endpoint)
+    {
+        if ((Schema.instance.getVersion() != null && Schema.instance.getVersion().equals(theirVersion)) || !shouldPullSchemaFrom(endpoint))
+        {
+            logger.debug("Not pulling schema because versions match or shouldPullSchemaFrom returned false");
+            return;
+        }
+
+        if (SchemaConstants.emptyVersion.equals(Schema.instance.getVersion()) || runtimeMXBean.getUptime() < MIGRATION_DELAY_IN_MS)
+        {
+            // If we think we may be bootstrapping or have recently started, submit MigrationTask immediately
+            logger.debug("Submitting migration task for {}", endpoint);
+            submitMigrationTask(endpoint);
+        }
+        else
+        {
+            // Include a delay to make sure we have a chance to apply any changes being
+            // pushed out simultaneously. See CASSANDRA-5025
+            Runnable runnable = () ->
+            {
+                // grab the latest version of the schema since it may have changed again since the initial scheduling
+                EndpointState epState = Gossiper.instance.getEndpointStateForEndpoint(endpoint);
+                if (epState == null)
+                {
+                    logger.debug("epState vanished for {}, not submitting migration task", endpoint);
+                    return;
+                }
+                VersionedValue value = epState.getApplicationState(ApplicationState.SCHEMA);
+                UUID currentVersion = UUID.fromString(value.value);
+                if (Schema.instance.getVersion().equals(currentVersion))
+                {
+                    logger.debug("not submitting migration task for {} because our versions match", endpoint);
+                    return;
+                }
+                logger.debug("submitting migration task for {}", endpoint);
+                submitMigrationTask(endpoint);
+            };
+            ScheduledExecutors.nonPeriodicTasks.schedule(runnable, MIGRATION_DELAY_IN_MS, TimeUnit.MILLISECONDS);
+        }
+    }
+
+    private static Future<?> submitMigrationTask(InetAddress endpoint)
+    {
+        /*
+         * Do not de-ref the future because that causes distributed deadlock (CASSANDRA-3832) because we are
+         * running in the gossip stage.
+         */
+        return StageManager.getStage(Stage.MIGRATION).submit(new MigrationTask(endpoint));
+    }
+
+    static boolean shouldPullSchemaFrom(InetAddress endpoint)
+    {
+        /*
+         * Don't request schema from nodes with a differnt or unknonw major version (may have incompatible schema)
+         * Don't request schema from fat clients
+         */
+        return MessagingService.instance().knowsVersion(endpoint)
+                && MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version
+                && !Gossiper.instance.isGossipOnlyMember(endpoint);
+    }
+
+    public static boolean isReadyForBootstrap()
+    {
+        return MigrationTask.getInflightTasks().isEmpty();
+    }
+
+    public static void waitUntilReadyForBootstrap()
+    {
+        CountDownLatch completionLatch;
+        while ((completionLatch = MigrationTask.getInflightTasks().poll()) != null)
+        {
+            try
+            {
+                if (!completionLatch.await(MIGRATION_TASK_WAIT_IN_SECONDS, TimeUnit.SECONDS))
+                    logger.error("Migration task failed to complete");
+            }
+            catch (InterruptedException e)
+            {
+                Thread.currentThread().interrupt();
+                logger.error("Migration task was interrupted");
+            }
+        }
+    }
+
+    public static void announceNewKeyspace(KeyspaceMetadata ksm) throws ConfigurationException
+    {
+        announceNewKeyspace(ksm, false);
+    }
+
+    public static void announceNewKeyspace(KeyspaceMetadata ksm, boolean announceLocally) throws ConfigurationException
+    {
+        announceNewKeyspace(ksm, FBUtilities.timestampMicros(), announceLocally);
+    }
+
+    public static void announceNewKeyspace(KeyspaceMetadata ksm, long timestamp, boolean announceLocally) throws ConfigurationException
+    {
+        ksm.validate();
+
+        if (Schema.instance.getKeyspaceMetadata(ksm.name) != null)
+            throw new AlreadyExistsException(ksm.name);
+
+        logger.info("Create new Keyspace: {}", ksm);
+        announce(SchemaKeyspace.makeCreateKeyspaceMutation(ksm, timestamp), announceLocally);
+    }
+
+    public static void announceNewTable(TableMetadata cfm) throws ConfigurationException
+    {
+        announceNewTable(cfm, false);
+    }
+
+    public static void announceNewTable(TableMetadata cfm, boolean announceLocally)
+    {
+        announceNewTable(cfm, announceLocally, true);
+    }
+
+    /**
+     * Announces the table even if the definition is already know locally.
+     * This should generally be avoided but is used internally when we want to force the most up to date version of
+     * a system table schema (Note that we don't know if the schema we force _is_ the most recent version or not, we
+     * just rely on idempotency to basically ignore that announce if it's not. That's why we can't use announceUpdateColumnFamily,
+     * it would for instance delete new columns if this is not called with the most up-to-date version)
+     *
+     * Note that this is only safe for system tables where we know the id is fixed and will be the same whatever version
+     * of the definition is used.
+     */
+    public static void forceAnnounceNewTable(TableMetadata cfm)
+    {
+        announceNewTable(cfm, false, false);
+    }
+
+    private static void announceNewTable(TableMetadata cfm, boolean announceLocally, boolean throwOnDuplicate)
+    {
+        cfm.validate();
+
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(cfm.keyspace);
+        if (ksm == null)
+            throw new ConfigurationException(String.format("Cannot add table '%s' to non existing keyspace '%s'.", cfm.name, cfm.keyspace));
+        // If we have a table or a view which has the same name, we can't add a new one
+        else if (throwOnDuplicate && ksm.getTableOrViewNullable(cfm.name) != null)
+            throw new AlreadyExistsException(cfm.keyspace, cfm.name);
+
+        logger.info("Create new table: {}", cfm);
+        announce(SchemaKeyspace.makeCreateTableMutation(ksm, cfm, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceNewView(ViewMetadata view, boolean announceLocally) throws ConfigurationException
+    {
+        view.metadata.validate();
+
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(view.keyspace);
+        if (ksm == null)
+            throw new ConfigurationException(String.format("Cannot add table '%s' to non existing keyspace '%s'.", view.name, view.keyspace));
+        else if (ksm.getTableOrViewNullable(view.name) != null)
+            throw new AlreadyExistsException(view.keyspace, view.name);
+
+        logger.info("Create new view: {}", view);
+        announce(SchemaKeyspace.makeCreateViewMutation(ksm, view, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceNewType(UserType newType, boolean announceLocally)
+    {
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(newType.keyspace);
+        announce(SchemaKeyspace.makeCreateTypeMutation(ksm, newType, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceNewFunction(UDFunction udf, boolean announceLocally)
+    {
+        logger.info("Create scalar function '{}'", udf.name());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(udf.name().keyspace);
+        announce(SchemaKeyspace.makeCreateFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceNewAggregate(UDAggregate udf, boolean announceLocally)
+    {
+        logger.info("Create aggregate function '{}'", udf.name());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(udf.name().keyspace);
+        announce(SchemaKeyspace.makeCreateAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceKeyspaceUpdate(KeyspaceMetadata ksm) throws ConfigurationException
+    {
+        announceKeyspaceUpdate(ksm, false);
+    }
+
+    public static void announceKeyspaceUpdate(KeyspaceMetadata ksm, boolean announceLocally) throws ConfigurationException
+    {
+        ksm.validate();
+
+        KeyspaceMetadata oldKsm = Schema.instance.getKeyspaceMetadata(ksm.name);
+        if (oldKsm == null)
+            throw new ConfigurationException(String.format("Cannot update non existing keyspace '%s'.", ksm.name));
+
+        logger.info("Update Keyspace '{}' From {} To {}", ksm.name, oldKsm, ksm);
+        announce(SchemaKeyspace.makeCreateKeyspaceMutation(ksm.name, ksm.params, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceTableUpdate(TableMetadata tm) throws ConfigurationException
+    {
+        announceTableUpdate(tm, false);
+    }
+
+    public static void announceTableUpdate(TableMetadata updated, boolean announceLocally) throws ConfigurationException
+    {
+        updated.validate();
+
+        TableMetadata current = Schema.instance.getTableMetadata(updated.keyspace, updated.name);
+        if (current == null)
+            throw new ConfigurationException(String.format("Cannot update non existing table '%s' in keyspace '%s'.", updated.name, updated.keyspace));
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(current.keyspace);
+
+        current.validateCompatibility(updated);
+
+        logger.info("Update table '{}/{}' From {} To {}", current.keyspace, current.name, current, updated);
+        announce(SchemaKeyspace.makeUpdateTableMutation(ksm, current, updated, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceViewUpdate(ViewMetadata view, boolean announceLocally) throws ConfigurationException
+    {
+        view.metadata.validate();
+
+        ViewMetadata oldView = Schema.instance.getView(view.keyspace, view.name);
+        if (oldView == null)
+            throw new ConfigurationException(String.format("Cannot update non existing materialized view '%s' in keyspace '%s'.", view.name, view.keyspace));
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(view.keyspace);
+
+        oldView.metadata.validateCompatibility(view.metadata);
+
+        logger.info("Update view '{}/{}' From {} To {}", view.keyspace, view.name, oldView, view);
+        announce(SchemaKeyspace.makeUpdateViewMutation(ksm, oldView, view, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceTypeUpdate(UserType updatedType, boolean announceLocally)
+    {
+        logger.info("Update type '{}.{}' to {}", updatedType.keyspace, updatedType.getNameAsString(), updatedType);
+        announceNewType(updatedType, announceLocally);
+    }
+
+    public static void announceKeyspaceDrop(String ksName) throws ConfigurationException
+    {
+        announceKeyspaceDrop(ksName, false);
+    }
+
+    public static void announceKeyspaceDrop(String ksName, boolean announceLocally) throws ConfigurationException
+    {
+        KeyspaceMetadata oldKsm = Schema.instance.getKeyspaceMetadata(ksName);
+        if (oldKsm == null)
+            throw new ConfigurationException(String.format("Cannot drop non existing keyspace '%s'.", ksName));
+
+        logger.info("Drop Keyspace '{}'", oldKsm.name);
+        announce(SchemaKeyspace.makeDropKeyspaceMutation(oldKsm, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceTableDrop(String ksName, String cfName) throws ConfigurationException
+    {
+        announceTableDrop(ksName, cfName, false);
+    }
+
+    public static void announceTableDrop(String ksName, String cfName, boolean announceLocally) throws ConfigurationException
+    {
+        TableMetadata tm = Schema.instance.getTableMetadata(ksName, cfName);
+        if (tm == null)
+            throw new ConfigurationException(String.format("Cannot drop non existing table '%s' in keyspace '%s'.", cfName, ksName));
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(ksName);
+
+        logger.info("Drop table '{}/{}'", tm.keyspace, tm.name);
+        announce(SchemaKeyspace.makeDropTableMutation(ksm, tm, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceViewDrop(String ksName, String viewName, boolean announceLocally) throws ConfigurationException
+    {
+        ViewMetadata view = Schema.instance.getView(ksName, viewName);
+        if (view == null)
+            throw new ConfigurationException(String.format("Cannot drop non existing materialized view '%s' in keyspace '%s'.", viewName, ksName));
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(ksName);
+
+        logger.info("Drop table '{}/{}'", view.keyspace, view.name);
+        announce(SchemaKeyspace.makeDropViewMutation(ksm, view, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceTypeDrop(UserType droppedType, boolean announceLocally)
+    {
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(droppedType.keyspace);
+        announce(SchemaKeyspace.dropTypeFromSchemaMutation(ksm, droppedType, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceFunctionDrop(UDFunction udf, boolean announceLocally)
+    {
+        logger.info("Drop scalar function overload '{}' args '{}'", udf.name(), udf.argTypes());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(udf.name().keyspace);
+        announce(SchemaKeyspace.makeDropFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceAggregateDrop(UDAggregate udf, boolean announceLocally)
+    {
+        logger.info("Drop aggregate function overload '{}' args '{}'", udf.name(), udf.argTypes());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(udf.name().keyspace);
+        announce(SchemaKeyspace.makeDropAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    /**
+     * actively announce a new version to active hosts via rpc
+     * @param schema The schema mutation to be applied
+     */
+    private static void announce(Mutation.SimpleBuilder schema, boolean announceLocally)
+    {
+        List<Mutation> mutations = Collections.singletonList(schema.build());
+
+        if (announceLocally)
+            Schema.instance.merge(mutations);
+        else
+            FBUtilities.waitOnFuture(announce(mutations));
+    }
+
+    private static void pushSchemaMutation(InetAddress endpoint, Collection<Mutation> schema)
+    {
+        MessageOut<Collection<Mutation>> msg = new MessageOut<>(MessagingService.Verb.DEFINITIONS_UPDATE,
+                                                                schema,
+                                                                MigrationsSerializer.instance);
+        MessagingService.instance().sendOneWay(msg, endpoint);
+    }
+
+    // Returns a future on the local application of the schema
+    private static Future<?> announce(final Collection<Mutation> schema)
+    {
+        Future<?> f = StageManager.getStage(Stage.MIGRATION).submit(new WrappedRunnable()
+        {
+            protected void runMayThrow() throws ConfigurationException
+            {
+                Schema.instance.mergeAndAnnounceVersion(schema);
+            }
+        });
+
+        for (InetAddress endpoint : Gossiper.instance.getLiveMembers())
+        {
+            // only push schema to nodes with known and equal versions
+            if (!endpoint.equals(FBUtilities.getBroadcastAddress()) &&
+                    MessagingService.instance().knowsVersion(endpoint) &&
+                    MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version)
+                pushSchemaMutation(endpoint, schema);
+        }
+
+        return f;
+    }
+
+    /**
+     * Announce my version passively over gossip.
+     * Used to notify nodes as they arrive in the cluster.
+     *
+     * @param version The schema version to announce
+     */
+    static void passiveAnnounce(UUID version)
+    {
+        Gossiper.instance.addLocalApplicationState(ApplicationState.SCHEMA, StorageService.instance.valueFactory.schema(version));
+        logger.debug("Gossiping my schema version {}", version);
+    }
+
+    /**
+     * Clear all locally stored schema information and reset schema to initial state.
+     * Called by user (via JMX) who wants to get rid of schema disagreement.
+     */
+    public static void resetLocalSchema()
+    {
+        logger.info("Starting local schema reset...");
+
+        logger.debug("Truncating schema tables...");
+
+        SchemaKeyspace.truncate();
+
+        logger.debug("Clearing local schema keyspace definitions...");
+
+        Schema.instance.clear();
+
+        Set<InetAddress> liveEndpoints = Gossiper.instance.getLiveMembers();
+        liveEndpoints.remove(FBUtilities.getBroadcastAddress());
+
+        // force migration if there are nodes around
+        for (InetAddress node : liveEndpoints)
+        {
+            if (shouldPullSchemaFrom(node))
+            {
+                logger.debug("Requesting schema from {}", node);
+                FBUtilities.waitOnFuture(submitMigrationTask(node));
+                break;
+            }
+        }
+
+        logger.info("Local schema reset is complete.");
+    }
+
+    public static class MigrationsSerializer implements IVersionedSerializer<Collection<Mutation>>
+    {
+        public static MigrationsSerializer instance = new MigrationsSerializer();
+
+        public void serialize(Collection<Mutation> schema, DataOutputPlus out, int version) throws IOException
+        {
+            out.writeInt(schema.size());
+            for (Mutation mutation : schema)
+                Mutation.serializer.serialize(mutation, out, version);
+        }
+
+        public Collection<Mutation> deserialize(DataInputPlus in, int version) throws IOException
+        {
+            int count = in.readInt();
+            Collection<Mutation> schema = new ArrayList<>(count);
+
+            for (int i = 0; i < count; i++)
+                schema.add(Mutation.serializer.deserialize(in, version));
+
+            return schema;
+        }
+
+        public long serializedSize(Collection<Mutation> schema, int version)
+        {
+            int size = TypeSizes.sizeof(schema.size());
+            for (Mutation mutation : schema)
+                size += Mutation.serializer.serializedSize(mutation, version);
+            return size;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/MigrationTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/MigrationTask.java b/src/java/org/apache/cassandra/schema/MigrationTask.java
new file mode 100644
index 0000000..a785e17
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/MigrationTask.java
@@ -0,0 +1,113 @@
+/*
+ * 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.net.InetAddress;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.SystemKeyspace.BootstrapState;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.gms.FailureDetector;
+import org.apache.cassandra.net.IAsyncCallback;
+import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.WrappedRunnable;
+
+final class MigrationTask extends WrappedRunnable
+{
+    private static final Logger logger = LoggerFactory.getLogger(MigrationTask.class);
+
+    private static final ConcurrentLinkedQueue<CountDownLatch> inflightTasks = new ConcurrentLinkedQueue<>();
+
+    private static final Set<BootstrapState> monitoringBootstrapStates = EnumSet.of(BootstrapState.NEEDS_BOOTSTRAP, BootstrapState.IN_PROGRESS);
+
+    private final InetAddress endpoint;
+
+    MigrationTask(InetAddress endpoint)
+    {
+        this.endpoint = endpoint;
+    }
+
+    static ConcurrentLinkedQueue<CountDownLatch> getInflightTasks()
+    {
+        return inflightTasks;
+    }
+
+    public void runMayThrow() throws Exception
+    {
+        // There is a chance that quite some time could have passed between now and the MM#maybeScheduleSchemaPull(),
+        // potentially enough for the endpoint node to restart - which is an issue if it does restart upgraded, with
+        // a higher major.
+        if (!MigrationManager.shouldPullSchemaFrom(endpoint))
+        {
+            logger.info("Skipped sending a migration request: node {} has a higher major version now.", endpoint);
+            return;
+        }
+
+        if (!FailureDetector.instance.isAlive(endpoint))
+        {
+            logger.debug("Can't send schema pull request: node {} is down.", endpoint);
+            return;
+        }
+
+        MessageOut message = new MessageOut<>(MessagingService.Verb.MIGRATION_REQUEST, null, MigrationManager.MigrationsSerializer.instance);
+
+        final CountDownLatch completionLatch = new CountDownLatch(1);
+
+        IAsyncCallback<Collection<Mutation>> cb = new IAsyncCallback<Collection<Mutation>>()
+        {
+            @Override
+            public void response(MessageIn<Collection<Mutation>> message)
+            {
+                try
+                {
+                    Schema.instance.mergeAndAnnounceVersion(message.payload);
+                }
+                catch (ConfigurationException e)
+                {
+                    logger.error("Configuration exception merging remote schema", e);
+                }
+                finally
+                {
+                    completionLatch.countDown();
+                }
+            }
+
+            public boolean isLatencyForSnitch()
+            {
+                return false;
+            }
+        };
+
+        // Only save the latches if we need bootstrap or are bootstrapping
+        if (monitoringBootstrapStates.contains(SystemKeyspace.getBootstrapState()))
+            inflightTasks.offer(completionLatch);
+
+        MessagingService.instance().sendRR(message, endpoint, cb);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Schema.java b/src/java/org/apache/cassandra/schema/Schema.java
new file mode 100644
index 0000000..5786804
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/Schema.java
@@ -0,0 +1,804 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.KeyspaceNotDefinedException;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.UnknownTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.utils.Pair;
+import org.cliffc.high_scale_lib.NonBlockingHashMap;
+
+import static java.lang.String.format;
+
+import static com.google.common.collect.Iterables.size;
+
+public final class Schema
+{
+    public static final Schema instance = new Schema();
+
+    private volatile Keyspaces keyspaces = Keyspaces.none();
+
+    // UUID -> mutable metadata ref map. We have to update these in place every time a table changes.
+    private final Map<TableId, TableMetadataRef> metadataRefs = new NonBlockingHashMap<>();
+
+    // (keyspace name, index name) -> mutable metadata ref map. We have to update these in place every time an index changes.
+    private final Map<Pair<String, String>, TableMetadataRef> indexMetadataRefs = new NonBlockingHashMap<>();
+
+    // Keyspace objects, one per keyspace. Only one instance should ever exist for any given keyspace.
+    private final Map<String, Keyspace> keyspaceInstances = new NonBlockingHashMap<>();
+
+    private volatile UUID version;
+
+    private final List<SchemaChangeListener> changeListeners = new CopyOnWriteArrayList<>();
+
+    /**
+     * Initialize empty schema object and load the hardcoded system tables
+     */
+    private Schema()
+    {
+        if (DatabaseDescriptor.isDaemonInitialized() || DatabaseDescriptor.isToolInitialized())
+        {
+            load(SchemaKeyspace.metadata());
+            load(SystemKeyspace.metadata());
+        }
+    }
+
+    /**
+     * Validates that the provided keyspace is not one of the system keyspace.
+     *
+     * @param keyspace the keyspace name to validate.
+     *
+     * @throws InvalidRequestException if {@code keyspace} is the name of a
+     * system keyspace.
+     */
+    public static void validateKeyspaceNotSystem(String keyspace)
+    {
+        if (SchemaConstants.isSystemKeyspace(keyspace))
+            throw new InvalidRequestException(format("%s keyspace is not user-modifiable", keyspace));
+    }
+
+    /**
+     * load keyspace (keyspace) definitions, but do not initialize the keyspace instances.
+     * Schema version may be updated as the result.
+     */
+    public void loadFromDisk()
+    {
+        loadFromDisk(true);
+    }
+
+    /**
+     * Load schema definitions from disk.
+     *
+     * @param updateVersion true if schema version needs to be updated
+     */
+    public void loadFromDisk(boolean updateVersion)
+    {
+        load(SchemaKeyspace.fetchNonSystemKeyspaces());
+        if (updateVersion)
+            updateVersion();
+    }
+
+    /**
+     * Load up non-system keyspaces
+     *
+     * @param keyspaceDefs The non-system keyspace definitions
+     */
+    private void load(Iterable<KeyspaceMetadata> keyspaceDefs)
+    {
+        keyspaceDefs.forEach(this::load);
+    }
+
+    /**
+     * Update (or insert) new keyspace definition
+     *
+     * @param ksm The metadata about keyspace
+     */
+    synchronized public void load(KeyspaceMetadata ksm)
+    {
+        KeyspaceMetadata previous = keyspaces.getNullable(ksm.name);
+
+        if (previous == null)
+            loadNew(ksm);
+        else
+            reload(previous, ksm);
+
+        keyspaces = keyspaces.withAddedOrUpdated(ksm);
+    }
+
+    private void loadNew(KeyspaceMetadata ksm)
+    {
+        ksm.tablesAndViews()
+           .forEach(metadata -> metadataRefs.put(metadata.id, new TableMetadataRef(metadata)));
+
+        ksm.tables
+           .indexTables()
+           .forEach((name, metadata) -> indexMetadataRefs.put(Pair.create(ksm.name, name), new TableMetadataRef(metadata)));
+    }
+
+    private void reload(KeyspaceMetadata previous, KeyspaceMetadata updated)
+    {
+        Keyspace keyspace = getKeyspaceInstance(updated.name);
+        if (keyspace != null)
+            keyspace.setMetadata(updated);
+
+        MapDifference<TableId, TableMetadata> tablesDiff = previous.tables.diff(updated.tables);
+        MapDifference<TableId, ViewMetadata> viewsDiff = previous.views.diff(updated.views);
+        MapDifference<String, TableMetadata> indexesDiff = previous.tables.indexesDiff(updated.tables);
+
+        // clean up after removed entries
+
+        tablesDiff.entriesOnlyOnLeft()
+                  .values()
+                  .forEach(table -> metadataRefs.remove(table.id));
+
+        viewsDiff.entriesOnlyOnLeft()
+                 .values()
+                 .forEach(view -> metadataRefs.remove(view.metadata.id));
+
+        indexesDiff.entriesOnlyOnLeft()
+                   .values()
+                   .forEach(indexTable -> indexMetadataRefs.remove(Pair.create(indexTable.keyspace, indexTable.indexName().get())));
+
+        // load up new entries
+
+        tablesDiff.entriesOnlyOnRight()
+                  .values()
+                  .forEach(table -> metadataRefs.put(table.id, new TableMetadataRef(table)));
+
+        viewsDiff.entriesOnlyOnRight()
+                 .values()
+                 .forEach(view -> metadataRefs.put(view.metadata.id, new TableMetadataRef(view.metadata)));
+
+        indexesDiff.entriesOnlyOnRight()
+                   .values()
+                   .forEach(indexTable -> indexMetadataRefs.put(Pair.create(indexTable.keyspace, indexTable.indexName().get()), new TableMetadataRef(indexTable)));
+
+        // refresh refs to updated ones
+
+        tablesDiff.entriesDiffering()
+                  .values()
+                  .forEach(diff -> metadataRefs.get(diff.rightValue().id).set(diff.rightValue()));
+
+        viewsDiff.entriesDiffering()
+                 .values()
+                 .forEach(diff -> metadataRefs.get(diff.rightValue().metadata.id).set(diff.rightValue().metadata));
+
+        indexesDiff.entriesDiffering()
+                   .values()
+                   .stream()
+                   .map(MapDifference.ValueDifference::rightValue)
+                   .forEach(indexTable -> indexMetadataRefs.get(Pair.create(indexTable.keyspace, indexTable.indexName().get())).set(indexTable));
+    }
+
+    public void registerListener(SchemaChangeListener listener)
+    {
+        changeListeners.add(listener);
+    }
+
+    @SuppressWarnings("unused")
+    public void unregisterListener(SchemaChangeListener listener)
+    {
+        changeListeners.remove(listener);
+    }
+
+    /**
+     * Get keyspace instance by name
+     *
+     * @param keyspaceName The name of the keyspace
+     *
+     * @return Keyspace object or null if keyspace was not found
+     */
+    public Keyspace getKeyspaceInstance(String keyspaceName)
+    {
+        return keyspaceInstances.get(keyspaceName);
+    }
+
+    public ColumnFamilyStore getColumnFamilyStoreInstance(TableId id)
+    {
+        TableMetadata metadata = getTableMetadata(id);
+        if (metadata == null)
+            return null;
+
+        Keyspace instance = getKeyspaceInstance(metadata.keyspace);
+        if (instance == null)
+            return null;
+
+        return instance.hasColumnFamilyStore(metadata.id)
+             ? instance.getColumnFamilyStore(metadata.id)
+             : null;
+    }
+
+    /**
+     * Store given Keyspace instance to the schema
+     *
+     * @param keyspace The Keyspace instance to store
+     *
+     * @throws IllegalArgumentException if Keyspace is already stored
+     */
+    public void storeKeyspaceInstance(Keyspace keyspace)
+    {
+        if (keyspaceInstances.containsKey(keyspace.getName()))
+            throw new IllegalArgumentException(String.format("Keyspace %s was already initialized.", keyspace.getName()));
+
+        keyspaceInstances.put(keyspace.getName(), keyspace);
+    }
+
+    /**
+     * Remove keyspace from schema
+     *
+     * @param keyspaceName The name of the keyspace to remove
+     *
+     * @return removed keyspace instance or null if it wasn't found
+     */
+    public Keyspace removeKeyspaceInstance(String keyspaceName)
+    {
+        return keyspaceInstances.remove(keyspaceName);
+    }
+
+    /**
+     * Remove keyspace definition from system
+     *
+     * @param ksm The keyspace definition to remove
+     */
+    synchronized void unload(KeyspaceMetadata ksm)
+    {
+        keyspaces = keyspaces.without(ksm.name);
+
+        ksm.tablesAndViews()
+           .forEach(t -> metadataRefs.remove(t.id));
+
+        ksm.tables
+           .indexTables()
+           .keySet()
+           .forEach(name -> indexMetadataRefs.remove(Pair.create(ksm.name, name)));
+    }
+
+    public int getNumberOfTables()
+    {
+        return keyspaces.stream().mapToInt(k -> size(k.tablesAndViews())).sum();
+    }
+
+    public ViewMetadata getView(String keyspaceName, String viewName)
+    {
+        assert keyspaceName != null;
+        KeyspaceMetadata ksm = keyspaces.getNullable(keyspaceName);
+        return (ksm == null) ? null : ksm.views.getNullable(viewName);
+    }
+
+    /**
+     * Get metadata about keyspace by its name
+     *
+     * @param keyspaceName The name of the keyspace
+     *
+     * @return The keyspace metadata or null if it wasn't found
+     */
+    public KeyspaceMetadata getKeyspaceMetadata(String keyspaceName)
+    {
+        assert keyspaceName != null;
+        return keyspaces.getNullable(keyspaceName);
+    }
+
+    private Set<String> getNonSystemKeyspacesSet()
+    {
+        return Sets.difference(keyspaces.names(), SchemaConstants.SYSTEM_KEYSPACE_NAMES);
+    }
+
+    /**
+     * @return collection of the non-system keyspaces (note that this count as system only the
+     * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
+     * returned. See getUserKeyspace() below if you don't want those)
+     */
+    public ImmutableList<String> getNonSystemKeyspaces()
+    {
+        return ImmutableList.copyOf(getNonSystemKeyspacesSet());
+    }
+
+    /**
+     * @return a collection of keyspaces that do not use LocalStrategy for replication
+     */
+    public List<String> getNonLocalStrategyKeyspaces()
+    {
+        return keyspaces.stream()
+                        .filter(keyspace -> keyspace.params.replication.klass != LocalStrategy.class)
+                        .map(keyspace -> keyspace.name)
+                        .collect(Collectors.toList());
+    }
+
+    /**
+     * @return collection of the user defined keyspaces
+     */
+    public List<String> getUserKeyspaces()
+    {
+        return ImmutableList.copyOf(Sets.difference(getNonSystemKeyspacesSet(), SchemaConstants.REPLICATED_SYSTEM_KEYSPACE_NAMES));
+    }
+
+    /**
+     * Get metadata about keyspace inner ColumnFamilies
+     *
+     * @param keyspaceName The name of the keyspace
+     *
+     * @return metadata about ColumnFamilies the belong to the given keyspace
+     */
+    public Iterable<TableMetadata> getTablesAndViews(String keyspaceName)
+    {
+        assert keyspaceName != null;
+        KeyspaceMetadata ksm = keyspaces.getNullable(keyspaceName);
+        assert ksm != null;
+        return ksm.tablesAndViews();
+    }
+
+    /**
+     * @return collection of the all keyspace names registered in the system (system and non-system)
+     */
+    public Set<String> getKeyspaces()
+    {
+        return keyspaces.names();
+    }
+
+    /* TableMetadata/Ref query/control methods */
+
+    /**
+     * Given a keyspace name and table/view name, get the table metadata
+     * reference. If the keyspace name or table/view name is not present
+     * this method returns null.
+     *
+     * @return TableMetadataRef object or null if it wasn't found
+     */
+    public TableMetadataRef getTableMetadataRef(String keyspace, String table)
+    {
+        TableMetadata tm = getTableMetadata(keyspace, table);
+        return tm == null
+             ? null
+             : metadataRefs.get(tm.id);
+    }
+
+    public TableMetadataRef getIndexTableMetadataRef(String keyspace, String index)
+    {
+        return indexMetadataRefs.get(Pair.create(keyspace, index));
+    }
+
+    /**
+     * Get Table metadata by its identifier
+     *
+     * @param id table or view identifier
+     *
+     * @return metadata about Table or View
+     */
+    public TableMetadataRef getTableMetadataRef(TableId id)
+    {
+        return metadataRefs.get(id);
+    }
+
+    public TableMetadataRef getTableMetadataRef(Descriptor descriptor)
+    {
+        return getTableMetadataRef(descriptor.ksname, descriptor.cfname);
+    }
+
+    /**
+     * Given a keyspace name and table name, get the table
+     * meta data. If the keyspace name or table name is not valid
+     * this function returns null.
+     *
+     * @param keyspace The keyspace name
+     * @param table The table name
+     *
+     * @return TableMetadata object or null if it wasn't found
+     */
+    public TableMetadata getTableMetadata(String keyspace, String table)
+    {
+        assert keyspace != null;
+        assert table != null;
+
+        KeyspaceMetadata ksm = keyspaces.getNullable(keyspace);
+        return ksm == null
+             ? null
+             : ksm.getTableOrViewNullable(table);
+    }
+
+    public TableMetadata getTableMetadata(TableId id)
+    {
+        return keyspaces.getTableOrViewNullable(id);
+    }
+
+    public TableMetadata validateTable(String keyspaceName, String tableName)
+    {
+        if (tableName.isEmpty())
+            throw new InvalidRequestException("non-empty table is required");
+
+        KeyspaceMetadata keyspace = keyspaces.getNullable(keyspaceName);
+        if (keyspace == null)
+            throw new KeyspaceNotDefinedException(format("keyspace %s does not exist", keyspaceName));
+
+        TableMetadata metadata = keyspace.getTableOrViewNullable(tableName);
+        if (metadata == null)
+            throw new InvalidRequestException(format("table %s does not exist", tableName));
+
+        return metadata;
+    }
+
+    public TableMetadata getTableMetadata(Descriptor descriptor)
+    {
+        return getTableMetadata(descriptor.ksname, descriptor.cfname);
+    }
+
+    /**
+     * @throws UnknownTableException if the table couldn't be found in the metadata
+     */
+    public TableMetadata getExistingTableMetadata(TableId id) throws UnknownTableException
+    {
+        TableMetadata metadata = getTableMetadata(id);
+        if (metadata != null)
+            return metadata;
+
+        String message =
+            String.format("Couldn't find table with id %s. If a table was just created, this is likely due to the schema"
+                          + "not being fully propagated.  Please wait for schema agreement on table creation.",
+                          id);
+        throw new UnknownTableException(message, id);
+    }
+
+    /* Function helpers */
+
+    /**
+     * Get all function overloads with the specified name
+     *
+     * @param name fully qualified function name
+     * @return an empty list if the keyspace or the function name are not found;
+     *         a non-empty collection of {@link Function} otherwise
+     */
+    public Collection<Function> getFunctions(FunctionName name)
+    {
+        if (!name.hasKeyspace())
+            throw new IllegalArgumentException(String.format("Function name must be fully quallified: got %s", name));
+
+        KeyspaceMetadata ksm = getKeyspaceMetadata(name.keyspace);
+        return ksm == null
+             ? Collections.emptyList()
+             : ksm.functions.get(name);
+    }
+
+    /**
+     * Find the function with the specified name
+     *
+     * @param name fully qualified function name
+     * @param argTypes function argument types
+     * @return an empty {@link Optional} if the keyspace or the function name are not found;
+     *         a non-empty optional of {@link Function} otherwise
+     */
+    public Optional<Function> findFunction(FunctionName name, List<AbstractType<?>> argTypes)
+    {
+        if (!name.hasKeyspace())
+            throw new IllegalArgumentException(String.format("Function name must be fully quallified: got %s", name));
+
+        KeyspaceMetadata ksm = getKeyspaceMetadata(name.keyspace);
+        return ksm == null
+             ? Optional.empty()
+             : ksm.functions.find(name, argTypes);
+    }
+
+    /* Version control */
+
+    /**
+     * @return current schema version
+     */
+    public UUID getVersion()
+    {
+        return version;
+    }
+
+    /**
+     * 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 void updateVersion()
+    {
+        version = SchemaKeyspace.calculateSchemaDigest();
+        SystemKeyspace.updateSchemaVersion(version);
+    }
+
+    /*
+     * Like updateVersion, but also announces via gossip
+     */
+    public void updateVersionAndAnnounce()
+    {
+        updateVersion();
+        MigrationManager.passiveAnnounce(version);
+    }
+
+    /**
+     * Clear all KS/CF metadata and reset version.
+     */
+    public synchronized void clear()
+    {
+        getNonSystemKeyspaces().forEach(k -> unload(getKeyspaceMetadata(k)));
+        updateVersionAndAnnounce();
+    }
+
+    /**
+     * 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
+     */
+    synchronized void mergeAndAnnounceVersion(Collection<Mutation> mutations)
+    {
+        merge(mutations);
+        updateVersionAndAnnounce();
+    }
+
+    synchronized void merge(Collection<Mutation> mutations)
+    {
+        // only compare the keyspaces affected by this set of schema mutations
+        Set<String> affectedKeyspaces = SchemaKeyspace.affectedKeyspaces(mutations);
+
+        // fetch the current state of schema for the affected keyspaces only
+        Keyspaces before = keyspaces.filter(k -> affectedKeyspaces.contains(k.name));
+
+        // apply the schema mutations
+        SchemaKeyspace.applyChanges(mutations);
+
+        // apply the schema mutations and fetch the new versions of the altered keyspaces
+        Keyspaces after = SchemaKeyspace.fetchKeyspaces(affectedKeyspaces);
+
+        MapDifference<String, KeyspaceMetadata> keyspacesDiff = before.diff(after);
+
+        // dropped keyspaces
+        keyspacesDiff.entriesOnlyOnLeft().values().forEach(this::dropKeyspace);
+
+        // new keyspaces
+        keyspacesDiff.entriesOnlyOnRight().values().forEach(this::createKeyspace);
+
+        // updated keyspaces
+        keyspacesDiff.entriesDiffering().entrySet().forEach(diff -> alterKeyspace(diff.getValue().leftValue(), diff.getValue().rightValue()));
+    }
+
+    private void alterKeyspace(KeyspaceMetadata before, KeyspaceMetadata after)
+    {
+        // calculate the deltas
+        MapDifference<TableId, TableMetadata> tablesDiff = before.tables.diff(after.tables);
+        MapDifference<TableId, ViewMetadata> viewsDiff = before.views.diff(after.views);
+        MapDifference<ByteBuffer, UserType> typesDiff = before.types.diff(after.types);
+        MapDifference<Pair<FunctionName, List<String>>, UDFunction> udfsDiff = before.functions.udfsDiff(after.functions);
+        MapDifference<Pair<FunctionName, List<String>>, UDAggregate> udasDiff = before.functions.udasDiff(after.functions);
+
+        // drop tables and views
+        viewsDiff.entriesOnlyOnLeft().values().forEach(this::dropView);
+        tablesDiff.entriesOnlyOnLeft().values().forEach(this::dropTable);
+
+        load(after);
+
+        // add tables and views
+        tablesDiff.entriesOnlyOnRight().values().forEach(this::createTable);
+        viewsDiff.entriesOnlyOnRight().values().forEach(this::createView);
+
+        // update tables and views
+        tablesDiff.entriesDiffering().values().forEach(diff -> alterTable(diff.rightValue()));
+        viewsDiff.entriesDiffering().values().forEach(diff -> alterView(diff.rightValue()));
+
+        // deal with all removed, added, and altered views
+        Keyspace.open(before.name).viewManager.reload();
+
+        // notify on everything dropped
+        udasDiff.entriesOnlyOnLeft().values().forEach(this::notifyDropAggregate);
+        udfsDiff.entriesOnlyOnLeft().values().forEach(this::notifyDropFunction);
+        viewsDiff.entriesOnlyOnLeft().values().forEach(this::notifyDropView);
+        tablesDiff.entriesOnlyOnLeft().values().forEach(this::notifyDropTable);
+        typesDiff.entriesOnlyOnLeft().values().forEach(this::notifyDropType);
+
+        // notify on everything created
+        typesDiff.entriesOnlyOnRight().values().forEach(this::notifyCreateType);
+        tablesDiff.entriesOnlyOnRight().values().forEach(this::notifyCreateTable);
+        viewsDiff.entriesOnlyOnRight().values().forEach(this::notifyCreateView);
+        udfsDiff.entriesOnlyOnRight().values().forEach(this::notifyCreateFunction);
+        udasDiff.entriesOnlyOnRight().values().forEach(this::notifyCreateAggregate);
+
+        // notify on everything altered
+        if (!before.params.equals(after.params))
+            notifyAlterKeyspace(after);
+        typesDiff.entriesDiffering().values().forEach(diff -> notifyAlterType(diff.rightValue()));
+        tablesDiff.entriesDiffering().values().forEach(diff -> notifyAlterTable(diff.leftValue(), diff.rightValue()));
+        viewsDiff.entriesDiffering().values().forEach(diff -> notifyAlterView(diff.leftValue(), diff.rightValue()));
+        udfsDiff.entriesDiffering().values().forEach(diff -> notifyAlterFunction(diff.rightValue()));
+        udasDiff.entriesDiffering().values().forEach(diff -> notifyAlterAggregate(diff.rightValue()));
+    }
+
+    private void createKeyspace(KeyspaceMetadata keyspace)
+    {
+        load(keyspace);
+        Keyspace.open(keyspace.name);
+
+        notifyCreateKeyspace(keyspace);
+        keyspace.types.forEach(this::notifyCreateType);
+        keyspace.tables.forEach(this::notifyCreateTable);
+        keyspace.views.forEach(this::notifyCreateView);
+        keyspace.functions.udfs().forEach(this::notifyCreateFunction);
+        keyspace.functions.udas().forEach(this::notifyCreateAggregate);
+    }
+
+    private void dropKeyspace(KeyspaceMetadata keyspace)
+    {
+        keyspace.views.forEach(this::dropView);
+        keyspace.tables.forEach(this::dropTable);
+
+        // remove the keyspace from the static instances.
+        Keyspace.clear(keyspace.name);
+        unload(keyspace);
+        Keyspace.writeOrder.awaitNewBarrier();
+
+        keyspace.functions.udas().forEach(this::notifyDropAggregate);
+        keyspace.functions.udfs().forEach(this::notifyDropFunction);
+        keyspace.views.forEach(this::notifyDropView);
+        keyspace.tables.forEach(this::notifyDropTable);
+        keyspace.types.forEach(this::notifyDropType);
+        notifyDropKeyspace(keyspace);
+    }
+
+    private void dropView(ViewMetadata metadata)
+    {
+        dropTable(metadata.metadata);
+    }
+
+    private void dropTable(TableMetadata metadata)
+    {
+        ColumnFamilyStore cfs = Keyspace.open(metadata.keyspace).getColumnFamilyStore(metadata.name);
+        assert cfs != null;
+        // make sure all the indexes are dropped, or else.
+        cfs.indexManager.markAllIndexesRemoved();
+        CompactionManager.instance.interruptCompactionFor(Collections.singleton(metadata), true);
+        if (DatabaseDescriptor.isAutoSnapshot())
+            cfs.snapshot(Keyspace.getTimestampedSnapshotNameWithPrefix(cfs.name, ColumnFamilyStore.SNAPSHOT_DROP_PREFIX));
+        CommitLog.instance.forceRecycleAllSegments(Collections.singleton(metadata.id));
+        Keyspace.open(metadata.keyspace).dropCf(metadata.id);
+    }
+
+    private void createTable(TableMetadata table)
+    {
+        Keyspace.open(table.keyspace).initCf(metadataRefs.get(table.id), true);
+    }
+
+    private void createView(ViewMetadata view)
+    {
+        Keyspace.open(view.keyspace).initCf(metadataRefs.get(view.metadata.id), true);
+    }
+
+    private void alterTable(TableMetadata updated)
+    {
+        Keyspace.open(updated.keyspace).getColumnFamilyStore(updated.name).reload();
+    }
+
+    private void alterView(ViewMetadata updated)
+    {
+        Keyspace.open(updated.keyspace).getColumnFamilyStore(updated.name).reload();
+    }
+
+    private void notifyCreateKeyspace(KeyspaceMetadata ksm)
+    {
+        changeListeners.forEach(l -> l.onCreateKeyspace(ksm.name));
+    }
+
+    private void notifyCreateTable(TableMetadata metadata)
+    {
+        changeListeners.forEach(l -> l.onCreateTable(metadata.keyspace, metadata.name));
+    }
+
+    private void notifyCreateView(ViewMetadata view)
+    {
+        changeListeners.forEach(l -> l.onCreateView(view.keyspace, view.name));
+    }
+
+    private void notifyCreateType(UserType ut)
+    {
+        changeListeners.forEach(l -> l.onCreateType(ut.keyspace, ut.getNameAsString()));
+    }
+
+    private void notifyCreateFunction(UDFunction udf)
+    {
+        changeListeners.forEach(l -> l.onCreateFunction(udf.name().keyspace, udf.name().name, udf.argTypes()));
+    }
+
+    private void notifyCreateAggregate(UDAggregate udf)
+    {
+        changeListeners.forEach(l -> l.onCreateAggregate(udf.name().keyspace, udf.name().name, udf.argTypes()));
+    }
+
+    private void notifyAlterKeyspace(KeyspaceMetadata ksm)
+    {
+        changeListeners.forEach(l -> l.onAlterKeyspace(ksm.name));
+    }
+
+    private void notifyAlterTable(TableMetadata current, TableMetadata updated)
+    {
+        boolean changeAffectedPreparedStatements = current.changeAffectsPreparedStatements(updated);
+        changeListeners.forEach(l -> l.onAlterTable(updated.keyspace, updated.name, changeAffectedPreparedStatements));
+    }
+
+    private void notifyAlterView(ViewMetadata current, ViewMetadata updated)
+    {
+        boolean changeAffectedPreparedStatements = current.metadata.changeAffectsPreparedStatements(updated.metadata);
+        changeListeners.forEach(l ->l.onAlterView(updated.keyspace, updated.name, changeAffectedPreparedStatements));
+    }
+
+    private void notifyAlterType(UserType ut)
+    {
+        changeListeners.forEach(l -> l.onAlterType(ut.keyspace, ut.getNameAsString()));
+    }
+
+    private void notifyAlterFunction(UDFunction udf)
+    {
+        changeListeners.forEach(l -> l.onAlterFunction(udf.name().keyspace, udf.name().name, udf.argTypes()));
+    }
+
+    private void notifyAlterAggregate(UDAggregate udf)
+    {
+        changeListeners.forEach(l -> l.onAlterAggregate(udf.name().keyspace, udf.name().name, udf.argTypes()));
+    }
+
+    private void notifyDropKeyspace(KeyspaceMetadata ksm)
+    {
+        changeListeners.forEach(l -> l.onDropKeyspace(ksm.name));
+    }
+
+    private void notifyDropTable(TableMetadata metadata)
+    {
+        changeListeners.forEach(l -> l.onDropTable(metadata.keyspace, metadata.name));
+    }
+
+    private void notifyDropView(ViewMetadata view)
+    {
+        changeListeners.forEach(l -> l.onDropView(view.keyspace, view.name));
+    }
+
+    private void notifyDropType(UserType ut)
+    {
+        changeListeners.forEach(l -> l.onDropType(ut.keyspace, ut.getNameAsString()));
+    }
+
+    private void notifyDropFunction(UDFunction udf)
+    {
+        changeListeners.forEach(l -> l.onDropFunction(udf.name().keyspace, udf.name().name, udf.argTypes()));
+    }
+
+    private void notifyDropAggregate(UDAggregate udf)
+    {
+        changeListeners.forEach(l -> l.onDropAggregate(udf.name().keyspace, udf.name().name, udf.argTypes()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/SchemaChangeListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaChangeListener.java b/src/java/org/apache/cassandra/schema/SchemaChangeListener.java
new file mode 100644
index 0000000..4390309
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/SchemaChangeListener.java
@@ -0,0 +1,102 @@
+/*
+ * 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.util.List;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public abstract class SchemaChangeListener
+{
+    public void onCreateKeyspace(String keyspace)
+    {
+    }
+
+    public void onCreateTable(String keyspace, String table)
+    {
+    }
+
+    public void onCreateView(String keyspace, String view)
+    {
+        onCreateTable(keyspace, view);
+    }
+
+    public void onCreateType(String keyspace, String type)
+    {
+    }
+
+    public void onCreateFunction(String keyspace, String function, List<AbstractType<?>> argumentTypes)
+    {
+    }
+
+    public void onCreateAggregate(String keyspace, String aggregate, List<AbstractType<?>> argumentTypes)
+    {
+    }
+
+    public void onAlterKeyspace(String keyspace)
+    {
+    }
+
+    // the boolean flag indicates whether the change that triggered this event may have a substantive
+    // impact on statements using the column family.
+    public void onAlterTable(String keyspace, String table, boolean affectsStatements)
+    {
+    }
+
+    public void onAlterView(String keyspace, String view, boolean affectsStataments)
+    {
+        onAlterTable(keyspace, view, affectsStataments);
+    }
+
+    public void onAlterType(String keyspace, String type)
+    {
+    }
+
+    public void onAlterFunction(String keyspace, String function, List<AbstractType<?>> argumentTypes)
+    {
+    }
+
+    public void onAlterAggregate(String keyspace, String aggregate, List<AbstractType<?>> argumentTypes)
+    {
+    }
+
+    public void onDropKeyspace(String keyspace)
+    {
+    }
+
+    public void onDropTable(String keyspace, String table)
+    {
+    }
+
+    public void onDropView(String keyspace, String view)
+    {
+        onDropTable(keyspace, view);
+    }
+
+    public void onDropType(String keyspace, String type)
+    {
+    }
+
+    public void onDropFunction(String keyspace, String function, List<AbstractType<?>> argumentTypes)
+    {
+    }
+
+    public void onDropAggregate(String keyspace, String aggregate, List<AbstractType<?>> argumentTypes)
+    {
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/SchemaConstants.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaConstants.java b/src/java/org/apache/cassandra/schema/SchemaConstants.java
new file mode 100644
index 0000000..5340f69
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/SchemaConstants.java
@@ -0,0 +1,82 @@
+/*
+ * 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.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Set;
+import java.util.UUID;
+import java.util.regex.Pattern;
+
+import com.google.common.collect.ImmutableSet;
+
+public final class SchemaConstants
+{
+    public static final Pattern PATTERN_WORD_CHARS = Pattern.compile("\\w+");
+
+    public static final String SYSTEM_KEYSPACE_NAME = "system";
+    public static final String SCHEMA_KEYSPACE_NAME = "system_schema";
+
+    public static final String TRACE_KEYSPACE_NAME = "system_traces";
+    public static final String AUTH_KEYSPACE_NAME = "system_auth";
+    public static final String DISTRIBUTED_KEYSPACE_NAME = "system_distributed";
+
+    /* system keyspace names (the ones with LocalStrategy replication strategy) */
+    public static final Set<String> SYSTEM_KEYSPACE_NAMES = ImmutableSet.of(SYSTEM_KEYSPACE_NAME, SCHEMA_KEYSPACE_NAME);
+
+    /* replicate system keyspace names (the ones with a "true" replication strategy) */
+    public static final Set<String> REPLICATED_SYSTEM_KEYSPACE_NAMES = ImmutableSet.of(TRACE_KEYSPACE_NAME,
+                                                                                       AUTH_KEYSPACE_NAME,
+                                                                                       DISTRIBUTED_KEYSPACE_NAME);
+    /**
+     * longest permissible KS or CF name.  Our main concern is that filename not be more than 255 characters;
+     * the filename will contain both the KS and CF names. Since non-schema-name components only take up
+     * ~64 characters, we could allow longer names than this, but on Windows, the entire path should be not greater than
+     * 255 characters, so a lower limit here helps avoid problems.  See CASSANDRA-4110.
+     */
+    public static final int NAME_LENGTH = 48;
+
+    // 59adb24e-f3cd-3e02-97f0-5b395827453f
+    public static final UUID emptyVersion;
+
+    public static boolean isValidName(String name)
+    {
+        return name != null && !name.isEmpty() && name.length() <= NAME_LENGTH && PATTERN_WORD_CHARS.matcher(name).matches();
+    }
+
+    static
+    {
+        try
+        {
+            emptyVersion = UUID.nameUUIDFromBytes(MessageDigest.getInstance("MD5").digest());
+        }
+        catch (NoSuchAlgorithmException e)
+        {
+            throw new AssertionError();
+        }
+    }
+
+    /**
+     * @return whether or not the keyspace is a really system one (w/ LocalStrategy, unmodifiable, hardcoded)
+     */
+    public static boolean isSystemKeyspace(String keyspaceName)
+    {
+        return SYSTEM_KEYSPACE_NAMES.contains(keyspaceName.toLowerCase());
+    }
+}


[13/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index 2bf4805..c2efb6a 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -21,6 +21,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.*;
 
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.index.sasi.SASIIndex;
 import org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder;
@@ -39,7 +40,7 @@ import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.index.StubIndex;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.schema.*;
-import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -119,44 +120,31 @@ public class SchemaLoader
                 KeyspaceParams.simple(1),
                 Tables.of(
                 // Column Families
-                standardCFMD(ks1, "Standard1").compaction(CompactionParams.scts(compactionOptions)),
-                standardCFMD(ks1, "Standard2"),
-                standardCFMD(ks1, "Standard3"),
-                standardCFMD(ks1, "Standard4"),
-                standardCFMD(ks1, "StandardGCGS0").gcGraceSeconds(0),
-                standardCFMD(ks1, "StandardLong1"),
-                standardCFMD(ks1, "StandardLong2"),
-                //CFMetaData.Builder.create(ks1, "ValuesWithQuotes").build(),
-                superCFMD(ks1, "Super1", LongType.instance),
-                superCFMD(ks1, "Super2", LongType.instance),
-                superCFMD(ks1, "Super3", LongType.instance),
-                superCFMD(ks1, "Super4", UTF8Type.instance),
-                superCFMD(ks1, "Super5", bytes),
-                superCFMD(ks1, "Super6", LexicalUUIDType.instance, UTF8Type.instance),
-                keysIndexCFMD(ks1, "Indexed1", true),
-                keysIndexCFMD(ks1, "Indexed2", false),
-                //CFMetaData.Builder.create(ks1, "StandardInteger1").withColumnNameComparator(IntegerType.instance).build(),
-                //CFMetaData.Builder.create(ks1, "StandardLong3").withColumnNameComparator(IntegerType.instance).build(),
-                //CFMetaData.Builder.create(ks1, "Counter1", false, false, true).build(),
-                //CFMetaData.Builder.create(ks1, "SuperCounter1", false, false, true, true).build(),
-                superCFMD(ks1, "SuperDirectGC", BytesType.instance).gcGraceSeconds(0),
-//                jdbcCFMD(ks1, "JdbcInteger", IntegerType.instance).addColumnDefinition(integerColumn(ks1, "JdbcInteger")),
-                jdbcCFMD(ks1, "JdbcUtf8", UTF8Type.instance).addColumnDefinition(utf8Column(ks1, "JdbcUtf8")),
-                jdbcCFMD(ks1, "JdbcLong", LongType.instance),
-                jdbcCFMD(ks1, "JdbcBytes", bytes),
-                jdbcCFMD(ks1, "JdbcAscii", AsciiType.instance),
-                //CFMetaData.Builder.create(ks1, "StandardComposite", false, true, false).withColumnNameComparator(composite).build(),
-                //CFMetaData.Builder.create(ks1, "StandardComposite2", false, true, false).withColumnNameComparator(compositeMaxMin).build(),
-                //CFMetaData.Builder.create(ks1, "StandardDynamicComposite", false, true, false).withColumnNameComparator(dynamicComposite).build(),
-                standardCFMD(ks1, "StandardLeveled").compaction(CompactionParams.lcs(leveledOptions)),
-                standardCFMD(ks1, "legacyleveled").compaction(CompactionParams.lcs(leveledOptions)),
+                standardCFMD(ks1, "Standard1").compaction(CompactionParams.scts(compactionOptions)).build(),
+                standardCFMD(ks1, "Standard2").build(),
+                standardCFMD(ks1, "Standard3").build(),
+                standardCFMD(ks1, "Standard4").build(),
+                standardCFMD(ks1, "StandardGCGS0").gcGraceSeconds(0).build(),
+                standardCFMD(ks1, "StandardLong1").build(),
+                standardCFMD(ks1, "StandardLong2").build(),
+                superCFMD(ks1, "Super1", LongType.instance).build(),
+                superCFMD(ks1, "Super2", LongType.instance).build(),
+                superCFMD(ks1, "Super3", LongType.instance).build(),
+                superCFMD(ks1, "Super4", UTF8Type.instance).build(),
+                superCFMD(ks1, "Super5", bytes).build(),
+                superCFMD(ks1, "Super6", LexicalUUIDType.instance, UTF8Type.instance).build(),
+                keysIndexCFMD(ks1, "Indexed1", true).build(),
+                keysIndexCFMD(ks1, "Indexed2", false).build(),
+                superCFMD(ks1, "SuperDirectGC", BytesType.instance).gcGraceSeconds(0).build(),
+                jdbcCFMD(ks1, "JdbcUtf8", UTF8Type.instance).addColumn(utf8Column(ks1, "JdbcUtf8")).build(),
+                jdbcCFMD(ks1, "JdbcLong", LongType.instance).build(),
+                jdbcCFMD(ks1, "JdbcBytes", bytes).build(),
+                jdbcCFMD(ks1, "JdbcAscii", AsciiType.instance).build(),
+                standardCFMD(ks1, "StandardLeveled").compaction(CompactionParams.lcs(leveledOptions)).build(),
+                standardCFMD(ks1, "legacyleveled").compaction(CompactionParams.lcs(leveledOptions)).build(),
                 standardCFMD(ks1, "StandardLowIndexInterval").minIndexInterval(8)
                                                              .maxIndexInterval(256)
-                                                             .caching(CachingParams.CACHE_NOTHING)
-                //CFMetaData.Builder.create(ks1, "UUIDKeys").addPartitionKey("key",UUIDType.instance).build(),
-                //CFMetaData.Builder.create(ks1, "MixedTypes").withColumnNameComparator(LongType.instance).addPartitionKey("key", UUIDType.instance).build(),
-                //CFMetaData.Builder.create(ks1, "MixedTypesComposite", false, true, false).withColumnNameComparator(composite).addPartitionKey("key", composite).build(),
-                //CFMetaData.Builder.create(ks1, "AsciiKeys").addPartitionKey("key", AsciiType.instance).build()
+                                                             .caching(CachingParams.CACHE_NOTHING).build()
         )));
 
         // Keyspace 2
@@ -164,118 +152,135 @@ public class SchemaLoader
                 KeyspaceParams.simple(1),
                 Tables.of(
                 // Column Families
-                standardCFMD(ks2, "Standard1"),
-                standardCFMD(ks2, "Standard3"),
-                superCFMD(ks2, "Super3", bytes),
-                superCFMD(ks2, "Super4", TimeUUIDType.instance),
-                keysIndexCFMD(ks2, "Indexed1", true),
-                compositeIndexCFMD(ks2, "Indexed2", true),
-                compositeIndexCFMD(ks2, "Indexed3", true).gcGraceSeconds(0))));
+                standardCFMD(ks2, "Standard1").build(),
+                standardCFMD(ks2, "Standard3").build(),
+                superCFMD(ks2, "Super3", bytes).build(),
+                superCFMD(ks2, "Super4", TimeUUIDType.instance).build(),
+                keysIndexCFMD(ks2, "Indexed1", true).build(),
+                compositeIndexCFMD(ks2, "Indexed2", true).build(),
+                compositeIndexCFMD(ks2, "Indexed3", true).gcGraceSeconds(0).build())));
 
         // Keyspace 3
         schema.add(KeyspaceMetadata.create(ks3,
                 KeyspaceParams.simple(5),
                 Tables.of(
-                standardCFMD(ks3, "Standard1"),
-                keysIndexCFMD(ks3, "Indexed1", true))));
+                standardCFMD(ks3, "Standard1").build(),
+                keysIndexCFMD(ks3, "Indexed1", true).build())));
 
         // Keyspace 4
         schema.add(KeyspaceMetadata.create(ks4,
                 KeyspaceParams.simple(3),
                 Tables.of(
-                standardCFMD(ks4, "Standard1"),
-                standardCFMD(ks4, "Standard3"),
-                superCFMD(ks4, "Super3", bytes),
-                superCFMD(ks4, "Super4", TimeUUIDType.instance),
-                superCFMD(ks4, "Super5", TimeUUIDType.instance, BytesType.instance))));
+                standardCFMD(ks4, "Standard1").build(),
+                standardCFMD(ks4, "Standard3").build(),
+                superCFMD(ks4, "Super3", bytes).build(),
+                superCFMD(ks4, "Super4", TimeUUIDType.instance).build(),
+                superCFMD(ks4, "Super5", TimeUUIDType.instance, BytesType.instance).build())));
 
         // Keyspace 5
         schema.add(KeyspaceMetadata.create(ks5,
                 KeyspaceParams.simple(2),
-                Tables.of(standardCFMD(ks5, "Standard1"))));
+                Tables.of(standardCFMD(ks5, "Standard1").build())));
 
         // Keyspace 6
         schema.add(KeyspaceMetadata.create(ks6,
                 KeyspaceParams.simple(1),
-                Tables.of(keysIndexCFMD(ks6, "Indexed1", true))));
+                Tables.of(keysIndexCFMD(ks6, "Indexed1", true).build())));
 
         // Keyspace 7
         schema.add(KeyspaceMetadata.create(ks7,
                 KeyspaceParams.simple(1),
-                Tables.of(customIndexCFMD(ks7, "Indexed1"))));
+                Tables.of(customIndexCFMD(ks7, "Indexed1").build())));
 
         // KeyCacheSpace
         schema.add(KeyspaceMetadata.create(ks_kcs,
                 KeyspaceParams.simple(1),
                 Tables.of(
-                standardCFMD(ks_kcs, "Standard1"),
-                standardCFMD(ks_kcs, "Standard2"),
-                standardCFMD(ks_kcs, "Standard3"))));
+                standardCFMD(ks_kcs, "Standard1").build(),
+                standardCFMD(ks_kcs, "Standard2").build(),
+                standardCFMD(ks_kcs, "Standard3").build())));
 
         // RowCacheSpace
         schema.add(KeyspaceMetadata.create(ks_rcs,
                 KeyspaceParams.simple(1),
                 Tables.of(
-                standardCFMD(ks_rcs, "CFWithoutCache").caching(CachingParams.CACHE_NOTHING),
-                standardCFMD(ks_rcs, "CachedCF").caching(CachingParams.CACHE_EVERYTHING),
-                standardCFMD(ks_rcs, "CachedNoClustering", 1, IntegerType.instance, IntegerType.instance, null).caching(CachingParams.CACHE_EVERYTHING),
-                standardCFMD(ks_rcs, "CachedIntCF").
-                        caching(new CachingParams(true, 100)))));
-
-        // CounterCacheSpace
-        /*schema.add(KeyspaceMetadata.testMetadata(ks_ccs,
-                simple,
-                opts_rf1,
-                CFMetaData.Builder.create(ks_ccs, "Counter1", false, false, true).build(),
-                CFMetaData.Builder.create(ks_ccs, "Counter1", false, false, true).build()));*/
+                standardCFMD(ks_rcs, "CFWithoutCache").caching(CachingParams.CACHE_NOTHING).build(),
+                standardCFMD(ks_rcs, "CachedCF").caching(CachingParams.CACHE_EVERYTHING).build(),
+                standardCFMD(ks_rcs, "CachedNoClustering", 1, IntegerType.instance, IntegerType.instance, null).caching(CachingParams.CACHE_EVERYTHING).build(),
+                standardCFMD(ks_rcs, "CachedIntCF").caching(new CachingParams(true, 100)).build())));
 
         schema.add(KeyspaceMetadata.create(ks_nocommit, KeyspaceParams.simpleTransient(1), Tables.of(
-                standardCFMD(ks_nocommit, "Standard1"))));
+                standardCFMD(ks_nocommit, "Standard1").build())));
 
         // CQLKeyspace
         schema.add(KeyspaceMetadata.create(ks_cql, KeyspaceParams.simple(1), Tables.of(
 
-                // Column Families
-                CFMetaData.compile("CREATE TABLE table1 ("
-                        + "k int PRIMARY KEY,"
-                        + "v1 text,"
-                        + "v2 int"
-                        + ")", ks_cql),
-
-                CFMetaData.compile("CREATE TABLE table2 ("
-                        + "k text,"
-                        + "c text,"
-                        + "v text,"
-                        + "PRIMARY KEY (k, c))", ks_cql),
-                CFMetaData.compile("CREATE TABLE foo ("
-                        + "bar text, "
-                        + "baz text, "
-                        + "qux text, "
-                        + "PRIMARY KEY(bar, baz) ) "
-                        + "WITH COMPACT STORAGE", ks_cql),
-                CFMetaData.compile("CREATE TABLE foofoo ("
-                        + "bar text, "
-                        + "baz text, "
-                        + "qux text, "
-                        + "quz text, "
-                        + "foo text, "
-                        + "PRIMARY KEY((bar, baz), qux, quz) ) "
-                        + "WITH COMPACT STORAGE", ks_cql)
+        // Column Families
+        CreateTableStatement.parse("CREATE TABLE table1 ("
+                                   + "k int PRIMARY KEY,"
+                                   + "v1 text,"
+                                   + "v2 int"
+                                   + ")", ks_cql)
+                            .build(),
+
+        CreateTableStatement.parse("CREATE TABLE table2 ("
+                                   + "k text,"
+                                   + "c text,"
+                                   + "v text,"
+                                   + "PRIMARY KEY (k, c))", ks_cql)
+                            .build(),
+
+        CreateTableStatement.parse("CREATE TABLE foo ("
+                                   + "bar text, "
+                                   + "baz text, "
+                                   + "qux text, "
+                                   + "PRIMARY KEY(bar, baz) ) "
+                                   + "WITH COMPACT STORAGE", ks_cql)
+                            .build(),
+
+        CreateTableStatement.parse("CREATE TABLE foofoo ("
+                                   + "bar text, "
+                                   + "baz text, "
+                                   + "qux text, "
+                                   + "quz text, "
+                                   + "foo text, "
+                                   + "PRIMARY KEY((bar, baz), qux, quz) ) "
+                                   + "WITH COMPACT STORAGE", ks_cql)
+                            .build()
         )));
 
         if (DatabaseDescriptor.getPartitioner() instanceof Murmur3Partitioner)
-            schema.add(KeyspaceMetadata.create("sasi", KeyspaceParams.simpleTransient(1), Tables.of(sasiCFMD("sasi", "test_cf"), clusteringSASICFMD("sasi", "clustering_test_cf"))));
-
-        if (Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false")))
-            useCompression(schema);
+        {
+            schema.add(KeyspaceMetadata.create("sasi",
+                                               KeyspaceParams.simpleTransient(1),
+                                               Tables.of(sasiCFMD("sasi", "test_cf").build(),
+                                                         clusteringSASICFMD("sasi", "clustering_test_cf").build())));
+        }
 
         // if you're messing with low-level sstable stuff, it can be useful to inject the schema directly
         // Schema.instance.load(schemaDefinition());
         for (KeyspaceMetadata ksm : schema)
             MigrationManager.announceNewKeyspace(ksm, false);
+
+        if (Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false")))
+            useCompression(schema);
+    }
+
+    public static void createKeyspace(String name, KeyspaceParams params)
+    {
+        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(name, params, Tables.of()), true);
     }
 
-    public static void createKeyspace(String name, KeyspaceParams params, CFMetaData... tables)
+    public static void createKeyspace(String name, KeyspaceParams params, TableMetadata.Builder... builders)
+    {
+        Tables.Builder tables = Tables.builder();
+        for (TableMetadata.Builder builder : builders)
+            tables.add(builder.build());
+
+        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(name, params, tables.build()), true);
+    }
+
+    public static void createKeyspace(String name, KeyspaceParams params, TableMetadata... tables)
     {
         MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(name, params, Tables.of(tables)), true);
     }
@@ -285,384 +290,374 @@ public class SchemaLoader
         MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(name, params, tables, Views.none(), types, Functions.none()), true);
     }
 
-    public static ColumnDefinition integerColumn(String ksName, String cfName)
+    public static ColumnMetadata integerColumn(String ksName, String cfName)
     {
-        return new ColumnDefinition(ksName,
-                                    cfName,
-                                    ColumnIdentifier.getInterned(IntegerType.instance.fromString("42"), IntegerType.instance),
-                                    UTF8Type.instance,
-                                    ColumnDefinition.NO_POSITION,
-                                    ColumnDefinition.Kind.REGULAR);
+        return new ColumnMetadata(ksName,
+                                  cfName,
+                                  ColumnIdentifier.getInterned(IntegerType.instance.fromString("42"), IntegerType.instance),
+                                  UTF8Type.instance,
+                                  ColumnMetadata.NO_POSITION,
+                                  ColumnMetadata.Kind.REGULAR);
     }
 
-    public static ColumnDefinition utf8Column(String ksName, String cfName)
+    public static ColumnMetadata utf8Column(String ksName, String cfName)
     {
-        return new ColumnDefinition(ksName,
-                                    cfName,
-                                    ColumnIdentifier.getInterned("fortytwo", true),
-                                    UTF8Type.instance,
-                                    ColumnDefinition.NO_POSITION,
-                                    ColumnDefinition.Kind.REGULAR);
+        return new ColumnMetadata(ksName,
+                                  cfName,
+                                  ColumnIdentifier.getInterned("fortytwo", true),
+                                  UTF8Type.instance,
+                                  ColumnMetadata.NO_POSITION,
+                                  ColumnMetadata.Kind.REGULAR);
     }
 
-    public static CFMetaData perRowIndexedCFMD(String ksName, String cfName)
+    public static TableMetadata perRowIndexedCFMD(String ksName, String cfName)
     {
-        final Map<String, String> indexOptions = Collections.singletonMap(
-                                                      IndexTarget.CUSTOM_INDEX_OPTION_NAME,
-                                                      StubIndex.class.getName());
+        ColumnMetadata indexedColumn = ColumnMetadata.regularColumn(ksName, cfName, "indexed", AsciiType.instance);
 
-        CFMetaData cfm =  CFMetaData.Builder.create(ksName, cfName)
-                .addPartitionKey("key", AsciiType.instance)
-                .build();
+        TableMetadata.Builder builder =
+            TableMetadata.builder(ksName, cfName)
+                         .addPartitionKeyColumn("key", AsciiType.instance)
+                         .addColumn(indexedColumn);
 
-        ColumnDefinition indexedColumn = ColumnDefinition.regularDef(ksName, cfName, "indexed", AsciiType.instance);
-        cfm.addOrReplaceColumnDefinition(indexedColumn);
+        final Map<String, String> indexOptions = Collections.singletonMap(IndexTarget.CUSTOM_INDEX_OPTION_NAME, StubIndex.class.getName());
+        builder.indexes(Indexes.of(IndexMetadata.fromIndexTargets(
+        Collections.singletonList(new IndexTarget(indexedColumn.name,
+                                                                                                            IndexTarget.Type.VALUES)),
+                                                                  "indexe1",
+                                                                  IndexMetadata.Kind.CUSTOM,
+                                                                  indexOptions)));
 
-        cfm.indexes(
-            cfm.getIndexes()
-               .with(IndexMetadata.fromIndexTargets(cfm,
-                                                    Collections.singletonList(new IndexTarget(indexedColumn.name,
-                                                                                              IndexTarget.Type.VALUES)),
-                                                    "indexe1",
-                                                    IndexMetadata.Kind.CUSTOM,
-                                                    indexOptions)));
-        return cfm;
+        return builder.build();
     }
 
     private static void useCompression(List<KeyspaceMetadata> schema)
     {
         for (KeyspaceMetadata ksm : schema)
-            for (CFMetaData cfm : ksm.tablesAndViews())
-                cfm.compression(CompressionParams.snappy());
+            for (TableMetadata cfm : ksm.tablesAndViews())
+                MigrationManager.announceTableUpdate(cfm.unbuild().compression(CompressionParams.snappy()).build(), true);
     }
 
-    public static CFMetaData counterCFMD(String ksName, String cfName)
+    public static TableMetadata.Builder counterCFMD(String ksName, String cfName)
     {
-        return CFMetaData.Builder.create(ksName, cfName, false, true, true)
-                .addPartitionKey("key", AsciiType.instance)
-                .addClusteringColumn("name", AsciiType.instance)
-                .addRegularColumn("val", CounterColumnType.instance)
-                .addRegularColumn("val2", CounterColumnType.instance)
-                .build()
-                .compression(getCompressionParameters());
+        return TableMetadata.builder(ksName, cfName)
+                            .isCounter(true)
+                            .addPartitionKeyColumn("key", AsciiType.instance)
+                            .addClusteringColumn("name", AsciiType.instance)
+                            .addRegularColumn("val", CounterColumnType.instance)
+                            .addRegularColumn("val2", CounterColumnType.instance)
+                            .compression(getCompressionParameters());
     }
 
-    public static CFMetaData standardCFMD(String ksName, String cfName)
+    public static TableMetadata.Builder standardCFMD(String ksName, String cfName)
     {
         return standardCFMD(ksName, cfName, 1, AsciiType.instance);
     }
 
-    public static CFMetaData standardCFMD(String ksName, String cfName, int columnCount, AbstractType<?> keyType)
+    public static TableMetadata.Builder standardCFMD(String ksName, String cfName, int columnCount, AbstractType<?> keyType)
     {
         return standardCFMD(ksName, cfName, columnCount, keyType, AsciiType.instance);
     }
 
-    public static CFMetaData standardCFMD(String ksName, String cfName, int columnCount, AbstractType<?> keyType, AbstractType<?> valType)
+    public static TableMetadata.Builder standardCFMD(String ksName, String cfName, int columnCount, AbstractType<?> keyType, AbstractType<?> valType)
     {
         return standardCFMD(ksName, cfName, columnCount, keyType, valType, AsciiType.instance);
     }
 
-    public static CFMetaData standardCFMD(String ksName, String cfName, int columnCount, AbstractType<?> keyType, AbstractType<?> valType, AbstractType<?> clusteringType)
+    public static TableMetadata.Builder standardCFMD(String ksName, String cfName, int columnCount, AbstractType<?> keyType, AbstractType<?> valType, AbstractType<?> clusteringType)
     {
-        CFMetaData.Builder builder;
-        builder = CFMetaData.Builder.create(ksName, cfName)
-                                    .addPartitionKey("key", keyType)
-                                    .addRegularColumn("val", valType);
+        TableMetadata.Builder builder =
+            TableMetadata.builder(ksName, cfName)
+                         .addPartitionKeyColumn("key", keyType)
+                         .addRegularColumn("val", valType)
+                         .compression(getCompressionParameters());
 
-        if(clusteringType != null)
-            builder = builder.addClusteringColumn("name", clusteringType);
+        if (clusteringType != null)
+            builder.addClusteringColumn("name", clusteringType);
 
         for (int i = 0; i < columnCount; i++)
             builder.addRegularColumn("val" + i, AsciiType.instance);
 
-        return builder.build()
-                      .compression(getCompressionParameters());
+        return builder;
     }
 
 
-    public static CFMetaData denseCFMD(String ksName, String cfName)
+    public static TableMetadata.Builder denseCFMD(String ksName, String cfName)
     {
         return denseCFMD(ksName, cfName, AsciiType.instance);
     }
-    public static CFMetaData denseCFMD(String ksName, String cfName, AbstractType cc)
+    public static TableMetadata.Builder denseCFMD(String ksName, String cfName, AbstractType cc)
     {
         return denseCFMD(ksName, cfName, cc, null);
     }
-    public static CFMetaData denseCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
+    public static TableMetadata.Builder denseCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
     {
         AbstractType comp = cc;
         if (subcc != null)
             comp = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{cc, subcc}));
 
-        return CFMetaData.Builder.createDense(ksName, cfName, subcc != null, false)
-            .addPartitionKey("key", AsciiType.instance)
-            .addClusteringColumn("cols", comp)
-            .addRegularColumn("val", AsciiType.instance)
-            .build()
-            .compression(getCompressionParameters());
+        return TableMetadata.builder(ksName, cfName)
+                            .isDense(true)
+                            .isCompound(subcc != null)
+                            .addPartitionKeyColumn("key", AsciiType.instance)
+                            .addClusteringColumn("cols", comp)
+                            .addRegularColumn("val", AsciiType.instance)
+                            .compression(getCompressionParameters());
     }
 
     // TODO: Fix superCFMD failing on legacy table creation. Seems to be applying composite comparator to partition key
-    public static CFMetaData superCFMD(String ksName, String cfName, AbstractType subcc)
+    public static TableMetadata.Builder superCFMD(String ksName, String cfName, AbstractType subcc)
     {
         return superCFMD(ksName, cfName, BytesType.instance, subcc);
     }
-    public static CFMetaData superCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
+    public static TableMetadata.Builder superCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
     {
         return superCFMD(ksName, cfName, "cols", cc, subcc);
     }
-    public static CFMetaData superCFMD(String ksName, String cfName, String ccName, AbstractType cc, AbstractType subcc)
+    public static TableMetadata.Builder superCFMD(String ksName, String cfName, String ccName, AbstractType cc, AbstractType subcc)
     {
-        //This is busted
-//        return CFMetaData.Builder.createSuper(ksName, cfName, false)
-//            .addPartitionKey("0", BytesType.instance)
-//            .addClusteringColumn("1", cc)
-//            .addClusteringColumn("2", subcc)
-//            .addRegularColumn("3", AsciiType.instance)
-//            .build();
         return standardCFMD(ksName, cfName);
 
     }
-    public static CFMetaData compositeIndexCFMD(String ksName, String cfName, boolean withRegularIndex) throws ConfigurationException
+    public static TableMetadata.Builder compositeIndexCFMD(String ksName, String cfName, boolean withRegularIndex) throws ConfigurationException
     {
         return compositeIndexCFMD(ksName, cfName, withRegularIndex, false);
     }
 
-    public static CFMetaData compositeIndexCFMD(String ksName, String cfName, boolean withRegularIndex, boolean withStaticIndex) throws ConfigurationException
+    public static TableMetadata.Builder compositeIndexCFMD(String ksName, String cfName, boolean withRegularIndex, boolean withStaticIndex) throws ConfigurationException
     {
         // the withIndex flag exists to allow tests index creation
         // on existing columns
-        CFMetaData cfm = CFMetaData.Builder.create(ksName, cfName)
-                .addPartitionKey("key", AsciiType.instance)
-                .addClusteringColumn("c1", AsciiType.instance)
-                .addRegularColumn("birthdate", LongType.instance)
-                .addRegularColumn("notbirthdate", LongType.instance)
-                .addStaticColumn("static", LongType.instance)
-                .build();
+        TableMetadata.Builder builder =
+            TableMetadata.builder(ksName, cfName)
+                         .addPartitionKeyColumn("key", AsciiType.instance)
+                         .addClusteringColumn("c1", AsciiType.instance)
+                         .addRegularColumn("birthdate", LongType.instance)
+                         .addRegularColumn("notbirthdate", LongType.instance)
+                         .addStaticColumn("static", LongType.instance)
+                         .compression(getCompressionParameters());
+
+        Indexes.Builder indexes = Indexes.builder();
 
         if (withRegularIndex)
         {
-            cfm.indexes(
-                cfm.getIndexes()
-                   .with(IndexMetadata.fromIndexTargets(cfm,
-                                                        Collections.singletonList(
-                                                            new IndexTarget(new ColumnIdentifier("birthdate", true),
-                                                                            IndexTarget.Type.VALUES)),
-                                                        "birthdate_key_index",
-                                                        IndexMetadata.Kind.COMPOSITES,
-                                                        Collections.EMPTY_MAP)));
+            indexes.add(IndexMetadata.fromIndexTargets(
+            Collections.singletonList(
+                                                           new IndexTarget(new ColumnIdentifier("birthdate", true),
+                                                                           IndexTarget.Type.VALUES)),
+                                                       cfName + "_birthdate_key_index",
+                                                       IndexMetadata.Kind.COMPOSITES,
+                                                       Collections.EMPTY_MAP));
         }
 
         if (withStaticIndex)
         {
-            cfm.indexes(
-                    cfm.getIndexes()
-                       .with(IndexMetadata.fromIndexTargets(cfm,
-                                                            Collections.singletonList(
-                                                                new IndexTarget(new ColumnIdentifier("static", true),
-                                                                                IndexTarget.Type.VALUES)),
-                                                            "static_index",
-                                                            IndexMetadata.Kind.COMPOSITES,
-                                                            Collections.EMPTY_MAP)));
+            indexes.add(IndexMetadata.fromIndexTargets(
+            Collections.singletonList(
+                                                           new IndexTarget(new ColumnIdentifier("static", true),
+                                                                           IndexTarget.Type.VALUES)),
+                                                       cfName + "_static_index",
+                                                       IndexMetadata.Kind.COMPOSITES,
+                                                       Collections.EMPTY_MAP));
         }
 
-        return cfm.compression(getCompressionParameters());
+        return builder.indexes(indexes.build());
     }
 
-    public static CFMetaData keysIndexCFMD(String ksName, String cfName, boolean withIndex) throws ConfigurationException
+    public static TableMetadata.Builder keysIndexCFMD(String ksName, String cfName, boolean withIndex)
     {
-        CFMetaData cfm = CFMetaData.Builder.createDense(ksName, cfName, false, false)
-                                           .addPartitionKey("key", AsciiType.instance)
-                                           .addClusteringColumn("c1", AsciiType.instance)
-                                           .addStaticColumn("birthdate", LongType.instance)
-                                           .addStaticColumn("notbirthdate", LongType.instance)
-                                           .addRegularColumn("value", LongType.instance)
-                                           .build();
+        TableMetadata.Builder builder =
+            TableMetadata.builder(ksName, cfName)
+                         .isCompound(false)
+                         .isDense(true)
+                         .addPartitionKeyColumn("key", AsciiType.instance)
+                         .addClusteringColumn("c1", AsciiType.instance)
+                         .addStaticColumn("birthdate", LongType.instance)
+                         .addStaticColumn("notbirthdate", LongType.instance)
+                         .addRegularColumn("value", LongType.instance)
+                         .compression(getCompressionParameters());
 
         if (withIndex)
-            cfm.indexes(
-                cfm.getIndexes()
-                   .with(IndexMetadata.fromIndexTargets(cfm,
-                                                        Collections.singletonList(
-                                                            new IndexTarget(new ColumnIdentifier("birthdate", true),
-                                                                            IndexTarget.Type.VALUES)),
-                                                         "birthdate_composite_index",
-                                                         IndexMetadata.Kind.KEYS,
-                                                         Collections.EMPTY_MAP)));
-
-
-        return cfm.compression(getCompressionParameters());
-    }
-
-    public static CFMetaData customIndexCFMD(String ksName, String cfName) throws ConfigurationException
-    {
-        CFMetaData cfm = CFMetaData.Builder.createDense(ksName, cfName, false, false)
-                                           .addPartitionKey("key", AsciiType.instance)
-                                           .addClusteringColumn("c1", AsciiType.instance)
-                                           .addRegularColumn("value", LongType.instance)
-                                           .build();
-
-            cfm.indexes(
-                cfm.getIndexes()
-                .with(IndexMetadata.fromIndexTargets(cfm,
-                                                     Collections.singletonList(
-                                                             new IndexTarget(new ColumnIdentifier("value", true),
-                                                                             IndexTarget.Type.VALUES)),
-                                                     "value_index",
-                                                     IndexMetadata.Kind.CUSTOM,
-                                                     Collections.singletonMap(
-                                                             IndexTarget.CUSTOM_INDEX_OPTION_NAME,
-                                                             StubIndex.class.getName()))));
-
-
-        return cfm.compression(getCompressionParameters());
-    }
-
-    public static CFMetaData jdbcCFMD(String ksName, String cfName, AbstractType comp)
-    {
-        return CFMetaData.Builder.create(ksName, cfName).addPartitionKey("key", BytesType.instance)
-                                                        .build()
-                                                        .compression(getCompressionParameters());
-    }
-
-    public static CFMetaData sasiCFMD(String ksName, String cfName)
-    {
-        CFMetaData cfm = CFMetaData.Builder.create(ksName, cfName)
-                                           .addPartitionKey("id", UTF8Type.instance)
-                                           .addRegularColumn("first_name", UTF8Type.instance)
-                                           .addRegularColumn("last_name", UTF8Type.instance)
-                                           .addRegularColumn("age", Int32Type.instance)
-                                           .addRegularColumn("height", Int32Type.instance)
-                                           .addRegularColumn("timestamp", LongType.instance)
-                                           .addRegularColumn("address", UTF8Type.instance)
-                                           .addRegularColumn("score", DoubleType.instance)
-                                           .addRegularColumn("comment", UTF8Type.instance)
-                                           .addRegularColumn("comment_suffix_split", UTF8Type.instance)
-                                           .addRegularColumn("/output/full-name/", UTF8Type.instance)
-                                           .addRegularColumn("/data/output/id", UTF8Type.instance)
-                                           .addRegularColumn("first_name_prefix", UTF8Type.instance)
-                                           .build();
-
-        cfm.indexes(cfm.getIndexes()
-                        .with(IndexMetadata.fromSchemaMetadata("first_name", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "first_name");
-                            put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
-                        }}))
-                        .with(IndexMetadata.fromSchemaMetadata("last_name", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "last_name");
-                            put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
-                        }}))
-                        .with(IndexMetadata.fromSchemaMetadata("age", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "age");
-
-                        }}))
-                        .with(IndexMetadata.fromSchemaMetadata("timestamp", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "timestamp");
-                            put("mode", OnDiskIndexBuilder.Mode.SPARSE.toString());
-
-                        }}))
-                        .with(IndexMetadata.fromSchemaMetadata("address", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put("analyzer_class", "org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer");
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "address");
-                            put("mode", OnDiskIndexBuilder.Mode.PREFIX.toString());
-                            put("case_sensitive", "false");
-                        }}))
-                        .with(IndexMetadata.fromSchemaMetadata("score", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "score");
-                        }}))
-                        .with(IndexMetadata.fromSchemaMetadata("comment", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "comment");
-                            put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
-                            put("analyzed", "true");
-                        }}))
-                        .with(IndexMetadata.fromSchemaMetadata("comment_suffix_split", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "comment_suffix_split");
-                            put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
-                            put("analyzed", "false");
-                        }}))
-                        .with(IndexMetadata.fromSchemaMetadata("output_full_name", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "/output/full-name/");
-                            put("analyzed", "true");
-                            put("analyzer_class", "org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer");
-                            put("case_sensitive", "false");
-                        }}))
-                        .with(IndexMetadata.fromSchemaMetadata("data_output_id", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "/data/output/id");
-                            put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
-                        }}))
-                        .with(IndexMetadata.fromSchemaMetadata("first_name_prefix", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
-                        {{
-                            put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
-                            put(IndexTarget.TARGET_OPTION_NAME, "first_name_prefix");
-                            put("analyzed", "true");
-                            put("tokenization_normalize_lowercase", "true");
-                        }})));
-
-        return cfm;
-    }
-
-    public static CFMetaData clusteringSASICFMD(String ksName, String cfName)
-    {
-        return clusteringSASICFMD(ksName, cfName, "location", "age", "height", "score");
-    }
-
-    public static CFMetaData clusteringSASICFMD(String ksName, String cfName, String...indexedColumns)
-    {
-        CFMetaData cfm = CFMetaData.Builder.create(ksName, cfName)
-                                           .addPartitionKey("name", UTF8Type.instance)
-                                           .addClusteringColumn("location", UTF8Type.instance)
-                                           .addClusteringColumn("age", Int32Type.instance)
-                                           .addRegularColumn("height", Int32Type.instance)
-                                           .addRegularColumn("score", DoubleType.instance)
-                                           .addStaticColumn("nickname", UTF8Type.instance)
-                                           .build();
-
-        Indexes indexes = cfm.getIndexes();
+        {
+            IndexMetadata index =
+                IndexMetadata.fromIndexTargets(
+                Collections.singletonList(new IndexTarget(new ColumnIdentifier("birthdate", true),
+                                                                                         IndexTarget.Type.VALUES)),
+                                                                                         cfName + "_birthdate_composite_index",
+                                                                                         IndexMetadata.Kind.KEYS,
+                                                                                         Collections.EMPTY_MAP);
+            builder.indexes(Indexes.builder().add(index).build());
+        }
+
+        return builder;
+    }
+
+    public static TableMetadata.Builder customIndexCFMD(String ksName, String cfName)
+    {
+        TableMetadata.Builder builder  =
+            TableMetadata.builder(ksName, cfName)
+                         .isCompound(false)
+                         .isDense(true)
+                         .addPartitionKeyColumn("key", AsciiType.instance)
+                         .addClusteringColumn("c1", AsciiType.instance)
+                         .addRegularColumn("value", LongType.instance)
+                         .compression(getCompressionParameters());
+
+        IndexMetadata index =
+            IndexMetadata.fromIndexTargets(
+            Collections.singletonList(new IndexTarget(new ColumnIdentifier("value", true), IndexTarget.Type.VALUES)),
+                                           cfName + "_value_index",
+                                           IndexMetadata.Kind.CUSTOM,
+                                           Collections.singletonMap(IndexTarget.CUSTOM_INDEX_OPTION_NAME, StubIndex.class.getName()));
+
+        builder.indexes(Indexes.of(index));
+
+        return builder;
+    }
+
+    public static TableMetadata.Builder jdbcCFMD(String ksName, String cfName, AbstractType comp)
+    {
+        return TableMetadata.builder(ksName, cfName)
+                            .addPartitionKeyColumn("key", BytesType.instance)
+                            .compression(getCompressionParameters());
+    }
+
+    public static TableMetadata.Builder sasiCFMD(String ksName, String cfName)
+    {
+        TableMetadata.Builder builder =
+            TableMetadata.builder(ksName, cfName)
+                         .addPartitionKeyColumn("id", UTF8Type.instance)
+                         .addRegularColumn("first_name", UTF8Type.instance)
+                         .addRegularColumn("last_name", UTF8Type.instance)
+                         .addRegularColumn("age", Int32Type.instance)
+                         .addRegularColumn("height", Int32Type.instance)
+                         .addRegularColumn("timestamp", LongType.instance)
+                         .addRegularColumn("address", UTF8Type.instance)
+                         .addRegularColumn("score", DoubleType.instance)
+                         .addRegularColumn("comment", UTF8Type.instance)
+                         .addRegularColumn("comment_suffix_split", UTF8Type.instance)
+                         .addRegularColumn("/output/full-name/", UTF8Type.instance)
+                         .addRegularColumn("/data/output/id", UTF8Type.instance)
+                         .addRegularColumn("first_name_prefix", UTF8Type.instance);
+
+        Indexes.Builder indexes = Indexes.builder();
+
+        indexes.add(IndexMetadata.fromSchemaMetadata(cfName + "_first_name", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "first_name");
+                        put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
+                    }}))
+               .add(IndexMetadata.fromSchemaMetadata(cfName + "_last_name", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "last_name");
+                        put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
+                    }}))
+               .add(IndexMetadata.fromSchemaMetadata(cfName + "_age", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "age");
+                    }}))
+               .add(IndexMetadata.fromSchemaMetadata(cfName + "_timestamp", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "timestamp");
+                        put("mode", OnDiskIndexBuilder.Mode.SPARSE.toString());
+
+                    }}))
+               .add(IndexMetadata.fromSchemaMetadata(cfName + "_address", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put("analyzer_class", "org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer");
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "address");
+                        put("mode", OnDiskIndexBuilder.Mode.PREFIX.toString());
+                        put("case_sensitive", "false");
+                    }}))
+               .add(IndexMetadata.fromSchemaMetadata(cfName + "_score", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "score");
+                    }}))
+               .add(IndexMetadata.fromSchemaMetadata(cfName + "_comment", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "comment");
+                        put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
+                        put("analyzed", "true");
+                    }}))
+               .add(IndexMetadata.fromSchemaMetadata(cfName + "_comment_suffix_split", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "comment_suffix_split");
+                        put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
+                        put("analyzed", "false");
+                    }}))
+               .add(IndexMetadata.fromSchemaMetadata(cfName + "_output_full_name", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "/output/full-name/");
+                        put("analyzed", "true");
+                        put("analyzer_class", "org.apache.cassandra.index.sasi.analyzer.NonTokenizingAnalyzer");
+                        put("case_sensitive", "false");
+                    }}))
+               .add(IndexMetadata.fromSchemaMetadata(cfName + "_data_output_id", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "/data/output/id");
+                        put("mode", OnDiskIndexBuilder.Mode.CONTAINS.toString());
+                    }}))
+               .add(IndexMetadata.fromSchemaMetadata(cfName + "_first_name_prefix", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+                    {{
+                        put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
+                        put(IndexTarget.TARGET_OPTION_NAME, "first_name_prefix");
+                        put("analyzed", "true");
+                        put("tokenization_normalize_lowercase", "true");
+                    }}));
+
+    return builder.indexes(indexes.build());
+}
+
+public static TableMetadata.Builder clusteringSASICFMD(String ksName, String cfName)
+{
+    return clusteringSASICFMD(ksName, cfName, "location", "age", "height", "score");
+}
+
+    public static TableMetadata.Builder clusteringSASICFMD(String ksName, String cfName, String...indexedColumns)
+    {
+        Indexes.Builder indexes = Indexes.builder();
         for (String indexedColumn : indexedColumns)
         {
-            indexes = indexes.with(IndexMetadata.fromSchemaMetadata(indexedColumn, IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+            indexes.add(IndexMetadata.fromSchemaMetadata(cfName + "_" + indexedColumn, IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
             {{
                 put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
                 put(IndexTarget.TARGET_OPTION_NAME, indexedColumn);
                 put("mode", OnDiskIndexBuilder.Mode.PREFIX.toString());
             }}));
         }
-        cfm.indexes(indexes);
-        return cfm;
+
+        return TableMetadata.builder(ksName, cfName)
+                            .addPartitionKeyColumn("name", UTF8Type.instance)
+                            .addClusteringColumn("location", UTF8Type.instance)
+                            .addClusteringColumn("age", Int32Type.instance)
+                            .addRegularColumn("height", Int32Type.instance)
+                            .addRegularColumn("score", DoubleType.instance)
+                            .addStaticColumn("nickname", UTF8Type.instance)
+                            .indexes(indexes.build());
     }
 
-    public static CFMetaData staticSASICFMD(String ksName, String cfName)
+    public static TableMetadata.Builder staticSASICFMD(String ksName, String cfName)
     {
-        CFMetaData cfm = CFMetaData.Builder.create(ksName, cfName)
-                                           .addPartitionKey("sensor_id", Int32Type.instance)
-                                           .addStaticColumn("sensor_type", UTF8Type.instance)
-                                           .addClusteringColumn("date", LongType.instance)
-                                           .addRegularColumn("value", DoubleType.instance)
-                                           .addRegularColumn("variance", Int32Type.instance)
-                                           .build();
+        TableMetadata.Builder builder =
+            TableMetadata.builder(ksName, cfName)
+                         .addPartitionKeyColumn("sensor_id", Int32Type.instance)
+                         .addStaticColumn("sensor_type", UTF8Type.instance)
+                         .addClusteringColumn("date", LongType.instance)
+                         .addRegularColumn("value", DoubleType.instance)
+                         .addRegularColumn("variance", Int32Type.instance);
+
+        Indexes.Builder indexes = Indexes.builder();
 
-        Indexes indexes = cfm.getIndexes();
-        indexes = indexes.with(IndexMetadata.fromSchemaMetadata("sensor_type", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+        indexes.add(IndexMetadata.fromSchemaMetadata(cfName + "_sensor_type", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
         {{
             put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
             put(IndexTarget.TARGET_OPTION_NAME, "sensor_type");
@@ -671,34 +666,34 @@ public class SchemaLoader
             put("case_sensitive", "false");
         }}));
 
-        indexes = indexes.with(IndexMetadata.fromSchemaMetadata("value", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+        indexes.add(IndexMetadata.fromSchemaMetadata(cfName + "_value", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
         {{
             put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
             put(IndexTarget.TARGET_OPTION_NAME, "value");
             put("mode", OnDiskIndexBuilder.Mode.PREFIX.toString());
         }}));
 
-        indexes = indexes.with(IndexMetadata.fromSchemaMetadata("variance", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+        indexes.add(IndexMetadata.fromSchemaMetadata(cfName + "_variance", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
         {{
             put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
             put(IndexTarget.TARGET_OPTION_NAME, "variance");
             put("mode", OnDiskIndexBuilder.Mode.PREFIX.toString());
         }}));
 
-        cfm.indexes(indexes);
-        return cfm;
+        return builder.indexes(indexes.build());
     }
 
-    public static CFMetaData fullTextSearchSASICFMD(String ksName, String cfName)
+    public static TableMetadata.Builder fullTextSearchSASICFMD(String ksName, String cfName)
     {
-        CFMetaData cfm = CFMetaData.Builder.create(ksName, cfName)
-                                           .addPartitionKey("song_id", UUIDType.instance)
-                                           .addRegularColumn("title", UTF8Type.instance)
-                                           .addRegularColumn("artist", UTF8Type.instance)
-                                           .build();
+        TableMetadata.Builder builder =
+            TableMetadata.builder(ksName, cfName)
+                         .addPartitionKeyColumn("song_id", UUIDType.instance)
+                         .addRegularColumn("title", UTF8Type.instance)
+                         .addRegularColumn("artist", UTF8Type.instance);
+
+        Indexes.Builder indexes = Indexes.builder();
 
-        Indexes indexes = cfm.getIndexes();
-        indexes = indexes.with(IndexMetadata.fromSchemaMetadata("title", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+        indexes.add(IndexMetadata.fromSchemaMetadata(cfName + "_title", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
         {{
             put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
             put(IndexTarget.TARGET_OPTION_NAME, "title");
@@ -710,7 +705,7 @@ public class SchemaLoader
             put("tokenization_normalize_lowercase", "true");
         }}));
 
-        indexes = indexes.with(IndexMetadata.fromSchemaMetadata("artist", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
+        indexes.add(IndexMetadata.fromSchemaMetadata(cfName + "_artist", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
         {{
             put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, SASIIndex.class.getName());
             put(IndexTarget.TARGET_OPTION_NAME, "artist");
@@ -720,8 +715,7 @@ public class SchemaLoader
 
         }}));
 
-        cfm.indexes(indexes);
-        return cfm;
+        return builder.indexes(indexes.build());
     }
 
     public static CompressionParams getCompressionParameters()
@@ -784,7 +778,7 @@ public class SchemaLoader
 
     public static void insertData(String keyspace, String columnFamily, int offset, int numberOfRows)
     {
-        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace, columnFamily);
+        TableMetadata cfm = Schema.instance.getTableMetadata(keyspace, columnFamily);
 
         for (int i = offset; i < offset + numberOfRows; i++)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/UpdateBuilder.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/UpdateBuilder.java b/test/unit/org/apache/cassandra/UpdateBuilder.java
index 9fcda15..0f3c918 100644
--- a/test/unit/org/apache/cassandra/UpdateBuilder.java
+++ b/test/unit/org/apache/cassandra/UpdateBuilder.java
@@ -17,13 +17,10 @@
  */
 package org.apache.cassandra;
 
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
-import org.apache.cassandra.utils.FBUtilities;
 
 
 /**
@@ -42,7 +39,7 @@ public class UpdateBuilder
         this.updateBuilder = updateBuilder;
     }
 
-    public static UpdateBuilder create(CFMetaData metadata, Object... partitionKey)
+    public static UpdateBuilder create(TableMetadata metadata, Object... partitionKey)
     {
         return new UpdateBuilder(PartitionUpdate.simpleBuilder(metadata, partitionKey));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index d77ca78..a01aab6 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -39,8 +39,9 @@ import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 
@@ -110,13 +111,6 @@ public class Util
         return PartitionPosition.ForKey.get(ByteBufferUtil.bytes(key), partitioner);
     }
 
-    public static Cell getRegularCell(CFMetaData metadata, Row row, String name)
-    {
-        ColumnDefinition column = metadata.getColumnDefinition(ByteBufferUtil.bytes(name));
-        assert column != null;
-        return row.getCell(column);
-    }
-
     public static Clustering clustering(ClusteringComparator comparator, Object... o)
     {
         return comparator.make(o);
@@ -179,12 +173,12 @@ public class Util
     {
         IMutation first = mutations.get(0);
         String keyspaceName = first.getKeyspaceName();
-        UUID cfid = first.getColumnFamilyIds().iterator().next();
+        TableId tableId = first.getTableIds().iterator().next();
 
         for (Mutation rm : mutations)
             rm.applyUnsafe();
 
-        ColumnFamilyStore store = Keyspace.open(keyspaceName).getColumnFamilyStore(cfid);
+        ColumnFamilyStore store = Keyspace.open(keyspaceName).getColumnFamilyStore(tableId);
         store.forceBlockingFlush();
         return store;
     }
@@ -270,7 +264,7 @@ public class Util
 
     public static AbstractReadCommandBuilder.SinglePartitionBuilder cmd(ColumnFamilyStore cfs, Object... partitionKey)
     {
-        return new AbstractReadCommandBuilder.SinglePartitionBuilder(cfs, makeKey(cfs.metadata, partitionKey));
+        return new AbstractReadCommandBuilder.SinglePartitionBuilder(cfs, makeKey(cfs.metadata(), partitionKey));
     }
 
     public static AbstractReadCommandBuilder.PartitionRangeBuilder cmd(ColumnFamilyStore cfs)
@@ -278,13 +272,13 @@ public class Util
         return new AbstractReadCommandBuilder.PartitionRangeBuilder(cfs);
     }
 
-    static DecoratedKey makeKey(CFMetaData metadata, Object... partitionKey)
+    static DecoratedKey makeKey(TableMetadata metadata, Object... partitionKey)
     {
         if (partitionKey.length == 1 && partitionKey[0] instanceof DecoratedKey)
             return (DecoratedKey)partitionKey[0];
 
-        ByteBuffer key = CFMetaData.serializePartitionKey(metadata.getKeyValidatorAsClusteringComparator().make(partitionKey));
-        return metadata.decorateKey(key);
+        ByteBuffer key = metadata.partitionKeyAsClusteringComparator().make(partitionKey).serializeAsPartitionKey();
+        return metadata.partitioner.decorateKey(key);
     }
 
     public static void assertEmptyUnfiltered(ReadCommand command)
@@ -296,7 +290,7 @@ public class Util
             {
                 try (UnfilteredRowIterator partition = iterator.next())
                 {
-                    throw new AssertionError("Expected no results for query " + command.toCQLString() + " but got key " + command.metadata().getKeyValidator().getString(partition.partitionKey().getKey()));
+                    throw new AssertionError("Expected no results for query " + command.toCQLString() + " but got key " + command.metadata().partitionKeyType.getString(partition.partitionKey().getKey()));
                 }
             }
         }
@@ -311,7 +305,7 @@ public class Util
             {
                 try (RowIterator partition = iterator.next())
                 {
-                    throw new AssertionError("Expected no results for query " + command.toCQLString() + " but got key " + command.metadata().getKeyValidator().getString(partition.partitionKey().getKey()));
+                    throw new AssertionError("Expected no results for query " + command.toCQLString() + " but got key " + command.metadata().partitionKeyType.getString(partition.partitionKey().getKey()));
                 }
             }
         }
@@ -423,7 +417,7 @@ public class Util
 
     public static Cell cell(ColumnFamilyStore cfs, Row row, String columnName)
     {
-        ColumnDefinition def = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes(columnName));
+        ColumnMetadata def = cfs.metadata().getColumn(ByteBufferUtil.bytes(columnName));
         assert def != null;
         return row.getCell(def);
     }
@@ -436,7 +430,7 @@ public class Util
     public static void assertCellValue(Object value, ColumnFamilyStore cfs, Row row, String columnName)
     {
         Cell cell = cell(cfs, row, columnName);
-        assert cell != null : "Row " + row.toString(cfs.metadata) + " has no cell for " + columnName;
+        assert cell != null : "Row " + row.toString(cfs.metadata()) + " has no cell for " + columnName;
         assertEquals(value, cell.column().type.compose(cell.value()));
     }
 
@@ -460,15 +454,6 @@ public class Util
         return size;
     }
 
-    public static CBuilder getCBuilderForCFM(CFMetaData cfm)
-    {
-        List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
-        List<AbstractType<?>> types = new ArrayList<>(clusteringColumns.size());
-        for (ColumnDefinition def : clusteringColumns)
-            types.add(def.type);
-        return CBuilder.create(new ClusteringComparator(types));
-    }
-
     public static boolean equal(UnfilteredRowIterator a, UnfilteredRowIterator b)
     {
         return Objects.equals(a.columns(), b.columns())
@@ -492,12 +477,12 @@ public class Util
 
     public static boolean sameContent(Mutation a, Mutation b)
     {
-        if (!a.key().equals(b.key()) || !a.getColumnFamilyIds().equals(b.getColumnFamilyIds()))
+        if (!a.key().equals(b.key()) || !a.getTableIds().equals(b.getTableIds()))
             return false;
 
-        for (UUID cfId : a.getColumnFamilyIds())
+        for (PartitionUpdate update : a.getPartitionUpdates())
         {
-            if (!sameContent(a.getPartitionUpdate(cfId).unfilteredIterator(), b.getPartitionUpdate(cfId).unfilteredIterator()))
+            if (!sameContent(update.unfilteredIterator(), b.getPartitionUpdate(update.metadata()).unfilteredIterator()))
                 return false;
         }
         return true;
@@ -521,9 +506,9 @@ public class Util
                         StringUtils.join(expectedColumnNames, ","));
     }
 
-    public static void assertColumn(CFMetaData cfm, Row row, String name, String value, long timestamp)
+    public static void assertColumn(TableMetadata cfm, Row row, String name, String value, long timestamp)
     {
-        Cell cell = row.getCell(cfm.getColumnDefinition(new ColumnIdentifier(name, true)));
+        Cell cell = row.getCell(cfm.getColumn(new ColumnIdentifier(name, true)));
         assertColumn(cell, value, timestamp);
     }
 
@@ -534,7 +519,7 @@ public class Util
         assertEquals(timestamp, cell.timestamp());
     }
 
-    public static void assertClustering(CFMetaData cfm, Row row, Object... clusteringValue)
+    public static void assertClustering(TableMetadata cfm, Row row, Object... clusteringValue)
     {
         assertEquals(row.clustering().size(), clusteringValue.length);
         assertEquals(0, cfm.comparator.compare(row.clustering(), cfm.comparator.make(clusteringValue)));
@@ -647,12 +632,12 @@ public class Util
     {
         Iterator<Unfiltered> content;
 
-        public UnfilteredSource(CFMetaData cfm, DecoratedKey partitionKey, Row staticRow, Iterator<Unfiltered> content)
+        public UnfilteredSource(TableMetadata metadata, DecoratedKey partitionKey, Row staticRow, Iterator<Unfiltered> content)
         {
-            super(cfm,
+            super(metadata,
                   partitionKey,
                   DeletionTime.LIVE,
-                  cfm.partitionColumns(),
+                  metadata.regularAndStaticColumns(),
                   staticRow != null ? staticRow : Rows.EMPTY_STATIC_ROW,
                   false,
                   EncodingStats.NO_STATS);
@@ -715,20 +700,20 @@ public class Util
 
     public static PagingState makeSomePagingState(ProtocolVersion protocolVersion)
     {
-        CFMetaData metadata = CFMetaData.Builder.create("ks", "tbl")
-                                                .addPartitionKey("k", AsciiType.instance)
-                                                .addClusteringColumn("c1", AsciiType.instance)
-                                                .addClusteringColumn("c1", Int32Type.instance)
-                                                .addRegularColumn("myCol", AsciiType.instance)
-                                                .build();
+        TableMetadata metadata =
+            TableMetadata.builder("ks", "tbl")
+                         .addPartitionKeyColumn("k", AsciiType.instance)
+                         .addClusteringColumn("c1", AsciiType.instance)
+                         .addClusteringColumn("c2", Int32Type.instance)
+                         .addRegularColumn("myCol", AsciiType.instance)
+                         .build();
 
         ByteBuffer pk = ByteBufferUtil.bytes("someKey");
 
-        ColumnDefinition def = metadata.getColumnDefinition(new ColumnIdentifier("myCol", false));
+        ColumnMetadata def = metadata.getColumn(new ColumnIdentifier("myCol", false));
         Clustering c = Clustering.make(ByteBufferUtil.bytes("c1"), ByteBufferUtil.bytes(42));
         Row row = BTreeRow.singleCellRow(c, BufferCell.live(def, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER));
         PagingState.RowMark mark = PagingState.RowMark.create(metadata, row, protocolVersion);
         return new PagingState(pk, mark, 10, 0);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/batchlog/BatchTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/batchlog/BatchTest.java b/test/unit/org/apache/cassandra/batchlog/BatchTest.java
index d2db9b9..8ed1811 100644
--- a/test/unit/org/apache/cassandra/batchlog/BatchTest.java
+++ b/test/unit/org/apache/cassandra/batchlog/BatchTest.java
@@ -28,8 +28,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.RowUpdateBuilder;
@@ -45,7 +44,6 @@ import org.apache.cassandra.utils.UUIDGen;
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 
 public class BatchTest
 {
@@ -64,7 +62,7 @@ public class BatchTest
     @Test
     public void testSerialization() throws IOException
     {
-        CFMetaData cfm = Keyspace.open(KEYSPACE).getColumnFamilyStore(CF_STANDARD).metadata;
+        TableMetadata cfm = Keyspace.open(KEYSPACE).getColumnFamilyStore(CF_STANDARD).metadata();
 
         long now = FBUtilities.timestampMicros();
         int version = MessagingService.current_version;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
index 038255b..34902fe 100644
--- a/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
+++ b/test/unit/org/apache/cassandra/batchlog/BatchlogManagerTest.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.util.*;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Lists;
 
@@ -30,10 +29,10 @@ import org.junit.*;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.Util.PartitionerSwitcher;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -49,7 +48,6 @@ import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -106,7 +104,7 @@ public class BatchlogManagerTest
     public void testDelete()
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
-        CFMetaData cfm = cfs.metadata;
+        TableMetadata cfm = cfs.metadata();
         new RowUpdateBuilder(cfm, FBUtilities.timestampMicros(), ByteBufferUtil.bytes("1234"))
                 .clustering("c")
                 .add("val", "val" + 1234)
@@ -134,7 +132,7 @@ public class BatchlogManagerTest
         long initialAllBatches = BatchlogManager.instance.countAllBatches();
         long initialReplayedBatches = BatchlogManager.instance.getTotalBatchesReplayed();
 
-        CFMetaData cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata;
+        TableMetadata cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata();
 
         // Generate 1000 mutations (100 batches of 10 mutations each) and put them all into the batchlog.
         // Half batches (50) ready to be replayed, half not.
@@ -205,8 +203,8 @@ public class BatchlogManagerTest
     @Test
     public void testTruncatedReplay() throws InterruptedException, ExecutionException
     {
-        CFMetaData cf2 = Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD2);
-        CFMetaData cf3 = Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD3);
+        TableMetadata cf2 = Schema.instance.getTableMetadata(KEYSPACE1, CF_STANDARD2);
+        TableMetadata cf3 = Schema.instance.getTableMetadata(KEYSPACE1, CF_STANDARD3);
         // Generate 2000 mutations (1000 batchlog entries) and put them all into the batchlog.
         // Each batchlog entry with a mutation for Standard2 and Standard3.
         // In the middle of the process, 'truncate' Standard2.
@@ -277,7 +275,7 @@ public class BatchlogManagerTest
     public void testAddBatch() throws IOException
     {
         long initialAllBatches = BatchlogManager.instance.countAllBatches();
-        CFMetaData cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD5).metadata;
+        TableMetadata cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD5).metadata();
 
         long timestamp = (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout() * 2) * 1000;
         UUID uuid = UUIDGen.getTimeUUID();
@@ -309,7 +307,7 @@ public class BatchlogManagerTest
     public void testRemoveBatch()
     {
         long initialAllBatches = BatchlogManager.instance.countAllBatches();
-        CFMetaData cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD5).metadata;
+        TableMetadata cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD5).metadata();
 
         long timestamp = (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout() * 2) * 1000;
         UUID uuid = UUIDGen.getTimeUUID();
@@ -351,7 +349,7 @@ public class BatchlogManagerTest
         long initialAllBatches = BatchlogManager.instance.countAllBatches();
         long initialReplayedBatches = BatchlogManager.instance.getTotalBatchesReplayed();
 
-        CFMetaData cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata;
+        TableMetadata cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata();
 
         long timestamp = (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout() * 2) * 1000;
         UUID uuid = UUIDGen.getTimeUUID();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
index c952470..bb5129a 100644
--- a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
+++ b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
@@ -17,8 +17,8 @@
  */
 package org.apache.cassandra.cache;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.AsciiType;
@@ -45,10 +45,9 @@ public class AutoSavingCacheTest
         SchemaLoader.prepareServer();
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     KeyspaceParams.simple(1),
-                                    CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD1)
-                                                      .addPartitionKey("pKey", AsciiType.instance)
-                                                      .addRegularColumn("col1", AsciiType.instance)
-                                                      .build());
+                                    TableMetadata.builder(KEYSPACE1, CF_STANDARD1)
+                                                 .addPartitionKeyColumn("pKey", AsciiType.instance)
+                                                 .addRegularColumn("col1", AsciiType.instance));
     }
 
     @Test
@@ -71,8 +70,8 @@ public class AutoSavingCacheTest
         cfs.truncateBlocking();
         for (int i = 0; i < 2; i++)
         {
-            ColumnDefinition colDef = ColumnDefinition.regularDef(cfs.metadata, ByteBufferUtil.bytes("col1"), AsciiType.instance);
-            RowUpdateBuilder rowBuilder = new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), "key1");
+            ColumnMetadata colDef = ColumnMetadata.regularColumn(cfs.metadata(), ByteBufferUtil.bytes("col1"), AsciiType.instance);
+            RowUpdateBuilder rowBuilder = new RowUpdateBuilder(cfs.metadata(), System.currentTimeMillis(), "key1");
             rowBuilder.add(colDef, "val1");
             rowBuilder.build().apply();
             cfs.forceBlockingFlush();
@@ -95,6 +94,6 @@ public class AutoSavingCacheTest
         // then load saved
         keyCache.loadSavedAsync().get();
         for (SSTableReader sstable : cfs.getLiveSSTables())
-            Assert.assertNotNull(keyCache.get(new KeyCacheKey(cfs.metadata.ksAndCFName, sstable.descriptor, ByteBufferUtil.bytes("key1"))));
+            Assert.assertNotNull(keyCache.get(new KeyCacheKey(cfs.metadata(), sstable.descriptor, ByteBufferUtil.bytes("key1"))));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
index f6a8acf..1aa536e 100644
--- a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
+++ b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
@@ -18,30 +18,33 @@
  */
 package org.apache.cassandra.cache;
 
-import com.github.benmanes.caffeine.cache.Weigher;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
 
+import com.github.benmanes.caffeine.cache.Weigher;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.utils.Pair;
-
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.partitions.CachedBTreePartition;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.IndexMetadata;
+import org.apache.cassandra.schema.Indexes;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.FBUtilities;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.List;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -59,20 +62,20 @@ public class CacheProviderTest
     private static final String KEYSPACE1 = "CacheProviderTest1";
     private static final String CF_STANDARD1 = "Standard1";
 
-    private static CFMetaData cfm;
+    private static TableMetadata cfm;
 
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
         SchemaLoader.prepareServer();
 
-        cfm = CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD1)
-                                        .addPartitionKey("pKey", AsciiType.instance)
-                                        .addRegularColumn("col1", AsciiType.instance)
-                                        .build();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        cfm =
+            TableMetadata.builder(KEYSPACE1, CF_STANDARD1)
+                         .addPartitionKeyColumn("pKey", AsciiType.instance)
+                         .addRegularColumn("col1", AsciiType.instance)
+                         .build();
+
+        SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1), cfm);
     }
 
     private CachedBTreePartition createPartition()
@@ -158,16 +161,43 @@ public class CacheProviderTest
     @Test
     public void testKeys()
     {
-        Pair<String, String> ksAndCFName = Pair.create(KEYSPACE1, CF_STANDARD1);
+        TableId id1 = TableId.generate();
         byte[] b1 = {1, 2, 3, 4};
-        RowCacheKey key1 = new RowCacheKey(ksAndCFName, ByteBuffer.wrap(b1));
+        RowCacheKey key1 = new RowCacheKey(id1, null, ByteBuffer.wrap(b1));
+        TableId id2 = TableId.fromString(id1.toString());
         byte[] b2 = {1, 2, 3, 4};
-        RowCacheKey key2 = new RowCacheKey(ksAndCFName, ByteBuffer.wrap(b2));
+        RowCacheKey key2 = new RowCacheKey(id2, null, ByteBuffer.wrap(b2));
         assertEquals(key1, key2);
         assertEquals(key1.hashCode(), key2.hashCode());
 
+        TableMetadata tm = TableMetadata.builder("ks", "tab", id1)
+                                        .addPartitionKeyColumn("pk", UTF8Type.instance)
+                                        .build();
+
+        assertTrue(key1.sameTable(tm));
+
         byte[] b3 = {1, 2, 3, 5};
-        RowCacheKey key3 = new RowCacheKey(ksAndCFName, ByteBuffer.wrap(b3));
+        RowCacheKey key3 = new RowCacheKey(id1, null, ByteBuffer.wrap(b3));
+        assertNotSame(key1, key3);
+        assertNotSame(key1.hashCode(), key3.hashCode());
+
+        // with index name
+
+        key1 = new RowCacheKey(id1, "indexFoo", ByteBuffer.wrap(b1));
+        assertNotSame(key1, key2);
+        assertNotSame(key1.hashCode(), key2.hashCode());
+
+        key2 = new RowCacheKey(id2, "indexFoo", ByteBuffer.wrap(b2));
+        assertEquals(key1, key2);
+        assertEquals(key1.hashCode(), key2.hashCode());
+
+        tm = TableMetadata.builder("ks", "tab.indexFoo", id1)
+                          .addPartitionKeyColumn("pk", UTF8Type.instance)
+                          .indexes(Indexes.of(IndexMetadata.fromSchemaMetadata("indexFoo", IndexMetadata.Kind.KEYS, Collections.emptyMap())))
+                          .build();
+        assertTrue(key1.sameTable(tm));
+
+        key3 = new RowCacheKey(id1, "indexFoo", ByteBuffer.wrap(b3));
         assertNotSame(key1, key3);
         assertNotSame(key1.hashCode(), key3.hashCode());
     }


[04/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
index deb7747..4eba4d8 100644
--- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.index.sasi;
 
-import java.io.File;
 import java.io.FileWriter;
 import java.io.Writer;
 import java.nio.ByteBuffer;
@@ -33,13 +32,12 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.cql3.statements.IndexTarget;
-import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.filter.DataLimits;
@@ -61,16 +59,10 @@ import org.apache.cassandra.index.sasi.memory.IndexMemtable;
 import org.apache.cassandra.index.sasi.plan.QueryController;
 import org.apache.cassandra.index.sasi.plan.QueryPlan;
 import org.apache.cassandra.io.sstable.SSTable;
-import org.apache.cassandra.io.sstable.format.big.BigFormat;
 import org.apache.cassandra.schema.IndexMetadata;
-import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.schema.Tables;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
@@ -102,13 +94,13 @@ public class SASIIndexTest
     public static void loadSchema() throws ConfigurationException
     {
         SchemaLoader.loadSchema();
-        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(KS_NAME,
-                                                                     KeyspaceParams.simpleTransient(1),
-                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME),
-                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_1),
-                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_2, "location"),
-                                                                               SchemaLoader.staticSASICFMD(KS_NAME, STATIC_CF_NAME),
-                                                                               SchemaLoader.fullTextSearchSASICFMD(KS_NAME, FTS_CF_NAME))));
+        SchemaLoader.createKeyspace(KS_NAME,
+                                    KeyspaceParams.simpleTransient(1),
+                                    SchemaLoader.sasiCFMD(KS_NAME, CF_NAME),
+                                    SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_1),
+                                    SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME_2, "location"),
+                                    SchemaLoader.staticSASICFMD(KS_NAME, STATIC_CF_NAME),
+                                    SchemaLoader.fullTextSearchSASICFMD(KS_NAME, FTS_CF_NAME));
     }
 
     @Before
@@ -771,25 +763,25 @@ public class SASIIndexTest
         ColumnFamilyStore store = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
 
         Mutation rm1 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key1")));
-        rm1.add(PartitionUpdate.singleRowUpdate(store.metadata,
+        rm1.add(PartitionUpdate.singleRowUpdate(store.metadata(),
                                                 rm1.key(),
-                                                buildRow(buildCell(store.metadata,
+                                                buildRow(buildCell(store.metadata(),
                                                                    UTF8Type.instance.decompose("/data/output/id"),
                                                                    AsciiType.instance.decompose("jason"),
                                                                    System.currentTimeMillis()))));
 
         Mutation rm2 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key2")));
-        rm2.add(PartitionUpdate.singleRowUpdate(store.metadata,
+        rm2.add(PartitionUpdate.singleRowUpdate(store.metadata(),
                                                 rm2.key(),
-                                                buildRow(buildCell(store.metadata,
+                                                buildRow(buildCell(store.metadata(),
                                                                    UTF8Type.instance.decompose("/data/output/id"),
                                                                    AsciiType.instance.decompose("pavel"),
                                                                    System.currentTimeMillis()))));
 
         Mutation rm3 = new Mutation(KS_NAME, decoratedKey(AsciiType.instance.decompose("key3")));
-        rm3.add(PartitionUpdate.singleRowUpdate(store.metadata,
+        rm3.add(PartitionUpdate.singleRowUpdate(store.metadata(),
                                                 rm3.key(),
-                                                buildRow(buildCell(store.metadata,
+                                                buildRow(buildCell(store.metadata(),
                                                                    UTF8Type.instance.decompose("/data/output/id"),
                                                                    AsciiType.instance.decompose("Aleksey"),
                                                                    System.currentTimeMillis()))));
@@ -822,14 +814,14 @@ public class SASIIndexTest
         Assert.assertTrue(rows.toString(), rows.isEmpty());
 
         // now let's trigger index rebuild and check if we got the data back
-        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
+        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName(store.name + "_data_output_id"));
 
         rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
         Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
 
         // also let's try to build an index for column which has no data to make sure that doesn't fail
-        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("first_name"));
-        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName("data_output_id"));
+        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName(store.name + "_first_name"));
+        store.indexManager.buildIndexBlocking(store.indexManager.getIndexByName(store.name + "_data_output_id"));
 
         rows = getIndexed(store, 10, buildExpression(dataOutputId, Operator.LIKE_CONTAINS, UTF8Type.instance.decompose("a")));
         Assert.assertTrue(rows.toString(), Arrays.equals(new String[] { "key1", "key2" }, rows.toArray(new String[rows.size()])));
@@ -1307,14 +1299,14 @@ public class SASIIndexTest
         ColumnFamilyStore store = loadData(data1, true);
 
         RowFilter filter = RowFilter.create();
-        filter.add(store.metadata.getColumnDefinition(firstName), Operator.LIKE_CONTAINS, AsciiType.instance.fromString("a"));
+        filter.add(store.metadata().getColumn(firstName), Operator.LIKE_CONTAINS, AsciiType.instance.fromString("a"));
 
-        ReadCommand command = new PartitionRangeReadCommand(store.metadata,
+        ReadCommand command = new PartitionRangeReadCommand(store.metadata(),
                                                             FBUtilities.nowInSeconds(),
-                                                            ColumnFilter.all(store.metadata),
+                                                            ColumnFilter.all(store.metadata()),
                                                             filter,
                                                             DataLimits.NONE,
-                                                            DataRange.allData(store.metadata.partitioner),
+                                                            DataRange.allData(store.metadata().partitioner),
                                                             Optional.empty());
 
         try
@@ -1602,7 +1594,7 @@ public class SASIIndexTest
         };
 
         // first let's check that we get 'false' for 'isLiteral' if we don't set the option with special comparator
-        ColumnDefinition columnA = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-A", stringType);
+        ColumnMetadata columnA = ColumnMetadata.regularColumn(KS_NAME, CF_NAME, "special-A", stringType);
 
         ColumnIndex indexA = new ColumnIndex(UTF8Type.instance, columnA, IndexMetadata.fromSchemaMetadata("special-index-A", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
         {{
@@ -1613,7 +1605,7 @@ public class SASIIndexTest
         Assert.assertEquals(false, indexA.isLiteral());
 
         // now let's double-check that we do get 'true' when we set it
-        ColumnDefinition columnB = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-B", stringType);
+        ColumnMetadata columnB = ColumnMetadata.regularColumn(KS_NAME, CF_NAME, "special-B", stringType);
 
         ColumnIndex indexB = new ColumnIndex(UTF8Type.instance, columnB, IndexMetadata.fromSchemaMetadata("special-index-B", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
         {{
@@ -1625,7 +1617,7 @@ public class SASIIndexTest
         Assert.assertEquals(true, indexB.isLiteral());
 
         // and finally we should also get a 'true' if it's built-in UTF-8/ASCII comparator
-        ColumnDefinition columnC = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-C", UTF8Type.instance);
+        ColumnMetadata columnC = ColumnMetadata.regularColumn(KS_NAME, CF_NAME, "special-C", UTF8Type.instance);
 
         ColumnIndex indexC = new ColumnIndex(UTF8Type.instance, columnC, IndexMetadata.fromSchemaMetadata("special-index-C", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
         {{
@@ -1635,7 +1627,7 @@ public class SASIIndexTest
         Assert.assertEquals(true, indexC.isIndexed());
         Assert.assertEquals(true, indexC.isLiteral());
 
-        ColumnDefinition columnD = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-D", AsciiType.instance);
+        ColumnMetadata columnD = ColumnMetadata.regularColumn(KS_NAME, CF_NAME, "special-D", AsciiType.instance);
 
         ColumnIndex indexD = new ColumnIndex(UTF8Type.instance, columnD, IndexMetadata.fromSchemaMetadata("special-index-D", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
         {{
@@ -1646,7 +1638,7 @@ public class SASIIndexTest
         Assert.assertEquals(true, indexD.isLiteral());
 
         // and option should supersedes the comparator type
-        ColumnDefinition columnE = ColumnDefinition.regularDef(KS_NAME, CF_NAME, "special-E", UTF8Type.instance);
+        ColumnMetadata columnE = ColumnMetadata.regularColumn(KS_NAME, CF_NAME, "special-E", UTF8Type.instance);
 
         ColumnIndex indexE = new ColumnIndex(UTF8Type.instance, columnE, IndexMetadata.fromSchemaMetadata("special-index-E", IndexMetadata.Kind.CUSTOM, new HashMap<String, String>()
         {{
@@ -1880,7 +1872,7 @@ public class SASIIndexTest
         store.forceBlockingFlush();
 
         SSTable ssTable = store.getSSTables(SSTableSet.LIVE).iterator().next();
-        Path path = FileSystems.getDefault().getPath(ssTable.getFilename().replace("-Data", "-SI_age"));
+        Path path = FileSystems.getDefault().getPath(ssTable.getFilename().replace("-Data", "-SI_" + CLUSTERING_CF_NAME_1 + "_age"));
 
         // Overwrite index file with garbage
         Writer writer = new FileWriter(path.toFile(), false);
@@ -1892,7 +1884,7 @@ public class SASIIndexTest
         Assert.assertTrue(executeCQL(CLUSTERING_CF_NAME_1, "SELECT * FROM %s.%s WHERE age = 27 AND name = 'Pavel'").isEmpty());
 
         // Rebuld index
-        store.rebuildSecondaryIndex("age");
+        store.rebuildSecondaryIndex(CLUSTERING_CF_NAME_1 + "_age");
 
         long size2 = Files.readAttributes(path, BasicFileAttributes.class).size();
         // Make sure that garbage was overwriten
@@ -1917,7 +1909,7 @@ public class SASIIndexTest
             // invalid index mode
             SASIIndex.validateOptions(new HashMap<String, String>()
                                       {{ put("target", "address"); put("mode", "NORMAL"); }},
-                                      store.metadata);
+                                      store.metadata());
             Assert.fail();
         }
         catch (ConfigurationException e)
@@ -1930,7 +1922,7 @@ public class SASIIndexTest
             // invalid SPARSE on the literal index
             SASIIndex.validateOptions(new HashMap<String, String>()
                                       {{ put("target", "address"); put("mode", "SPARSE"); }},
-                                      store.metadata);
+                                      store.metadata());
             Assert.fail();
         }
         catch (ConfigurationException e)
@@ -1943,7 +1935,7 @@ public class SASIIndexTest
             // invalid SPARSE on the explicitly literal index
             SASIIndex.validateOptions(new HashMap<String, String>()
                                       {{ put("target", "height"); put("mode", "SPARSE"); put("is_literal", "true"); }},
-                    store.metadata);
+                                      store.metadata());
             Assert.fail();
         }
         catch (ConfigurationException e)
@@ -1956,7 +1948,7 @@ public class SASIIndexTest
             //  SPARSE with analyzer
             SASIIndex.validateOptions(new HashMap<String, String>()
                                       {{ put("target", "height"); put("mode", "SPARSE"); put("analyzed", "true"); }},
-                                      store.metadata);
+                                      store.metadata());
             Assert.fail();
         }
         catch (ConfigurationException e)
@@ -2229,12 +2221,12 @@ public class SASIIndexTest
             put("key1", Pair.create("Pavel", 14));
         }}, false);
 
-        ColumnIndex index = ((SASIIndex) store.indexManager.getIndexByName("first_name")).getIndex();
+        ColumnIndex index = ((SASIIndex) store.indexManager.getIndexByName(store.name + "_first_name")).getIndex();
         IndexMemtable beforeFlushMemtable = index.getCurrentMemtable();
 
-        PartitionRangeReadCommand command = new PartitionRangeReadCommand(store.metadata,
+        PartitionRangeReadCommand command = new PartitionRangeReadCommand(store.metadata(),
                                                                           FBUtilities.nowInSeconds(),
-                                                                          ColumnFilter.all(store.metadata),
+                                                                          ColumnFilter.all(store.metadata()),
                                                                           RowFilter.NONE,
                                                                           DataLimits.NONE,
                                                                           DataRange.allData(store.getPartitioner()),
@@ -2322,7 +2314,7 @@ public class SASIIndexTest
 
     private static Set<String> getIndexed(ColumnFamilyStore store, int maxResults, Expression... expressions)
     {
-        return getIndexed(store, ColumnFilter.all(store.metadata), maxResults, expressions);
+        return getIndexed(store, ColumnFilter.all(store.metadata()), maxResults, expressions);
     }
 
     private static Set<String> getIndexed(ColumnFamilyStore store, ColumnFilter columnFilter, int maxResults, Expression... expressions)
@@ -2341,7 +2333,7 @@ public class SASIIndexTest
         do
         {
             count = 0;
-            currentPage = getIndexed(store, ColumnFilter.all(store.metadata), lastKey, pageSize, expressions);
+            currentPage = getIndexed(store, ColumnFilter.all(store.metadata()), lastKey, pageSize, expressions);
             if (currentPage == null)
                 break;
 
@@ -2370,9 +2362,9 @@ public class SASIIndexTest
 
         RowFilter filter = RowFilter.create();
         for (Expression e : expressions)
-            filter.add(store.metadata.getColumnDefinition(e.name), e.op, e.value);
+            filter.add(store.metadata().getColumn(e.name), e.op, e.value);
 
-        ReadCommand command = new PartitionRangeReadCommand(store.metadata,
+        ReadCommand command = new PartitionRangeReadCommand(store.metadata(),
                                                             FBUtilities.nowInSeconds(),
                                                             columnFilter,
                                                             filter,
@@ -2473,13 +2465,13 @@ public class SASIIndexTest
 
     private static Cell buildCell(ByteBuffer name, ByteBuffer value, long timestamp)
     {
-        CFMetaData cfm = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).metadata;
-        return BufferCell.live(cfm.getColumnDefinition(name), timestamp, value);
+        TableMetadata cfm = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).metadata();
+        return BufferCell.live(cfm.getColumn(name), timestamp, value);
     }
 
-    private static Cell buildCell(CFMetaData cfm, ByteBuffer name, ByteBuffer value, long timestamp)
+    private static Cell buildCell(TableMetadata cfm, ByteBuffer name, ByteBuffer value, long timestamp)
     {
-        ColumnDefinition column = cfm.getColumnDefinition(name);
+        ColumnMetadata column = cfm.getColumn(name);
         assert column != null;
         return BufferCell.live(column, timestamp, value);
     }
@@ -2491,14 +2483,14 @@ public class SASIIndexTest
 
     private static void update(Mutation rm, ByteBuffer name, ByteBuffer value, long timestamp)
     {
-        CFMetaData metadata = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).metadata;
+        TableMetadata metadata = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).metadata();
         rm.add(PartitionUpdate.singleRowUpdate(metadata, rm.key(), buildRow(buildCell(metadata, name, value, timestamp))));
     }
 
 
     private static void update(Mutation rm, List<Cell> cells)
     {
-        CFMetaData metadata = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).metadata;
+        TableMetadata metadata = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).metadata();
         rm.add(PartitionUpdate.singleRowUpdate(metadata, rm.key(), buildRow(cells)));
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java b/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
index d940186..97b3433 100644
--- a/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
@@ -24,8 +24,6 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ThreadLocalRandom;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -43,10 +41,12 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Tables;
-import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import com.google.common.util.concurrent.Futures;
@@ -67,7 +67,7 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
         SchemaLoader.loadSchema();
         MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(KS_NAME,
                                                                      KeyspaceParams.simpleTransient(1),
-                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME))));
+                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME).build())));
     }
 
     @Test
@@ -78,9 +78,9 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
         final long timestamp = System.currentTimeMillis();
 
         ColumnFamilyStore cfs = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
-        ColumnDefinition column = cfs.metadata.getColumnDefinition(UTF8Type.instance.decompose("age"));
+        ColumnMetadata column = cfs.metadata().getColumn(UTF8Type.instance.decompose("age"));
 
-        SASIIndex sasi = (SASIIndex) cfs.indexManager.getIndexByName("age");
+        SASIIndex sasi = (SASIIndex) cfs.indexManager.getIndexByName(cfs.name + "_age");
 
         File directory = cfs.getDirectories().getDirectoryForNewSSTables();
         Descriptor descriptor = cfs.newSSTableDescriptor(directory);
@@ -91,7 +91,7 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
         for (int i = 0; i < maxKeys; i++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.format(keyFormat, i));
-            expectedKeys.put(cfs.metadata.partitioner.decorateKey(key),
+            expectedKeys.put(cfs.metadata().partitioner.decorateKey(key),
                              BTreeRow.singleCellRow(Clustering.EMPTY,
                                                     BufferCell.live(column, timestamp, Int32Type.instance.decompose(i))));
         }
@@ -136,7 +136,7 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
 
         OnDiskIndex index = new OnDiskIndex(new File(indexFile), Int32Type.instance, keyPosition -> {
             ByteBuffer key = ByteBufferUtil.bytes(String.format(keyFormat, keyPosition));
-            return cfs.metadata.partitioner.decorateKey(key);
+            return cfs.metadata().partitioner.decorateKey(key);
         });
 
         Assert.assertEquals(0, UTF8Type.instance.compare(index.minKey(), ByteBufferUtil.bytes(String.format(keyFormat, 0))));
@@ -170,9 +170,9 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
         final String columnName = "timestamp";
 
         ColumnFamilyStore cfs = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
-        ColumnDefinition column = cfs.metadata.getColumnDefinition(UTF8Type.instance.decompose(columnName));
+        ColumnMetadata column = cfs.metadata().getColumn(UTF8Type.instance.decompose(columnName));
 
-        SASIIndex sasi = (SASIIndex) cfs.indexManager.getIndexByName(columnName);
+        SASIIndex sasi = (SASIIndex) cfs.indexManager.getIndexByName(cfs.name + "_" + columnName);
 
         File directory = cfs.getDirectories().getDirectoryForNewSSTables();
         Descriptor descriptor = cfs.newSSTableDescriptor(directory);
@@ -183,7 +183,7 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
         indexWriter.begin();
         indexWriter.indexes.put(column, indexWriter.newIndex(sasi.getIndex()));
 
-        populateSegment(cfs.metadata, indexWriter.getIndex(column), new HashMap<Long, Set<Integer>>()
+        populateSegment(cfs.metadata(), indexWriter.getIndex(column), new HashMap<Long, Set<Integer>>()
         {{
             put(now,     new HashSet<>(Arrays.asList(0, 1)));
             put(now + 1, new HashSet<>(Arrays.asList(2, 3)));
@@ -201,7 +201,7 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
         // now let's test multiple correct segments with yield incorrect final segment
         for (int i = 0; i < 3; i++)
         {
-            populateSegment(cfs.metadata, index, new HashMap<Long, Set<Integer>>()
+            populateSegment(cfs.metadata(), index, new HashMap<Long, Set<Integer>>()
             {{
                 put(now,     new HashSet<>(Arrays.asList(random.nextInt(), random.nextInt(), random.nextInt())));
                 put(now + 1, new HashSet<>(Arrays.asList(random.nextInt(), random.nextInt(), random.nextInt())));
@@ -236,7 +236,7 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
         Assert.assertFalse(new File(index.outputFile).exists());
     }
 
-    private static void populateSegment(CFMetaData metadata, PerSSTableIndexWriter.Index index, Map<Long, Set<Integer>> data)
+    private static void populateSegment(TableMetadata metadata, PerSSTableIndexWriter.Index index, Map<Long, Set<Integer>> data)
     {
         for (Map.Entry<Long, Set<Integer>> value : data.entrySet())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java b/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
index e388cd4..8273dec 100644
--- a/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
@@ -25,8 +25,8 @@ import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.RowFilter;
@@ -37,10 +37,7 @@ import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.schema.Tables;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.utils.FBUtilities;
 
 import org.junit.*;
@@ -61,11 +58,11 @@ public class OperationTest extends SchemaLoader
     {
         System.setProperty("cassandra.config", "cassandra-murmur.yaml");
         SchemaLoader.loadSchema();
-        MigrationManager.announceNewKeyspace(KeyspaceMetadata.create(KS_NAME,
-                                                                     KeyspaceParams.simpleTransient(1),
-                                                                     Tables.of(SchemaLoader.sasiCFMD(KS_NAME, CF_NAME),
-                                                                               SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME),
-                                                                               SchemaLoader.staticSASICFMD(KS_NAME, STATIC_CF_NAME))));
+        SchemaLoader.createKeyspace(KS_NAME,
+                                    KeyspaceParams.simpleTransient(1),
+                                    SchemaLoader.sasiCFMD(KS_NAME, CF_NAME),
+                                    SchemaLoader.clusteringSASICFMD(KS_NAME, CLUSTERING_CF_NAME),
+                                    SchemaLoader.staticSASICFMD(KS_NAME, STATIC_CF_NAME));
 
         BACKEND = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME);
         CLUSTERING_BACKEND = Keyspace.open(KS_NAME).getColumnFamilyStore(CLUSTERING_CF_NAME);
@@ -78,7 +75,7 @@ public class OperationTest extends SchemaLoader
     public void beforeTest()
     {
         controller = new QueryController(BACKEND,
-                                         PartitionRangeReadCommand.allDataRead(BACKEND.metadata, FBUtilities.nowInSeconds()),
+                                         PartitionRangeReadCommand.allDataRead(BACKEND.metadata(), FBUtilities.nowInSeconds()),
                                          TimeUnit.SECONDS.toMillis(10));
     }
 
@@ -91,9 +88,9 @@ public class OperationTest extends SchemaLoader
     @Test
     public void testAnalyze() throws Exception
     {
-        final ColumnDefinition firstName = getColumn(UTF8Type.instance.decompose("first_name"));
-        final ColumnDefinition age = getColumn(UTF8Type.instance.decompose("age"));
-        final ColumnDefinition comment = getColumn(UTF8Type.instance.decompose("comment"));
+        final ColumnMetadata firstName = getColumn(UTF8Type.instance.decompose("first_name"));
+        final ColumnMetadata age = getColumn(UTF8Type.instance.decompose("age"));
+        final ColumnMetadata comment = getColumn(UTF8Type.instance.decompose("comment"));
 
         // age != 5 AND age > 1 AND age != 6 AND age <= 10
         Map<Expression.Op, Expression> expressions = convert(Operation.analyzeGroup(controller, OperationType.AND,
@@ -184,8 +181,8 @@ public class OperationTest extends SchemaLoader
                             }}, expressions.get(Expression.Op.EQ));
 
         // comment = 'soft eng' and comment != 'likes do'
-        ListMultimap<ColumnDefinition, Expression> e = Operation.analyzeGroup(controller, OperationType.OR,
-                                                    Arrays.asList(new SimpleExpression(comment, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("soft eng")),
+        ListMultimap<ColumnMetadata, Expression> e = Operation.analyzeGroup(controller, OperationType.OR,
+                                                                            Arrays.asList(new SimpleExpression(comment, Operator.LIKE_MATCHES, UTF8Type.instance.decompose("soft eng")),
                                                                   new SimpleExpression(comment, Operator.NEQ, UTF8Type.instance.decompose("likes do"))));
 
         List<Expression> expectedExpressions = new ArrayList<Expression>(2)
@@ -274,8 +271,8 @@ public class OperationTest extends SchemaLoader
     @Test
     public void testSatisfiedBy() throws Exception
     {
-        final ColumnDefinition timestamp = getColumn(UTF8Type.instance.decompose("timestamp"));
-        final ColumnDefinition age = getColumn(UTF8Type.instance.decompose("age"));
+        final ColumnMetadata timestamp = getColumn(UTF8Type.instance.decompose("timestamp"));
+        final ColumnMetadata age = getColumn(UTF8Type.instance.decompose("age"));
 
         Operation.Builder builder = new Operation.Builder(OperationType.AND, controller, new SimpleExpression(age, Operator.NEQ, Int32Type.instance.decompose(5)));
         Operation op = builder.complete();
@@ -438,8 +435,8 @@ public class OperationTest extends SchemaLoader
     @Test
     public void testAnalyzeNotIndexedButDefinedColumn() throws Exception
     {
-        final ColumnDefinition firstName = getColumn(UTF8Type.instance.decompose("first_name"));
-        final ColumnDefinition height = getColumn(UTF8Type.instance.decompose("height"));
+        final ColumnMetadata firstName = getColumn(UTF8Type.instance.decompose("first_name"));
+        final ColumnMetadata height = getColumn(UTF8Type.instance.decompose("height"));
 
         // first_name = 'a' AND height != 10
         Map<Expression.Op, Expression> expressions;
@@ -490,7 +487,7 @@ public class OperationTest extends SchemaLoader
     @Test
     public void testSatisfiedByWithMultipleTerms()
     {
-        final ColumnDefinition comment = getColumn(UTF8Type.instance.decompose("comment"));
+        final ColumnMetadata comment = getColumn(UTF8Type.instance.decompose("comment"));
 
         Unfiltered row = buildRow(buildCell(comment,UTF8Type.instance.decompose("software engineer is working on a project"),System.currentTimeMillis()));
         Row staticRow = buildRow(Clustering.STATIC_CLUSTERING);
@@ -511,10 +508,10 @@ public class OperationTest extends SchemaLoader
     @Test
     public void testSatisfiedByWithClustering()
     {
-        ColumnDefinition location = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("location"));
-        ColumnDefinition age = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("age"));
-        ColumnDefinition height = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("height"));
-        ColumnDefinition score = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("score"));
+        ColumnMetadata location = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("location"));
+        ColumnMetadata age = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("age"));
+        ColumnMetadata height = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("height"));
+        ColumnMetadata score = getColumn(CLUSTERING_BACKEND, UTF8Type.instance.decompose("score"));
 
         Unfiltered row = buildRow(Clustering.make(UTF8Type.instance.fromString("US"), Int32Type.instance.decompose(27)),
                                   buildCell(height, Int32Type.instance.decompose(182), System.currentTimeMillis()),
@@ -567,7 +564,7 @@ public class OperationTest extends SchemaLoader
         Assert.assertTrue(builder.complete().satisfiedBy(row, staticRow, false));
     }
 
-    private Map<Expression.Op, Expression> convert(Multimap<ColumnDefinition, Expression> expressions)
+    private Map<Expression.Op, Expression> convert(Multimap<ColumnMetadata, Expression> expressions)
     {
         Map<Expression.Op, Expression> converted = new HashMap<>();
         for (Expression expression : expressions.values())
@@ -583,8 +580,8 @@ public class OperationTest extends SchemaLoader
     @Test
     public void testSatisfiedByWithStatic()
     {
-        final ColumnDefinition sensorType = getColumn(STATIC_BACKEND, UTF8Type.instance.decompose("sensor_type"));
-        final ColumnDefinition value = getColumn(STATIC_BACKEND, UTF8Type.instance.decompose("value"));
+        final ColumnMetadata sensorType = getColumn(STATIC_BACKEND, UTF8Type.instance.decompose("sensor_type"));
+        final ColumnMetadata value = getColumn(STATIC_BACKEND, UTF8Type.instance.decompose("value"));
 
         Unfiltered row = buildRow(Clustering.make(UTF8Type.instance.fromString("date"), LongType.instance.decompose(20160401L)),
                           buildCell(value, DoubleType.instance.decompose(24.56), System.currentTimeMillis()));
@@ -638,7 +635,7 @@ public class OperationTest extends SchemaLoader
 
     private static class SimpleExpression extends RowFilter.Expression
     {
-        SimpleExpression(ColumnDefinition column, Operator operator, ByteBuffer value)
+        SimpleExpression(ColumnMetadata column, Operator operator, ByteBuffer value)
         {
             super(column, operator, value);
         }
@@ -650,7 +647,7 @@ public class OperationTest extends SchemaLoader
         }
 
         @Override
-        public boolean isSatisfiedBy(CFMetaData metadata, DecoratedKey partitionKey, Row row)
+        public boolean isSatisfiedBy(TableMetadata metadata, DecoratedKey partitionKey, Row row)
         {
             throw new UnsupportedOperationException();
         }
@@ -684,23 +681,23 @@ public class OperationTest extends SchemaLoader
         return rowBuilder.build();
     }
 
-    private static Cell buildCell(ColumnDefinition column, ByteBuffer value, long timestamp)
+    private static Cell buildCell(ColumnMetadata column, ByteBuffer value, long timestamp)
     {
         return BufferCell.live(column, timestamp, value);
     }
 
-    private static Cell deletedCell(ColumnDefinition column, long timestamp, int nowInSeconds)
+    private static Cell deletedCell(ColumnMetadata column, long timestamp, int nowInSeconds)
     {
         return BufferCell.tombstone(column, timestamp, nowInSeconds);
     }
 
-    private static ColumnDefinition getColumn(ByteBuffer name)
+    private static ColumnMetadata getColumn(ByteBuffer name)
     {
         return getColumn(BACKEND, name);
     }
 
-    private static ColumnDefinition getColumn(ColumnFamilyStore cfs, ByteBuffer name)
+    private static ColumnMetadata getColumn(ColumnFamilyStore cfs, ByteBuffer name)
     {
-        return cfs.metadata.getColumnDefinition(name);
+        return cfs.metadata().getColumn(name);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/compress/CQLCompressionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CQLCompressionTest.java b/test/unit/org/apache/cassandra/io/compress/CQLCompressionTest.java
index a2aff2f..1efccd3 100644
--- a/test/unit/org/apache/cassandra/io/compress/CQLCompressionTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CQLCompressionTest.java
@@ -32,13 +32,13 @@ public class CQLCompressionTest extends CQLTester
     public void lz4ParamsTest()
     {
         createTable("create table %s (id int primary key, uh text) with compression = {'class':'LZ4Compressor', 'lz4_high_compressor_level':3}");
-        assertTrue(((LZ4Compressor)getCurrentColumnFamilyStore().metadata.params.compression.getSstableCompressor()).compressorType.equals(LZ4Compressor.LZ4_FAST_COMPRESSOR));
+        assertTrue(((LZ4Compressor)getCurrentColumnFamilyStore().metadata().params.compression.getSstableCompressor()).compressorType.equals(LZ4Compressor.LZ4_FAST_COMPRESSOR));
         createTable("create table %s (id int primary key, uh text) with compression = {'class':'LZ4Compressor', 'lz4_compressor_type':'high', 'lz4_high_compressor_level':13}");
-        assertEquals(((LZ4Compressor)getCurrentColumnFamilyStore().metadata.params.compression.getSstableCompressor()).compressorType, LZ4Compressor.LZ4_HIGH_COMPRESSOR);
-        assertEquals(((LZ4Compressor)getCurrentColumnFamilyStore().metadata.params.compression.getSstableCompressor()).compressionLevel, (Integer)13);
+        assertEquals(((LZ4Compressor)getCurrentColumnFamilyStore().metadata().params.compression.getSstableCompressor()).compressorType, LZ4Compressor.LZ4_HIGH_COMPRESSOR);
+        assertEquals(((LZ4Compressor)getCurrentColumnFamilyStore().metadata().params.compression.getSstableCompressor()).compressionLevel, (Integer)13);
         createTable("create table %s (id int primary key, uh text) with compression = {'class':'LZ4Compressor'}");
-        assertEquals(((LZ4Compressor)getCurrentColumnFamilyStore().metadata.params.compression.getSstableCompressor()).compressorType, LZ4Compressor.LZ4_FAST_COMPRESSOR);
-        assertEquals(((LZ4Compressor)getCurrentColumnFamilyStore().metadata.params.compression.getSstableCompressor()).compressionLevel, (Integer)9);
+        assertEquals(((LZ4Compressor)getCurrentColumnFamilyStore().metadata().params.compression.getSstableCompressor()).compressorType, LZ4Compressor.LZ4_FAST_COMPRESSOR);
+        assertEquals(((LZ4Compressor)getCurrentColumnFamilyStore().metadata().params.compression.getSstableCompressor()).compressionLevel, (Integer)9);
     }
 
     @Test(expected = ConfigurationException.class)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
index 4985342..7e8c1fb 100644
--- a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
@@ -69,7 +69,7 @@ public class BigTableWriterTest extends AbstractTransactionalTest
 
         private TestableBTW(Descriptor desc)
         {
-            this(desc, SSTableTxnWriter.create(cfs, desc, 0, 0, new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS)));
+            this(desc, SSTableTxnWriter.create(cfs, desc, 0, 0, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS)));
         }
 
         private TestableBTW(Descriptor desc, SSTableTxnWriter sw)
@@ -81,7 +81,7 @@ public class BigTableWriterTest extends AbstractTransactionalTest
 
             for (int i = 0; i < 100; i++)
             {
-                UpdateBuilder update = UpdateBuilder.create(cfs.metadata, i);
+                UpdateBuilder update = UpdateBuilder.create(cfs.metadata(), i);
                 for (int j = 0; j < 10; j++)
                     update.newRow(j).add("val", SSTableRewriterTest.random(0, 1000));
                 writer.append(update.build().unfilteredIterator());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index ac7f4ad..970d7ab 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -38,6 +38,8 @@ import org.apache.cassandra.cql3.functions.UDHelper;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.*;
 import com.datastax.driver.core.DataType;
@@ -602,9 +604,9 @@ public class CQLSSTableWriterTest
                     addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
             }
 
-            public CFMetaData getTableMetadata(String cfName)
+            public TableMetadataRef getTableMetadata(String cfName)
             {
-                return Schema.instance.getCFMetaData(keyspace, cfName);
+                return Schema.instance.getTableMetadataRef(keyspace, cfName);
             }
         }, new OutputHandler.SystemOutput(false, false));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
index f287912..186f0e8 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
@@ -52,6 +52,8 @@ import org.apache.cassandra.metrics.CompactionMetrics;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.schema.CachingParams;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static com.google.common.collect.ImmutableMap.of;
@@ -64,7 +66,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class IndexSummaryManagerTest
 {
@@ -102,8 +103,8 @@ public class IndexSummaryManagerTest
         String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
-        originalMinIndexInterval = cfs.metadata.params.minIndexInterval;
-        originalMaxIndexInterval = cfs.metadata.params.maxIndexInterval;
+        originalMinIndexInterval = cfs.metadata().params.minIndexInterval;
+        originalMaxIndexInterval = cfs.metadata().params.maxIndexInterval;
         originalCapacity = IndexSummaryManager.instance.getMemoryPoolCapacityInMB();
     }
 
@@ -119,8 +120,10 @@ public class IndexSummaryManagerTest
         String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
-        cfs.metadata.minIndexInterval(originalMinIndexInterval);
-        cfs.metadata.maxIndexInterval(originalMaxIndexInterval);
+
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().minIndexInterval(originalMinIndexInterval).build(), true);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().maxIndexInterval(originalMaxIndexInterval).build(), true);
+
         IndexSummaryManager.instance.setMemoryPoolCapacityInMB(originalCapacity);
     }
 
@@ -139,7 +142,7 @@ public class IndexSummaryManagerTest
 
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), originalOffHeapSize * sstables.size());
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), originalOffHeapSize * sstables.size());
         }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
@@ -152,7 +155,7 @@ public class IndexSummaryManagerTest
         for (int i = 0; i < numPartition; i++)
         {
             Row row = Util.getOnlyRowUnfiltered(Util.cmd(cfs, String.format("%3d", i)).build());
-            Cell cell = row.getCell(cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val")));
+            Cell cell = row.getCell(cfs.metadata().getColumn(ByteBufferUtil.bytes("val")));
             assertNotNull(cell);
             assertEquals(100, cell.value().array().length);
 
@@ -182,7 +185,7 @@ public class IndexSummaryManagerTest
             {
 
                 String key = String.format("%3d", p);
-                new RowUpdateBuilder(cfs.metadata, 0, key)
+                new RowUpdateBuilder(cfs.metadata(), 0, key)
                     .clustering("column")
                     .add("val", value)
                     .build()
@@ -221,34 +224,34 @@ public class IndexSummaryManagerTest
             sstable.overrideReadMeter(new RestorableMeter(100.0, 100.0));
 
         for (SSTableReader sstable : sstables)
-            assertEquals(cfs.metadata.params.minIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
+            assertEquals(cfs.metadata().params.minIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
 
         // double the min_index_interval
-        cfs.metadata.minIndexInterval(originalMinIndexInterval * 2);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().minIndexInterval(originalMinIndexInterval * 2).build(), true);
         IndexSummaryManager.instance.redistributeSummaries();
         for (SSTableReader sstable : cfs.getLiveSSTables())
         {
-            assertEquals(cfs.metadata.params.minIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
-            assertEquals(numRows / cfs.metadata.params.minIndexInterval, sstable.getIndexSummarySize());
+            assertEquals(cfs.metadata().params.minIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
+            assertEquals(numRows / cfs.metadata().params.minIndexInterval, sstable.getIndexSummarySize());
         }
 
         // return min_index_interval to its original value
-        cfs.metadata.minIndexInterval(originalMinIndexInterval);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().minIndexInterval(originalMinIndexInterval).build(), true);
         IndexSummaryManager.instance.redistributeSummaries();
         for (SSTableReader sstable : cfs.getLiveSSTables())
         {
-            assertEquals(cfs.metadata.params.minIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
-            assertEquals(numRows / cfs.metadata.params.minIndexInterval, sstable.getIndexSummarySize());
+            assertEquals(cfs.metadata().params.minIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
+            assertEquals(numRows / cfs.metadata().params.minIndexInterval, sstable.getIndexSummarySize());
         }
 
         // halve the min_index_interval, but constrain the available space to exactly what we have now; as a result,
         // the summary shouldn't change
-        cfs.metadata.minIndexInterval(originalMinIndexInterval / 2);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().minIndexInterval(originalMinIndexInterval / 2).build(), true);
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
         long summarySpace = sstable.getIndexSummaryOffHeapSize();
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
         {
-            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), summarySpace);
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), summarySpace);
         }
 
         sstable = cfs.getLiveSSTables().iterator().next();
@@ -260,7 +263,7 @@ public class IndexSummaryManagerTest
         int previousSize = sstable.getIndexSummarySize();
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
         {
-            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) Math.ceil(summarySpace * 1.5));
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (long) Math.ceil(summarySpace * 1.5));
         }
         sstable = cfs.getLiveSSTables().iterator().next();
         assertEquals(previousSize * 1.5, (double) sstable.getIndexSummarySize(), 1);
@@ -268,10 +271,10 @@ public class IndexSummaryManagerTest
 
         // return min_index_interval to it's original value (double it), but only give the summary enough space
         // to have an effective index interval of twice the new min
-        cfs.metadata.minIndexInterval(originalMinIndexInterval);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().minIndexInterval(originalMinIndexInterval).build(), true);
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
         {
-            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) Math.ceil(summarySpace / 2.0));
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (long) Math.ceil(summarySpace / 2.0));
         }
         sstable = cfs.getLiveSSTables().iterator().next();
         assertEquals(originalMinIndexInterval * 2, sstable.getEffectiveIndexInterval(), 0.001);
@@ -280,14 +283,14 @@ public class IndexSummaryManagerTest
         // raise the min_index_interval above our current effective interval, but set the max_index_interval lower
         // than what we actually have space for (meaning the index summary would ideally be smaller, but this would
         // result in an effective interval above the new max)
-        cfs.metadata.minIndexInterval(originalMinIndexInterval * 4);
-        cfs.metadata.maxIndexInterval(originalMinIndexInterval * 4);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().minIndexInterval(originalMinIndexInterval * 4).build(), true);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().maxIndexInterval(originalMinIndexInterval * 4).build(), true);
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
         {
-            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), 10);
         }
         sstable = cfs.getLiveSSTables().iterator().next();
-        assertEquals(cfs.metadata.params.minIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
+        assertEquals(cfs.metadata().params.minIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
     }
 
     @Test
@@ -307,35 +310,35 @@ public class IndexSummaryManagerTest
 
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), 10);
         }
         sstables = new ArrayList<>(cfs.getLiveSSTables());
         for (SSTableReader sstable : sstables)
-            assertEquals(cfs.metadata.params.maxIndexInterval, sstable.getEffectiveIndexInterval(), 0.01);
+            assertEquals(cfs.metadata().params.maxIndexInterval, sstable.getEffectiveIndexInterval(), 0.01);
 
         // halve the max_index_interval
-        cfs.metadata.maxIndexInterval(cfs.metadata.params.maxIndexInterval / 2);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().maxIndexInterval(cfs.metadata().params.maxIndexInterval / 2).build(), true);
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 1);
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), 1);
         }
         sstables = new ArrayList<>(cfs.getLiveSSTables());
         for (SSTableReader sstable : sstables)
         {
-            assertEquals(cfs.metadata.params.maxIndexInterval, sstable.getEffectiveIndexInterval(), 0.01);
-            assertEquals(numRows / cfs.metadata.params.maxIndexInterval, sstable.getIndexSummarySize());
+            assertEquals(cfs.metadata().params.maxIndexInterval, sstable.getEffectiveIndexInterval(), 0.01);
+            assertEquals(numRows / cfs.metadata().params.maxIndexInterval, sstable.getIndexSummarySize());
         }
 
         // return max_index_interval to its original value
-        cfs.metadata.maxIndexInterval(cfs.metadata.params.maxIndexInterval * 2);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().maxIndexInterval(cfs.metadata().params.maxIndexInterval * 2).build(), true);
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 1);
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), 1);
         }
         for (SSTableReader sstable : cfs.getLiveSSTables())
         {
-            assertEquals(cfs.metadata.params.maxIndexInterval, sstable.getEffectiveIndexInterval(), 0.01);
-            assertEquals(numRows / cfs.metadata.params.maxIndexInterval, sstable.getIndexSummarySize());
+            assertEquals(cfs.metadata().params.maxIndexInterval, sstable.getEffectiveIndexInterval(), 0.01);
+            assertEquals(numRows / cfs.metadata().params.maxIndexInterval, sstable.getIndexSummarySize());
         }
     }
 
@@ -350,7 +353,7 @@ public class IndexSummaryManagerTest
         int numRows = 256;
         createSSTables(ksname, cfname, numSSTables, numRows);
 
-        int minSamplingLevel = (BASE_SAMPLING_LEVEL * cfs.metadata.params.minIndexInterval) / cfs.metadata.params.maxIndexInterval;
+        int minSamplingLevel = (BASE_SAMPLING_LEVEL * cfs.metadata().params.minIndexInterval) / cfs.metadata().params.maxIndexInterval;
 
         List<SSTableReader> sstables = new ArrayList<>(cfs.getLiveSSTables());
         for (SSTableReader sstable : sstables)
@@ -361,7 +364,7 @@ public class IndexSummaryManagerTest
         // there should be enough space to not downsample anything
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * numSSTables));
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (singleSummaryOffHeapSpace * numSSTables));
         }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
@@ -372,7 +375,7 @@ public class IndexSummaryManagerTest
         assert sstables.size() == 4;
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 2)));
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (singleSummaryOffHeapSpace * (numSSTables / 2)));
         }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL / 2, sstable.getIndexSummarySamplingLevel());
@@ -381,7 +384,7 @@ public class IndexSummaryManagerTest
         // everything should get cut to a quarter
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 4)));
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (singleSummaryOffHeapSpace * (numSSTables / 4)));
         }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL / 4, sstable.getIndexSummarySamplingLevel());
@@ -390,7 +393,7 @@ public class IndexSummaryManagerTest
         // upsample back up to half
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 2) + 4));
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (singleSummaryOffHeapSpace * (numSSTables / 2) + 4));
         }
         assert sstables.size() == 4;
         for (SSTableReader sstable : sstables)
@@ -400,7 +403,7 @@ public class IndexSummaryManagerTest
         // upsample back up to the original index summary
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * numSSTables));
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (singleSummaryOffHeapSpace * numSSTables));
         }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
@@ -412,7 +415,7 @@ public class IndexSummaryManagerTest
         sstables.get(1).overrideReadMeter(new RestorableMeter(50.0, 50.0));
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3));
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (singleSummaryOffHeapSpace * 3));
         }
         Collections.sort(sstables, hotnessComparator);
         assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(0).getIndexSummarySamplingLevel());
@@ -428,7 +431,7 @@ public class IndexSummaryManagerTest
         sstables.get(1).overrideReadMeter(new RestorableMeter(higherRate, higherRate));
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3));
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (singleSummaryOffHeapSpace * 3));
         }
         Collections.sort(sstables, hotnessComparator);
         assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(0).getIndexSummarySamplingLevel());
@@ -446,7 +449,7 @@ public class IndexSummaryManagerTest
 
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3) + 50);
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (singleSummaryOffHeapSpace * 3) + 50);
         }
         Collections.sort(sstables, hotnessComparator);
 
@@ -470,7 +473,7 @@ public class IndexSummaryManagerTest
         sstables.get(3).overrideReadMeter(new RestorableMeter(128.0, 128.0));
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) (singleSummaryOffHeapSpace + (singleSummaryOffHeapSpace * (92.0 / BASE_SAMPLING_LEVEL))));
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), (long) (singleSummaryOffHeapSpace + (singleSummaryOffHeapSpace * (92.0 / BASE_SAMPLING_LEVEL))));
         }
         Collections.sort(sstables, hotnessComparator);
         assertEquals(1, sstables.get(0).getIndexSummarySize());  // at the min sampling level
@@ -483,7 +486,7 @@ public class IndexSummaryManagerTest
         // Don't leave enough space for even the minimal index summaries
         try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
         {
-            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.id, txn), 10);
         }
         for (SSTableReader sstable : sstables)
             assertEquals(1, sstable.getIndexSummarySize());  // at the min sampling level
@@ -506,7 +509,7 @@ public class IndexSummaryManagerTest
         for (int row = 0; row < numRows; row++)
         {
             String key = String.format("%3d", row);
-            new RowUpdateBuilder(cfs.metadata, 0, key)
+            new RowUpdateBuilder(cfs.metadata(), 0, key)
             .clustering("column")
             .add("val", value)
             .build()
@@ -526,7 +529,7 @@ public class IndexSummaryManagerTest
             {
                 sstable = sstable.cloneWithNewSummarySamplingLevel(cfs, samplingLevel);
                 assertEquals(samplingLevel, sstable.getIndexSummarySamplingLevel());
-                int expectedSize = (numRows * samplingLevel) / (sstable.metadata.params.minIndexInterval * BASE_SAMPLING_LEVEL);
+                int expectedSize = (numRows * samplingLevel) / (cfs.metadata().params.minIndexInterval * BASE_SAMPLING_LEVEL);
                 assertEquals(expectedSize, sstable.getIndexSummarySize(), 1);
                 txn.update(sstable, true);
                 txn.checkpoint();
@@ -572,7 +575,7 @@ public class IndexSummaryManagerTest
             for (int row = 0; row < numRows; row++)
             {
                 String key = String.format("%3d", row);
-                new RowUpdateBuilder(cfs.metadata, 0, key)
+                new RowUpdateBuilder(cfs.metadata(), 0, key)
                 .clustering("column")
                 .add("val", value)
                 .build()
@@ -581,20 +584,20 @@ public class IndexSummaryManagerTest
             cfs.forceBlockingFlush();
         }
 
-        assertTrue(manager.getAverageIndexInterval() >= cfs.metadata.params.minIndexInterval);
+        assertTrue(manager.getAverageIndexInterval() >= cfs.metadata().params.minIndexInterval);
         Map<String, Integer> intervals = manager.getIndexIntervals();
         for (Map.Entry<String, Integer> entry : intervals.entrySet())
             if (entry.getKey().contains(CF_STANDARDLOWiINTERVAL))
-                assertEquals(cfs.metadata.params.minIndexInterval, entry.getValue(), 0.001);
+                assertEquals(cfs.metadata().params.minIndexInterval, entry.getValue(), 0.001);
 
         manager.setMemoryPoolCapacityInMB(0);
         manager.redistributeSummaries();
-        assertTrue(manager.getAverageIndexInterval() > cfs.metadata.params.minIndexInterval);
+        assertTrue(manager.getAverageIndexInterval() > cfs.metadata().params.minIndexInterval);
         intervals = manager.getIndexIntervals();
         for (Map.Entry<String, Integer> entry : intervals.entrySet())
         {
             if (entry.getKey().contains(CF_STANDARDLOWiINTERVAL))
-                assertTrue(entry.getValue() >= cfs.metadata.params.minIndexInterval);
+                assertTrue(entry.getValue() >= cfs.metadata().params.minIndexInterval);
         }
     }
 
@@ -630,7 +633,7 @@ public class IndexSummaryManagerTest
                     try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
                     {
                         IndexSummaryManager.redistributeSummaries(new ObservableRedistribution(Collections.EMPTY_LIST,
-                                                                                               of(cfs.metadata.cfId, txn),
+                                                                                               of(cfs.metadata.id, txn),
                                                                                                singleSummaryOffHeapSpace,
                                                                                                barrier));
                     }
@@ -669,7 +672,7 @@ public class IndexSummaryManagerTest
     }
 
     private static List<SSTableReader> redistributeSummaries(List<SSTableReader> compacting,
-                                                             Map<UUID, LifecycleTransaction> transactions,
+                                                             Map<TableId, LifecycleTransaction> transactions,
                                                              long memoryPoolBytes)
     throws IOException
     {
@@ -683,7 +686,7 @@ public class IndexSummaryManagerTest
         CountDownLatch barrier;
 
         ObservableRedistribution(List<SSTableReader> compacting,
-                                 Map<UUID, LifecycleTransaction> transactions,
+                                 Map<TableId, LifecycleTransaction> transactions,
                                  long memoryPoolBytes,
                                  CountDownLatch barrier)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java
index 1f2221e..bc82128 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableCorruptionDetectionTest.java
@@ -73,18 +73,16 @@ public class SSTableCorruptionDetectionTest extends SSTableWriterTestBase
     @BeforeClass
     public static void setUp()
     {
-        CFMetaData cfm = CFMetaData.Builder.create(keyspace, table)
-                                           .addPartitionKey("pk", AsciiType.instance)
-                                           .addClusteringColumn("ck1", AsciiType.instance)
-                                           .addClusteringColumn("ck2", AsciiType.instance)
-                                           .addRegularColumn("reg1", BytesType.instance)
-                                           .addRegularColumn("reg2", BytesType.instance)
-                                           .build();
-
-        cfm.compression(CompressionParams.noCompression());
-        SchemaLoader.createKeyspace(keyspace,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        TableMetadata.Builder cfm =
+            TableMetadata.builder(keyspace, table)
+                         .addPartitionKeyColumn("pk", AsciiType.instance)
+                         .addClusteringColumn("ck1", AsciiType.instance)
+                         .addClusteringColumn("ck2", AsciiType.instance)
+                         .addRegularColumn("reg1", BytesType.instance)
+                         .addRegularColumn("reg2", BytesType.instance)
+                         .compression(CompressionParams.noCompression());
+
+        SchemaLoader.createKeyspace(keyspace, KeyspaceParams.simple(1), cfm);
 
         cfs = Keyspace.open(keyspace).getColumnFamilyStore(table);
         cfs.disableAutoCompaction();
@@ -104,7 +102,7 @@ public class SSTableCorruptionDetectionTest extends SSTableWriterTestBase
         writer = getWriter(cfs, dir, txn);
         for (int i = 0; i < numberOfPks; i++)
         {
-            UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, String.format("pkvalue_%07d", i)).withTimestamp(1);
+            UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), String.format("pkvalue_%07d", i)).withTimestamp(1);
             byte[] reg1 = new byte[valueSize];
             random.nextBytes(reg1);
             byte[] reg2 = new byte[valueSize];
@@ -210,7 +208,7 @@ public class SSTableCorruptionDetectionTest extends SSTableWriterTestBase
             for (int i = 0; i < numberOfPks; i++)
             {
                 DecoratedKey dk = Util.dk(String.format("pkvalue_%07d", i));
-                try (UnfilteredRowIterator rowIter = sstable.iterator(dk, Slices.ALL, ColumnFilter.all(cfs.metadata), false))
+                try (UnfilteredRowIterator rowIter = sstable.iterator(dk, Slices.ALL, ColumnFilter.all(cfs.metadata()), false))
                 {
                     while (rowIter.hasNext())
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
index 72c7467..4f3739f 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
@@ -31,8 +31,9 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.marshal.AsciiType;
@@ -106,9 +107,9 @@ public class SSTableLoaderTest
                 addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
         }
 
-        public CFMetaData getTableMetadata(String tableName)
+        public TableMetadataRef getTableMetadata(String tableName)
         {
-            return Schema.instance.getCFMetaData(keyspace, tableName);
+            return Schema.instance.getTableMetadataRef(keyspace, tableName);
         }
     }
 
@@ -117,7 +118,7 @@ public class SSTableLoaderTest
     {
         File dataDir = new File(tmpdir.getAbsolutePath() + File.separator + KEYSPACE1 + File.separator + CF_STANDARD1);
         assert dataDir.mkdirs();
-        CFMetaData cfmeta = Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD1);
+        TableMetadata metadata = Schema.instance.getTableMetadata(KEYSPACE1, CF_STANDARD1);
 
         String schema = "CREATE TABLE %s.%s (key ascii, name ascii, val ascii, val1 ascii, PRIMARY KEY (key, name))";
         String query = "INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)";
@@ -143,7 +144,7 @@ public class SSTableLoaderTest
         assertEquals(1, partitions.size());
         assertEquals("key1", AsciiType.instance.getString(partitions.get(0).partitionKey().getKey()));
         assertEquals(ByteBufferUtil.bytes("100"), partitions.get(0).getRow(Clustering.make(ByteBufferUtil.bytes("col1")))
-                                                                   .getCell(cfmeta.getColumnDefinition(ByteBufferUtil.bytes("val")))
+                                                                   .getCell(metadata.getColumn(ByteBufferUtil.bytes("val")))
                                                                    .value());
 
         // The stream future is signalled when the work is complete but before releasing references. Wait for release

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
index ffe7b06..b922ca8 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
@@ -27,7 +27,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
@@ -60,11 +60,11 @@ public class SSTableMetadataTest
                                     SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
                                     SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
                                     SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD3),
-                                    CFMetaData.Builder.create(KEYSPACE1, CF_STANDARDCOMPOSITE2)
-                                                      .addPartitionKey("key", AsciiType.instance)
-                                                      .addClusteringColumn("name", AsciiType.instance)
-                                                      .addClusteringColumn("int", IntegerType.instance)
-                                                      .addRegularColumn("val", AsciiType.instance).build(),
+                                    TableMetadata.builder(KEYSPACE1, CF_STANDARDCOMPOSITE2)
+                                                 .addPartitionKeyColumn("key", AsciiType.instance)
+                                                 .addClusteringColumn("name", AsciiType.instance)
+                                                 .addClusteringColumn("int", IntegerType.instance)
+                                                 .addRegularColumn("val", AsciiType.instance),
                                     SchemaLoader.counterCFMD(KEYSPACE1, CF_COUNTER1));
     }
 
@@ -78,7 +78,7 @@ public class SSTableMetadataTest
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
             for (int j = 0; j < 10; j++)
-                new RowUpdateBuilder(store.metadata, timestamp, 10 + j, Integer.toString(i))
+                new RowUpdateBuilder(store.metadata(), timestamp, 10 + j, Integer.toString(i))
                     .clustering(Integer.toString(j))
                     .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
@@ -86,7 +86,7 @@ public class SSTableMetadataTest
 
         }
 
-        new RowUpdateBuilder(store.metadata, timestamp, 10000, "longttl")
+        new RowUpdateBuilder(store.metadata(), timestamp, 10000, "longttl")
             .clustering("col")
             .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
@@ -104,7 +104,7 @@ public class SSTableMetadataTest
 
         }
 
-        new RowUpdateBuilder(store.metadata, timestamp, 20000, "longttl2")
+        new RowUpdateBuilder(store.metadata(), timestamp, 20000, "longttl2")
         .clustering("col")
         .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
         .build()
@@ -153,14 +153,14 @@ public class SSTableMetadataTest
         long timestamp = System.currentTimeMillis();
         DecoratedKey key = Util.dk("deletetest");
         for (int i = 0; i<5; i++)
-            new RowUpdateBuilder(store.metadata, timestamp, 100, "deletetest")
+            new RowUpdateBuilder(store.metadata(), timestamp, 100, "deletetest")
                 .clustering("deletecolumn" + i)
                 .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                 .build()
                 .applyUnsafe();
 
 
-        new RowUpdateBuilder(store.metadata, timestamp, 1000, "deletetest")
+        new RowUpdateBuilder(store.metadata(), timestamp, 1000, "deletetest")
         .clustering("todelete")
         .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
         .build()
@@ -176,7 +176,7 @@ public class SSTableMetadataTest
             assertEquals(ttltimestamp + 1000, firstMaxDelTime, 10);
         }
 
-        RowUpdateBuilder.deleteRow(store.metadata, timestamp + 1, "deletetest", "todelete").applyUnsafe();
+        RowUpdateBuilder.deleteRow(store.metadata(), timestamp + 1, "deletetest", "todelete").applyUnsafe();
 
         store.forceBlockingFlush();
         assertEquals(2,store.getLiveSSTables().size());
@@ -208,7 +208,7 @@ public class SSTableMetadataTest
             String key = "row" + j;
             for (int i = 100; i<150; i++)
             {
-                new RowUpdateBuilder(store.metadata, System.currentTimeMillis(), key)
+                new RowUpdateBuilder(store.metadata(), System.currentTimeMillis(), key)
                     .clustering(j + "col" + i)
                     .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
@@ -226,7 +226,7 @@ public class SSTableMetadataTest
 
         for (int i = 101; i<299; i++)
         {
-            new RowUpdateBuilder(store.metadata, System.currentTimeMillis(), key)
+            new RowUpdateBuilder(store.metadata(), System.currentTimeMillis(), key)
             .clustering(9 + "col" + i)
             .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
@@ -263,7 +263,7 @@ public class SSTableMetadataTest
 
         for (int i = 0; i < 10; i++)
         {
-            new RowUpdateBuilder(cfs.metadata, 0, "k")
+            new RowUpdateBuilder(cfs.metadata(), 0, "k")
                 .clustering("a" + (9 - i), getBytes(i))
                 .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                 .build()
@@ -274,7 +274,7 @@ public class SSTableMetadataTest
 
         for (int i = 0; i < 10; i++)
         {
-            new RowUpdateBuilder(cfs.metadata, 0, "k2")
+            new RowUpdateBuilder(cfs.metadata(), 0, "k2")
             .clustering("b" + (9 - i), getBytes(i))
             .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 982fc9c..64d0252 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -98,7 +98,7 @@ public class SSTableReaderTest
         CompactionManager.instance.disableAutoCompaction();
         for (int j = 0; j < 10; j++)
         {
-            new RowUpdateBuilder(store.metadata, j, String.valueOf(j))
+            new RowUpdateBuilder(store.metadata(), j, String.valueOf(j))
                 .clustering("0")
                 .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                 .build()
@@ -144,7 +144,7 @@ public class SSTableReaderTest
             CompactionManager.instance.disableAutoCompaction();
             for (int j = 0; j < 100; j += 2)
             {
-                new RowUpdateBuilder(store.metadata, j, String.valueOf(j))
+                new RowUpdateBuilder(store.metadata(), j, String.valueOf(j))
                 .clustering("0")
                 .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                 .build()
@@ -186,7 +186,7 @@ public class SSTableReaderTest
 
         for (int j = 0; j < 100; j += 2)
         {
-            new RowUpdateBuilder(store.metadata, j, String.valueOf(j))
+            new RowUpdateBuilder(store.metadata(), j, String.valueOf(j))
             .clustering("0")
             .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
@@ -214,7 +214,7 @@ public class SSTableReaderTest
 
         for (int j = 0; j < 10; j++)
         {
-            new RowUpdateBuilder(store.metadata, j, String.valueOf(j))
+            new RowUpdateBuilder(store.metadata(), j, String.valueOf(j))
             .clustering("0")
             .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
@@ -247,7 +247,7 @@ public class SSTableReaderTest
         for (int j = 0; j < 10; j++)
         {
 
-            new RowUpdateBuilder(store.metadata, j, String.valueOf(j))
+            new RowUpdateBuilder(store.metadata(), j, String.valueOf(j))
             .clustering("0")
             .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
@@ -280,7 +280,7 @@ public class SSTableReaderTest
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_INDEXED);
         partitioner = store.getPartitioner();
 
-        new RowUpdateBuilder(store.metadata, System.currentTimeMillis(), "k1")
+        new RowUpdateBuilder(store.metadata(), System.currentTimeMillis(), "k1")
             .clustering("0")
             .add("birthdate", 1L)
             .build()
@@ -302,7 +302,7 @@ public class SSTableReaderTest
         CompactionManager.instance.disableAutoCompaction();
         for (int j = 0; j < 10; j++)
         {
-            new RowUpdateBuilder(store.metadata, j, String.valueOf(j))
+            new RowUpdateBuilder(store.metadata(), j, String.valueOf(j))
             .clustering("0")
             .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
@@ -339,18 +339,18 @@ public class SSTableReaderTest
 
         DecoratedKey firstKey = null, lastKey = null;
         long timestamp = System.currentTimeMillis();
-        for (int i = 0; i < store.metadata.params.minIndexInterval; i++)
+        for (int i = 0; i < store.metadata().params.minIndexInterval; i++)
         {
             DecoratedKey key = Util.dk(String.valueOf(i));
             if (firstKey == null)
                 firstKey = key;
             if (lastKey == null)
                 lastKey = key;
-            if (store.metadata.getKeyValidator().compare(lastKey.getKey(), key.getKey()) < 0)
+            if (store.metadata().partitionKeyType.compare(lastKey.getKey(), key.getKey()) < 0)
                 lastKey = key;
 
 
-            new RowUpdateBuilder(store.metadata, timestamp, key.getKey())
+            new RowUpdateBuilder(store.metadata(), timestamp, key.getKey())
                 .clustering("col")
                 .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                 .build()
@@ -377,7 +377,7 @@ public class SSTableReaderTest
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
 
-        new RowUpdateBuilder(store.metadata, System.currentTimeMillis(), "k1")
+        new RowUpdateBuilder(store.metadata(), System.currentTimeMillis(), "k1")
         .clustering("0")
         .add("birthdate", 1L)
         .build()
@@ -406,7 +406,7 @@ public class SSTableReaderTest
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         partitioner = store.getPartitioner();
 
-        new RowUpdateBuilder(store.metadata, 0, "k1")
+        new RowUpdateBuilder(store.metadata(), 0, "k1")
             .clustering("xyz")
             .add("val", "abc")
             .build()
@@ -439,7 +439,7 @@ public class SSTableReaderTest
         for (int j = 0; j < 130; j++)
         {
 
-            new RowUpdateBuilder(store.metadata, j, String.valueOf(j))
+            new RowUpdateBuilder(store.metadata(), j, String.valueOf(j))
             .clustering("0")
             .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
@@ -478,7 +478,7 @@ public class SSTableReaderTest
         final int NUM_PARTITIONS = 512;
         for (int j = 0; j < NUM_PARTITIONS; j++)
         {
-            new RowUpdateBuilder(store.metadata, j, String.format("%3d", j))
+            new RowUpdateBuilder(store.metadata(), j, String.format("%3d", j))
             .clustering("0")
             .add("val", String.format("%3d", j))
             .build()
@@ -557,7 +557,7 @@ public class SSTableReaderTest
         final int NUM_PARTITIONS = 512;
         for (int j = 0; j < NUM_PARTITIONS; j++)
         {
-            new RowUpdateBuilder(store.metadata, j, String.format("%3d", j))
+            new RowUpdateBuilder(store.metadata(), j, String.format("%3d", j))
             .clustering("0")
             .add("val", String.format("%3d", j))
             .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
index 9ea29e5..e3afaeb 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -75,7 +75,7 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
 
         for (int j = 0; j < 100; j ++)
         {
-            new RowUpdateBuilder(cfs.metadata, j, String.valueOf(j))
+            new RowUpdateBuilder(cfs.metadata(), j, String.valueOf(j))
                 .clustering("0")
                 .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                 .build()
@@ -692,7 +692,7 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
             String key = Integer.toString(i);
 
             for (int j = 0; j < 10; j++)
-                new RowUpdateBuilder(cfs.metadata, 100, key)
+                new RowUpdateBuilder(cfs.metadata(), 100, key)
                     .clustering(Integer.toString(j))
                     .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
@@ -936,12 +936,12 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
             File dir = cfs.getDirectories().getDirectoryForNewSSTables();
             Descriptor desc = cfs.newSSTableDescriptor(dir);
 
-            try (SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, desc, 0, 0, new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS)))
+            try (SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, desc, 0, 0, new SerializationHeader(true, cfs.metadata(), cfs.metadata().regularAndStaticColumns(), EncodingStats.NO_STATS)))
             {
                 int end = f == fileCount - 1 ? partitionCount : ((f + 1) * partitionCount) / fileCount;
                 for ( ; i < end ; i++)
                 {
-                    UpdateBuilder builder = UpdateBuilder.create(cfs.metadata, ByteBufferUtil.bytes(i));
+                    UpdateBuilder builder = UpdateBuilder.create(cfs.metadata(), ByteBufferUtil.bytes(i));
                     for (int j = 0; j < cellCount ; j++)
                         builder.newRow(Integer.toString(i)).add("val", random(0, 1000));
 


[06/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
index b3f1f57..eccf671 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
@@ -27,7 +27,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.*;
@@ -59,23 +59,27 @@ public class CompactionsPurgeTest
     public static void defineSchema() throws ConfigurationException
     {
         SchemaLoader.prepareServer();
+
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     KeyspaceParams.simple(1),
                                     SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
                                     SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+
         SchemaLoader.createKeyspace(KEYSPACE2,
                                     KeyspaceParams.simple(1),
                                     SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD1));
+
         SchemaLoader.createKeyspace(KEYSPACE_CACHED,
                                     KeyspaceParams.simple(1),
                                     SchemaLoader.standardCFMD(KEYSPACE_CACHED, CF_CACHED).caching(CachingParams.CACHE_EVERYTHING));
+
         SchemaLoader.createKeyspace(KEYSPACE_CQL,
                                     KeyspaceParams.simple(1),
-                                    CFMetaData.compile("CREATE TABLE " + CF_CQL + " ("
-                                            + "k int PRIMARY KEY,"
-                                            + "v1 text,"
-                                            + "v2 int"
-                                            + ")", KEYSPACE_CQL));
+                                    CreateTableStatement.parse("CREATE TABLE " + CF_CQL + " ("
+                                                               + "k int PRIMARY KEY,"
+                                                               + "v1 text,"
+                                                               + "v2 int"
+                                                               + ")", KEYSPACE_CQL));
     }
 
     @Test
@@ -92,7 +96,7 @@ public class CompactionsPurgeTest
         // inserts
         for (int i = 0; i < 10; i++)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 0, key);
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 0, key);
             builder.clustering(String.valueOf(i))
                    .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                    .build().applyUnsafe();
@@ -103,12 +107,12 @@ public class CompactionsPurgeTest
         // deletes
         for (int i = 0; i < 10; i++)
         {
-            RowUpdateBuilder.deleteRow(cfs.metadata, 1, key, String.valueOf(i)).applyUnsafe();
+            RowUpdateBuilder.deleteRow(cfs.metadata(), 1, key, String.valueOf(i)).applyUnsafe();
         }
         cfs.forceBlockingFlush();
 
         // resurrect one column
-        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 2, key);
+        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 2, key);
         builder.clustering(String.valueOf(5))
                .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                .build().applyUnsafe();
@@ -137,7 +141,7 @@ public class CompactionsPurgeTest
         // inserts
         for (int i = 0; i < 10; i++)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 0, key);
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 0, key);
             builder.clustering(String.valueOf(i))
                    .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                    .build().applyUnsafe();
@@ -147,7 +151,7 @@ public class CompactionsPurgeTest
         // deletes
         for (int i = 0; i < 10; i++)
         {
-            RowUpdateBuilder.deleteRow(cfs.metadata, Long.MAX_VALUE, key, String.valueOf(i)).applyUnsafe();
+            RowUpdateBuilder.deleteRow(cfs.metadata(), Long.MAX_VALUE, key, String.valueOf(i)).applyUnsafe();
         }
         cfs.forceBlockingFlush();
 
@@ -155,7 +159,7 @@ public class CompactionsPurgeTest
         FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false));
 
         // resurrect one column
-        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 2, key);
+        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 2, key);
         builder.clustering(String.valueOf(5))
                .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                .build().applyUnsafe();
@@ -182,7 +186,7 @@ public class CompactionsPurgeTest
         // inserts
         for (int i = 0; i < 10; i++)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 0, key);
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 0, key);
             builder.clustering(String.valueOf(i))
                    .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                    .build().applyUnsafe();
@@ -190,7 +194,7 @@ public class CompactionsPurgeTest
         cfs.forceBlockingFlush();
 
         new Mutation(KEYSPACE1, dk(key))
-            .add(PartitionUpdate.fullPartitionDelete(cfs.metadata, dk(key), Long.MAX_VALUE, FBUtilities.nowInSeconds()))
+            .add(PartitionUpdate.fullPartitionDelete(cfs.metadata(), dk(key), Long.MAX_VALUE, FBUtilities.nowInSeconds()))
             .applyUnsafe();
         cfs.forceBlockingFlush();
 
@@ -198,7 +202,7 @@ public class CompactionsPurgeTest
         FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false));
 
         // resurrect one column
-        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 2, key);
+        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 2, key);
         builder.clustering(String.valueOf(5))
                .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                .build().applyUnsafe();
@@ -225,14 +229,14 @@ public class CompactionsPurgeTest
         // inserts
         for (int i = 0; i < 10; i++)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 0, key);
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 0, key);
             builder.clustering(String.valueOf(i))
                    .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                    .build().applyUnsafe();
         }
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, Long.MAX_VALUE, dk(key))
+        new RowUpdateBuilder(cfs.metadata(), Long.MAX_VALUE, dk(key))
             .addRangeTombstone(String.valueOf(0), String.valueOf(9)).build().applyUnsafe();
         cfs.forceBlockingFlush();
 
@@ -240,7 +244,7 @@ public class CompactionsPurgeTest
         FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false));
 
         // resurrect one column
-        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 2, key);
+        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 2, key);
         builder.clustering(String.valueOf(5))
                .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                .build().applyUnsafe();
@@ -268,7 +272,7 @@ public class CompactionsPurgeTest
             // inserts
             for (int i = 0; i < 10; i++)
             {
-                RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 0, key);
+                RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 0, key);
                 builder.clustering(String.valueOf(i))
                         .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                         .build().applyUnsafe();
@@ -278,7 +282,7 @@ public class CompactionsPurgeTest
             // deletes
             for (int i = 0; i < 10; i++)
             {
-                RowUpdateBuilder.deleteRow(cfs.metadata, 1, key, String.valueOf(i)).applyUnsafe();
+                RowUpdateBuilder.deleteRow(cfs.metadata(), 1, key, String.valueOf(i)).applyUnsafe();
             }
 
             cfs.forceBlockingFlush();
@@ -292,7 +296,7 @@ public class CompactionsPurgeTest
         cfs.forceBlockingFlush();
         Collection<SSTableReader> sstablesIncomplete = cfs.getLiveSSTables();
 
-        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 2, "key1");
+        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 2, "key1");
         builder.clustering(String.valueOf(5))
                 .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                 .build().applyUnsafe();
@@ -326,25 +330,25 @@ public class CompactionsPurgeTest
         String key3 = "key3";
 
         // inserts
-        new RowUpdateBuilder(cfs.metadata, 8, key3)
+        new RowUpdateBuilder(cfs.metadata(), 8, key3)
             .clustering("c1")
             .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build().applyUnsafe();
 
-        new RowUpdateBuilder(cfs.metadata, 8, key3)
+        new RowUpdateBuilder(cfs.metadata(), 8, key3)
         .clustering("c2")
         .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
         .build().applyUnsafe();
 
         cfs.forceBlockingFlush();
         // delete c1
-        RowUpdateBuilder.deleteRow(cfs.metadata, 10, key3, "c1").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), 10, key3, "c1").applyUnsafe();
 
         cfs.forceBlockingFlush();
         Collection<SSTableReader> sstablesIncomplete = cfs.getLiveSSTables();
 
         // delete c2 so we have new delete in a diffrent SSTable
-        RowUpdateBuilder.deleteRow(cfs.metadata, 9, key3, "c2").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), 9, key3, "c2").applyUnsafe();
         cfs.forceBlockingFlush();
 
         // compact the sstables with the c1/c2 data and the c1 tombstone
@@ -374,7 +378,7 @@ public class CompactionsPurgeTest
         // inserts
         for (int i = 0; i < 5; i++)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 0, key);
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 0, key);
             builder.clustering(String.valueOf(i))
                    .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                    .build().applyUnsafe();
@@ -383,7 +387,7 @@ public class CompactionsPurgeTest
         // deletes
         for (int i = 0; i < 5; i++)
         {
-            RowUpdateBuilder.deleteRow(cfs.metadata, 1, key, String.valueOf(i)).applyUnsafe();
+            RowUpdateBuilder.deleteRow(cfs.metadata(), 1, key, String.valueOf(i)).applyUnsafe();
         }
         cfs.forceBlockingFlush();
         assertEquals(String.valueOf(cfs.getLiveSSTables()), 1, cfs.getLiveSSTables().size()); // inserts & deletes were in the same memtable -> only deletes in sstable
@@ -411,7 +415,7 @@ public class CompactionsPurgeTest
         // inserts
         for (int i = 0; i < 10; i++)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 0, key);
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 0, key);
             builder.clustering(String.valueOf(i))
                    .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                    .build().applyUnsafe();
@@ -419,12 +423,12 @@ public class CompactionsPurgeTest
 
         // deletes partition
         Mutation rm = new Mutation(KEYSPACE_CACHED, dk(key));
-        rm.add(PartitionUpdate.fullPartitionDelete(cfs.metadata, dk(key), 1, FBUtilities.nowInSeconds()));
+        rm.add(PartitionUpdate.fullPartitionDelete(cfs.metadata(), dk(key), 1, FBUtilities.nowInSeconds()));
         rm.applyUnsafe();
 
         // Adds another unrelated partition so that the sstable is not considered fully expired. We do not
         // invalidate the row cache in that latter case.
-        new RowUpdateBuilder(cfs.metadata, 0, "key4").clustering("c").add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER).build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "key4").clustering("c").add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER).build().applyUnsafe();
 
         // move the key up in row cache (it should not be empty since we have the partition deletion info)
         assertFalse(Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build()).isEmpty());
@@ -451,7 +455,7 @@ public class CompactionsPurgeTest
         // inserts
         for (int i = 0; i < 10; i++)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, i, key);
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), i, key);
             builder.clustering(String.valueOf(i))
                    .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                    .build().applyUnsafe();
@@ -459,7 +463,7 @@ public class CompactionsPurgeTest
 
         // deletes partition with timestamp such that not all columns are deleted
         Mutation rm = new Mutation(KEYSPACE1, dk(key));
-        rm.add(PartitionUpdate.fullPartitionDelete(cfs.metadata, dk(key), 4, FBUtilities.nowInSeconds()));
+        rm.add(PartitionUpdate.fullPartitionDelete(cfs.metadata(), dk(key), 4, FBUtilities.nowInSeconds()));
         rm.applyUnsafe();
 
         ImmutableBTreePartition partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build());
@@ -473,7 +477,7 @@ public class CompactionsPurgeTest
         // re-inserts with timestamp lower than delete
         for (int i = 0; i < 5; i++)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, i, key);
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), i, key);
             builder.clustering(String.valueOf(i))
                    .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                    .build().applyUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index 47dc7f6..feea50a 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -29,7 +29,7 @@ import org.junit.runner.RunWith;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -37,6 +37,7 @@ import org.apache.cassandra.dht.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.schema.CompactionParams;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -82,7 +83,7 @@ public class CompactionsTest
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_DENSE1);
         store.clearUnsafe();
-        store.metadata.gcGraceSeconds(1);
+        MigrationManager.announceTableUpdate(store.metadata().unbuild().gcGraceSeconds(1).build(), true);
 
         // disable compaction while flushing
         store.disableAutoCompaction();
@@ -118,7 +119,7 @@ public class CompactionsTest
     public static long populate(String ks, String cf, int startRowKey, int endRowKey, int ttl)
     {
         long timestamp = System.currentTimeMillis();
-        CFMetaData cfm = Keyspace.open(ks).getColumnFamilyStore(cf).metadata;
+        TableMetadata cfm = Keyspace.open(ks).getColumnFamilyStore(cf).metadata();
         for (int i = startRowKey; i <= endRowKey; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
@@ -156,21 +157,21 @@ public class CompactionsTest
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Super1");
-        CFMetaData cfm = cfs.metadata;
+        CFMetaData table = cfs.metadata;
         cfs.disableAutoCompaction();
 
         DecoratedKey key = Util.dk("tskey");
         ByteBuffer scName = ByteBufferUtil.bytes("TestSuperColumn");
 
         // a subcolumn
-        new RowUpdateBuilder(cfm, FBUtilities.timestampMicros(), key.getKey())
+        new RowUpdateBuilder(table, FBUtilities.timestampMicros(), key.getKey())
             .clustering(ByteBufferUtil.bytes("cols"))
             .add("val", "val1")
             .build().applyUnsafe();
         cfs.forceBlockingFlush();
 
         // shadow the subcolumn with a supercolumn tombstone
-        RowUpdateBuilder.deleteRow(cfm, FBUtilities.timestampMicros(), key.getKey(), ByteBufferUtil.bytes("cols")).applyUnsafe();
+        RowUpdateBuilder.deleteRow(table, FBUtilities.timestampMicros(), key.getKey(), ByteBufferUtil.bytes("cols")).applyUnsafe();
         cfs.forceBlockingFlush();
 
         CompactionManager.instance.performMaximal(cfs);
@@ -543,7 +544,7 @@ public class CompactionsTest
     {
         long timestamp = System.currentTimeMillis();
         DecoratedKey dk = Util.dk(String.format("%03d", key));
-        new RowUpdateBuilder(Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata, timestamp, dk.getKey())
+        new RowUpdateBuilder(Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata(), timestamp, dk.getKey())
                 .add("val", "val")
                 .build()
                 .applyUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
index 5f470b7..e4cd37b 100644
--- a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
@@ -223,7 +223,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < numSSTables; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            new RowUpdateBuilder(cfs.metadata, r, key.getKey())
+            new RowUpdateBuilder(cfs.metadata(), r, key.getKey())
                 .clustering("column")
                 .add("val", value).build().applyUnsafe();
 
@@ -259,7 +259,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < numSSTables; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            new RowUpdateBuilder(cfs.metadata, r, key.getKey())
+            new RowUpdateBuilder(cfs.metadata(), r, key.getKey())
                 .clustering("column")
                 .add("val", value).build().applyUnsafe();
 
@@ -296,7 +296,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
 
         // create 2 sstables
         DecoratedKey key = Util.dk(String.valueOf("expired"));
-        new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), 1, key.getKey())
+        new RowUpdateBuilder(cfs.metadata(), System.currentTimeMillis(), 1, key.getKey())
             .clustering("column")
             .add("val", value).build().applyUnsafe();
 
@@ -305,7 +305,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
         Thread.sleep(10);
 
         key = Util.dk(String.valueOf("nonexpired"));
-        new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), key.getKey())
+        new RowUpdateBuilder(cfs.metadata(), System.currentTimeMillis(), key.getKey())
             .clustering("column")
             .add("val", value).build().applyUnsafe();
 
@@ -349,7 +349,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
             for (int i = 0; i < 10; i++)
             {
                 DecoratedKey key = Util.dk(String.valueOf(r));
-                new RowUpdateBuilder(cfs.metadata, timestamp, key.getKey())
+                new RowUpdateBuilder(cfs.metadata(), timestamp, key.getKey())
                     .clustering("column")
                     .add("val", bigValue).build().applyUnsafe();
             }
@@ -359,7 +359,7 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < numSSTables / 2; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            new RowUpdateBuilder(cfs.metadata, timestamp, key.getKey())
+            new RowUpdateBuilder(cfs.metadata(), timestamp, key.getKey())
                 .clustering("column")
                 .add("val", value).build().applyUnsafe();
             cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 12144eb..9a8371e 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -117,7 +117,7 @@ public class LeveledCompactionStrategyTest
         // Adds enough data to trigger multiple sstable per level
         for (int r = 0; r < rows; r++)
         {
-            UpdateBuilder update = UpdateBuilder.create(cfs.metadata, String.valueOf(r));
+            UpdateBuilder update = UpdateBuilder.create(cfs.metadata(), String.valueOf(r));
             for (int c = 0; c < columns; c++)
                 update.newRow("column" + c).add("val", value);
             update.applyUnsafe();
@@ -173,7 +173,7 @@ public class LeveledCompactionStrategyTest
         // Adds enough data to trigger multiple sstable per level
         for (int r = 0; r < rows; r++)
         {
-            UpdateBuilder update = UpdateBuilder.create(cfs.metadata, String.valueOf(r));
+            UpdateBuilder update = UpdateBuilder.create(cfs.metadata(), String.valueOf(r));
             for (int c = 0; c < columns; c++)
                 update.newRow("column" + c).add("val", value);
             update.applyUnsafe();
@@ -239,7 +239,7 @@ public class LeveledCompactionStrategyTest
         int columns = 10;
         for (int r = 0; r < rows; r++)
         {
-            UpdateBuilder update = UpdateBuilder.create(cfs.metadata, String.valueOf(r));
+            UpdateBuilder update = UpdateBuilder.create(cfs.metadata(), String.valueOf(r));
             for (int c = 0; c < columns; c++)
                 update.newRow("column" + c).add("val", value);
             update.applyUnsafe();
@@ -276,7 +276,7 @@ public class LeveledCompactionStrategyTest
         // Adds enough data to trigger multiple sstable per level
         for (int r = 0; r < rows; r++)
         {
-            UpdateBuilder update = UpdateBuilder.create(cfs.metadata, String.valueOf(r));
+            UpdateBuilder update = UpdateBuilder.create(cfs.metadata(), String.valueOf(r));
             for (int c = 0; c < columns; c++)
                 update.newRow("column" + c).add("val", value);
             update.applyUnsafe();
@@ -317,7 +317,7 @@ public class LeveledCompactionStrategyTest
         // Adds enough data to trigger multiple sstable per level
         for (int r = 0; r < rows; r++)
         {
-            UpdateBuilder update = UpdateBuilder.create(cfs.metadata, String.valueOf(r));
+            UpdateBuilder update = UpdateBuilder.create(cfs.metadata(), String.valueOf(r));
             for (int c = 0; c < columns; c++)
                 update.newRow("column" + c).add("val", value);
             update.applyUnsafe();
@@ -395,7 +395,7 @@ public class LeveledCompactionStrategyTest
         // create 10 sstables that contain data for both key1 and key2
         for (int i = 0; i < numIterations; i++) {
             for (DecoratedKey key : keys) {
-                UpdateBuilder update = UpdateBuilder.create(cfs.metadata, key);
+                UpdateBuilder update = UpdateBuilder.create(cfs.metadata(), key);
                 for (int c = 0; c < columns; c++)
                     update.newRow("column" + c).add("val", value);
                 update.applyUnsafe();
@@ -406,7 +406,7 @@ public class LeveledCompactionStrategyTest
         // create 20 more sstables with 10 containing data for key1 and other 10 containing data for key2
         for (int i = 0; i < numIterations; i++) {
             for (DecoratedKey key : keys) {
-                UpdateBuilder update = UpdateBuilder.create(cfs.metadata, key);
+                UpdateBuilder update = UpdateBuilder.create(cfs.metadata(), key);
                 for (int c = 0; c < columns; c++)
                     update.newRow("column" + c).add("val", value);
                 update.applyUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
index f55bf52..0c469dc 100644
--- a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
@@ -64,7 +64,7 @@ public class OneCompactionTest
         Set<String> inserted = new HashSet<>();
         for (int j = 0; j < insertsPerTable; j++) {
             String key = String.valueOf(j);
-            new RowUpdateBuilder(store.metadata, j, key)
+            new RowUpdateBuilder(store.metadata(), j, key)
                 .clustering("0")
                 .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                 .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
index f4dbea8..7c86f87 100644
--- a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
@@ -158,7 +158,7 @@ public class SizeTieredCompactionStrategyTest
         for (int r = 0; r < numSSTables; r++)
         {
             String key = String.valueOf(r);
-            new RowUpdateBuilder(cfs.metadata, 0, key)
+            new RowUpdateBuilder(cfs.metadata(), 0, key)
                 .clustering("column").add("val", value)
                 .build().applyUnsafe();
             cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
index 55bff5e..6e4ba0b 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
@@ -31,7 +31,7 @@ import org.junit.runner.RunWith;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
@@ -41,6 +41,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.tools.SSTableExpiredBlockers;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -59,17 +60,17 @@ public class TTLExpiryTest
         SchemaLoader.prepareServer();
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     KeyspaceParams.simple(1),
-                                    CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD1)
-                                                      .addPartitionKey("pKey", AsciiType.instance)
-                                                      .addRegularColumn("col1", AsciiType.instance)
-                                                      .addRegularColumn("col", AsciiType.instance)
-                                                      .addRegularColumn("col311", AsciiType.instance)
-                                                      .addRegularColumn("col2", AsciiType.instance)
-                                                      .addRegularColumn("col3", AsciiType.instance)
-                                                      .addRegularColumn("col7", AsciiType.instance)
-                                                      .addRegularColumn("col8", MapType.getInstance(AsciiType.instance, AsciiType.instance, true))
-                                                      .addRegularColumn("shadow", AsciiType.instance)
-                                                      .build().gcGraceSeconds(0));
+                                    TableMetadata.builder(KEYSPACE1, CF_STANDARD1)
+                                                 .addPartitionKeyColumn("pKey", AsciiType.instance)
+                                                 .addRegularColumn("col1", AsciiType.instance)
+                                                 .addRegularColumn("col", AsciiType.instance)
+                                                 .addRegularColumn("col311", AsciiType.instance)
+                                                 .addRegularColumn("col2", AsciiType.instance)
+                                                 .addRegularColumn("col3", AsciiType.instance)
+                                                 .addRegularColumn("col7", AsciiType.instance)
+                                                 .addRegularColumn("col8", MapType.getInstance(AsciiType.instance, AsciiType.instance, true))
+                                                 .addRegularColumn("shadow", AsciiType.instance)
+                                                 .gcGraceSeconds(0));
     }
 
     @Test
@@ -77,36 +78,36 @@ public class TTLExpiryTest
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
         cfs.disableAutoCompaction();
-        cfs.metadata.gcGraceSeconds(0);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().gcGraceSeconds(0).build(), true);
         String key = "ttl";
-        new RowUpdateBuilder(cfs.metadata, 1L, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), 1L, 1, key)
                     .add("col1", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
                     .applyUnsafe();
 
-        new RowUpdateBuilder(cfs.metadata, 3L, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), 3L, 1, key)
                     .add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
                     .applyUnsafe();
         cfs.forceBlockingFlush();
-        new RowUpdateBuilder(cfs.metadata, 2L, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), 2L, 1, key)
                     .add("col1", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
                     .applyUnsafe();
 
-        new RowUpdateBuilder(cfs.metadata, 5L, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), 5L, 1, key)
                     .add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
                     .applyUnsafe();
 
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 4L, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), 4L, 1, key)
                     .add("col1", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
                     .applyUnsafe();
 
-        new RowUpdateBuilder(cfs.metadata, 7L, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), 7L, 1, key)
                     .add("shadow", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
                     .applyUnsafe();
@@ -114,12 +115,12 @@ public class TTLExpiryTest
         cfs.forceBlockingFlush();
 
 
-        new RowUpdateBuilder(cfs.metadata, 6L, 3, key)
+        new RowUpdateBuilder(cfs.metadata(), 6L, 3, key)
                     .add("shadow", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
                     .applyUnsafe();
 
-        new RowUpdateBuilder(cfs.metadata, 8L, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), 8L, 1, key)
                     .add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
                     .applyUnsafe();
@@ -158,10 +159,10 @@ public class TTLExpiryTest
         cfs.truncateBlocking();
         cfs.disableAutoCompaction();
         // To reproduce #10944, we need our gcBefore to be equal to the locaDeletionTime. A gcGrace of 1 will (almost always) give us that.
-        cfs.metadata.gcGraceSeconds(force10944Bug ? 1 : 0);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().gcGraceSeconds(force10944Bug ? 1 : 0).build(), true);
         long timestamp = System.currentTimeMillis();
         String key = "ttl";
-        new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), timestamp, 1, key)
                         .add("col", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                         .add("col7", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                         .build()
@@ -169,7 +170,7 @@ public class TTLExpiryTest
 
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), timestamp, 1, key)
             .add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .add("col8", Collections.singletonMap("bar", "foo"))
             .delete("col1")
@@ -180,14 +181,14 @@ public class TTLExpiryTest
         cfs.forceBlockingFlush();
         // To reproduce #10944, we need to avoid the optimization that get rid of full sstable because everything
         // is known to be gcAble, so keep some data non-expiring in that case.
-        new RowUpdateBuilder(cfs.metadata, timestamp, force10944Bug ? 0 : 1, key)
+        new RowUpdateBuilder(cfs.metadata(), timestamp, force10944Bug ? 0 : 1, key)
                     .add("col3", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                     .build()
                     .applyUnsafe();
 
 
         cfs.forceBlockingFlush();
-        new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), timestamp, 1, key)
                             .add("col311", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                             .build()
                             .applyUnsafe();
@@ -206,28 +207,28 @@ public class TTLExpiryTest
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
         cfs.truncateBlocking();
         cfs.disableAutoCompaction();
-        cfs.metadata.gcGraceSeconds(0);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().gcGraceSeconds(0).build(), true);
         long timestamp = System.currentTimeMillis();
         String key = "ttl";
-        new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), timestamp, 1, key)
             .add("col", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .add("col7", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
             .applyUnsafe();
 
         cfs.forceBlockingFlush();
-        new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), timestamp, 1, key)
             .add("col2", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
             .applyUnsafe();
         cfs.forceBlockingFlush();
-        new RowUpdateBuilder(cfs.metadata, timestamp, 1, key)
+        new RowUpdateBuilder(cfs.metadata(), timestamp, 1, key)
             .add("col3", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
             .applyUnsafe();
         cfs.forceBlockingFlush();
         String noTTLKey = "nottl";
-        new RowUpdateBuilder(cfs.metadata, timestamp, noTTLKey)
+        new RowUpdateBuilder(cfs.metadata(), timestamp, noTTLKey)
             .add("col311", ByteBufferUtil.EMPTY_BYTE_BUFFER)
             .build()
             .applyUnsafe();
@@ -238,7 +239,7 @@ public class TTLExpiryTest
         cfs.enableAutoCompaction(true);
         assertEquals(1, cfs.getLiveSSTables().size());
         SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
-        ISSTableScanner scanner = sstable.getScanner(ColumnFilter.all(sstable.metadata), DataRange.allData(cfs.getPartitioner()));
+        ISSTableScanner scanner = sstable.getScanner(ColumnFilter.all(cfs.metadata()), DataRange.allData(cfs.getPartitioner()));
         assertTrue(scanner.hasNext());
         while(scanner.hasNext())
         {
@@ -254,9 +255,9 @@ public class TTLExpiryTest
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
         cfs.truncateBlocking();
         cfs.disableAutoCompaction();
-        cfs.metadata.gcGraceSeconds(0);
+        MigrationManager.announceTableUpdate(cfs.metadata().unbuild().gcGraceSeconds(0).build(), true);
 
-        new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), "test")
+        new RowUpdateBuilder(cfs.metadata(), System.currentTimeMillis(), "test")
                 .noRowMarker()
                 .add("col1", ByteBufferUtil.EMPTY_BYTE_BUFFER)
                 .build()
@@ -266,7 +267,7 @@ public class TTLExpiryTest
         SSTableReader blockingSSTable = cfs.getSSTables(SSTableSet.LIVE).iterator().next();
         for (int i = 0; i < 10; i++)
         {
-            new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), "test")
+            new RowUpdateBuilder(cfs.metadata(), System.currentTimeMillis(), "test")
                             .noRowMarker()
                             .delete("col1")
                             .build()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java
index 5041b31..930fc79 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TimeWindowCompactionStrategyTest.java
@@ -148,7 +148,7 @@ public class TimeWindowCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < 3; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            new RowUpdateBuilder(cfs.metadata, r, key.getKey())
+            new RowUpdateBuilder(cfs.metadata(), r, key.getKey())
                 .clustering("column")
                 .add("val", value).build().applyUnsafe();
 
@@ -159,7 +159,7 @@ public class TimeWindowCompactionStrategyTest extends SchemaLoader
         {
             // And add progressively more cells into each sstable
             DecoratedKey key = Util.dk(String.valueOf(r));
-            new RowUpdateBuilder(cfs.metadata, r, key.getKey())
+            new RowUpdateBuilder(cfs.metadata(), r, key.getKey())
                 .clustering("column")
                 .add("val", value).build().applyUnsafe();
             cfs.forceBlockingFlush();
@@ -200,7 +200,7 @@ public class TimeWindowCompactionStrategyTest extends SchemaLoader
             DecoratedKey key = Util.dk(String.valueOf(r));
             for(int i = 0 ; i < r ; i++)
             {
-                new RowUpdateBuilder(cfs.metadata, tstamp + r, key.getKey())
+                new RowUpdateBuilder(cfs.metadata(), tstamp + r, key.getKey())
                     .clustering("column")
                     .add("val", value).build().applyUnsafe();
             }
@@ -232,7 +232,7 @@ public class TimeWindowCompactionStrategyTest extends SchemaLoader
 
         // create 2 sstables
         DecoratedKey key = Util.dk(String.valueOf("expired"));
-        new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), 1, key.getKey())
+        new RowUpdateBuilder(cfs.metadata(), System.currentTimeMillis(), 1, key.getKey())
             .clustering("column")
             .add("val", value).build().applyUnsafe();
 
@@ -241,7 +241,7 @@ public class TimeWindowCompactionStrategyTest extends SchemaLoader
         Thread.sleep(10);
 
         key = Util.dk(String.valueOf("nonexpired"));
-        new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), key.getKey())
+        new RowUpdateBuilder(cfs.metadata(), System.currentTimeMillis(), key.getKey())
             .clustering("column")
             .add("val", value).build().applyUnsafe();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
index 1db7944..7e1540a 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
@@ -31,11 +31,11 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import junit.framework.Assert;
-import org.apache.cassandra.MockSchema;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.MockSchema;
 
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertNotNull;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
index 4514b72..0cc5b72 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -28,15 +28,15 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import junit.framework.Assert;
-import org.apache.cassandra.MockSchema;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Memtable;
 import org.apache.cassandra.db.commitlog.CommitLogPosition;
 import org.apache.cassandra.db.compaction.OperationType;
-import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.MockSchema;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
 import org.apache.cassandra.utils.concurrent.Transactional.AbstractTransactional.State;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
index dd3f370..2021538 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LogTransactionTest.java
@@ -32,15 +32,8 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.assertNull;
-import static junit.framework.Assert.fail;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 import junit.framework.Assert;
-import org.apache.cassandra.MockSchema;
+
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.SerializationHeader;
@@ -51,12 +44,20 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.schema.MockSchema;
 import org.apache.cassandra.utils.AlwaysPresentFilter;
 import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
 import org.apache.cassandra.utils.concurrent.Transactional;
 
+import static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertNull;
+import static junit.framework.Assert.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 public class LogTransactionTest extends AbstractTransactionalTest
 {
     private static final String KEYSPACE = "TransactionLogsTest";
@@ -89,7 +90,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
             {
                 this.cfs = cfs;
                 this.txnLogs = txnLogs;
-                this.dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+                this.dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
                 this.sstableOld = sstable(dataFolder, cfs, 0, 128);
                 this.sstableNew = sstable(dataFolder, cfs, 1, 128);
 
@@ -201,7 +202,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testUntrack() throws Throwable
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128);
 
         // complete a transaction without keep the new files since they were untracked
@@ -224,7 +225,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testCommitSameDesc() throws Throwable
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstableOld1 = sstable(dataFolder, cfs, 0, 128);
         SSTableReader sstableOld2 = sstable(dataFolder, cfs, 0, 256);
         SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128);
@@ -255,7 +256,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testCommitOnlyNew() throws Throwable
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);
 
         LogTransaction log = new LogTransaction(OperationType.COMPACTION);
@@ -273,7 +274,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testCommitOnlyOld() throws Throwable
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);
 
         LogTransaction log = new LogTransaction(OperationType.COMPACTION);
@@ -294,7 +295,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
 
-        File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File origiFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         File dataFolder1 = new File(origiFolder, "1");
         File dataFolder2 = new File(origiFolder, "2");
         Files.createDirectories(dataFolder1.toPath());
@@ -330,7 +331,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testAbortOnlyNew() throws Throwable
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);
 
         LogTransaction log = new LogTransaction(OperationType.COMPACTION);
@@ -348,7 +349,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testAbortOnlyOld() throws Throwable
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);
 
         LogTransaction log = new LogTransaction(OperationType.COMPACTION);
@@ -370,7 +371,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
 
-        File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File origiFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         File dataFolder1 = new File(origiFolder, "1");
         File dataFolder2 = new File(origiFolder, "2");
         Files.createDirectories(dataFolder1.toPath());
@@ -405,7 +406,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testRemoveUnfinishedLeftovers_abort() throws Throwable
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstableOld = sstable(dataFolder, cfs, 0, 128);
         SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128);
 
@@ -424,10 +425,10 @@ public class LogTransactionTest extends AbstractTransactionalTest
         Assert.assertEquals(tmpFiles, getTemporaryFiles(sstableNew.descriptor.directory));
 
         // normally called at startup
-        LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+        LogTransaction.removeUnfinishedLeftovers(cfs.metadata());
 
         // sstableOld should be only table left
-        Directories directories = new Directories(cfs.metadata);
+        Directories directories = new Directories(cfs.metadata());
         Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
         assertEquals(1, sstables.size());
 
@@ -442,7 +443,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testRemoveUnfinishedLeftovers_commit() throws Throwable
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstableOld = sstable(dataFolder, cfs, 0, 128);
         SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128);
 
@@ -464,10 +465,10 @@ public class LogTransactionTest extends AbstractTransactionalTest
         Assert.assertEquals(tmpFiles, getTemporaryFiles(sstableOld.descriptor.directory));
 
         // normally called at startup
-        LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+        LogTransaction.removeUnfinishedLeftovers(cfs.metadata());
 
         // sstableNew should be only table left
-        Directories directories = new Directories(cfs.metadata);
+        Directories directories = new Directories(cfs.metadata());
         Map<Descriptor, Set<Component>> sstables = directories.sstableLister(Directories.OnTxnErr.THROW).list();
         assertEquals(1, sstables.size());
 
@@ -483,7 +484,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
 
-        File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File origiFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         File dataFolder1 = new File(origiFolder, "1");
         File dataFolder2 = new File(origiFolder, "2");
         Files.createDirectories(dataFolder1.toPath());
@@ -534,7 +535,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
 
-        File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File origiFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         File dataFolder1 = new File(origiFolder, "1");
         File dataFolder2 = new File(origiFolder, "2");
         Files.createDirectories(dataFolder1.toPath());
@@ -709,7 +710,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
 
-        File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File origiFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         File dataFolder1 = new File(origiFolder, "1");
         File dataFolder2 = new File(origiFolder, "2");
         Files.createDirectories(dataFolder1.toPath());
@@ -766,7 +767,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testGetTemporaryFiles() throws IOException
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstable1 = sstable(dataFolder, cfs, 0, 128);
 
         Set<File> tmpFiles = getTemporaryFiles(dataFolder);
@@ -775,7 +776,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
 
         try(LogTransaction log = new LogTransaction(OperationType.WRITE))
         {
-            Directories directories = new Directories(cfs.metadata);
+            Directories directories = new Directories(cfs.metadata());
 
             File[] beforeSecondSSTable = dataFolder.listFiles(pathname -> !pathname.isDirectory());
 
@@ -834,7 +835,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
 
-        File origiFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File origiFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         File dataFolder1 = new File(origiFolder, "1");
         File dataFolder2 = new File(origiFolder, "2");
         Files.createDirectories(dataFolder1.toPath());
@@ -993,7 +994,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     private static void testCorruptRecord(BiConsumer<LogTransaction, SSTableReader> modifier, boolean isRecoverable) throws IOException
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstableOld = sstable(dataFolder, cfs, 0, 128);
         SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128);
 
@@ -1028,7 +1029,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
         { // the corruption is recoverable but the commit record is unreadable so the transaction is still in progress
 
             //This should remove new files
-            LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+            LogTransaction.removeUnfinishedLeftovers(cfs.metadata());
 
             // make sure to exclude the old files that were deleted by the modifier
             assertFiles(dataFolder.getPath(), oldFiles);
@@ -1037,7 +1038,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
         { // if an intermediate line was also modified, it should ignore the tx log file
 
             //This should not remove any files
-            LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+            LogTransaction.removeUnfinishedLeftovers(cfs.metadata());
 
             assertFiles(dataFolder.getPath(), Sets.newHashSet(Iterables.concat(newFiles,
                                                                                oldFiles,
@@ -1065,7 +1066,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     private static void testObsoletedFilesChanged(Consumer<SSTableReader> modifier) throws IOException
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstableOld = sstable(dataFolder, cfs, 0, 128);
         SSTableReader sstableNew = sstable(dataFolder, cfs, 1, 128);
 
@@ -1083,7 +1084,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
         log.txnFile().commit();
 
         //This should not remove the old files
-        LogTransaction.removeUnfinishedLeftovers(cfs.metadata);
+        LogTransaction.removeUnfinishedLeftovers(cfs.metadata());
 
         assertFiles(dataFolder.getPath(), Sets.newHashSet(Iterables.concat(
                                                                           sstableNew.getAllFilePaths(),
@@ -1108,7 +1109,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testGetTemporaryFilesSafeAfterObsoletion() throws Throwable
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);
 
         LogTransaction logs = new LogTransaction(OperationType.COMPACTION);
@@ -1132,7 +1133,7 @@ public class LogTransactionTest extends AbstractTransactionalTest
     public void testGetTemporaryFilesThrowsIfCompletingAfterObsoletion() throws Throwable
     {
         ColumnFamilyStore cfs = MockSchema.newCFS(KEYSPACE);
-        File dataFolder = new Directories(cfs.metadata).getDirectoryForNewSSTables();
+        File dataFolder = new Directories(cfs.metadata()).getDirectoryForNewSSTables();
         SSTableReader sstable = sstable(dataFolder, cfs, 0, 128);
 
         LogTransaction logs = new LogTransaction(OperationType.COMPACTION);
@@ -1180,9 +1181,9 @@ public class LogTransactionTest extends AbstractTransactionalTest
         FileHandle dFile = new FileHandle.Builder(descriptor.filenameFor(Component.DATA)).complete();
         FileHandle iFile = new FileHandle.Builder(descriptor.filenameFor(Component.PRIMARY_INDEX)).complete();
 
-        SerializationHeader header = SerializationHeader.make(cfs.metadata, Collections.emptyList());
-        StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
-                                                 .finalizeMetadata(cfs.metadata.partitioner.getClass().getCanonicalName(), 0.01f, -1, header)
+        SerializationHeader header = SerializationHeader.make(cfs.metadata(), Collections.emptyList());
+        StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata().comparator)
+                                                 .finalizeMetadata(cfs.metadata().partitioner.getClass().getCanonicalName(), 0.01f, -1, header)
                                                  .get(MetadataType.STATS);
         SSTableReader reader = SSTableReader.internalOpen(descriptor,
                                                           components,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java b/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
index b825f52..f4443f1 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/RealTransactionsTest.java
@@ -30,8 +30,8 @@ import org.junit.Test;
 
 import junit.framework.Assert;
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SerializationHeader;
@@ -158,13 +158,13 @@ public class RealTransactionsTest extends SchemaLoader
                 long lastCheckObsoletion = System.nanoTime();
                 File directory = txn.originals().iterator().next().descriptor.directory;
                 Descriptor desc = cfs.newSSTableDescriptor(directory);
-                CFMetaData metadata = Schema.instance.getCFMetaData(desc);
+                TableMetadataRef metadata = Schema.instance.getTableMetadataRef(desc);
                 rewriter.switchWriter(SSTableWriter.create(metadata,
                                                            desc,
                                                            0,
                                                            0,
                                                            0,
-                                                           SerializationHeader.make(cfs.metadata, txn.originals()),
+                                                           SerializationHeader.make(cfs.metadata(), txn.originals()),
                                                            cfs.indexManager.listIndexes(),
                                                            txn));
                 while (ci.hasNext())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
index 65c7d0e..a5a1baf 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -34,7 +34,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import junit.framework.Assert;
-import org.apache.cassandra.MockSchema;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Memtable;
@@ -43,6 +42,7 @@ import org.apache.cassandra.db.commitlog.CommitLogPosition;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.notifications.*;
+import org.apache.cassandra.schema.MockSchema;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
 import static com.google.common.collect.ImmutableSet.copyOf;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
index a0e6e5f..f092854 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
@@ -32,13 +32,13 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import junit.framework.Assert;
-import org.apache.cassandra.MockSchema;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Memtable;
 import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.MockSchema;
 
 import static com.google.common.collect.ImmutableSet.copyOf;
 import static com.google.common.collect.ImmutableSet.of;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
index 2bf4082..9813fd9 100644
--- a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
@@ -29,7 +29,7 @@ import static org.junit.Assert.assertEquals;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.Row;
@@ -38,7 +38,6 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.serializers.UTF8Serializer;
 import org.apache.cassandra.utils.*;
 
 public class CompositeTypeTest
@@ -190,13 +189,13 @@ public class CompositeTypeTest
         ByteBuffer key = ByteBufferUtil.bytes("k");
 
         long ts = FBUtilities.timestampMicros();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname5).add("val", "cname5").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname1).add("val", "cname1").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname4).add("val", "cname4").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname2).add("val", "cname2").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname3).add("val", "cname3").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname5).add("val", "cname5").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname1).add("val", "cname1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname4).add("val", "cname4").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname2).add("val", "cname2").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname3).add("val", "cname3").build().applyUnsafe();
 
-        ColumnDefinition cdef = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
+        ColumnMetadata cdef = cfs.metadata().getColumn(ByteBufferUtil.bytes("val"));
 
         ImmutableBTreePartition readPartition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build());
         Iterator<Row> iter = readPartition.iterator();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
index 0a3c39c..068daf6 100644
--- a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
@@ -31,7 +31,7 @@ import static org.junit.Assert.fail;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.Row;
@@ -193,13 +193,13 @@ public class DynamicCompositeTypeTest
 
         ByteBuffer key = ByteBufferUtil.bytes("k");
         long ts = FBUtilities.timestampMicros();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname5).add("val", "cname5").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname1).add("val", "cname1").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname4).add("val", "cname4").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname2).add("val", "cname2").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname3).add("val", "cname3").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname5).add("val", "cname5").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname1).add("val", "cname1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname4).add("val", "cname4").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname2).add("val", "cname2").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname3).add("val", "cname3").build().applyUnsafe();
 
-        ColumnDefinition cdef = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
+        ColumnMetadata cdef = cfs.metadata().getColumn(ByteBufferUtil.bytes("val"));
 
         ImmutableBTreePartition readPartition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build());
         Iterator<Row> iter = readPartition.iterator();
@@ -230,13 +230,13 @@ public class DynamicCompositeTypeTest
         ByteBuffer key = ByteBufferUtil.bytes("kr");
 
         long ts = FBUtilities.timestampMicros();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname5).add("val", "cname5").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname1).add("val", "cname1").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname4).add("val", "cname4").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname2).add("val", "cname2").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, ts, key).clustering(cname3).add("val", "cname3").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname5).add("val", "cname5").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname1).add("val", "cname1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname4).add("val", "cname4").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname2).add("val", "cname2").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), ts, key).clustering(cname3).add("val", "cname3").build().applyUnsafe();
 
-        ColumnDefinition cdef = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
+        ColumnMetadata cdef = cfs.metadata().getColumn(ByteBufferUtil.bytes("val"));
 
         ImmutableBTreePartition readPartition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, key).build());
         Iterator<Row> iter = readPartition.iterator();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java b/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
index 90d6310..7bdfdb5 100644
--- a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
+++ b/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
@@ -35,8 +35,8 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
@@ -62,7 +62,7 @@ public class PartitionImplementationTest
 
     private static final int TIMESTAMP = KEY_RANGE + 1;
 
-    private static CFMetaData cfm;
+    private static TableMetadata metadata;
     private Random rand = new Random(2);
 
     @BeforeClass
@@ -70,15 +70,15 @@ public class PartitionImplementationTest
     {
         SchemaLoader.prepareServer();
 
-        cfm = CFMetaData.Builder.create(KEYSPACE, CF)
-                                        .addPartitionKey("pk", AsciiType.instance)
-                                        .addClusteringColumn("ck", AsciiType.instance)
-                                        .addRegularColumn("col", AsciiType.instance)
-                                        .addStaticColumn("static_col", AsciiType.instance)
-                                        .build();
-        SchemaLoader.createKeyspace(KEYSPACE,
-                                    KeyspaceParams.simple(1),
-                                    cfm);
+        metadata =
+            TableMetadata.builder(KEYSPACE, CF)
+                         .addPartitionKeyColumn("pk", AsciiType.instance)
+                         .addClusteringColumn("ck", AsciiType.instance)
+                         .addRegularColumn("col", AsciiType.instance)
+                         .addStaticColumn("static_col", AsciiType.instance)
+                         .build();
+
+        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), metadata);
     }
 
     private List<Row> generateRows()
@@ -100,7 +100,7 @@ public class PartitionImplementationTest
 
     Row makeRow(Clustering clustering, String colValue)
     {
-        ColumnDefinition defCol = cfm.getColumnDefinition(new ColumnIdentifier("col", true));
+        ColumnMetadata defCol = metadata.getColumn(new ColumnIdentifier("col", true));
         Row.Builder row = BTreeRow.unsortedBuilder(TIMESTAMP);
         row.newRow(clustering);
         row.addCell(BufferCell.live(defCol, TIMESTAMP, ByteBufferUtil.bytes(colValue)));
@@ -109,7 +109,7 @@ public class PartitionImplementationTest
 
     Row makeStaticRow()
     {
-        ColumnDefinition defCol = cfm.getColumnDefinition(new ColumnIdentifier("static_col", true));
+        ColumnMetadata defCol = metadata.getColumn(new ColumnIdentifier("static_col", true));
         Row.Builder row = BTreeRow.unsortedBuilder(TIMESTAMP);
         row.newRow(Clustering.STATIC_CLUSTERING);
         row.addCell(BufferCell.live(defCol, TIMESTAMP, ByteBufferUtil.bytes("static value")));
@@ -148,7 +148,7 @@ public class PartitionImplementationTest
             markers.add(open);
             markers.add(close);
         }
-        markers.sort(cfm.comparator);
+        markers.sort(metadata.comparator);
 
         RangeTombstoneMarker toAdd = null;
         Set<DeletionTime> open = new HashSet<>();
@@ -163,7 +163,7 @@ public class PartitionImplementationTest
                 {
                     if (toAdd != null)
                     {
-                        if (cfm.comparator.compare(toAdd, marker) != 0)
+                        if (metadata.comparator.compare(toAdd, marker) != 0)
                             content.add(toAdd);
                         else
                         {
@@ -187,7 +187,7 @@ public class PartitionImplementationTest
                 {
                     if (toAdd != null)
                     {
-                        if (cfm.comparator.compare(toAdd, marker) != 0)
+                        if (metadata.comparator.compare(toAdd, marker) != 0)
                             content.add(toAdd);
                         else
                         {
@@ -212,7 +212,7 @@ public class PartitionImplementationTest
 
     private Clustering clustering(int i)
     {
-        return cfm.comparator.make(String.format("Row%06d", i));
+        return metadata.comparator.make(String.format("Row%06d", i));
     }
 
     private void test(Supplier<Collection<? extends Unfiltered>> content, Row staticRow)
@@ -233,18 +233,18 @@ public class PartitionImplementationTest
 
     private void testIter(Supplier<Collection<? extends Unfiltered>> contentSupplier, Row staticRow)
     {
-        NavigableSet<Clusterable> sortedContent = new TreeSet<Clusterable>(cfm.comparator);
+        NavigableSet<Clusterable> sortedContent = new TreeSet<Clusterable>(metadata.comparator);
         sortedContent.addAll(contentSupplier.get());
         AbstractBTreePartition partition;
-        try (UnfilteredRowIterator iter = new Util.UnfilteredSource(cfm, Util.dk("pk"), staticRow, sortedContent.stream().map(x -> (Unfiltered) x).iterator()))
+        try (UnfilteredRowIterator iter = new Util.UnfilteredSource(metadata, Util.dk("pk"), staticRow, sortedContent.stream().map(x -> (Unfiltered) x).iterator()))
         {
             partition = ImmutableBTreePartition.create(iter);
         }
 
-        ColumnDefinition defCol = cfm.getColumnDefinition(new ColumnIdentifier("col", true));
+        ColumnMetadata defCol = metadata.getColumn(new ColumnIdentifier("col", true));
         ColumnFilter cf = ColumnFilter.selectionBuilder().add(defCol).build();
-        Function<? super Clusterable, ? extends Clusterable> colFilter = x -> x instanceof Row ? ((Row) x).filter(cf, cfm) : x;
-        Slices slices = Slices.with(cfm.comparator, Slice.make(clustering(KEY_RANGE / 4), clustering(KEY_RANGE * 3 / 4)));
+        Function<? super Clusterable, ? extends Clusterable> colFilter = x -> x instanceof Row ? ((Row) x).filter(cf, metadata) : x;
+        Slices slices = Slices.with(metadata.comparator, Slice.make(clustering(KEY_RANGE / 4), clustering(KEY_RANGE * 3 / 4)));
         Slices multiSlices = makeSlices();
 
         // lastRow
@@ -278,44 +278,44 @@ public class PartitionImplementationTest
 
         // unfiltered iterator
         assertIteratorsEqual(sortedContent.iterator(),
-                             partition.unfilteredIterator(ColumnFilter.all(cfm), Slices.ALL, false));
+                             partition.unfilteredIterator(ColumnFilter.all(metadata), Slices.ALL, false));
         // column-filtered
         assertIteratorsEqual(sortedContent.stream().map(colFilter).iterator(),
                              partition.unfilteredIterator(cf, Slices.ALL, false));
         // sliced
         assertIteratorsEqual(slice(sortedContent, slices.get(0)),
-                             partition.unfilteredIterator(ColumnFilter.all(cfm), slices, false));
+                             partition.unfilteredIterator(ColumnFilter.all(metadata), slices, false));
         assertIteratorsEqual(streamOf(slice(sortedContent, slices.get(0))).map(colFilter).iterator(),
                              partition.unfilteredIterator(cf, slices, false));
         // randomly multi-sliced
         assertIteratorsEqual(slice(sortedContent, multiSlices),
-                             partition.unfilteredIterator(ColumnFilter.all(cfm), multiSlices, false));
+                             partition.unfilteredIterator(ColumnFilter.all(metadata), multiSlices, false));
         assertIteratorsEqual(streamOf(slice(sortedContent, multiSlices)).map(colFilter).iterator(),
                              partition.unfilteredIterator(cf, multiSlices, false));
         // reversed
         assertIteratorsEqual(sortedContent.descendingIterator(),
-                             partition.unfilteredIterator(ColumnFilter.all(cfm), Slices.ALL, true));
+                             partition.unfilteredIterator(ColumnFilter.all(metadata), Slices.ALL, true));
         assertIteratorsEqual(sortedContent.descendingSet().stream().map(colFilter).iterator(),
                              partition.unfilteredIterator(cf, Slices.ALL, true));
         assertIteratorsEqual(invert(slice(sortedContent, slices.get(0))),
-                             partition.unfilteredIterator(ColumnFilter.all(cfm), slices, true));
+                             partition.unfilteredIterator(ColumnFilter.all(metadata), slices, true));
         assertIteratorsEqual(streamOf(invert(slice(sortedContent, slices.get(0)))).map(colFilter).iterator(),
                              partition.unfilteredIterator(cf, slices, true));
         assertIteratorsEqual(invert(slice(sortedContent, multiSlices)),
-                             partition.unfilteredIterator(ColumnFilter.all(cfm), multiSlices, true));
+                             partition.unfilteredIterator(ColumnFilter.all(metadata), multiSlices, true));
         assertIteratorsEqual(streamOf(invert(slice(sortedContent, multiSlices))).map(colFilter).iterator(),
                              partition.unfilteredIterator(cf, multiSlices, true));
 
         // search iterator
-        testSearchIterator(sortedContent, partition, ColumnFilter.all(cfm), false);
+        testSearchIterator(sortedContent, partition, ColumnFilter.all(metadata), false);
         testSearchIterator(sortedContent, partition, cf, false);
-        testSearchIterator(sortedContent, partition, ColumnFilter.all(cfm), true);
+        testSearchIterator(sortedContent, partition, ColumnFilter.all(metadata), true);
         testSearchIterator(sortedContent, partition, cf, true);
 
         // sliceable iter
-        testSlicingOfIterators(sortedContent, partition, ColumnFilter.all(cfm), false);
+        testSlicingOfIterators(sortedContent, partition, ColumnFilter.all(metadata), false);
         testSlicingOfIterators(sortedContent, partition, cf, false);
-        testSlicingOfIterators(sortedContent, partition, ColumnFilter.all(cfm), true);
+        testSlicingOfIterators(sortedContent, partition, ColumnFilter.all(metadata), true);
         testSlicingOfIterators(sortedContent, partition, cf, true);
     }
 
@@ -338,14 +338,14 @@ public class PartitionImplementationTest
             assertEquals(expected == null, row == null);
             if (row == null)
                 continue;
-            assertRowsEqual(expected.filter(cf, cfm), row);
+            assertRowsEqual(expected.filter(cf, metadata), row);
         }
     }
 
     Slices makeSlices()
     {
         int pos = 0;
-        Slices.Builder builder = new Slices.Builder(cfm.comparator);
+        Slices.Builder builder = new Slices.Builder(metadata.comparator);
         while (pos <= KEY_RANGE)
         {
             int skip = rand.nextInt(KEY_RANGE / 10) * (rand.nextInt(3) + 2 / 3); // increased chance of getting 0
@@ -362,13 +362,13 @@ public class PartitionImplementationTest
 
     void testSlicingOfIterators(NavigableSet<Clusterable> sortedContent, AbstractBTreePartition partition, ColumnFilter cf, boolean reversed)
     {
-        Function<? super Clusterable, ? extends Clusterable> colFilter = x -> x instanceof Row ? ((Row) x).filter(cf, cfm) : x;
+        Function<? super Clusterable, ? extends Clusterable> colFilter = x -> x instanceof Row ? ((Row) x).filter(cf, metadata) : x;
         Slices slices = makeSlices();
 
         // fetch each slice in turn
         for (Slice slice : (Iterable<Slice>) () -> directed(slices, reversed))
         {
-            try (UnfilteredRowIterator slicedIter = partition.unfilteredIterator(cf, Slices.with(cfm.comparator, slice), reversed))
+            try (UnfilteredRowIterator slicedIter = partition.unfilteredIterator(cf, Slices.with(metadata.comparator, slice), reversed))
             {
                 assertIteratorsEqual(streamOf(directed(slice(sortedContent, slice), reversed)).map(colFilter).iterator(),
                                      slicedIter);
@@ -441,8 +441,8 @@ public class PartitionImplementationTest
         Clusterable[] a2 = (Clusterable[]) Iterators.toArray(it2, Clusterable.class);
         if (Arrays.equals(a1, a2))
             return;
-        String a1s = Stream.of(a1).map(x -> "\n" + (x instanceof Unfiltered ? ((Unfiltered) x).toString(cfm) : x.toString())).collect(Collectors.toList()).toString();
-        String a2s = Stream.of(a2).map(x -> "\n" + (x instanceof Unfiltered ? ((Unfiltered) x).toString(cfm) : x.toString())).collect(Collectors.toList()).toString();
+        String a1s = Stream.of(a1).map(x -> "\n" + (x instanceof Unfiltered ? ((Unfiltered) x).toString(metadata) : x.toString())).collect(Collectors.toList()).toString();
+        String a2s = Stream.of(a2).map(x -> "\n" + (x instanceof Unfiltered ? ((Unfiltered) x).toString(metadata) : x.toString())).collect(Collectors.toList()).toString();
         assertArrayEquals("Arrays differ. Expected " + a1s + " was " + a2s, a1, a2);
     }
 
@@ -451,7 +451,7 @@ public class PartitionImplementationTest
         NavigableSet<Clusterable> nexts = sortedContent.tailSet(cl, true);
         if (nexts.isEmpty())
             return null;
-        Row row = nexts.first() instanceof Row && cfm.comparator.compare(cl, nexts.first()) == 0 ? (Row) nexts.first() : null;
+        Row row = nexts.first() instanceof Row && metadata.comparator.compare(cl, nexts.first()) == 0 ? (Row) nexts.first() : null;
         for (Clusterable next : nexts)
             if (next instanceof RangeTombstoneMarker)
             {
@@ -459,7 +459,7 @@ public class PartitionImplementationTest
                 if (!rt.isClose(false))
                     return row;
                 DeletionTime delTime = rt.closeDeletionTime(false);
-                return row == null ? BTreeRow.emptyDeletedRow(cl, Deletion.regular(delTime)) : row.filter(ColumnFilter.all(cfm), delTime, true, cfm);
+                return row == null ? BTreeRow.emptyDeletedRow(cl, Deletion.regular(delTime)) : row.filter(ColumnFilter.all(metadata), delTime, true, metadata);
             }
         return row;
     }


[34/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/Relation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Relation.java b/src/java/org/apache/cassandra/cql3/Relation.java
index 097b88e..1d27874 100644
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.cql3;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.restrictions.Restriction;
 import org.apache.cassandra.cql3.statements.Bound;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -132,31 +132,30 @@ public abstract class Relation
     /**
      * Converts this <code>Relation</code> into a <code>Restriction</code>.
      *
-     * @param cfm the Column Family meta data
+     * @param table the Column Family meta data
      * @param boundNames the variables specification where to collect the bind variables
      * @return the <code>Restriction</code> corresponding to this <code>Relation</code>
      * @throws InvalidRequestException if this <code>Relation</code> is not valid
      */
-    public final Restriction toRestriction(CFMetaData cfm,
-                                           VariableSpecifications boundNames) throws InvalidRequestException
+    public final Restriction toRestriction(TableMetadata table, VariableSpecifications boundNames)
     {
         switch (relationType)
         {
-            case EQ: return newEQRestriction(cfm, boundNames);
-            case LT: return newSliceRestriction(cfm, boundNames, Bound.END, false);
-            case LTE: return newSliceRestriction(cfm, boundNames, Bound.END, true);
-            case GTE: return newSliceRestriction(cfm, boundNames, Bound.START, true);
-            case GT: return newSliceRestriction(cfm, boundNames, Bound.START, false);
-            case IN: return newINRestriction(cfm, boundNames);
-            case CONTAINS: return newContainsRestriction(cfm, boundNames, false);
-            case CONTAINS_KEY: return newContainsRestriction(cfm, boundNames, true);
-            case IS_NOT: return newIsNotRestriction(cfm, boundNames);
+            case EQ: return newEQRestriction(table, boundNames);
+            case LT: return newSliceRestriction(table, boundNames, Bound.END, false);
+            case LTE: return newSliceRestriction(table, boundNames, Bound.END, true);
+            case GTE: return newSliceRestriction(table, boundNames, Bound.START, true);
+            case GT: return newSliceRestriction(table, boundNames, Bound.START, false);
+            case IN: return newINRestriction(table, boundNames);
+            case CONTAINS: return newContainsRestriction(table, boundNames, false);
+            case CONTAINS_KEY: return newContainsRestriction(table, boundNames, true);
+            case IS_NOT: return newIsNotRestriction(table, boundNames);
             case LIKE_PREFIX:
             case LIKE_SUFFIX:
             case LIKE_CONTAINS:
             case LIKE_MATCHES:
             case LIKE:
-                return newLikeRestriction(cfm, boundNames, relationType);
+                return newLikeRestriction(table, boundNames, relationType);
             default: throw invalidRequest("Unsupported \"!=\" relation: %s", this);
         }
     }
@@ -164,59 +163,52 @@ public abstract class Relation
     /**
      * Creates a new EQ restriction instance.
      *
-     * @param cfm the Column Family meta data
+     * @param table the table meta data
      * @param boundNames the variables specification where to collect the bind variables
      * @return a new EQ restriction instance.
      * @throws InvalidRequestException if the relation cannot be converted into an EQ restriction.
      */
-    protected abstract Restriction newEQRestriction(CFMetaData cfm,
-                                                    VariableSpecifications boundNames) throws InvalidRequestException;
+    protected abstract Restriction newEQRestriction(TableMetadata table, VariableSpecifications boundNames);
 
     /**
      * Creates a new IN restriction instance.
      *
-     * @param cfm the Column Family meta data
+     * @param table the table meta data
      * @param boundNames the variables specification where to collect the bind variables
      * @return a new IN restriction instance
      * @throws InvalidRequestException if the relation cannot be converted into an IN restriction.
      */
-    protected abstract Restriction newINRestriction(CFMetaData cfm,
-                                                    VariableSpecifications boundNames) throws InvalidRequestException;
+    protected abstract Restriction newINRestriction(TableMetadata table, VariableSpecifications boundNames);
 
     /**
      * Creates a new Slice restriction instance.
      *
-     * @param cfm the Column Family meta data
+     * @param table the table meta data
      * @param boundNames the variables specification where to collect the bind variables
      * @param bound the slice bound
      * @param inclusive <code>true</code> if the bound is included.
      * @return a new slice restriction instance
      * @throws InvalidRequestException if the <code>Relation</code> is not valid
      */
-    protected abstract Restriction newSliceRestriction(CFMetaData cfm,
+    protected abstract Restriction newSliceRestriction(TableMetadata table,
                                                        VariableSpecifications boundNames,
                                                        Bound bound,
-                                                       boolean inclusive) throws InvalidRequestException;
+                                                       boolean inclusive);
 
     /**
      * Creates a new Contains restriction instance.
      *
-     * @param cfm the Column Family meta data
+     * @param table the table meta data
      * @param boundNames the variables specification where to collect the bind variables
      * @param isKey <code>true</code> if the restriction to create is a CONTAINS KEY
      * @return a new Contains <code>Restriction</code> instance
      * @throws InvalidRequestException if the <code>Relation</code> is not valid
      */
-    protected abstract Restriction newContainsRestriction(CFMetaData cfm,
-                                                          VariableSpecifications boundNames,
-                                                          boolean isKey) throws InvalidRequestException;
+    protected abstract Restriction newContainsRestriction(TableMetadata table, VariableSpecifications boundNames, boolean isKey);
 
-    protected abstract Restriction newIsNotRestriction(CFMetaData cfm,
-                                                       VariableSpecifications boundNames) throws InvalidRequestException;
+    protected abstract Restriction newIsNotRestriction(TableMetadata table, VariableSpecifications boundNames);
 
-    protected abstract Restriction newLikeRestriction(CFMetaData cfm,
-                                                      VariableSpecifications boundNames,
-                                                      Operator operator) throws InvalidRequestException;
+    protected abstract Restriction newLikeRestriction(TableMetadata table, VariableSpecifications boundNames, Operator operator);
 
     /**
      * Converts the specified <code>Raw</code> into a <code>Term</code>.
@@ -231,8 +223,7 @@ public abstract class Relation
     protected abstract Term toTerm(List<? extends ColumnSpecification> receivers,
                                    Term.Raw raw,
                                    String keyspace,
-                                   VariableSpecifications boundNames)
-                                   throws InvalidRequestException;
+                                   VariableSpecifications boundNames);
 
     /**
      * Converts the specified <code>Raw</code> terms into a <code>Term</code>s.
@@ -247,7 +238,7 @@ public abstract class Relation
     protected final List<Term> toTerms(List<? extends ColumnSpecification> receivers,
                                        List<? extends Term.Raw> raws,
                                        String keyspace,
-                                       VariableSpecifications boundNames) throws InvalidRequestException
+                                       VariableSpecifications boundNames)
     {
         if (raws == null)
             return null;
@@ -266,5 +257,5 @@ public abstract class Relation
      * @return this object, if the old identifier is not in the set of entities that this relation covers; otherwise
      *         a new Relation with "from" replaced by "to" is returned.
      */
-    public abstract Relation renameIdentifier(ColumnDefinition.Raw from, ColumnDefinition.Raw to);
+    public abstract Relation renameIdentifier(ColumnMetadata.Raw from, ColumnMetadata.Raw to);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/Sets.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Sets.java b/src/java/org/apache/cassandra/cql3/Sets.java
index e79bda7..2920ed7 100644
--- a/src/java/org/apache/cassandra/cql3/Sets.java
+++ b/src/java/org/apache/cassandra/cql3/Sets.java
@@ -24,7 +24,7 @@ import java.util.*;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.rows.*;
@@ -301,7 +301,7 @@ public abstract class Sets
 
     public static class Setter extends Operation
     {
-        public Setter(ColumnDefinition column, Term t)
+        public Setter(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -321,7 +321,7 @@ public abstract class Sets
 
     public static class Adder extends Operation
     {
-        public Adder(ColumnDefinition column, Term t)
+        public Adder(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -334,7 +334,7 @@ public abstract class Sets
                 doAdd(value, column, params);
         }
 
-        static void doAdd(Term.Terminal value, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
+        static void doAdd(Term.Terminal value, ColumnMetadata column, UpdateParameters params) throws InvalidRequestException
         {
             if (column.type.isMultiCell())
             {
@@ -363,7 +363,7 @@ public abstract class Sets
     // Note that this is reused for Map subtraction too (we subtract a set from a map)
     public static class Discarder extends Operation
     {
-        public Discarder(ColumnDefinition column, Term t)
+        public Discarder(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -388,7 +388,7 @@ public abstract class Sets
 
     public static class ElementDiscarder extends Operation
     {
-        public ElementDiscarder(ColumnDefinition column, Term k)
+        public ElementDiscarder(ColumnMetadata column, Term k)
         {
             super(column, k);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
index 719ef68..412eb16 100644
--- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@ -21,8 +21,8 @@ import java.util.Collections;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.Term.Raw;
 import org.apache.cassandra.cql3.restrictions.Restriction;
 import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction;
@@ -44,12 +44,12 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.invalidReq
  */
 public final class SingleColumnRelation extends Relation
 {
-    private final ColumnDefinition.Raw entity;
+    private final ColumnMetadata.Raw entity;
     private final Term.Raw mapKey;
     private final Term.Raw value;
     private final List<Term.Raw> inValues;
 
-    private SingleColumnRelation(ColumnDefinition.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value, List<Term.Raw> inValues)
+    private SingleColumnRelation(ColumnMetadata.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value, List<Term.Raw> inValues)
     {
         this.entity = entity;
         this.mapKey = mapKey;
@@ -69,7 +69,7 @@ public final class SingleColumnRelation extends Relation
      * @param type the type that describes how this entity relates to the value.
      * @param value the value being compared.
      */
-    public SingleColumnRelation(ColumnDefinition.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value)
+    public SingleColumnRelation(ColumnMetadata.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value)
     {
         this(entity, mapKey, type, value, null);
     }
@@ -81,7 +81,7 @@ public final class SingleColumnRelation extends Relation
      * @param type the type that describes how this entity relates to the value.
      * @param value the value being compared.
      */
-    public SingleColumnRelation(ColumnDefinition.Raw entity, Operator type, Term.Raw value)
+    public SingleColumnRelation(ColumnMetadata.Raw entity, Operator type, Term.Raw value)
     {
         this(entity, null, type, value);
     }
@@ -96,12 +96,12 @@ public final class SingleColumnRelation extends Relation
         return inValues;
     }
 
-    public static SingleColumnRelation createInRelation(ColumnDefinition.Raw entity, List<Term.Raw> inValues)
+    public static SingleColumnRelation createInRelation(ColumnMetadata.Raw entity, List<Term.Raw> inValues)
     {
         return new SingleColumnRelation(entity, null, Operator.IN, null, inValues);
     }
 
-    public ColumnDefinition.Raw getEntity()
+    public ColumnMetadata.Raw getEntity()
     {
         return entity;
     }
@@ -135,7 +135,7 @@ public final class SingleColumnRelation extends Relation
         }
     }
 
-    public Relation renameIdentifier(ColumnDefinition.Raw from, ColumnDefinition.Raw to)
+    public Relation renameIdentifier(ColumnMetadata.Raw from, ColumnMetadata.Raw to)
     {
         return entity.equals(from)
                ? new SingleColumnRelation(to, mapKey, operator(), value, inValues)
@@ -156,31 +156,29 @@ public final class SingleColumnRelation extends Relation
     }
 
     @Override
-    protected Restriction newEQRestriction(CFMetaData cfm,
-                                           VariableSpecifications boundNames) throws InvalidRequestException
+    protected Restriction newEQRestriction(TableMetadata table, VariableSpecifications boundNames)
     {
-        ColumnDefinition columnDef = entity.prepare(cfm);
+        ColumnMetadata columnDef = entity.prepare(table);
         if (mapKey == null)
         {
-            Term term = toTerm(toReceivers(columnDef), value, cfm.ksName, boundNames);
+            Term term = toTerm(toReceivers(columnDef), value, table.keyspace, boundNames);
             return new SingleColumnRestriction.EQRestriction(columnDef, term);
         }
         List<? extends ColumnSpecification> receivers = toReceivers(columnDef);
-        Term entryKey = toTerm(Collections.singletonList(receivers.get(0)), mapKey, cfm.ksName, boundNames);
-        Term entryValue = toTerm(Collections.singletonList(receivers.get(1)), value, cfm.ksName, boundNames);
+        Term entryKey = toTerm(Collections.singletonList(receivers.get(0)), mapKey, table.keyspace, boundNames);
+        Term entryValue = toTerm(Collections.singletonList(receivers.get(1)), value, table.keyspace, boundNames);
         return new SingleColumnRestriction.ContainsRestriction(columnDef, entryKey, entryValue);
     }
 
     @Override
-    protected Restriction newINRestriction(CFMetaData cfm,
-                                           VariableSpecifications boundNames) throws InvalidRequestException
+    protected Restriction newINRestriction(TableMetadata table, VariableSpecifications boundNames)
     {
-        ColumnDefinition columnDef = entity.prepare(cfm);
+        ColumnMetadata columnDef = entity.prepare(table);
         List<? extends ColumnSpecification> receivers = toReceivers(columnDef);
-        List<Term> terms = toTerms(receivers, inValues, cfm.ksName, boundNames);
+        List<Term> terms = toTerms(receivers, inValues, table.keyspace, boundNames);
         if (terms == null)
         {
-            Term term = toTerm(receivers, value, cfm.ksName, boundNames);
+            Term term = toTerm(receivers, value, table.keyspace, boundNames);
             return new SingleColumnRestriction.InRestrictionWithMarker(columnDef, (Lists.Marker) term);
         }
 
@@ -192,46 +190,46 @@ public final class SingleColumnRelation extends Relation
     }
 
     @Override
-    protected Restriction newSliceRestriction(CFMetaData cfm,
+    protected Restriction newSliceRestriction(TableMetadata table,
                                               VariableSpecifications boundNames,
                                               Bound bound,
-                                              boolean inclusive) throws InvalidRequestException
+                                              boolean inclusive)
     {
-        ColumnDefinition columnDef = entity.prepare(cfm);
+        ColumnMetadata columnDef = entity.prepare(table);
         checkFalse(columnDef.type instanceof DurationType, "Slice restriction are not supported on duration columns");
 
-        Term term = toTerm(toReceivers(columnDef), value, cfm.ksName, boundNames);
+        Term term = toTerm(toReceivers(columnDef), value, table.keyspace, boundNames);
         return new SingleColumnRestriction.SliceRestriction(columnDef, bound, inclusive, term);
     }
 
     @Override
-    protected Restriction newContainsRestriction(CFMetaData cfm,
+    protected Restriction newContainsRestriction(TableMetadata table,
                                                  VariableSpecifications boundNames,
                                                  boolean isKey) throws InvalidRequestException
     {
-        ColumnDefinition columnDef = entity.prepare(cfm);
-        Term term = toTerm(toReceivers(columnDef), value, cfm.ksName, boundNames);
+        ColumnMetadata columnDef = entity.prepare(table);
+        Term term = toTerm(toReceivers(columnDef), value, table.keyspace, boundNames);
         return new SingleColumnRestriction.ContainsRestriction(columnDef, term, isKey);
     }
 
     @Override
-    protected Restriction newIsNotRestriction(CFMetaData cfm,
+    protected Restriction newIsNotRestriction(TableMetadata table,
                                               VariableSpecifications boundNames) throws InvalidRequestException
     {
-        ColumnDefinition columnDef = entity.prepare(cfm);
+        ColumnMetadata columnDef = entity.prepare(table);
         // currently enforced by the grammar
         assert value == Constants.NULL_LITERAL : "Expected null literal for IS NOT relation: " + this.toString();
         return new SingleColumnRestriction.IsNotNullRestriction(columnDef);
     }
 
     @Override
-    protected Restriction newLikeRestriction(CFMetaData cfm, VariableSpecifications boundNames, Operator operator) throws InvalidRequestException
+    protected Restriction newLikeRestriction(TableMetadata table, VariableSpecifications boundNames, Operator operator)
     {
         if (mapKey != null)
             throw invalidRequest("%s can't be used with collections.", operator());
 
-        ColumnDefinition columnDef = entity.prepare(cfm);
-        Term term = toTerm(toReceivers(columnDef), value, cfm.ksName, boundNames);
+        ColumnMetadata columnDef = entity.prepare(table);
+        Term term = toTerm(toReceivers(columnDef), value, table.keyspace, boundNames);
 
         return new SingleColumnRestriction.LikeRestriction(columnDef, operator, term);
     }
@@ -242,7 +240,7 @@ public final class SingleColumnRelation extends Relation
      * @return the receivers for the specified relation.
      * @throws InvalidRequestException if the relation is invalid
      */
-    private List<? extends ColumnSpecification> toReceivers(ColumnDefinition columnDef) throws InvalidRequestException
+    private List<? extends ColumnSpecification> toReceivers(ColumnMetadata columnDef) throws InvalidRequestException
     {
         ColumnSpecification receiver = columnDef;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/TokenRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/TokenRelation.java b/src/java/org/apache/cassandra/cql3/TokenRelation.java
index 42464ef..62f603b 100644
--- a/src/java/org/apache/cassandra/cql3/TokenRelation.java
+++ b/src/java/org/apache/cassandra/cql3/TokenRelation.java
@@ -24,8 +24,8 @@ import java.util.stream.Collectors;
 
 import com.google.common.base.Joiner;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.Term.Raw;
 import org.apache.cassandra.cql3.restrictions.Restriction;
 import org.apache.cassandra.cql3.restrictions.TokenRestriction;
@@ -47,11 +47,11 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.invalidReq
  */
 public final class TokenRelation extends Relation
 {
-    private final List<ColumnDefinition.Raw> entities;
+    private final List<ColumnMetadata.Raw> entities;
 
     private final Term.Raw value;
 
-    public TokenRelation(List<ColumnDefinition.Raw> entities, Operator type, Term.Raw value)
+    public TokenRelation(List<ColumnMetadata.Raw> entities, Operator type, Term.Raw value)
     {
         this.entities = entities;
         this.relationType = type;
@@ -75,44 +75,44 @@ public final class TokenRelation extends Relation
     }
 
     @Override
-    protected Restriction newEQRestriction(CFMetaData cfm, VariableSpecifications boundNames) throws InvalidRequestException
+    protected Restriction newEQRestriction(TableMetadata table, VariableSpecifications boundNames)
     {
-        List<ColumnDefinition> columnDefs = getColumnDefinitions(cfm);
-        Term term = toTerm(toReceivers(cfm, columnDefs), value, cfm.ksName, boundNames);
-        return new TokenRestriction.EQRestriction(cfm, columnDefs, term);
+        List<ColumnMetadata> columnDefs = getColumnDefinitions(table);
+        Term term = toTerm(toReceivers(table, columnDefs), value, table.keyspace, boundNames);
+        return new TokenRestriction.EQRestriction(table, columnDefs, term);
     }
 
     @Override
-    protected Restriction newINRestriction(CFMetaData cfm, VariableSpecifications boundNames) throws InvalidRequestException
+    protected Restriction newINRestriction(TableMetadata table, VariableSpecifications boundNames)
     {
         throw invalidRequest("%s cannot be used with the token function", operator());
     }
 
     @Override
-    protected Restriction newSliceRestriction(CFMetaData cfm,
+    protected Restriction newSliceRestriction(TableMetadata table,
                                               VariableSpecifications boundNames,
                                               Bound bound,
-                                              boolean inclusive) throws InvalidRequestException
+                                              boolean inclusive)
     {
-        List<ColumnDefinition> columnDefs = getColumnDefinitions(cfm);
-        Term term = toTerm(toReceivers(cfm, columnDefs), value, cfm.ksName, boundNames);
-        return new TokenRestriction.SliceRestriction(cfm, columnDefs, bound, inclusive, term);
+        List<ColumnMetadata> columnDefs = getColumnDefinitions(table);
+        Term term = toTerm(toReceivers(table, columnDefs), value, table.keyspace, boundNames);
+        return new TokenRestriction.SliceRestriction(table, columnDefs, bound, inclusive, term);
     }
 
     @Override
-    protected Restriction newContainsRestriction(CFMetaData cfm, VariableSpecifications boundNames, boolean isKey) throws InvalidRequestException
+    protected Restriction newContainsRestriction(TableMetadata table, VariableSpecifications boundNames, boolean isKey)
     {
         throw invalidRequest("%s cannot be used with the token function", operator());
     }
 
     @Override
-    protected Restriction newIsNotRestriction(CFMetaData cfm, VariableSpecifications boundNames) throws InvalidRequestException
+    protected Restriction newIsNotRestriction(TableMetadata table, VariableSpecifications boundNames)
     {
         throw invalidRequest("%s cannot be used with the token function", operator());
     }
 
     @Override
-    protected Restriction newLikeRestriction(CFMetaData cfm, VariableSpecifications boundNames, Operator operator) throws InvalidRequestException
+    protected Restriction newLikeRestriction(TableMetadata table, VariableSpecifications boundNames, Operator operator)
     {
         throw invalidRequest("%s cannot be used with the token function", operator);
     }
@@ -128,12 +128,12 @@ public final class TokenRelation extends Relation
         return term;
     }
 
-    public Relation renameIdentifier(ColumnDefinition.Raw from, ColumnDefinition.Raw to)
+    public Relation renameIdentifier(ColumnMetadata.Raw from, ColumnMetadata.Raw to)
     {
         if (!entities.contains(from))
             return this;
 
-        List<ColumnDefinition.Raw> newEntities = entities.stream().map(e -> e.equals(from) ? to : e).collect(Collectors.toList());
+        List<ColumnMetadata.Raw> newEntities = entities.stream().map(e -> e.equals(from) ? to : e).collect(Collectors.toList());
         return new TokenRelation(newEntities, operator(), value);
     }
 
@@ -146,48 +146,48 @@ public final class TokenRelation extends Relation
     /**
      * Returns the definition of the columns to which apply the token restriction.
      *
-     * @param cfm the column family metadata
+     * @param table the table metadata
      * @return the definition of the columns to which apply the token restriction.
      * @throws InvalidRequestException if the entity cannot be resolved
      */
-    private List<ColumnDefinition> getColumnDefinitions(CFMetaData cfm) throws InvalidRequestException
+    private List<ColumnMetadata> getColumnDefinitions(TableMetadata table)
     {
-        List<ColumnDefinition> columnDefs = new ArrayList<>(entities.size());
-        for ( ColumnDefinition.Raw raw : entities)
-            columnDefs.add(raw.prepare(cfm));
+        List<ColumnMetadata> columnDefs = new ArrayList<>(entities.size());
+        for ( ColumnMetadata.Raw raw : entities)
+            columnDefs.add(raw.prepare(table));
         return columnDefs;
     }
 
     /**
      * Returns the receivers for this relation.
      *
-     * @param cfm the Column Family meta data
+     * @param table the table meta data
      * @param columnDefs the column definitions
      * @return the receivers for the specified relation.
      * @throws InvalidRequestException if the relation is invalid
      */
-    private static List<? extends ColumnSpecification> toReceivers(CFMetaData cfm,
-                                                                   List<ColumnDefinition> columnDefs)
+    private static List<? extends ColumnSpecification> toReceivers(TableMetadata table,
+                                                                   List<ColumnMetadata> columnDefs)
                                                                    throws InvalidRequestException
     {
 
-        if (!columnDefs.equals(cfm.partitionKeyColumns()))
+        if (!columnDefs.equals(table.partitionKeyColumns()))
         {
-            checkTrue(columnDefs.containsAll(cfm.partitionKeyColumns()),
+            checkTrue(columnDefs.containsAll(table.partitionKeyColumns()),
                       "The token() function must be applied to all partition key components or none of them");
 
             checkContainsNoDuplicates(columnDefs, "The token() function contains duplicate partition key components");
 
-            checkContainsOnly(columnDefs, cfm.partitionKeyColumns(), "The token() function must contains only partition key components");
+            checkContainsOnly(columnDefs, table.partitionKeyColumns(), "The token() function must contains only partition key components");
 
             throw invalidRequest("The token function arguments must be in the partition key order: %s",
-                                 Joiner.on(", ").join(ColumnDefinition.toIdentifiers(cfm.partitionKeyColumns())));
+                                 Joiner.on(", ").join(ColumnMetadata.toIdentifiers(table.partitionKeyColumns())));
         }
 
-        ColumnDefinition firstColumn = columnDefs.get(0);
+        ColumnMetadata firstColumn = columnDefs.get(0);
         return Collections.singletonList(new ColumnSpecification(firstColumn.ksName,
                                                                  firstColumn.cfName,
                                                                  new ColumnIdentifier("partition key token", true),
-                                                                 cfm.partitioner.getTokenValidator()));
+                                                                 table.partitioner.getTokenValidator()));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
index d896071..9733830 100644
--- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
@@ -22,17 +22,16 @@ import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.transport.ProtocolVersion;
-import org.apache.cassandra.utils.AbstractIterator;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.*;
-import org.apache.cassandra.db.partitions.PartitionIterator;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.pager.QueryPager;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.AbstractIterator;
 import org.apache.cassandra.utils.FBUtilities;
 
 /** a utility for doing internal cql-based queries */
@@ -221,19 +220,19 @@ public abstract class UntypedResultSet implements Iterable<UntypedResultSet.Row>
                 data.put(names.get(i).name.toString(), columns.get(i));
         }
 
-        public static Row fromInternalRow(CFMetaData metadata, DecoratedKey key, org.apache.cassandra.db.rows.Row row)
+        public static Row fromInternalRow(TableMetadata metadata, DecoratedKey key, org.apache.cassandra.db.rows.Row row)
         {
             Map<String, ByteBuffer> data = new HashMap<>();
 
             ByteBuffer[] keyComponents = SelectStatement.getComponents(metadata, key);
-            for (ColumnDefinition def : metadata.partitionKeyColumns())
+            for (ColumnMetadata def : metadata.partitionKeyColumns())
                 data.put(def.name.toString(), keyComponents[def.position()]);
 
             Clustering clustering = row.clustering();
-            for (ColumnDefinition def : metadata.clusteringColumns())
+            for (ColumnMetadata def : metadata.clusteringColumns())
                 data.put(def.name.toString(), clustering.get(def.position()));
 
-            for (ColumnDefinition def : metadata.partitionColumns())
+            for (ColumnMetadata def : metadata.regularAndStaticColumns())
             {
                 if (def.isSimple())
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/UpdateParameters.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
index d065269..0be8f51 100644
--- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java
+++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.cql3;
 import java.nio.ByteBuffer;
 import java.util.Map;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.ColumnFilter;
@@ -35,8 +35,8 @@ import org.apache.cassandra.utils.FBUtilities;
  */
 public class UpdateParameters
 {
-    public final CFMetaData metadata;
-    public final PartitionColumns updatedColumns;
+    public final TableMetadata metadata;
+    public final RegularAndStaticColumns updatedColumns;
     public final QueryOptions options;
 
     private final int nowInSec;
@@ -54,8 +54,8 @@ public class UpdateParameters
     // The builder currently in use. Will alias either staticBuilder or regularBuilder, which are themselves built lazily.
     private Row.Builder builder;
 
-    public UpdateParameters(CFMetaData metadata,
-                            PartitionColumns updatedColumns,
+    public UpdateParameters(TableMetadata metadata,
+                            RegularAndStaticColumns updatedColumns,
                             QueryOptions options,
                             long timestamp,
                             int ttl,
@@ -124,27 +124,27 @@ public class UpdateParameters
         // the "compact" one. As such, deleting the row or deleting that single cell is equivalent. We favor the later
         // for backward compatibility (thought it doesn't truly matter anymore).
         if (metadata.isCompactTable() && builder.clustering() != Clustering.STATIC_CLUSTERING)
-            addTombstone(metadata.compactValueColumn());
+            addTombstone(metadata.compactValueColumn);
         else
             builder.addRowDeletion(Row.Deletion.regular(deletionTime));
     }
 
-    public void addTombstone(ColumnDefinition column) throws InvalidRequestException
+    public void addTombstone(ColumnMetadata column) throws InvalidRequestException
     {
         addTombstone(column, null);
     }
 
-    public void addTombstone(ColumnDefinition column, CellPath path) throws InvalidRequestException
+    public void addTombstone(ColumnMetadata column, CellPath path) throws InvalidRequestException
     {
         builder.addCell(BufferCell.tombstone(column, timestamp, nowInSec, path));
     }
 
-    public void addCell(ColumnDefinition column, ByteBuffer value) throws InvalidRequestException
+    public void addCell(ColumnMetadata column, ByteBuffer value) throws InvalidRequestException
     {
         addCell(column, null, value);
     }
 
-    public void addCell(ColumnDefinition column, CellPath path, ByteBuffer value) throws InvalidRequestException
+    public void addCell(ColumnMetadata column, CellPath path, ByteBuffer value) throws InvalidRequestException
     {
         Cell cell = ttl == LivenessInfo.NO_TTL
                   ? BufferCell.live(column, timestamp, value, path)
@@ -152,7 +152,7 @@ public class UpdateParameters
         builder.addCell(cell);
     }
 
-    public void addCounter(ColumnDefinition column, long increment) throws InvalidRequestException
+    public void addCounter(ColumnMetadata column, long increment) throws InvalidRequestException
     {
         assert ttl == LivenessInfo.NO_TTL;
 
@@ -168,12 +168,12 @@ public class UpdateParameters
         builder.addCell(BufferCell.live(column, timestamp, CounterContext.instance().createLocal(increment)));
     }
 
-    public void setComplexDeletionTime(ColumnDefinition column)
+    public void setComplexDeletionTime(ColumnMetadata column)
     {
         builder.addComplexDeletion(column, deletionTime);
     }
 
-    public void setComplexDeletionTimeForOverwrite(ColumnDefinition column)
+    public void setComplexDeletionTimeForOverwrite(ColumnMetadata column)
     {
         builder.addComplexDeletion(column, new DeletionTime(deletionTime.markedForDeleteAt() - 1, deletionTime.localDeletionTime()));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/UserTypes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UserTypes.java b/src/java/org/apache/cassandra/cql3/UserTypes.java
index c8ddad0..9b87352 100644
--- a/src/java/org/apache/cassandra/cql3/UserTypes.java
+++ b/src/java/org/apache/cassandra/cql3/UserTypes.java
@@ -21,7 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.stream.Collectors;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.*;
@@ -315,7 +315,7 @@ public abstract class UserTypes
 
     public static class Setter extends Operation
     {
-        public Setter(ColumnDefinition column, Term t)
+        public Setter(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -361,7 +361,7 @@ public abstract class UserTypes
     {
         private final FieldIdentifier field;
 
-        public SetterByField(ColumnDefinition column, FieldIdentifier field, Term t)
+        public SetterByField(ColumnMetadata column, FieldIdentifier field, Term t)
         {
             super(column, t);
             this.field = field;
@@ -388,7 +388,7 @@ public abstract class UserTypes
     {
         private final FieldIdentifier field;
 
-        public DeleterByField(ColumnDefinition column, FieldIdentifier field)
+        public DeleterByField(ColumnMetadata column, FieldIdentifier field)
         {
             super(column, null);
             this.field = field;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/Validation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Validation.java b/src/java/org/apache/cassandra/cql3/Validation.java
index 3f388eb..34a4027 100644
--- a/src/java/org/apache/cassandra/cql3/Validation.java
+++ b/src/java/org/apache/cassandra/cql3/Validation.java
@@ -19,12 +19,9 @@ package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
-import org.apache.cassandra.db.KeyspaceNotDefinedException;
-import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -36,37 +33,6 @@ import org.apache.cassandra.utils.FBUtilities;
  */
 public abstract class Validation
 {
-    /**
-     * Retrieves the metadata for the provided keyspace and table name, throwing
-     * a meaningful user exception if those doen't exist.
-     *
-     * @param keyspaceName the keyspace name.
-     * @param tableName the table name.
-     * @return the metadata for table {@code keyspaceName.tableName} if it
-     * exists (otherwise an {@code InvalidRequestException} is thrown).
-     *
-     * @throws InvalidRequestException if the table requested doesn't exist.
-     */
-    public static CFMetaData validateColumnFamily(String keyspaceName, String tableName)
-    throws InvalidRequestException
-    {
-        validateKeyspace(keyspaceName);
-        if (tableName.isEmpty())
-            throw new InvalidRequestException("non-empty table is required");
-
-        CFMetaData metadata = Schema.instance.getCFMetaData(keyspaceName, tableName);
-        if (metadata == null)
-            throw new InvalidRequestException("unconfigured table " + tableName);
-
-        return metadata;
-    }
-
-    private static void validateKeyspace(String keyspaceName)
-    throws KeyspaceNotDefinedException
-    {
-        if (!Schema.instance.getKeyspaces().contains(keyspaceName))
-            throw new KeyspaceNotDefinedException("Keyspace " + keyspaceName + " does not exist");
-    }
 
     /**
      * Validates a (full serialized) partition key.
@@ -76,8 +42,7 @@ public abstract class Validation
      *
      * @throws InvalidRequestException if the provided {@code key} is invalid.
      */
-    public static void validateKey(CFMetaData metadata, ByteBuffer key)
-    throws InvalidRequestException
+    public static void validateKey(TableMetadata metadata, ByteBuffer key)
     {
         if (key == null || key.remaining() == 0)
             throw new InvalidRequestException("Key may not be empty");
@@ -92,26 +57,11 @@ public abstract class Validation
 
         try
         {
-            metadata.getKeyValidator().validate(key);
+            metadata.partitionKeyType.validate(key);
         }
         catch (MarshalException e)
         {
             throw new InvalidRequestException(e.getMessage());
         }
     }
-
-    /**
-     * Validates that the provided keyspace is not one of the system keyspace.
-     *
-     * @param keyspace the keyspace name to validate.
-     *
-     * @throws InvalidRequestException if {@code keyspace} is the name of a
-     * system keyspace.
-     */
-    public static void validateKeyspaceNotSystem(String keyspace)
-    throws InvalidRequestException
-    {
-        if (SchemaConstants.isSystemKeyspace(keyspace))
-            throw new InvalidRequestException(String.format("%s keyspace is not user-modifiable", keyspace));
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/VariableSpecifications.java b/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
index 24f71e4..96290a6 100644
--- a/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
+++ b/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
@@ -17,24 +17,24 @@
  */
 package org.apache.cassandra.cql3;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+
 public class VariableSpecifications
 {
     private final List<ColumnIdentifier> variableNames;
     private final ColumnSpecification[] specs;
-    private final ColumnDefinition[] targetColumns;
+    private final ColumnMetadata[] targetColumns;
 
     public VariableSpecifications(List<ColumnIdentifier> variableNames)
     {
         this.variableNames = variableNames;
         this.specs = new ColumnSpecification[variableNames.size()];
-        this.targetColumns = new ColumnDefinition[variableNames.size()];
+        this.targetColumns = new ColumnMetadata[variableNames.size()];
     }
 
     /**
@@ -58,21 +58,21 @@ public class VariableSpecifications
 
     /**
      * Returns an array with the same length as the number of partition key columns for the table corresponding
-     * to cfm.  Each short in the array represents the bind index of the marker that holds the value for that
+     * to table.  Each short in the array represents the bind index of the marker that holds the value for that
      * partition key column.  If there are no bind markers for any of the partition key columns, null is returned.
      *
      * Callers of this method should ensure that all statements operate on the same table.
      */
-    public short[] getPartitionKeyBindIndexes(CFMetaData cfm)
+    public short[] getPartitionKeyBindIndexes(TableMetadata metadata)
     {
-        short[] partitionKeyPositions = new short[cfm.partitionKeyColumns().size()];
+        short[] partitionKeyPositions = new short[metadata.partitionKeyColumns().size()];
         boolean[] set = new boolean[partitionKeyPositions.length];
         for (int i = 0; i < targetColumns.length; i++)
         {
-            ColumnDefinition targetColumn = targetColumns[i];
+            ColumnMetadata targetColumn = targetColumns[i];
             if (targetColumn != null && targetColumn.isPartitionKey())
             {
-                assert targetColumn.ksName.equals(cfm.ksName) && targetColumn.cfName.equals(cfm.cfName);
+                assert targetColumn.ksName.equals(metadata.keyspace) && targetColumn.cfName.equals(metadata.name);
                 partitionKeyPositions[targetColumn.position()] = (short) i;
                 set[targetColumn.position()] = true;
             }
@@ -87,8 +87,8 @@ public class VariableSpecifications
 
     public void add(int bindIndex, ColumnSpecification spec)
     {
-        if (spec instanceof ColumnDefinition)
-            targetColumns[bindIndex] = (ColumnDefinition) spec;
+        if (spec instanceof ColumnMetadata)
+            targetColumns[bindIndex] = (ColumnMetadata) spec;
 
         ColumnIdentifier bindMarkerName = variableNames.get(bindIndex);
         // Use the user name, if there is one

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/conditions/AbstractConditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/conditions/AbstractConditions.java b/src/java/org/apache/cassandra/cql3/conditions/AbstractConditions.java
index 9b6dd7e..0e2646e 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/AbstractConditions.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/AbstractConditions.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.cql3.conditions;
 
 import java.util.List;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.functions.Function;
 
 /**
@@ -32,7 +32,7 @@ abstract class AbstractConditions implements Conditions
     {
     }
 
-    public Iterable<ColumnDefinition> getColumns()
+    public Iterable<ColumnMetadata> getColumns()
     {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
index cfd62f5..31b7185 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/ColumnCondition.java
@@ -22,13 +22,13 @@ import java.util.*;
 
 import com.google.common.collect.Iterators;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.Term.Terminal;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -39,11 +39,11 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.*;
  */
 public abstract class ColumnCondition
 {
-    public final ColumnDefinition column;
+    public final ColumnMetadata column;
     public final Operator operator;
     private final Terms terms;
 
-    private ColumnCondition(ColumnDefinition column, Operator op, Terms terms)
+    private ColumnCondition(ColumnMetadata column, Operator op, Terms terms)
     {
         this.column = column;
         this.operator = op;
@@ -116,7 +116,7 @@ public abstract class ColumnCondition
      */
     private static final class SimpleColumnCondition extends ColumnCondition
     {
-        public SimpleColumnCondition(ColumnDefinition column, Operator op, Terms values)
+        public SimpleColumnCondition(ColumnMetadata column, Operator op, Terms values)
         {
             super(column, op, values);
         }
@@ -140,7 +140,7 @@ public abstract class ColumnCondition
     {
         private final Term collectionElement;
 
-        public CollectionElementCondition(ColumnDefinition column, Term collectionElement, Operator op, Terms values)
+        public CollectionElementCondition(ColumnMetadata column, Term collectionElement, Operator op, Terms values)
         {
             super(column, op, values);
             this.collectionElement = collectionElement;
@@ -171,7 +171,7 @@ public abstract class ColumnCondition
     {
         private final FieldIdentifier udtField;
 
-        public UDTFieldCondition(ColumnDefinition column, FieldIdentifier udtField, Operator op, Terms values)
+        public UDTFieldCondition(ColumnMetadata column, FieldIdentifier udtField, Operator op, Terms values)
         {
             super(column, op, values);
             assert udtField != null;
@@ -187,7 +187,7 @@ public abstract class ColumnCondition
     /**
      *  A regular column, simple condition.
      */
-    public static ColumnCondition condition(ColumnDefinition column, Operator op, Terms terms)
+    public static ColumnCondition condition(ColumnMetadata column, Operator op, Terms terms)
     {
         return new SimpleColumnCondition(column, op, terms);
     }
@@ -195,7 +195,7 @@ public abstract class ColumnCondition
     /**
      * A collection column, simple condition.
      */
-    public static ColumnCondition condition(ColumnDefinition column, Term collectionElement, Operator op, Terms terms)
+    public static ColumnCondition condition(ColumnMetadata column, Term collectionElement, Operator op, Terms terms)
     {
         return new CollectionElementCondition(column, collectionElement, op, terms);
     }
@@ -203,17 +203,17 @@ public abstract class ColumnCondition
     /**
      * A UDT column, simple condition.
      */
-    public static ColumnCondition condition(ColumnDefinition column, FieldIdentifier udtField, Operator op, Terms terms)
+    public static ColumnCondition condition(ColumnMetadata column, FieldIdentifier udtField, Operator op, Terms terms)
     {
         return new UDTFieldCondition(column, udtField, op, terms);
     }
 
     public static abstract class Bound
     {
-        public final ColumnDefinition column;
+        public final ColumnMetadata column;
         public final Operator comparisonOperator;
 
-        protected Bound(ColumnDefinition column, Operator operator)
+        protected Bound(ColumnMetadata column, Operator operator)
         {
             this.column = column;
             // If the operator is an IN we want to compare the value using an EQ.
@@ -257,21 +257,21 @@ public abstract class ColumnCondition
         }
     }
 
-    protected static final Cell getCell(Row row, ColumnDefinition column)
+    protected static final Cell getCell(Row row, ColumnMetadata column)
     {
         // If we're asking for a given cell, and we didn't got any row from our read, it's
         // the same as not having said cell.
         return row == null ? null : row.getCell(column);
     }
 
-    protected static final Cell getCell(Row row, ColumnDefinition column, CellPath path)
+    protected static final Cell getCell(Row row, ColumnMetadata column, CellPath path)
     {
         // If we're asking for a given cell, and we didn't got any row from our read, it's
         // the same as not having said cell.
         return row == null ? null : row.getCell(column, path);
     }
 
-    protected static final Iterator<Cell> getCells(Row row, ColumnDefinition column)
+    protected static final Iterator<Cell> getCells(Row row, ColumnMetadata column)
     {
         // If we're asking for a complex cells, and we didn't got any row from our read, it's
         // the same as not having any cells for that column.
@@ -312,7 +312,7 @@ public abstract class ColumnCondition
          */
         private final List<ByteBuffer> values;
 
-        private SimpleBound(ColumnDefinition column, Operator operator, List<ByteBuffer> values)
+        private SimpleBound(ColumnMetadata column, Operator operator, List<ByteBuffer> values)
         {
             super(column, operator);
             this.values = values;
@@ -356,10 +356,10 @@ public abstract class ColumnCondition
          */
         private final List<ByteBuffer> values;
 
-        private ElementAccessBound(ColumnDefinition column,
-                                     ByteBuffer collectionElement,
-                                     Operator operator,
-                                     List<ByteBuffer> values)
+        private ElementAccessBound(ColumnMetadata column,
+                                   ByteBuffer collectionElement,
+                                   Operator operator,
+                                   List<ByteBuffer> values)
         {
             super(column, operator);
 
@@ -452,7 +452,7 @@ public abstract class ColumnCondition
     {
         private final List<Term.Terminal> values;
 
-        public MultiCellCollectionBound(ColumnDefinition column, Operator operator, List<Term.Terminal> values)
+        public MultiCellCollectionBound(ColumnMetadata column, Operator operator, List<Term.Terminal> values)
         {
             super(column, operator);
             assert column.type.isMultiCell();
@@ -586,7 +586,7 @@ public abstract class ColumnCondition
          */
         private final List<ByteBuffer> values;
 
-        private UDTFieldAccessBound(ColumnDefinition column, FieldIdentifier field, Operator operator, List<ByteBuffer> values)
+        private UDTFieldAccessBound(ColumnMetadata column, FieldIdentifier field, Operator operator, List<ByteBuffer> values)
         {
             super(column, operator);
             assert column.type.isUDT() && field != null;
@@ -645,7 +645,7 @@ public abstract class ColumnCondition
          */
         private final ProtocolVersion protocolVersion;
 
-        private MultiCellUdtBound(ColumnDefinition column, Operator op, List<ByteBuffer> values, ProtocolVersion protocolVersion)
+        private MultiCellUdtBound(ColumnMetadata column, Operator op, List<ByteBuffer> values, ProtocolVersion protocolVersion)
         {
             super(column, op);
             assert column.type.isMultiCell();
@@ -756,7 +756,7 @@ public abstract class ColumnCondition
             return new Raw(null, null, inMarker, null, udtField, Operator.IN);
         }
 
-        public ColumnCondition prepare(String keyspace, ColumnDefinition receiver, CFMetaData cfm)
+        public ColumnCondition prepare(String keyspace, ColumnMetadata receiver, TableMetadata cfm)
         {
             if (receiver.type instanceof CounterColumnType)
                 throw invalidRequest("Conditions on counters are not supported");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java b/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java
index b0378b7..3057b05 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java
@@ -24,11 +24,11 @@ import java.util.List;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.statements.CQL3CasRequest;
 import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.schema.ColumnMetadata;
 
 /**
  * A set of <code>ColumnCondition</code>s.
@@ -68,7 +68,7 @@ public final class ColumnConditions extends AbstractConditions
     }
 
     @Override
-    public Collection<ColumnDefinition> getColumns()
+    public Collection<ColumnMetadata> getColumns()
     {
         return Stream.concat(columnConditions.stream(), staticConditions.stream())
                      .map(e -> e.column)
@@ -136,7 +136,7 @@ public final class ColumnConditions extends AbstractConditions
          */
         public Builder add(ColumnCondition condition)
         {
-            List<ColumnCondition> conds = null;
+            List<ColumnCondition> conds;
             if (condition.column.isStatic())
             {
                 if (staticConditions.isEmpty())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/conditions/Conditions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/conditions/Conditions.java b/src/java/org/apache/cassandra/cql3/conditions/Conditions.java
index 0214fcf..1622be0 100644
--- a/src/java/org/apache/cassandra/cql3/conditions/Conditions.java
+++ b/src/java/org/apache/cassandra/cql3/conditions/Conditions.java
@@ -19,11 +19,11 @@ package org.apache.cassandra.cql3.conditions;
 
 import java.util.List;
 
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.statements.CQL3CasRequest;
 import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.schema.ColumnMetadata;
 
 /**
  * Conditions that can be applied to a mutation statement.
@@ -56,7 +56,7 @@ public interface Conditions
      * Returns the column definitions to which apply the conditions.
      * @return the column definitions to which apply the conditions.
      */
-    Iterable<ColumnDefinition> getColumns();
+    Iterable<ColumnMetadata> getColumns();
 
     /**
      * Checks if this <code>Conditions</code> is empty.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
index aa7555f..5e10e9f 100644
--- a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
@@ -22,10 +22,13 @@ import java.util.List;
 import com.google.common.base.Objects;
 
 import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.commons.lang3.text.StrBuilder;
 
+import static java.util.stream.Collectors.toList;
+
 /**
  * Base class for our native/hardcoded functions.
  */
@@ -57,6 +60,14 @@ public abstract class AbstractFunction implements Function
         return returnType;
     }
 
+    public List<String> argumentsList()
+    {
+        return argTypes().stream()
+                         .map(AbstractType::asCQL3Type)
+                         .map(CQL3Type::toString)
+                         .collect(toList());
+    }
+
     @Override
     public boolean equals(Object o)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/functions/FunctionName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/FunctionName.java b/src/java/org/apache/cassandra/cql3/functions/FunctionName.java
index aa980e9..be71d52 100644
--- a/src/java/org/apache/cassandra/cql3/functions/FunctionName.java
+++ b/src/java/org/apache/cassandra/cql3/functions/FunctionName.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.cql3.functions;
 
 import com.google.common.base.Objects;
 
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 
 public final class FunctionName
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java b/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java
index 7234d1f..9c7e85f 100644
--- a/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java
+++ b/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java
@@ -21,11 +21,11 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.AbstractMarker;
 import org.apache.cassandra.cql3.AssignmentTestable;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
@@ -70,7 +70,7 @@ public final class FunctionResolver
     throws InvalidRequestException
     {
         if (name.equalsNativeFunction(TOKEN_FUNCTION_NAME))
-            return new TokenFct(Schema.instance.getCFMetaData(receiverKs, receiverCf));
+            return new TokenFct(Schema.instance.getTableMetadata(receiverKs, receiverCf));
 
         // The toJson() function can accept any type of argument, so instances of it are not pre-declared.  Instead,
         // we create new instances as needed while handling selectors (which is the only place that toJson() is supported,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java b/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java
index 1f115a9..0a039c0 100644
--- a/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java
+++ b/src/java/org/apache/cassandra/cql3/functions/OperationFcts.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.OperationExecutionException;
 import org.apache.cassandra.transport.ProtocolVersion;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/TokenFct.java b/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
index 1907641..e93084f 100644
--- a/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
+++ b/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.cql3.functions;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.CBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -29,26 +29,26 @@ import org.apache.cassandra.transport.ProtocolVersion;
 
 public class TokenFct extends NativeScalarFunction
 {
-    private final CFMetaData cfm;
+    private final TableMetadata metadata;
 
-    public TokenFct(CFMetaData cfm)
+    public TokenFct(TableMetadata metadata)
     {
-        super("token", cfm.partitioner.getTokenValidator(), getKeyTypes(cfm));
-        this.cfm = cfm;
+        super("token", metadata.partitioner.getTokenValidator(), getKeyTypes(metadata));
+        this.metadata = metadata;
     }
 
-    private static AbstractType[] getKeyTypes(CFMetaData cfm)
+    private static AbstractType[] getKeyTypes(TableMetadata metadata)
     {
-        AbstractType[] types = new AbstractType[cfm.partitionKeyColumns().size()];
+        AbstractType[] types = new AbstractType[metadata.partitionKeyColumns().size()];
         int i = 0;
-        for (ColumnDefinition def : cfm.partitionKeyColumns())
+        for (ColumnMetadata def : metadata.partitionKeyColumns())
             types[i++] = def.type;
         return types;
     }
 
     public ByteBuffer execute(ProtocolVersion protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException
     {
-        CBuilder builder = CBuilder.create(cfm.getKeyValidatorAsClusteringComparator());
+        CBuilder builder = CBuilder.create(metadata.partitionKeyAsClusteringComparator());
         for (int i = 0; i < parameters.size(); i++)
         {
             ByteBuffer bb = parameters.get(i);
@@ -56,6 +56,6 @@ public class TokenFct extends NativeScalarFunction
                 return null;
             builder.add(bb);
         }
-        return cfm.partitioner.getTokenFactory().toByteArray(cfm.partitioner.getToken(CFMetaData.serializePartitionKey(builder.build())));
+        return metadata.partitioner.getTokenFactory().toByteArray(metadata.partitioner.getToken(builder.build().serializeAsPartitionKey()));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 f8af619..b6fedcc 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -42,10 +42,9 @@ import org.slf4j.LoggerFactory;
 
 import com.datastax.driver.core.DataType;
 import com.datastax.driver.core.TypeCodec;
-import com.datastax.driver.core.UserType;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.FunctionExecutionException;
@@ -53,10 +52,8 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.schema.Functions;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientWarn;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.ProtocolVersion;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 /**
@@ -212,7 +209,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
         this.argCodecs = UDHelper.codecsFor(argDataTypes);
         this.returnCodec = UDHelper.codecFor(returnDataType);
         this.calledOnNullInput = calledOnNullInput;
-        KeyspaceMetadata keyspaceMetadata = Schema.instance.getKSMetaData(name.keyspace);
+        KeyspaceMetadata keyspaceMetadata = Schema.instance.getKeyspaceMetadata(name.keyspace);
         this.udfContext = new UDFContextImpl(argNames, argCodecs, returnCodec,
                                              keyspaceMetadata);
     }
@@ -605,37 +602,6 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
         return Objects.hashCode(name, Functions.typeHashCode(argTypes), Functions.typeHashCode(returnType), returnType, language, body);
     }
 
-    public void userTypeUpdated(String ksName, String typeName)
-    {
-        boolean updated = false;
-
-        for (int i = 0; i < argCodecs.length; i++)
-        {
-            DataType dataType = argCodecs[i].getCqlType();
-            if (dataType instanceof UserType)
-            {
-                UserType userType = (UserType) dataType;
-                if (userType.getKeyspace().equals(ksName) && userType.getTypeName().equals(typeName))
-                {
-                    KeyspaceMetadata ksm = Schema.instance.getKSMetaData(ksName);
-                    assert ksm != null;
-
-                    org.apache.cassandra.db.marshal.UserType ut = ksm.types.get(ByteBufferUtil.bytes(typeName)).get();
-
-                    DataType newUserType = UDHelper.driverType(ut);
-                    argCodecs[i] = UDHelper.codecFor(newUserType);
-
-                    argTypes.set(i, ut);
-
-                    updated = true;
-                }
-            }
-        }
-
-        if (updated)
-            MigrationManager.announceNewFunction(this, true);
-    }
-
     private static class UDFClassLoader extends ClassLoader
     {
         // insecureClassLoader is the C* class loader

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
index ed0d325..f537255 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictions.java
@@ -19,8 +19,8 @@ package org.apache.cassandra.cql3.restrictions;
 
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.statements.Bound;
 import org.apache.cassandra.db.*;
@@ -47,14 +47,14 @@ final class ClusteringColumnRestrictions extends RestrictionSetWrapper
      */
     private final boolean allowFiltering;
 
-    public ClusteringColumnRestrictions(CFMetaData cfm)
+    public ClusteringColumnRestrictions(TableMetadata table)
     {
-        this(cfm, false);
+        this(table, false);
     }
 
-    public ClusteringColumnRestrictions(CFMetaData cfm, boolean allowFiltering)
+    public ClusteringColumnRestrictions(TableMetadata table, boolean allowFiltering)
     {
-        this(cfm.comparator, new RestrictionSet(), allowFiltering);
+        this(table.comparator, new RestrictionSet(), allowFiltering);
     }
 
     private ClusteringColumnRestrictions(ClusteringComparator comparator,
@@ -74,8 +74,8 @@ final class ClusteringColumnRestrictions extends RestrictionSetWrapper
         if (!isEmpty() && !allowFiltering)
         {
             SingleRestriction lastRestriction = restrictions.lastRestriction();
-            ColumnDefinition lastRestrictionStart = lastRestriction.getFirstColumn();
-            ColumnDefinition newRestrictionStart = restriction.getFirstColumn();
+            ColumnMetadata lastRestrictionStart = lastRestriction.getFirstColumn();
+            ColumnMetadata newRestrictionStart = restriction.getFirstColumn();
 
             checkFalse(lastRestriction.isSlice() && newRestrictionStart.position() > lastRestrictionStart.position(),
                        "Clustering column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)",

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java b/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java
index eb91928..6ea6842 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/CustomIndexExpression.java
@@ -18,7 +18,7 @@
 
 package org.apache.cassandra.cql3.restrictions;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.filter.RowFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -38,21 +38,19 @@ public class CustomIndexExpression
         this.valueRaw = value;
     }
 
-    public void prepareValue(CFMetaData cfm, AbstractType<?> expressionType, VariableSpecifications boundNames)
+    public void prepareValue(TableMetadata table, AbstractType<?> expressionType, VariableSpecifications boundNames)
     {
-        ColumnSpecification spec = new ColumnSpecification(cfm.ksName, cfm.ksName, valueColId, expressionType);
-        value = valueRaw.prepare(cfm.ksName, spec);
+        ColumnSpecification spec = new ColumnSpecification(table.keyspace, table.keyspace, valueColId, expressionType);
+        value = valueRaw.prepare(table.keyspace, spec);
         value.collectMarkerSpecification(boundNames);
     }
 
-    public void addToRowFilter(RowFilter filter,
-                               CFMetaData cfm,
-                               QueryOptions options)
+    public void addToRowFilter(RowFilter filter, TableMetadata table, QueryOptions options)
     {
-        filter.addCustomIndexExpression(cfm,
-                                        cfm.getIndexes()
-                                           .get(targetIndex.getIdx())
-                                           .orElseThrow(() -> IndexRestrictions.indexNotFound(targetIndex, cfm)),
+        filter.addCustomIndexExpression(table,
+                                        table.indexes
+                                             .get(targetIndex.getIdx())
+                                             .orElseThrow(() -> IndexRestrictions.indexNotFound(targetIndex, table)),
                                         value.bindAndGet(options));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java
index c7f6b5f..ced04ed 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/IndexRestrictions.java
@@ -21,14 +21,14 @@ package org.apache.cassandra.cql3.restrictions;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.IndexName;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
 public class IndexRestrictions
 {
-    public static final String INDEX_NOT_FOUND = "Invalid index expression, index %s not found for %s.%s";
-    public static final String INVALID_INDEX = "Target index %s cannot be used to query %s.%s";
+    public static final String INDEX_NOT_FOUND = "Invalid index expression, index %s not found for %s";
+    public static final String INVALID_INDEX = "Target index %s cannot be used to query %s";
     public static final String CUSTOM_EXPRESSION_NOT_SUPPORTED = "Index %s does not support custom expressions";
     public static final String NON_CUSTOM_INDEX_IN_EXPRESSION = "Only CUSTOM indexes may be used in custom index expressions, %s is not valid";
     public static final String MULTIPLE_EXPRESSIONS = "Multiple custom index expressions in a single query are not supported";
@@ -61,14 +61,14 @@ public class IndexRestrictions
         return customExpressions;
     }
 
-    static InvalidRequestException invalidIndex(IndexName indexName, CFMetaData cfm)
+    static InvalidRequestException invalidIndex(IndexName indexName, TableMetadata table)
     {
-        return new InvalidRequestException(String.format(INVALID_INDEX, indexName.getIdx(), cfm.ksName, cfm.cfName));
+        return new InvalidRequestException(String.format(INVALID_INDEX, indexName.getIdx(), table.toString()));
     }
 
-    static InvalidRequestException indexNotFound(IndexName indexName, CFMetaData cfm)
+    static InvalidRequestException indexNotFound(IndexName indexName, TableMetadata table)
     {
-        return new InvalidRequestException(String.format(INDEX_NOT_FOUND,indexName.getIdx(), cfm.ksName, cfm.cfName));
+        return new InvalidRequestException(String.format(INDEX_NOT_FOUND, indexName.getIdx(), table.toString()));
     }
 
     static InvalidRequestException nonCustomIndexInExpression(IndexName indexName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
index b0cbdff..bf10024 100644
--- a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
+++ b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.cql3.restrictions;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.Term.Terminal;
 import org.apache.cassandra.cql3.functions.Function;
@@ -40,9 +40,9 @@ public abstract class MultiColumnRestriction implements SingleRestriction
     /**
      * The columns to which the restriction apply.
      */
-    protected final List<ColumnDefinition> columnDefs;
+    protected final List<ColumnMetadata> columnDefs;
 
-    public MultiColumnRestriction(List<ColumnDefinition> columnDefs)
+    public MultiColumnRestriction(List<ColumnMetadata> columnDefs)
     {
         this.columnDefs = columnDefs;
     }
@@ -54,19 +54,19 @@ public abstract class MultiColumnRestriction implements SingleRestriction
     }
 
     @Override
-    public ColumnDefinition getFirstColumn()
+    public ColumnMetadata getFirstColumn()
     {
         return columnDefs.get(0);
     }
 
     @Override
-    public ColumnDefinition getLastColumn()
+    public ColumnMetadata getLastColumn()
     {
         return columnDefs.get(columnDefs.size() - 1);
     }
 
     @Override
-    public List<ColumnDefinition> getColumnDefs()
+    public List<ColumnMetadata> getColumnDefs()
     {
         return columnDefs;
     }
@@ -96,14 +96,14 @@ public abstract class MultiColumnRestriction implements SingleRestriction
      */
     protected final String getColumnsInCommons(Restriction otherRestriction)
     {
-        Set<ColumnDefinition> commons = new HashSet<>(getColumnDefs());
+        Set<ColumnMetadata> commons = new HashSet<>(getColumnDefs());
         commons.retainAll(otherRestriction.getColumnDefs());
         StringBuilder builder = new StringBuilder();
-        for (ColumnDefinition columnDefinition : commons)
+        for (ColumnMetadata columnMetadata : commons)
         {
             if (builder.length() != 0)
                 builder.append(" ,");
-            builder.append(columnDefinition.name);
+            builder.append(columnMetadata.name);
         }
         return builder.toString();
     }
@@ -130,7 +130,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
     {
         protected final Term value;
 
-        public EQRestriction(List<ColumnDefinition> columnDefs, Term value)
+        public EQRestriction(List<ColumnMetadata> columnDefs, Term value)
         {
             super(columnDefs);
             this.value = value;
@@ -158,7 +158,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
         @Override
         protected boolean isSupportedBy(Index index)
         {
-            for(ColumnDefinition column : columnDefs)
+            for(ColumnMetadata column : columnDefs)
                 if (index.supportsExpression(column, Operator.EQ))
                     return true;
             return false;
@@ -185,7 +185,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
 
             for (int i = 0, m = columnDefs.size(); i < m; i++)
             {
-                ColumnDefinition columnDef = columnDefs.get(i);
+                ColumnMetadata columnDef = columnDefs.get(i);
                 filter.add(columnDef, Operator.EQ, values.get(i));
             }
         }
@@ -193,7 +193,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
 
     public abstract static class INRestriction extends MultiColumnRestriction
     {
-        public INRestriction(List<ColumnDefinition> columnDefs)
+        public INRestriction(List<ColumnMetadata> columnDefs)
         {
             super(columnDefs);
         }
@@ -208,7 +208,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
             builder.addAllElementsToAll(splitInValues);
 
             if (builder.containsNull())
-                throw invalidRequest("Invalid null value in condition for columns: %s", ColumnDefinition.toIdentifiers(columnDefs));
+                throw invalidRequest("Invalid null value in condition for columns: %s", ColumnMetadata.toIdentifiers(columnDefs));
             return builder;
         }
 
@@ -228,7 +228,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
         @Override
         protected boolean isSupportedBy(Index index)
         {
-            for (ColumnDefinition column: columnDefs)
+            for (ColumnMetadata column: columnDefs)
                 if (index.supportsExpression(column, Operator.IN))
                     return true;
             return false;
@@ -253,7 +253,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
     {
         protected final List<Term> values;
 
-        public InRestrictionWithValues(List<ColumnDefinition> columnDefs, List<Term> values)
+        public InRestrictionWithValues(List<ColumnMetadata> columnDefs, List<Term> values)
         {
             super(columnDefs);
             this.values = values;
@@ -292,7 +292,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
     {
         protected final AbstractMarker marker;
 
-        public InRestrictionWithMarker(List<ColumnDefinition> columnDefs, AbstractMarker marker)
+        public InRestrictionWithMarker(List<ColumnMetadata> columnDefs, AbstractMarker marker)
         {
             super(columnDefs);
             this.marker = marker;
@@ -323,12 +323,12 @@ public abstract class MultiColumnRestriction implements SingleRestriction
     {
         private final TermSlice slice;
 
-        public SliceRestriction(List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
+        public SliceRestriction(List<ColumnMetadata> columnDefs, Bound bound, boolean inclusive, Term term)
         {
             this(columnDefs, TermSlice.newInstance(bound, inclusive, term));
         }
 
-        SliceRestriction(List<ColumnDefinition> columnDefs, TermSlice slice)
+        SliceRestriction(List<ColumnMetadata> columnDefs, TermSlice slice)
         {
             super(columnDefs);
             this.slice = slice;
@@ -360,7 +360,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
 
             for (int i = 0, m = columnDefs.size(); i < m; i++)
             {
-                ColumnDefinition column = columnDefs.get(i);
+                ColumnMetadata column = columnDefs.get(i);
                 Bound b = bound.reverseIfNeeded(column);
 
                 // For mixed order columns, we need to create additional slices when 2 columns are in reverse order
@@ -410,7 +410,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
         @Override
         protected boolean isSupportedBy(Index index)
         {
-            for(ColumnDefinition def : columnDefs)
+            for(ColumnMetadata def : columnDefs)
                 if (slice.isSupportedBy(def, index))
                     return true;
             return false;
@@ -443,7 +443,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
 
             if (!getFirstColumn().equals(otherRestriction.getFirstColumn()))
             {
-                ColumnDefinition column = getFirstColumn().position() > otherRestriction.getFirstColumn().position()
+                ColumnMetadata column = getFirstColumn().position() > otherRestriction.getFirstColumn().position()
                         ? getFirstColumn() : otherRestriction.getFirstColumn();
 
                 throw invalidRequest("Column \"%s\" cannot be restricted by two inequalities not starting with the same column",
@@ -458,7 +458,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
                        getColumnsInCommons(otherRestriction));
 
             SliceRestriction otherSlice = (SliceRestriction) otherRestriction;
-            List<ColumnDefinition> newColumnDefs = columnDefs.size() >= otherSlice.columnDefs.size() ?  columnDefs : otherSlice.columnDefs;
+            List<ColumnMetadata> newColumnDefs = columnDefs.size() >= otherSlice.columnDefs.size() ? columnDefs : otherSlice.columnDefs;
 
             return new SliceRestriction(newColumnDefs, slice.merge(otherSlice.slice));
         }
@@ -507,7 +507,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
 
     public static class NotNullRestriction extends MultiColumnRestriction
     {
-        public NotNullRestriction(List<ColumnDefinition> columnDefs)
+        public NotNullRestriction(List<ColumnMetadata> columnDefs)
         {
             super(columnDefs);
             assert columnDefs.size() == 1;
@@ -540,7 +540,7 @@ public abstract class MultiColumnRestriction implements SingleRestriction
         @Override
         protected boolean isSupportedBy(Index index)
         {
-            for(ColumnDefinition column : columnDefs)
+            for(ColumnMetadata column : columnDefs)
                 if (index.supportsExpression(column, Operator.IS_NOT))
                     return true;
             return false;


[28/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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
deleted file mode 100644
index 8b3e121..0000000
--- a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
+++ /dev/null
@@ -1,55 +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.util.Collection;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.utils.WrappedRunnable;
-
-/**
- * Called when node receives updated schema state from the schema migration coordinator node.
- * Such happens when user makes local schema migration on one of the nodes in the ring
- * (which is going to act as coordinator) and that node sends (pushes) it's updated schema state
- * (in form of mutations) to all the alive nodes in the cluster.
- */
-public class DefinitionsUpdateVerbHandler implements IVerbHandler<Collection<Mutation>>
-{
-    private static final Logger logger = LoggerFactory.getLogger(DefinitionsUpdateVerbHandler.class);
-
-    public void doVerb(final MessageIn<Collection<Mutation>> message, int id)
-    {
-        logger.trace("Received schema mutation push from {}", message.from);
-
-        StageManager.getStage(Stage.MIGRATION).submit(new WrappedRunnable()
-        {
-            public void runMayThrow() throws ConfigurationException
-            {
-                SchemaKeyspace.mergeSchemaAndAnnounceVersion(message.payload);
-            }
-        });
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 2bb4784..a3e80e5 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -28,7 +28,6 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.function.BiFunction;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 
@@ -43,8 +42,8 @@ import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.DirectorySizeCalculator;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
@@ -60,13 +59,13 @@ import org.apache.cassandra.utils.Pair;
  * } </pre>
  *
  * Until v2.0, {@code <cf dir>} is just column family name.
- * Since v2.1, {@code <cf dir>} has column family ID(cfId) added to its end.
+ * Since v2.1, {@code <cf dir>} has column family ID(tableId) added to its end.
  *
  * SSTables from secondary indexes were put in the same directory as their parent.
  * Since v2.2, they have their own directory under the parent directory whose name is index name.
  * Upon startup, those secondary index files are moved to new directory when upgrading.
  *
- * For backward compatibility, Directories can use directory without cfId if exists.
+ * For backward compatibility, Directories can use directory without tableId if exists.
  *
  * In addition, more that one 'root' data directory can be specified so that
  * {@code <path_to_data_dir>} potentially represents multiple locations.
@@ -174,16 +173,16 @@ public class Directories
         }
     }
 
-    private final CFMetaData metadata;
+    private final TableMetadata metadata;
     private final DataDirectory[] paths;
     private final File[] dataPaths;
 
-    public Directories(final CFMetaData metadata)
+    public Directories(final TableMetadata metadata)
     {
         this(metadata, dataDirectories);
     }
 
-    public Directories(final CFMetaData metadata, Collection<DataDirectory> paths)
+    public Directories(final TableMetadata metadata, Collection<DataDirectory> paths)
     {
         this(metadata, paths.toArray(new DataDirectory[paths.size()]));
     }
@@ -194,35 +193,29 @@ public class Directories
      *
      * @param metadata metadata of ColumnFamily
      */
-    public Directories(final CFMetaData metadata, DataDirectory[] paths)
+    public Directories(final TableMetadata metadata, DataDirectory[] paths)
     {
         this.metadata = metadata;
         this.paths = paths;
 
-        String cfId = ByteBufferUtil.bytesToHex(ByteBufferUtil.bytes(metadata.cfId));
-        int idx = metadata.cfName.indexOf(SECONDARY_INDEX_NAME_SEPARATOR);
-        String cfName = idx >= 0 ? metadata.cfName.substring(0, idx) : metadata.cfName;
-        String indexNameWithDot = idx >= 0 ? metadata.cfName.substring(idx) : null;
+        String tableId = metadata.id.toHexString();
+        int idx = metadata.name.indexOf(SECONDARY_INDEX_NAME_SEPARATOR);
+        String cfName = idx >= 0 ? metadata.name.substring(0, idx) : metadata.name;
+        String indexNameWithDot = idx >= 0 ? metadata.name.substring(idx) : null;
 
         this.dataPaths = new File[paths.length];
         // If upgraded from version less than 2.1, use existing directories
-        String oldSSTableRelativePath = join(metadata.ksName, cfName);
+        String oldSSTableRelativePath = join(metadata.keyspace, cfName);
         for (int i = 0; i < paths.length; ++i)
         {
             // check if old SSTable directory exists
             dataPaths[i] = new File(paths[i].location, oldSSTableRelativePath);
         }
-        boolean olderDirectoryExists = Iterables.any(Arrays.asList(dataPaths), new Predicate<File>()
-        {
-            public boolean apply(File file)
-            {
-                return file.exists();
-            }
-        });
+        boolean olderDirectoryExists = Iterables.any(Arrays.asList(dataPaths), File::exists);
         if (!olderDirectoryExists)
         {
             // use 2.1+ style
-            String newSSTableRelativePath = join(metadata.ksName, cfName + '-' + cfId);
+            String newSSTableRelativePath = join(metadata.keyspace, cfName + '-' + tableId);
             for (int i = 0; i < paths.length; ++i)
                 dataPaths[i] = new File(paths[i].location, newSSTableRelativePath);
         }
@@ -261,7 +254,7 @@ public class Directories
                             return false;
 
                         Descriptor desc = SSTable.tryDescriptorFromFilename(file);
-                        return desc != null && desc.ksname.equals(metadata.ksName) && desc.cfname.equals(metadata.cfName);
+                        return desc != null && desc.ksname.equals(metadata.keyspace) && desc.cfname.equals(metadata.name);
 
                     }
                 });
@@ -760,7 +753,7 @@ public class Directories
                             return false;
 
                         // we are only interested in the SSTable files that belong to the specific ColumnFamily
-                        if (!pair.left.ksname.equals(metadata.ksName) || !pair.left.cfname.equals(metadata.cfName))
+                        if (!pair.left.ksname.equals(metadata.keyspace) || !pair.left.cfname.equals(metadata.name))
                             return false;
 
                         Set<Component> previous = components.get(pair.left);
@@ -1027,8 +1020,8 @@ public class Directories
             File file = path.toFile();
             Descriptor desc = SSTable.tryDescriptorFromFilename(file);
             return desc != null
-                && desc.ksname.equals(metadata.ksName)
-                && desc.cfname.equals(metadata.cfName)
+                && desc.ksname.equals(metadata.keyspace)
+                && desc.cfname.equals(metadata.name)
                 && !toSkip.contains(file);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/EmptyIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/EmptyIterators.java b/src/java/org/apache/cassandra/db/EmptyIterators.java
index 24c923f..04ff31b 100644
--- a/src/java/org/apache/cassandra/db/EmptyIterators.java
+++ b/src/java/org/apache/cassandra/db/EmptyIterators.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.db;
 
 import java.util.NoSuchElementException;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.partitions.BasePartitionIterator;
 import org.apache.cassandra.db.partitions.PartitionIterator;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
@@ -52,14 +52,14 @@ public class EmptyIterators
 
     private static class EmptyUnfilteredPartitionIterator extends EmptyBasePartitionIterator<UnfilteredRowIterator> implements UnfilteredPartitionIterator
     {
-        final CFMetaData metadata;
+        final TableMetadata metadata;
 
-        public EmptyUnfilteredPartitionIterator(CFMetaData metadata)
+        public EmptyUnfilteredPartitionIterator(TableMetadata metadata)
         {
             this.metadata = metadata;
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
             return metadata;
         }
@@ -76,13 +76,13 @@ public class EmptyIterators
 
     private static class EmptyBaseRowIterator<U extends Unfiltered> implements BaseRowIterator<U>
     {
-        final PartitionColumns columns;
-        final CFMetaData metadata;
+        final RegularAndStaticColumns columns;
+        final TableMetadata metadata;
         final DecoratedKey partitionKey;
         final boolean isReverseOrder;
         final Row staticRow;
 
-        EmptyBaseRowIterator(PartitionColumns columns, CFMetaData metadata, DecoratedKey partitionKey, boolean isReverseOrder, Row staticRow)
+        EmptyBaseRowIterator(RegularAndStaticColumns columns, TableMetadata metadata, DecoratedKey partitionKey, boolean isReverseOrder, Row staticRow)
         {
             this.columns = columns;
             this.metadata = metadata;
@@ -91,7 +91,7 @@ public class EmptyIterators
             this.staticRow = staticRow;
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
             return metadata;
         }
@@ -101,7 +101,7 @@ public class EmptyIterators
             return isReverseOrder;
         }
 
-        public PartitionColumns columns()
+        public RegularAndStaticColumns columns()
         {
             return columns;
         }
@@ -139,7 +139,7 @@ public class EmptyIterators
     private static class EmptyUnfilteredRowIterator extends EmptyBaseRowIterator<Unfiltered> implements UnfilteredRowIterator
     {
         final DeletionTime partitionLevelDeletion;
-        public EmptyUnfilteredRowIterator(PartitionColumns columns, CFMetaData metadata, DecoratedKey partitionKey,
+        public EmptyUnfilteredRowIterator(RegularAndStaticColumns columns, TableMetadata metadata, DecoratedKey partitionKey,
                                           boolean isReverseOrder, Row staticRow, DeletionTime partitionLevelDeletion)
         {
             super(columns, metadata, partitionKey, isReverseOrder, staticRow);
@@ -164,13 +164,13 @@ public class EmptyIterators
 
     private static class EmptyRowIterator extends EmptyBaseRowIterator<Row> implements RowIterator
     {
-        public EmptyRowIterator(CFMetaData metadata, DecoratedKey partitionKey, boolean isReverseOrder, Row staticRow)
+        public EmptyRowIterator(TableMetadata metadata, DecoratedKey partitionKey, boolean isReverseOrder, Row staticRow)
         {
-            super(PartitionColumns.NONE, metadata, partitionKey, isReverseOrder, staticRow);
+            super(RegularAndStaticColumns.NONE, metadata, partitionKey, isReverseOrder, staticRow);
         }
     }
 
-    public static UnfilteredPartitionIterator unfilteredPartition(CFMetaData metadata)
+    public static UnfilteredPartitionIterator unfilteredPartition(TableMetadata metadata)
     {
         return new EmptyUnfilteredPartitionIterator(metadata);
     }
@@ -181,11 +181,11 @@ public class EmptyIterators
     }
 
     // this method is the only one that can return a non-empty iterator, but it still has no rows, so it seems cleanest to keep it here
-    public static UnfilteredRowIterator unfilteredRow(CFMetaData metadata, DecoratedKey partitionKey, boolean isReverseOrder, Row staticRow, DeletionTime partitionDeletion)
+    public static UnfilteredRowIterator unfilteredRow(TableMetadata metadata, DecoratedKey partitionKey, boolean isReverseOrder, Row staticRow, DeletionTime partitionDeletion)
     {
-        PartitionColumns columns = PartitionColumns.NONE;
+        RegularAndStaticColumns columns = RegularAndStaticColumns.NONE;
         if (!staticRow.isEmpty())
-            columns = new PartitionColumns(Columns.from(staticRow.columns()), Columns.NONE);
+            columns = new RegularAndStaticColumns(Columns.from(staticRow.columns()), Columns.NONE);
         else
             staticRow = Rows.EMPTY_STATIC_ROW;
 
@@ -195,12 +195,12 @@ public class EmptyIterators
         return new EmptyUnfilteredRowIterator(columns, metadata, partitionKey, isReverseOrder, staticRow, partitionDeletion);
     }
 
-    public static UnfilteredRowIterator unfilteredRow(CFMetaData metadata, DecoratedKey partitionKey, boolean isReverseOrder)
+    public static UnfilteredRowIterator unfilteredRow(TableMetadata metadata, DecoratedKey partitionKey, boolean isReverseOrder)
     {
-        return new EmptyUnfilteredRowIterator(PartitionColumns.NONE, metadata, partitionKey, isReverseOrder, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE);
+        return new EmptyUnfilteredRowIterator(RegularAndStaticColumns.NONE, metadata, partitionKey, isReverseOrder, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE);
     }
 
-    public static RowIterator row(CFMetaData metadata, DecoratedKey partitionKey, boolean isReverseOrder)
+    public static RowIterator row(TableMetadata metadata, DecoratedKey partitionKey, boolean isReverseOrder)
     {
         return new EmptyRowIterator(metadata, partitionKey, isReverseOrder, Rows.EMPTY_STATIC_ROW);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/IMutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/IMutation.java b/src/java/org/apache/cassandra/db/IMutation.java
index 0ac89f7..3d4b1b2 100644
--- a/src/java/org/apache/cassandra/db/IMutation.java
+++ b/src/java/org/apache/cassandra/db/IMutation.java
@@ -18,15 +18,15 @@
 package org.apache.cassandra.db;
 
 import java.util.Collection;
-import java.util.UUID;
 
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.TableId;
 
 public interface IMutation
 {
     public void apply();
     public String getKeyspaceName();
-    public Collection<UUID> getColumnFamilyIds();
+    public Collection<TableId> getTableIds();
     public DecoratedKey key();
     public long getTimeout();
     public String toString(boolean shallow);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 b477fa3..f0fd5aa 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -46,6 +46,11 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.metrics.KeyspaceMetrics;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.*;
@@ -79,7 +84,8 @@ public class Keyspace
     public static final OpOrder writeOrder = new OpOrder();
 
     /* ColumnFamilyStore per column family */
-    private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<>();
+    private final ConcurrentMap<TableId, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<>();
+
     private volatile AbstractReplicationStrategy replicationStrategy;
     public final ViewManager viewManager;
 
@@ -152,9 +158,14 @@ public class Keyspace
         }
     }
 
-    public static ColumnFamilyStore openAndGetStore(CFMetaData cfm)
+    public static ColumnFamilyStore openAndGetStore(TableMetadataRef tableRef)
+    {
+        return open(tableRef.keyspace).getColumnFamilyStore(tableRef.id);
+    }
+
+    public static ColumnFamilyStore openAndGetStore(TableMetadata table)
     {
-        return open(cfm.ksName).getColumnFamilyStore(cfm.cfId);
+        return open(table.keyspace).getColumnFamilyStore(table.id);
     }
 
     /**
@@ -191,13 +202,13 @@ public class Keyspace
 
     public ColumnFamilyStore getColumnFamilyStore(String cfName)
     {
-        UUID id = Schema.instance.getId(getName(), cfName);
-        if (id == null)
+        TableMetadata table = Schema.instance.getTableMetadata(getName(), cfName);
+        if (table == null)
             throw new IllegalArgumentException(String.format("Unknown keyspace/cf pair (%s.%s)", getName(), cfName));
-        return getColumnFamilyStore(id);
+        return getColumnFamilyStore(table.id);
     }
 
-    public ColumnFamilyStore getColumnFamilyStore(UUID id)
+    public ColumnFamilyStore getColumnFamilyStore(TableId id)
     {
         ColumnFamilyStore cfs = columnFamilyStores.get(id);
         if (cfs == null)
@@ -205,7 +216,7 @@ public class Keyspace
         return cfs;
     }
 
-    public boolean hasColumnFamilyStore(UUID id)
+    public boolean hasColumnFamilyStore(TableId id)
     {
         return columnFamilyStores.containsKey(id);
     }
@@ -310,16 +321,16 @@ public class Keyspace
 
     private Keyspace(String keyspaceName, boolean loadSSTables)
     {
-        metadata = Schema.instance.getKSMetaData(keyspaceName);
+        metadata = Schema.instance.getKeyspaceMetadata(keyspaceName);
         assert metadata != null : "Unknown keyspace " + keyspaceName;
         createReplicationStrategy(metadata);
 
         this.metric = new KeyspaceMetrics(this);
         this.viewManager = new ViewManager(this);
-        for (CFMetaData cfm : metadata.tablesAndViews())
+        for (TableMetadata cfm : metadata.tablesAndViews())
         {
-            logger.trace("Initializing {}.{}", getName(), cfm.cfName);
-            initCf(cfm, loadSSTables);
+            logger.trace("Initializing {}.{}", getName(), cfm.name);
+            initCf(Schema.instance.getTableMetadataRef(cfm.id), loadSSTables);
         }
         this.viewManager.reload();
     }
@@ -347,10 +358,10 @@ public class Keyspace
     }
 
     // best invoked on the compaction mananger.
-    public void dropCf(UUID cfId)
+    public void dropCf(TableId tableId)
     {
-        assert columnFamilyStores.containsKey(cfId);
-        ColumnFamilyStore cfs = columnFamilyStores.remove(cfId);
+        assert columnFamilyStores.containsKey(tableId);
+        ColumnFamilyStore cfs = columnFamilyStores.remove(tableId);
         if (cfs == null)
             return;
 
@@ -376,17 +387,17 @@ public class Keyspace
      */
     public void initCfCustom(ColumnFamilyStore newCfs)
     {
-        ColumnFamilyStore cfs = columnFamilyStores.get(newCfs.metadata.cfId);
+        ColumnFamilyStore cfs = columnFamilyStores.get(newCfs.metadata.id);
 
         if (cfs == null)
         {
             // CFS being created for the first time, either on server startup or new CF being added.
             // We don't worry about races here; startup is safe, and adding multiple idential CFs
             // simultaneously is a "don't do that" scenario.
-            ColumnFamilyStore oldCfs = columnFamilyStores.putIfAbsent(newCfs.metadata.cfId, newCfs);
+            ColumnFamilyStore oldCfs = columnFamilyStores.putIfAbsent(newCfs.metadata.id, newCfs);
             // CFS mbean instantiation will error out before we hit this, but in case that changes...
             if (oldCfs != null)
-                throw new IllegalStateException("added multiple mappings for cf id " + newCfs.metadata.cfId);
+                throw new IllegalStateException("added multiple mappings for cf id " + newCfs.metadata.id);
         }
         else
         {
@@ -397,25 +408,25 @@ public class Keyspace
     /**
      * adds a cf to internal structures, ends up creating disk files).
      */
-    public void initCf(CFMetaData metadata, boolean loadSSTables)
+    public void initCf(TableMetadataRef metadata, boolean loadSSTables)
     {
-        ColumnFamilyStore cfs = columnFamilyStores.get(metadata.cfId);
+        ColumnFamilyStore cfs = columnFamilyStores.get(metadata.id);
 
         if (cfs == null)
         {
             // CFS being created for the first time, either on server startup or new CF being added.
             // We don't worry about races here; startup is safe, and adding multiple idential CFs
             // simultaneously is a "don't do that" scenario.
-            ColumnFamilyStore oldCfs = columnFamilyStores.putIfAbsent(metadata.cfId, ColumnFamilyStore.createColumnFamilyStore(this, metadata, loadSSTables));
+            ColumnFamilyStore oldCfs = columnFamilyStores.putIfAbsent(metadata.id, ColumnFamilyStore.createColumnFamilyStore(this, metadata, loadSSTables));
             // CFS mbean instantiation will error out before we hit this, but in case that changes...
             if (oldCfs != null)
-                throw new IllegalStateException("added multiple mappings for cf id " + metadata.cfId);
+                throw new IllegalStateException("added multiple mappings for cf id " + metadata.id);
         }
         else
         {
             // re-initializing an existing CF.  This will happen if you cleared the schema
             // on this node and it's getting repopulated from the rest of the cluster.
-            assert cfs.name.equals(metadata.cfName);
+            assert cfs.name.equals(metadata.name);
             cfs.reload();
         }
     }
@@ -490,14 +501,14 @@ public class Keyspace
             mutation.viewLockAcquireStart.compareAndSet(0L, System.currentTimeMillis());
 
             // the order of lock acquisition doesn't matter (from a deadlock perspective) because we only use tryLock()
-            Collection<UUID> columnFamilyIds = mutation.getColumnFamilyIds();
-            Iterator<UUID> idIterator = columnFamilyIds.iterator();
+            Collection<TableId> tableIds = mutation.getTableIds();
+            Iterator<TableId> idIterator = tableIds.iterator();
 
-            locks = new Lock[columnFamilyIds.size()];
-            for (int i = 0; i < columnFamilyIds.size(); i++)
+            locks = new Lock[tableIds.size()];
+            for (int i = 0; i < tableIds.size(); i++)
             {
-                UUID cfid = idIterator.next();
-                int lockKey = Objects.hash(mutation.key().getKey(), cfid);
+                TableId tableId = idIterator.next();
+                int lockKey = Objects.hash(mutation.key().getKey(), tableId);
                 while (true)
                 {
                     Lock lock = null;
@@ -515,7 +526,7 @@ public class Keyspace
                             for (int j = 0; j < i; j++)
                                 locks[j].unlock();
 
-                            logger.trace("Could not acquire lock for {} and table {}", ByteBufferUtil.bytesToHex(mutation.key().getKey()), columnFamilyStores.get(cfid).name);
+                            logger.trace("Could not acquire lock for {} and table {}", ByteBufferUtil.bytesToHex(mutation.key().getKey()), columnFamilyStores.get(tableId).name);
                             Tracing.trace("Could not acquire MV lock");
                             if (future != null)
                             {
@@ -569,8 +580,8 @@ public class Keyspace
             // Bulk non-droppable operations (e.g. commitlog replay, hint delivery) are not measured
             if (isDroppable)
             {
-                for(UUID cfid : columnFamilyIds)
-                    columnFamilyStores.get(cfid).metric.viewLockAcquireTime.update(acquireTime, TimeUnit.MILLISECONDS);
+                for(TableId tableId : tableIds)
+                    columnFamilyStores.get(tableId).metric.viewLockAcquireTime.update(acquireTime, TimeUnit.MILLISECONDS);
             }
         }
         int nowInSec = FBUtilities.nowInSeconds();
@@ -586,10 +597,10 @@ public class Keyspace
 
             for (PartitionUpdate upd : mutation.getPartitionUpdates())
             {
-                ColumnFamilyStore cfs = columnFamilyStores.get(upd.metadata().cfId);
+                ColumnFamilyStore cfs = columnFamilyStores.get(upd.metadata().id);
                 if (cfs == null)
                 {
-                    logger.error("Attempting to mutate non-existant table {} ({}.{})", upd.metadata().cfId, upd.metadata().ksName, upd.metadata().cfName);
+                    logger.error("Attempting to mutate non-existant table {} ({}.{})", upd.metadata().id, upd.metadata().keyspace, upd.metadata().name);
                     continue;
                 }
                 AtomicLong baseComplete = new AtomicLong(Long.MAX_VALUE);
@@ -599,18 +610,18 @@ public class Keyspace
                     try
                     {
                         Tracing.trace("Creating materialized view mutations from base table replica");
-                        viewManager.forTable(upd.metadata()).pushViewReplicaUpdates(upd, writeCommitLog, baseComplete);
+                        viewManager.forTable(upd.metadata().id).pushViewReplicaUpdates(upd, writeCommitLog, baseComplete);
                     }
                     catch (Throwable t)
                     {
                         JVMStabilityInspector.inspectThrowable(t);
-                        logger.error(String.format("Unknown exception caught while attempting to update MaterializedView! %s.%s",
-                                     upd.metadata().ksName, upd.metadata().cfName), t);
+                        logger.error(String.format("Unknown exception caught while attempting to update MaterializedView! %s",
+                                                   upd.metadata().toString()), t);
                         throw t;
                     }
                 }
 
-                Tracing.trace("Adding to {} memtable", upd.metadata().cfName);
+                Tracing.trace("Adding to {} memtable", upd.metadata().name);
                 UpdateTransaction indexTransaction = updateIndexes
                                                      ? cfs.indexManager.newUpdateTransaction(upd, opGroup, nowInSec)
                                                      : UpdateTransaction.NO_OP;
@@ -683,7 +694,7 @@ public class Keyspace
                 Index index = baseCfs.indexManager.getIndexByName(indexName);
                 if (index == null)
                     throw new IllegalArgumentException(String.format("Invalid index specified: %s/%s.",
-                                                                     baseCfs.metadata.cfName,
+                                                                     baseCfs.metadata.name,
                                                                      indexName));
 
                 if (index.getBackingTable().isPresent())
@@ -706,7 +717,7 @@ public class Keyspace
         Set<ColumnFamilyStore> stores = new HashSet<>();
         for (ColumnFamilyStore indexCfs : baseCfs.indexManager.getAllIndexColumnFamilyStores())
         {
-            logger.info("adding secondary index table {} to operation", indexCfs.metadata.cfName);
+            logger.info("adding secondary index table {} to operation", indexCfs.metadata.name);
             stores.add(indexCfs);
         }
         return stores;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 e2b0fc6..01e741d 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -29,14 +29,13 @@ import com.google.common.base.Throwables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogPosition;
 import org.apache.cassandra.db.commitlog.IntervalSet;
-import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
@@ -145,19 +144,19 @@ public class Memtable implements Comparable<Memtable>
         this.cfs = cfs;
         this.commitLogLowerBound = commitLogLowerBound;
         this.allocator = MEMORY_POOL.newAllocator();
-        this.initialComparator = cfs.metadata.comparator;
+        this.initialComparator = cfs.metadata().comparator;
         this.cfs.scheduleFlush();
-        this.columnsCollector = new ColumnsCollector(cfs.metadata.partitionColumns());
+        this.columnsCollector = new ColumnsCollector(cfs.metadata().regularAndStaticColumns());
     }
 
     // ONLY to be used for testing, to create a mock Memtable
     @VisibleForTesting
-    public Memtable(CFMetaData metadata)
+    public Memtable(TableMetadata metadata)
     {
         this.initialComparator = metadata.comparator;
         this.cfs = null;
         this.allocator = null;
-        this.columnsCollector = new ColumnsCollector(metadata.partitionColumns());
+        this.columnsCollector = new ColumnsCollector(metadata.regularAndStaticColumns());
     }
 
     public MemtableAllocator getAllocator()
@@ -249,7 +248,7 @@ public class Memtable implements Comparable<Memtable>
      */
     public boolean isExpired()
     {
-        int period = cfs.metadata.params.memtableFlushPeriodInMs;
+        int period = cfs.metadata().params.memtableFlushPeriodInMs;
         return period > 0 && (System.nanoTime() - creationNano >= TimeUnit.MILLISECONDS.toNanos(period));
     }
 
@@ -500,17 +499,17 @@ public class Memtable implements Comparable<Memtable>
 
         public SSTableMultiWriter createFlushWriter(LifecycleTransaction txn,
                                                     Descriptor descriptor,
-                                                    PartitionColumns columns,
+                                                    RegularAndStaticColumns columns,
                                                     EncodingStats stats)
         {
-            MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator)
+            MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata().comparator)
                     .commitLogIntervals(new IntervalSet<>(commitLogLowerBound.get(), commitLogUpperBound.get()));
 
             return cfs.createSSTableMultiWriter(descriptor,
                                                 toFlush.size(),
                                                 ActiveRepairService.UNREPAIRED_SSTABLE,
                                                 sstableMetadataCollector,
-                                                new SerializationHeader(true, cfs.metadata, columns, stats), txn);
+                                                new SerializationHeader(true, cfs.metadata(), columns, stats), txn);
         }
 
         @Override
@@ -564,9 +563,9 @@ public class Memtable implements Comparable<Memtable>
             return minLocalDeletionTime;
         }
 
-        public CFMetaData metadata()
+        public TableMetadata metadata()
         {
-            return cfs.metadata;
+            return cfs.metadata();
         }
 
         public boolean hasNext()
@@ -588,25 +587,25 @@ public class Memtable implements Comparable<Memtable>
 
     private static class ColumnsCollector
     {
-        private final HashMap<ColumnDefinition, AtomicBoolean> predefined = new HashMap<>();
-        private final ConcurrentSkipListSet<ColumnDefinition> extra = new ConcurrentSkipListSet<>();
-        ColumnsCollector(PartitionColumns columns)
+        private final HashMap<ColumnMetadata, AtomicBoolean> predefined = new HashMap<>();
+        private final ConcurrentSkipListSet<ColumnMetadata> extra = new ConcurrentSkipListSet<>();
+        ColumnsCollector(RegularAndStaticColumns columns)
         {
-            for (ColumnDefinition def : columns.statics)
+            for (ColumnMetadata def : columns.statics)
                 predefined.put(def, new AtomicBoolean());
-            for (ColumnDefinition def : columns.regulars)
+            for (ColumnMetadata def : columns.regulars)
                 predefined.put(def, new AtomicBoolean());
         }
 
-        public void update(PartitionColumns columns)
+        public void update(RegularAndStaticColumns columns)
         {
-            for (ColumnDefinition s : columns.statics)
+            for (ColumnMetadata s : columns.statics)
                 update(s);
-            for (ColumnDefinition r : columns.regulars)
+            for (ColumnMetadata r : columns.regulars)
                 update(r);
         }
 
-        private void update(ColumnDefinition definition)
+        private void update(ColumnMetadata definition)
         {
             AtomicBoolean present = predefined.get(definition);
             if (present != null)
@@ -620,10 +619,10 @@ public class Memtable implements Comparable<Memtable>
             }
         }
 
-        public PartitionColumns get()
+        public RegularAndStaticColumns get()
         {
-            PartitionColumns.Builder builder = PartitionColumns.builder();
-            for (Map.Entry<ColumnDefinition, AtomicBoolean> e : predefined.entrySet())
+            RegularAndStaticColumns.Builder builder = RegularAndStaticColumns.builder();
+            for (Map.Entry<ColumnMetadata, AtomicBoolean> e : predefined.entrySet())
                 if (e.getValue().get())
                     builder.add(e.getKey());
             return builder.addAll(extra).build();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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
deleted file mode 100644
index 3666b27..0000000
--- a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
+++ /dev/null
@@ -1,48 +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.util.Collection;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-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.SchemaKeyspace;
-import org.apache.cassandra.service.MigrationManager;
-
-/**
- * Sends it's current schema state in form of mutations in reply to the remote node's request.
- * Such a request is made when one of the nodes, by means of Gossip, detects schema disagreement in the ring.
- */
-public class MigrationRequestVerbHandler implements IVerbHandler
-{
-    private static final Logger logger = LoggerFactory.getLogger(MigrationRequestVerbHandler.class);
-
-    public void doVerb(MessageIn message, int id)
-    {
-        logger.trace("Received migration request from {}.", message.from);
-        MessageOut<Collection<Mutation>> response = new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE,
-                                                                     SchemaKeyspace.convertSchemaToMutations(),
-                                                                     MigrationManager.MigrationsSerializer.instance);
-        MessagingService.instance().sendReply(response, id, message.from);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/MultiCBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/MultiCBuilder.java b/src/java/org/apache/cassandra/db/MultiCBuilder.java
index ae8c26c..c4cff02 100644
--- a/src/java/org/apache/cassandra/db/MultiCBuilder.java
+++ b/src/java/org/apache/cassandra/db/MultiCBuilder.java
@@ -23,7 +23,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.NavigableSet;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.btree.BTreeSet;
 
@@ -177,7 +177,7 @@ public abstract class MultiCBuilder
     public abstract NavigableSet<ClusteringBound> buildBoundForSlice(boolean isStart,
                                                                  boolean isInclusive,
                                                                  boolean isOtherBoundInclusive,
-                                                                 List<ColumnDefinition> columnDefs);
+                                                                 List<ColumnMetadata> columnDefs);
 
     /**
      * Builds the <code>ClusteringBound</code>s
@@ -266,7 +266,7 @@ public abstract class MultiCBuilder
         public NavigableSet<ClusteringBound> buildBoundForSlice(boolean isStart,
                                                                 boolean isInclusive,
                                                                 boolean isOtherBoundInclusive,
-                                                                List<ColumnDefinition> columnDefs)
+                                                                List<ColumnMetadata> columnDefs)
         {
             return buildBound(isStart, columnDefs.get(0).isReversedType() ? isOtherBoundInclusive : isInclusive);
         }
@@ -421,7 +421,7 @@ public abstract class MultiCBuilder
         public NavigableSet<ClusteringBound> buildBoundForSlice(boolean isStart,
                                                             boolean isInclusive,
                                                             boolean isOtherBoundInclusive,
-                                                            List<ColumnDefinition> columnDefs)
+                                                            List<ColumnMetadata> columnDefs)
         {
             built = true;
 
@@ -454,7 +454,7 @@ public abstract class MultiCBuilder
                 // For example: if we have clustering_0 DESC and clustering_1 ASC a slice like (clustering_0, clustering_1) > (1, 2)
                 // will produce 2 slices: [BOTTOM, 1) and (1.2, 1]
                 // So, the END bound will return 2 bounds with the same values 1
-                ColumnDefinition lastColumn = columnDefs.get(columnDefs.size() - 1);
+                ColumnMetadata lastColumn = columnDefs.get(columnDefs.size() - 1);
                 if (elements.size() <= lastColumn.position() && i < m - 1 && elements.equals(elementsList.get(i + 1)))
                 {
                     set.add(builder.buildBoundWith(elements, isStart, false));
@@ -463,7 +463,7 @@ public abstract class MultiCBuilder
                 }
 
                 // Handle the normal bounds
-                ColumnDefinition column = columnDefs.get(elements.size() - 1 - offset);
+                ColumnMetadata column = columnDefs.get(elements.size() - 1 - offset);
                 set.add(builder.buildBoundWith(elements, isStart, column.isReversedType() ? isOtherBoundInclusive : isInclusive));
             }
             return set.build();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/Mutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java
index f7ba43b..062e1fe 100644
--- a/src/java/org/apache/cassandra/db/Mutation.java
+++ b/src/java/org/apache/cassandra/db/Mutation.java
@@ -18,18 +18,13 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicLong;
 
-import com.google.common.base.Throwables;
-import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.cassandra.config.CFMetaData;
+
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.SerializationHelper;
 import org.apache.cassandra.io.IVersionedSerializer;
@@ -37,6 +32,9 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 // TODO convert this to a Builder pattern instead of encouraging M.add directly,
@@ -54,7 +52,7 @@ public class Mutation implements IMutation
 
     private final DecoratedKey key;
     // map of column family id to mutations for that column family.
-    private final Map<UUID, PartitionUpdate> modifications;
+    private final Map<TableId, PartitionUpdate> modifications;
 
     // Time at which this mutation was instantiated
     public final long createdAt = System.currentTimeMillis();
@@ -70,10 +68,10 @@ public class Mutation implements IMutation
 
     public Mutation(PartitionUpdate update)
     {
-        this(update.metadata().ksName, update.partitionKey(), Collections.singletonMap(update.metadata().cfId, update));
+        this(update.metadata().keyspace, update.partitionKey(), Collections.singletonMap(update.metadata().id, update));
     }
 
-    protected Mutation(String keyspaceName, DecoratedKey key, Map<UUID, PartitionUpdate> modifications)
+    protected Mutation(String keyspaceName, DecoratedKey key, Map<TableId, PartitionUpdate> modifications)
     {
         this.keyspaceName = keyspaceName;
         this.key = key;
@@ -87,13 +85,14 @@ public class Mutation implements IMutation
         return new Mutation(keyspaceName, key, new HashMap<>(modifications));
     }
 
-    public Mutation without(Set<UUID> cfIds)
+    public Mutation without(Set<TableId> tableIds)
     {
-        if (cfIds.isEmpty())
+        if (tableIds.isEmpty())
             return this;
 
         Mutation copy = copy();
-        copy.modifications.keySet().removeAll(cfIds);
+
+        copy.modifications.keySet().removeAll(tableIds);
 
         copy.cdcEnabled = false;
         for (PartitionUpdate pu : modifications.values())
@@ -102,9 +101,9 @@ public class Mutation implements IMutation
         return copy;
     }
 
-    public Mutation without(UUID cfId)
+    public Mutation without(TableId tableId)
     {
-        return without(Collections.singleton(cfId));
+        return without(Collections.singleton(tableId));
     }
 
     public String getKeyspaceName()
@@ -112,7 +111,7 @@ public class Mutation implements IMutation
         return keyspaceName;
     }
 
-    public Collection<UUID> getColumnFamilyIds()
+    public Collection<TableId> getTableIds()
     {
         return modifications.keySet();
     }
@@ -127,9 +126,9 @@ public class Mutation implements IMutation
         return modifications.values();
     }
 
-    public PartitionUpdate getPartitionUpdate(UUID cfId)
+    public PartitionUpdate getPartitionUpdate(TableMetadata table)
     {
-        return modifications.get(cfId);
+        return table == null ? null : modifications.get(table.id);
     }
 
     /**
@@ -147,16 +146,16 @@ public class Mutation implements IMutation
 
         cdcEnabled |= update.metadata().params.cdc;
 
-        PartitionUpdate prev = modifications.put(update.metadata().cfId, update);
+        PartitionUpdate prev = modifications.put(update.metadata().id, update);
         if (prev != null)
             // developer error
-            throw new IllegalArgumentException("Table " + update.metadata().cfName + " already has modifications in this mutation: " + prev);
+            throw new IllegalArgumentException("Table " + update.metadata().name + " already has modifications in this mutation: " + prev);
         return this;
     }
 
-    public PartitionUpdate get(CFMetaData cfm)
+    public PartitionUpdate get(TableMetadata metadata)
     {
-        return modifications.get(cfm.cfId);
+        return modifications.get(metadata.id);
     }
 
     public boolean isEmpty()
@@ -182,7 +181,7 @@ public class Mutation implements IMutation
         if (mutations.size() == 1)
             return mutations.get(0);
 
-        Set<UUID> updatedTables = new HashSet<>();
+        Set<TableId> updatedTables = new HashSet<>();
         String ks = null;
         DecoratedKey key = null;
         for (Mutation mutation : mutations)
@@ -197,8 +196,8 @@ public class Mutation implements IMutation
         }
 
         List<PartitionUpdate> updates = new ArrayList<>(mutations.size());
-        Map<UUID, PartitionUpdate> modifications = new HashMap<>(updatedTables.size());
-        for (UUID table : updatedTables)
+        Map<TableId, PartitionUpdate> modifications = new HashMap<>(updatedTables.size());
+        for (TableId table : updatedTables)
         {
             for (Mutation mutation : mutations)
             {
@@ -288,10 +287,10 @@ public class Mutation implements IMutation
         if (shallow)
         {
             List<String> cfnames = new ArrayList<>(modifications.size());
-            for (UUID cfid : modifications.keySet())
+            for (TableId tableId : modifications.keySet())
             {
-                CFMetaData cfm = Schema.instance.getCFMetaData(cfid);
-                cfnames.add(cfm == null ? "-dropped-" : cfm.cfName);
+                TableMetadata cfm = Schema.instance.getTableMetadata(tableId);
+                cfnames.add(cfm == null ? "-dropped-" : cfm.name);
             }
             buff.append(StringUtils.join(cfnames, ", "));
         }
@@ -349,7 +348,7 @@ public class Mutation implements IMutation
          * @return a builder for the partition identified by {@code metadata} (and the partition key for which this is a
          * mutation of).
          */
-        public PartitionUpdate.SimpleBuilder update(CFMetaData metadata);
+        public PartitionUpdate.SimpleBuilder update(TableMetadata metadata);
 
         /**
          * Adds an update for table identified by the provided name and return a builder for that partition.
@@ -377,7 +376,7 @@ public class Mutation implements IMutation
             out.writeUnsignedVInt(size);
 
             assert size > 0;
-            for (Map.Entry<UUID, PartitionUpdate> entry : mutation.modifications.entrySet())
+            for (Map.Entry<TableId, PartitionUpdate> entry : mutation.modifications.entrySet())
                 PartitionUpdate.serializer.serialize(entry.getValue(), out, version);
         }
 
@@ -390,17 +389,17 @@ public class Mutation implements IMutation
             if (size == 1)
                 return new Mutation(update);
 
-            Map<UUID, PartitionUpdate> modifications = new HashMap<>(size);
+            Map<TableId, PartitionUpdate> modifications = new HashMap<>(size);
             DecoratedKey dk = update.partitionKey();
 
-            modifications.put(update.metadata().cfId, update);
+            modifications.put(update.metadata().id, update);
             for (int i = 1; i < size; ++i)
             {
                 update = PartitionUpdate.serializer.deserialize(in, version, flag);
-                modifications.put(update.metadata().cfId, update);
+                modifications.put(update.metadata().id, update);
             }
 
-            return new Mutation(update.metadata().ksName, dk, modifications);
+            return new Mutation(update.metadata().keyspace, dk, modifications);
         }
 
         public Mutation deserialize(DataInputPlus in, int version) throws IOException
@@ -411,7 +410,7 @@ public class Mutation implements IMutation
         public long serializedSize(Mutation mutation, int version)
         {
             int size = TypeSizes.sizeofUnsignedVInt(mutation.modifications.size());
-            for (Map.Entry<UUID, PartitionUpdate> entry : mutation.modifications.entrySet())
+            for (Map.Entry<TableId, PartitionUpdate> entry : mutation.modifications.entrySet())
                 size += PartitionUpdate.serializer.serializedSize(entry.getValue(), version);
 
             return size;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/PartitionColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PartitionColumns.java b/src/java/org/apache/cassandra/db/PartitionColumns.java
deleted file mode 100644
index bf4ac43..0000000
--- a/src/java/org/apache/cassandra/db/PartitionColumns.java
+++ /dev/null
@@ -1,199 +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.util.*;
-
-import com.google.common.collect.Iterators;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.utils.btree.BTreeSet;
-
-import static java.util.Comparator.naturalOrder;
-
-/**
- * Columns (or a subset of the columns) that a partition contains.
- * This mainly groups both static and regular columns for convenience.
- */
-public class PartitionColumns implements Iterable<ColumnDefinition>
-{
-    public static PartitionColumns NONE = new PartitionColumns(Columns.NONE, Columns.NONE);
-
-    public final Columns statics;
-    public final Columns regulars;
-
-    public PartitionColumns(Columns statics, Columns regulars)
-    {
-        assert statics != null && regulars != null;
-        this.statics = statics;
-        this.regulars = regulars;
-    }
-
-    public static PartitionColumns of(ColumnDefinition column)
-    {
-        return new PartitionColumns(column.isStatic() ? Columns.of(column) : Columns.NONE,
-                                    column.isStatic() ? Columns.NONE : Columns.of(column));
-    }
-
-    public PartitionColumns without(ColumnDefinition column)
-    {
-        return new PartitionColumns(column.isStatic() ? statics.without(column) : statics,
-                                    column.isStatic() ? regulars : regulars.without(column));
-    }
-
-    public PartitionColumns withoutStatics()
-    {
-        return statics.isEmpty() ? this : new PartitionColumns(Columns.NONE, regulars);
-    }
-
-    public PartitionColumns mergeTo(PartitionColumns that)
-    {
-        if (this == that)
-            return this;
-        Columns statics = this.statics.mergeTo(that.statics);
-        Columns regulars = this.regulars.mergeTo(that.regulars);
-        if (statics == this.statics && regulars == this.regulars)
-            return this;
-        if (statics == that.statics && regulars == that.regulars)
-            return that;
-        return new PartitionColumns(statics, regulars);
-    }
-
-    public boolean isEmpty()
-    {
-        return statics.isEmpty() && regulars.isEmpty();
-    }
-
-    public Columns columns(boolean isStatic)
-    {
-        return isStatic ? statics : regulars;
-    }
-
-    public boolean contains(ColumnDefinition column)
-    {
-        return column.isStatic() ? statics.contains(column) : regulars.contains(column);
-    }
-
-    public boolean includes(PartitionColumns columns)
-    {
-        return statics.containsAll(columns.statics) && regulars.containsAll(columns.regulars);
-    }
-
-    public Iterator<ColumnDefinition> iterator()
-    {
-        return Iterators.concat(statics.iterator(), regulars.iterator());
-    }
-
-    public Iterator<ColumnDefinition> selectOrderIterator()
-    {
-        return Iterators.concat(statics.selectOrderIterator(), regulars.selectOrderIterator());
-    }
-
-    /** * Returns the total number of static and regular columns. */
-    public int size()
-    {
-        return regulars.size() + statics.size();
-    }
-
-    @Override
-    public String toString()
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append("[").append(statics).append(" | ").append(regulars).append("]");
-        return sb.toString();
-    }
-
-    @Override
-    public boolean equals(Object other)
-    {
-        if (!(other instanceof PartitionColumns))
-            return false;
-
-        PartitionColumns that = (PartitionColumns)other;
-        return this.statics.equals(that.statics)
-            && this.regulars.equals(that.regulars);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return Objects.hash(statics, regulars);
-    }
-
-    public static Builder builder()
-    {
-        return new Builder();
-    }
-
-    public static class Builder
-    {
-        // Note that we do want to use sorted sets because we want the column definitions to be compared
-        // through compareTo, not equals. The former basically check it's the same column name, while the latter
-        // check it's the same object, including the same type.
-        private BTreeSet.Builder<ColumnDefinition> regularColumns;
-        private BTreeSet.Builder<ColumnDefinition> staticColumns;
-
-        public Builder add(ColumnDefinition c)
-        {
-            if (c.isStatic())
-            {
-                if (staticColumns == null)
-                    staticColumns = BTreeSet.builder(naturalOrder());
-                staticColumns.add(c);
-            }
-            else
-            {
-                assert c.isRegular();
-                if (regularColumns == null)
-                    regularColumns = BTreeSet.builder(naturalOrder());
-                regularColumns.add(c);
-            }
-            return this;
-        }
-
-        public Builder addAll(Iterable<ColumnDefinition> columns)
-        {
-            for (ColumnDefinition c : columns)
-                add(c);
-            return this;
-        }
-
-        public Builder addAll(PartitionColumns columns)
-        {
-            if (regularColumns == null && !columns.regulars.isEmpty())
-                regularColumns = BTreeSet.builder(naturalOrder());
-
-            for (ColumnDefinition c : columns.regulars)
-                regularColumns.add(c);
-
-            if (staticColumns == null && !columns.statics.isEmpty())
-                staticColumns = BTreeSet.builder(naturalOrder());
-
-            for (ColumnDefinition c : columns.statics)
-                staticColumns.add(c);
-
-            return this;
-        }
-
-        public PartitionColumns build()
-        {
-            return new PartitionColumns(staticColumns == null ? Columns.NONE : Columns.from(staticColumns.build()),
-                                        regularColumns == null ? Columns.NONE : Columns.from(regularColumns.build()));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
index df4d63c..a47302b 100644
--- a/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
+++ b/src/java/org/apache/cassandra/db/PartitionRangeReadCommand.java
@@ -24,7 +24,7 @@ import java.util.Optional;
 
 import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.lifecycle.View;
@@ -60,7 +60,7 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     public PartitionRangeReadCommand(boolean isDigest,
                                      int digestVersion,
-                                     CFMetaData metadata,
+                                     TableMetadata metadata,
                                      int nowInSec,
                                      ColumnFilter columnFilter,
                                      RowFilter rowFilter,
@@ -73,7 +73,7 @@ public class PartitionRangeReadCommand extends ReadCommand
         this.index = index;
     }
 
-    public PartitionRangeReadCommand(CFMetaData metadata,
+    public PartitionRangeReadCommand(TableMetadata metadata,
                                      int nowInSec,
                                      ColumnFilter columnFilter,
                                      RowFilter rowFilter,
@@ -92,7 +92,7 @@ public class PartitionRangeReadCommand extends ReadCommand
      *
      * @return a newly created read command that queries everything in the table.
      */
-    public static PartitionRangeReadCommand allDataRead(CFMetaData metadata, int nowInSec)
+    public static PartitionRangeReadCommand allDataRead(TableMetadata metadata, int nowInSec)
     {
         return new PartitionRangeReadCommand(metadata,
                                              nowInSec,
@@ -165,7 +165,7 @@ public class PartitionRangeReadCommand extends ReadCommand
         if (!dataRange().contains(key))
             return false;
 
-        return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().getKeyValidator());
+        return rowFilter().partitionKeyRestrictionsAreSatisfiedBy(key, metadata().partitionKeyType);
     }
 
     public boolean selectsClustering(DecoratedKey key, Clustering clustering)
@@ -196,7 +196,7 @@ public class PartitionRangeReadCommand extends ReadCommand
     protected UnfilteredPartitionIterator queryStorage(final ColumnFamilyStore cfs, ReadExecutionController executionController)
     {
         ColumnFamilyStore.ViewFragment view = cfs.select(View.selectLive(dataRange().keyRange()));
-        Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().getKeyValidator()));
+        Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), dataRange().keyRange().getString(metadata().partitionKeyType));
 
         // fetch data from current memtable, historical memtables, and SSTables in the correct order.
         final List<UnfilteredPartitionIterator> iterators = new ArrayList<>(Iterables.size(view.memtables) + view.sstables.size());
@@ -303,7 +303,7 @@ public class PartitionRangeReadCommand extends ReadCommand
      */
     public PartitionIterator postReconciliationProcessing(PartitionIterator result)
     {
-        ColumnFamilyStore cfs = Keyspace.open(metadata().ksName).getColumnFamilyStore(metadata().cfName);
+        ColumnFamilyStore cfs = Keyspace.open(metadata().keyspace).getColumnFamilyStore(metadata().name);
         Index index = getIndex(cfs);
         return index == null ? result : index.postProcessorFor(this).apply(result, this);
     }
@@ -311,9 +311,8 @@ public class PartitionRangeReadCommand extends ReadCommand
     @Override
     public String toString()
     {
-        return String.format("Read(%s.%s columns=%s rowfilter=%s limits=%s %s)",
-                             metadata().ksName,
-                             metadata().cfName,
+        return String.format("Read(%s columns=%s rowfilter=%s limits=%s %s)",
+                             metadata().toString(),
                              columnFilter(),
                              rowFilter(),
                              limits(),
@@ -332,7 +331,7 @@ public class PartitionRangeReadCommand extends ReadCommand
 
     private static class Deserializer extends SelectionDeserializer
     {
-        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
+        public ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, TableMetadata metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index)
         throws IOException
         {
             DataRange range = DataRange.serializer.deserialize(in, version, metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 51f8188..b8856a3 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -18,16 +18,13 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.function.Predicate;
 
-import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.*;
-import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.monitoring.ApproximateTime;
 import org.apache.cassandra.db.monitoring.MonitorableImpl;
@@ -35,7 +32,6 @@ import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.transform.StoppingTransformation;
 import org.apache.cassandra.db.transform.Transformation;
-import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.index.IndexNotAvailableException;
 import org.apache.cassandra.io.IVersionedSerializer;
@@ -43,14 +39,15 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.metrics.TableMetrics;
 import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.IndexMetadata;
-import org.apache.cassandra.schema.UnknownIndexException;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.exceptions.UnknownIndexException;
 import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
 
 /**
  * General interface for storage-engine read commands (common to both range and
@@ -65,7 +62,7 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
     public static final IVersionedSerializer<ReadCommand> serializer = new Serializer();
 
     private final Kind kind;
-    private final CFMetaData metadata;
+    private final TableMetadata metadata;
     private final int nowInSec;
 
     private final ColumnFilter columnFilter;
@@ -88,7 +85,7 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
 
     protected static abstract class SelectionDeserializer
     {
-        public abstract ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, CFMetaData metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index) throws IOException;
+        public abstract ReadCommand deserialize(DataInputPlus in, int version, boolean isDigest, int digestVersion, TableMetadata metadata, int nowInSec, ColumnFilter columnFilter, RowFilter rowFilter, DataLimits limits, Optional<IndexMetadata> index) throws IOException;
     }
 
     protected enum Kind
@@ -107,7 +104,7 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
     protected ReadCommand(Kind kind,
                           boolean isDigestQuery,
                           int digestVersion,
-                          CFMetaData metadata,
+                          TableMetadata metadata,
                           int nowInSec,
                           ColumnFilter columnFilter,
                           RowFilter rowFilter,
@@ -139,7 +136,7 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
      *
      * @return the metadata for the table queried.
      */
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
         return metadata;
     }
@@ -349,7 +346,7 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
                 throw new IndexNotAvailableException(index);
 
             searcher = index.searcherFor(this);
-            Tracing.trace("Executing read on {}.{} using index {}", cfs.metadata.ksName, cfs.metadata.cfName, index.getIndexMetadata().name);
+            Tracing.trace("Executing read on {}.{} using index {}", cfs.metadata.keyspace, cfs.metadata.name, index.getIndexMetadata().name);
         }
 
         UnfilteredPartitionIterator resultIterator = searcher == null
@@ -403,7 +400,7 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
             private final int failureThreshold = DatabaseDescriptor.getTombstoneFailureThreshold();
             private final int warningThreshold = DatabaseDescriptor.getTombstoneWarnThreshold();
 
-            private final boolean respectTombstoneThresholds = !SchemaConstants.isSystemKeyspace(ReadCommand.this.metadata().ksName);
+            private final boolean respectTombstoneThresholds = !SchemaConstants.isSystemKeyspace(ReadCommand.this.metadata().keyspace);
 
             private int liveRows = 0;
             private int tombstones = 0;
@@ -524,7 +521,7 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
 
         private void maybeDelayForTesting()
         {
-            if (!metadata.ksName.startsWith("system"))
+            if (!metadata.keyspace.startsWith("system"))
                 FBUtilities.sleepQuietly(TEST_ITERATION_DELAY_MILLIS);
         }
     }
@@ -574,7 +571,7 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
     {
         StringBuilder sb = new StringBuilder();
         sb.append("SELECT ").append(columnFilter());
-        sb.append(" FROM ").append(metadata().ksName).append('.').append(metadata.cfName);
+        sb.append(" FROM ").append(metadata().keyspace).append('.').append(metadata.name);
         appendCQLWhereClause(sb);
 
         if (limits() != DataLimits.NONE)
@@ -626,7 +623,7 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
             out.writeByte(digestFlag(command.isDigestQuery()) | indexFlag(command.index.isPresent()));
             if (command.isDigestQuery())
                 out.writeUnsignedVInt(command.digestVersion());
-            CFMetaData.serializer.serialize(command.metadata(), out, version);
+            command.metadata.id.serialize(out);
             out.writeInt(command.nowInSec());
             ColumnFilter.serializer.serialize(command.columnFilter(), out, version);
             RowFilter.serializer.serialize(command.rowFilter(), out, version);
@@ -652,11 +649,11 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
 
             boolean hasIndex = hasIndex(flags);
             int digestVersion = isDigest ? (int)in.readUnsignedVInt() : 0;
-            CFMetaData metadata = CFMetaData.serializer.deserialize(in, version);
+            TableMetadata metadata = Schema.instance.getExistingTableMetadata(TableId.deserialize(in));
             int nowInSec = in.readInt();
             ColumnFilter columnFilter = ColumnFilter.serializer.deserialize(in, version, metadata);
             RowFilter rowFilter = RowFilter.serializer.deserialize(in, version, metadata);
-            DataLimits limits = DataLimits.serializer.deserialize(in, version,  metadata.comparator);
+            DataLimits limits = DataLimits.serializer.deserialize(in, version, metadata.comparator);
             Optional<IndexMetadata> index = hasIndex
                                           ? deserializeIndexMetadata(in, version, metadata)
                                           : Optional.empty();
@@ -664,11 +661,11 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
             return kind.selectionDeserializer.deserialize(in, version, isDigest, digestVersion, metadata, nowInSec, columnFilter, rowFilter, limits, index);
         }
 
-        private Optional<IndexMetadata> deserializeIndexMetadata(DataInputPlus in, int version, CFMetaData cfm) throws IOException
+        private Optional<IndexMetadata> deserializeIndexMetadata(DataInputPlus in, int version, TableMetadata metadata) throws IOException
         {
             try
             {
-                return Optional.of(IndexMetadata.serializer.deserialize(in, version, cfm));
+                return Optional.of(IndexMetadata.serializer.deserialize(in, version, metadata));
             }
             catch (UnknownIndexException e)
             {
@@ -676,7 +673,7 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
                             "If an index was just created, this is likely due to the schema not " +
                             "being fully propagated. Local read will proceed without using the " +
                             "index. Please wait for schema agreement after index creation.",
-                            cfm.ksName, cfm.cfName, e.indexId);
+                            metadata.keyspace, metadata.name, e.indexId);
                 return Optional.empty();
             }
         }
@@ -684,14 +681,14 @@ public abstract class ReadCommand extends MonitorableImpl implements ReadQuery
         public long serializedSize(ReadCommand command, int version)
         {
             return 2 // kind + flags
-                 + (command.isDigestQuery() ? TypeSizes.sizeofUnsignedVInt(command.digestVersion()) : 0)
-                 + CFMetaData.serializer.serializedSize(command.metadata(), version)
-                 + TypeSizes.sizeof(command.nowInSec())
-                 + ColumnFilter.serializer.serializedSize(command.columnFilter(), version)
-                 + RowFilter.serializer.serializedSize(command.rowFilter(), version)
-                 + DataLimits.serializer.serializedSize(command.limits(), version, command.metadata.comparator)
-                 + command.selectionSerializedSize(version)
-                 + command.indexSerializedSize(version);
+                   + (command.isDigestQuery() ? TypeSizes.sizeofUnsignedVInt(command.digestVersion()) : 0)
+                   + command.metadata.id.serializedSize()
+                   + TypeSizes.sizeof(command.nowInSec())
+                   + ColumnFilter.serializer.serializedSize(command.columnFilter(), version)
+                   + RowFilter.serializer.serializedSize(command.rowFilter(), version)
+                   + DataLimits.serializer.serializedSize(command.limits(), version, command.metadata.comparator)
+                   + command.selectionSerializedSize(version)
+                   + command.indexSerializedSize(version);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ReadExecutionController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadExecutionController.java b/src/java/org/apache/cassandra/db/ReadExecutionController.java
index 56bb0d3..9114212 100644
--- a/src/java/org/apache/cassandra/db/ReadExecutionController.java
+++ b/src/java/org/apache/cassandra/db/ReadExecutionController.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.db;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
@@ -25,13 +25,13 @@ public class ReadExecutionController implements AutoCloseable
 {
     // For every reads
     private final OpOrder.Group baseOp;
-    private final CFMetaData baseMetadata; // kept to sanity check that we have take the op order on the right table
+    private final TableMetadata baseMetadata; // kept to sanity check that we have take the op order on the right table
 
     // For index reads
     private final ReadExecutionController indexController;
     private final OpOrder.Group writeOp;
 
-    private ReadExecutionController(OpOrder.Group baseOp, CFMetaData baseMetadata, ReadExecutionController indexController, OpOrder.Group writeOp)
+    private ReadExecutionController(OpOrder.Group baseOp, TableMetadata baseMetadata, ReadExecutionController indexController, OpOrder.Group writeOp)
     {
         // We can have baseOp == null, but only when empty() is called, in which case the controller will never really be used
         // (which validForReadOn should ensure). But if it's not null, we should have the proper metadata too.
@@ -54,7 +54,7 @@ public class ReadExecutionController implements AutoCloseable
 
     public boolean validForReadOn(ColumnFamilyStore cfs)
     {
-        return baseOp != null && cfs.metadata.cfId.equals(baseMetadata.cfId);
+        return baseOp != null && cfs.metadata.id.equals(baseMetadata.id);
     }
 
     public static ReadExecutionController empty()
@@ -79,7 +79,7 @@ public class ReadExecutionController implements AutoCloseable
 
         if (indexCfs == null)
         {
-            return new ReadExecutionController(baseCfs.readOrdering.start(), baseCfs.metadata, null, null);
+            return new ReadExecutionController(baseCfs.readOrdering.start(), baseCfs.metadata(), null, null);
         }
         else
         {
@@ -89,11 +89,11 @@ public class ReadExecutionController implements AutoCloseable
             try
             {
                 baseOp = baseCfs.readOrdering.start();
-                indexController = new ReadExecutionController(indexCfs.readOrdering.start(), indexCfs.metadata, null, null);
+                indexController = new ReadExecutionController(indexCfs.readOrdering.start(), indexCfs.metadata(), null, null);
                 // TODO: this should perhaps not open and maintain a writeOp for the full duration, but instead only *try* to delete stale entries, without blocking if there's no room
                 // as it stands, we open a writeOp and keep it open for the duration to ensure that should this CF get flushed to make room we don't block the reclamation of any room being made
                 writeOp = Keyspace.writeOrder.start();
-                return new ReadExecutionController(baseOp, baseCfs.metadata, indexController, writeOp);
+                return new ReadExecutionController(baseOp, baseCfs.metadata(), indexController, writeOp);
             }
             catch (RuntimeException e)
             {
@@ -120,7 +120,7 @@ public class ReadExecutionController implements AutoCloseable
         return index == null ? null : index.getBackingTable().orElse(null);
     }
 
-    public CFMetaData metaData()
+    public TableMetadata metadata()
     {
         return baseMetadata;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ReadQuery.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadQuery.java b/src/java/org/apache/cassandra/db/ReadQuery.java
index 806cfac..338ab1e 100644
--- a/src/java/org/apache/cassandra/db/ReadQuery.java
+++ b/src/java/org/apache/cassandra/db/ReadQuery.java
@@ -53,7 +53,7 @@ public interface ReadQuery
 
         public UnfilteredPartitionIterator executeLocally(ReadExecutionController executionController)
         {
-            return EmptyIterators.unfilteredPartition(executionController.metaData());
+            return EmptyIterators.unfilteredPartition(executionController.metadata());
         }
 
         public DataLimits limits()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ReadResponse.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadResponse.java b/src/java/org/apache/cassandra/db/ReadResponse.java
index 7cf04a4..52d30c2 100644
--- a/src/java/org/apache/cassandra/db/ReadResponse.java
+++ b/src/java/org/apache/cassandra/db/ReadResponse.java
@@ -24,13 +24,13 @@ import java.security.MessageDigest;
 import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java b/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java
new file mode 100644
index 0000000..fab7730
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/RegularAndStaticColumns.java
@@ -0,0 +1,194 @@
+/*
+ * 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.util.*;
+
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.utils.btree.BTreeSet;
+
+import static java.util.Comparator.naturalOrder;
+
+/**
+ * Columns (or a subset of the columns) that a partition contains.
+ * This mainly groups both static and regular columns for convenience.
+ */
+public class RegularAndStaticColumns implements Iterable<ColumnMetadata>
+{
+    public static RegularAndStaticColumns NONE = new RegularAndStaticColumns(Columns.NONE, Columns.NONE);
+
+    public final Columns statics;
+    public final Columns regulars;
+
+    public RegularAndStaticColumns(Columns statics, Columns regulars)
+    {
+        assert statics != null && regulars != null;
+        this.statics = statics;
+        this.regulars = regulars;
+    }
+
+    public static RegularAndStaticColumns of(ColumnMetadata column)
+    {
+        return new RegularAndStaticColumns(column.isStatic() ? Columns.of(column) : Columns.NONE,
+                                           column.isStatic() ? Columns.NONE : Columns.of(column));
+    }
+
+    public RegularAndStaticColumns without(ColumnMetadata column)
+    {
+        return new RegularAndStaticColumns(column.isStatic() ? statics.without(column) : statics,
+                                           column.isStatic() ? regulars : regulars.without(column));
+    }
+
+    public RegularAndStaticColumns mergeTo(RegularAndStaticColumns that)
+    {
+        if (this == that)
+            return this;
+        Columns statics = this.statics.mergeTo(that.statics);
+        Columns regulars = this.regulars.mergeTo(that.regulars);
+        if (statics == this.statics && regulars == this.regulars)
+            return this;
+        if (statics == that.statics && regulars == that.regulars)
+            return that;
+        return new RegularAndStaticColumns(statics, regulars);
+    }
+
+    public boolean isEmpty()
+    {
+        return statics.isEmpty() && regulars.isEmpty();
+    }
+
+    public Columns columns(boolean isStatic)
+    {
+        return isStatic ? statics : regulars;
+    }
+
+    public boolean contains(ColumnMetadata column)
+    {
+        return column.isStatic() ? statics.contains(column) : regulars.contains(column);
+    }
+
+    public boolean includes(RegularAndStaticColumns columns)
+    {
+        return statics.containsAll(columns.statics) && regulars.containsAll(columns.regulars);
+    }
+
+    public Iterator<ColumnMetadata> iterator()
+    {
+        return Iterators.concat(statics.iterator(), regulars.iterator());
+    }
+
+    public Iterator<ColumnMetadata> selectOrderIterator()
+    {
+        return Iterators.concat(statics.selectOrderIterator(), regulars.selectOrderIterator());
+    }
+
+    /** * Returns the total number of static and regular columns. */
+    public int size()
+    {
+        return regulars.size() + statics.size();
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append("[").append(statics).append(" | ").append(regulars).append("]");
+        return sb.toString();
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (!(other instanceof RegularAndStaticColumns))
+            return false;
+
+        RegularAndStaticColumns that = (RegularAndStaticColumns)other;
+        return this.statics.equals(that.statics)
+            && this.regulars.equals(that.regulars);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(statics, regulars);
+    }
+
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static class Builder
+    {
+        // Note that we do want to use sorted sets because we want the column definitions to be compared
+        // through compareTo, not equals. The former basically check it's the same column name, while the latter
+        // check it's the same object, including the same type.
+        private BTreeSet.Builder<ColumnMetadata> regularColumns;
+        private BTreeSet.Builder<ColumnMetadata> staticColumns;
+
+        public Builder add(ColumnMetadata c)
+        {
+            if (c.isStatic())
+            {
+                if (staticColumns == null)
+                    staticColumns = BTreeSet.builder(naturalOrder());
+                staticColumns.add(c);
+            }
+            else
+            {
+                assert c.isRegular();
+                if (regularColumns == null)
+                    regularColumns = BTreeSet.builder(naturalOrder());
+                regularColumns.add(c);
+            }
+            return this;
+        }
+
+        public Builder addAll(Iterable<ColumnMetadata> columns)
+        {
+            for (ColumnMetadata c : columns)
+                add(c);
+            return this;
+        }
+
+        public Builder addAll(RegularAndStaticColumns columns)
+        {
+            if (regularColumns == null && !columns.regulars.isEmpty())
+                regularColumns = BTreeSet.builder(naturalOrder());
+
+            for (ColumnMetadata c : columns.regulars)
+                regularColumns.add(c);
+
+            if (staticColumns == null && !columns.statics.isEmpty())
+                staticColumns = BTreeSet.builder(naturalOrder());
+
+            for (ColumnMetadata c : columns.statics)
+                staticColumns.add(c);
+
+            return this;
+        }
+
+        public RegularAndStaticColumns build()
+        {
+            return new RegularAndStaticColumns(staticColumns  == null ? Columns.NONE : Columns.from(staticColumns.build()),
+                                               regularColumns == null ? Columns.NONE : Columns.from(regularColumns.build()));
+        }
+    }
+}


[35/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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
deleted file mode 100644
index c6fc2a8..0000000
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ /dev/null
@@ -1,776 +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.util.*;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Sets;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.cql3.functions.*;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.UserType;
-import org.apache.cassandra.index.Index;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.locator.LocalStrategy;
-import org.apache.cassandra.schema.*;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.utils.ConcurrentBiMap;
-import org.apache.cassandra.utils.Pair;
-import org.cliffc.high_scale_lib.NonBlockingHashMap;
-
-public class Schema
-{
-    private static final Logger logger = LoggerFactory.getLogger(Schema.class);
-
-    public static final Schema instance = new Schema();
-
-    /* metadata map for faster keyspace lookup */
-    private final Map<String, KeyspaceMetadata> keyspaces = new NonBlockingHashMap<>();
-
-    /* Keyspace objects, one per keyspace. Only one instance should ever exist for any given keyspace. */
-    private final Map<String, Keyspace> keyspaceInstances = new NonBlockingHashMap<>();
-
-    /* metadata map for faster ColumnFamily lookup */
-    private final ConcurrentBiMap<Pair<String, String>, UUID> cfIdMap = new ConcurrentBiMap<>();
-
-    private volatile UUID version;
-
-    /**
-     * Initialize empty schema object and load the hardcoded system tables
-     */
-    public Schema()
-    {
-        if (DatabaseDescriptor.isDaemonInitialized() || DatabaseDescriptor.isToolInitialized())
-        {
-            load(SchemaKeyspace.metadata());
-            load(SystemKeyspace.metadata());
-        }
-    }
-
-    /**
-     * load keyspace (keyspace) definitions, but do not initialize the keyspace instances.
-     * Schema version may be updated as the result.
-     */
-    public Schema loadFromDisk()
-    {
-        return loadFromDisk(true);
-    }
-
-    /**
-     * Load schema definitions from disk.
-     *
-     * @param updateVersion true if schema version needs to be updated
-     */
-    public Schema loadFromDisk(boolean updateVersion)
-    {
-        load(SchemaKeyspace.fetchNonSystemKeyspaces());
-        if (updateVersion)
-            updateVersion();
-        return this;
-    }
-
-    /**
-     * Load up non-system keyspaces
-     *
-     * @param keyspaceDefs The non-system keyspace definitions
-     *
-     * @return self to support chaining calls
-     */
-    public Schema load(Iterable<KeyspaceMetadata> keyspaceDefs)
-    {
-        keyspaceDefs.forEach(this::load);
-        return this;
-    }
-
-    /**
-     * Load specific keyspace into Schema
-     *
-     * @param keyspaceDef The keyspace to load up
-     *
-     * @return self to support chaining calls
-     */
-    public Schema load(KeyspaceMetadata keyspaceDef)
-    {
-        keyspaceDef.tables.forEach(this::load);
-        keyspaceDef.views.forEach(this::load);
-        setKeyspaceMetadata(keyspaceDef);
-        return this;
-    }
-
-    /**
-     * Get keyspace instance by name
-     *
-     * @param keyspaceName The name of the keyspace
-     *
-     * @return Keyspace object or null if keyspace was not found
-     */
-    public Keyspace getKeyspaceInstance(String keyspaceName)
-    {
-        return keyspaceInstances.get(keyspaceName);
-    }
-
-    /**
-     * Retrieve a CFS by name even if that CFS is an index
-     *
-     * An index is identified by looking for '.' in the CF name and separating to find the base table
-     * containing the index
-     * @param ksNameAndCFName
-     * @return The named CFS or null if the keyspace, base table, or index don't exist
-     */
-    public ColumnFamilyStore getColumnFamilyStoreIncludingIndexes(Pair<String, String> ksNameAndCFName)
-    {
-        String ksName = ksNameAndCFName.left;
-        String cfName = ksNameAndCFName.right;
-        Pair<String, String> baseTable;
-
-        /*
-         * Split does special case a one character regex, and it looks like it can detect
-         * if you use two characters to escape '.', but it still allocates a useless array.
-         */
-        int indexOfSeparator = cfName.indexOf('.');
-        if (indexOfSeparator > -1)
-            baseTable = Pair.create(ksName, cfName.substring(0, indexOfSeparator));
-        else
-            baseTable = ksNameAndCFName;
-
-        UUID cfId = cfIdMap.get(baseTable);
-        if (cfId == null)
-            return null;
-
-        Keyspace ks = keyspaceInstances.get(ksName);
-        if (ks == null)
-            return null;
-
-        ColumnFamilyStore baseCFS = ks.getColumnFamilyStore(cfId);
-
-        //Not an index
-        if (indexOfSeparator == -1)
-            return baseCFS;
-
-        if (baseCFS == null)
-            return null;
-
-        Index index = baseCFS.indexManager.getIndexByName(cfName.substring(indexOfSeparator + 1, cfName.length()));
-        if (index == null)
-            return null;
-
-        //Shouldn't ask for a backing table if there is none so just throw?
-        //Or should it return null?
-        return index.getBackingTable().get();
-    }
-
-    public ColumnFamilyStore getColumnFamilyStoreInstance(UUID cfId)
-    {
-        Pair<String, String> pair = cfIdMap.inverse().get(cfId);
-        if (pair == null)
-            return null;
-        Keyspace instance = getKeyspaceInstance(pair.left);
-        if (instance == null)
-            return null;
-        return instance.getColumnFamilyStore(cfId);
-    }
-
-    /**
-     * Store given Keyspace instance to the schema
-     *
-     * @param keyspace The Keyspace instance to store
-     *
-     * @throws IllegalArgumentException if Keyspace is already stored
-     */
-    public void storeKeyspaceInstance(Keyspace keyspace)
-    {
-        if (keyspaceInstances.containsKey(keyspace.getName()))
-            throw new IllegalArgumentException(String.format("Keyspace %s was already initialized.", keyspace.getName()));
-
-        keyspaceInstances.put(keyspace.getName(), keyspace);
-    }
-
-    /**
-     * Remove keyspace from schema
-     *
-     * @param keyspaceName The name of the keyspace to remove
-     *
-     * @return removed keyspace instance or null if it wasn't found
-     */
-    public Keyspace removeKeyspaceInstance(String keyspaceName)
-    {
-        return keyspaceInstances.remove(keyspaceName);
-    }
-
-    /**
-     * Remove keyspace definition from system
-     *
-     * @param ksm The keyspace definition to remove
-     */
-    public void clearKeyspaceMetadata(KeyspaceMetadata ksm)
-    {
-        keyspaces.remove(ksm.name);
-    }
-
-    /**
-     * Given a keyspace name and column family name, get the column family
-     * meta data. If the keyspace name or column family name is not valid
-     * this function returns null.
-     *
-     * @param keyspaceName The keyspace name
-     * @param cfName The ColumnFamily name
-     *
-     * @return ColumnFamily Metadata object or null if it wasn't found
-     */
-    public CFMetaData getCFMetaData(String keyspaceName, String cfName)
-    {
-        assert keyspaceName != null;
-
-        KeyspaceMetadata ksm = keyspaces.get(keyspaceName);
-        return ksm == null
-             ? null
-             : ksm.getTableOrViewNullable(cfName);
-    }
-
-    /**
-     * Get ColumnFamily metadata by its identifier
-     *
-     * @param cfId The ColumnFamily identifier
-     *
-     * @return metadata about ColumnFamily
-     */
-    public CFMetaData getCFMetaData(UUID cfId)
-    {
-        Pair<String,String> cf = getCF(cfId);
-        return (cf == null) ? null : getCFMetaData(cf.left, cf.right);
-    }
-
-    public CFMetaData getCFMetaData(Descriptor descriptor)
-    {
-        return getCFMetaData(descriptor.ksname, descriptor.cfname);
-    }
-
-    public int getNumberOfTables()
-    {
-        return cfIdMap.size();
-    }
-
-    public ViewDefinition getView(String keyspaceName, String viewName)
-    {
-        assert keyspaceName != null;
-        KeyspaceMetadata ksm = keyspaces.get(keyspaceName);
-        return (ksm == null) ? null : ksm.views.getNullable(viewName);
-    }
-
-    /**
-     * Get metadata about keyspace by its name
-     *
-     * @param keyspaceName The name of the keyspace
-     *
-     * @return The keyspace metadata or null if it wasn't found
-     */
-    public KeyspaceMetadata getKSMetaData(String keyspaceName)
-    {
-        assert keyspaceName != null;
-        return keyspaces.get(keyspaceName);
-    }
-
-    private Set<String> getNonSystemKeyspacesSet()
-    {
-        return Sets.difference(keyspaces.keySet(), SchemaConstants.SYSTEM_KEYSPACE_NAMES);
-    }
-
-    /**
-     * @return collection of the non-system keyspaces (note that this count as system only the
-     * non replicated keyspaces, so keyspace like system_traces which are replicated are actually
-     * returned. See getUserKeyspace() below if you don't want those)
-     */
-    public List<String> getNonSystemKeyspaces()
-    {
-        return ImmutableList.copyOf(getNonSystemKeyspacesSet());
-    }
-
-    /**
-     * @return a collection of keyspaces that do not use LocalStrategy for replication
-     */
-    public List<String> getNonLocalStrategyKeyspaces()
-    {
-        return keyspaces.values().stream()
-                .filter(keyspace -> keyspace.params.replication.klass != LocalStrategy.class)
-                .map(keyspace -> keyspace.name)
-                .collect(Collectors.toList());
-    }
-
-    /**
-     * @return collection of the user defined keyspaces
-     */
-    public List<String> getUserKeyspaces()
-    {
-        return ImmutableList.copyOf(Sets.difference(getNonSystemKeyspacesSet(), SchemaConstants.REPLICATED_SYSTEM_KEYSPACE_NAMES));
-    }
-
-    /**
-     * Get metadata about keyspace inner ColumnFamilies
-     *
-     * @param keyspaceName The name of the keyspace
-     *
-     * @return metadata about ColumnFamilies the belong to the given keyspace
-     */
-    public Iterable<CFMetaData> getTablesAndViews(String keyspaceName)
-    {
-        assert keyspaceName != null;
-        KeyspaceMetadata ksm = keyspaces.get(keyspaceName);
-        assert ksm != null;
-        return ksm.tablesAndViews();
-    }
-
-    /**
-     * @return collection of the all keyspace names registered in the system (system and non-system)
-     */
-    public Set<String> getKeyspaces()
-    {
-        return keyspaces.keySet();
-    }
-
-    public Keyspaces getKeyspaces(Set<String> includedKeyspaceNames)
-    {
-        Keyspaces.Builder builder = Keyspaces.builder();
-        keyspaces.values()
-                 .stream()
-                 .filter(k -> includedKeyspaceNames.contains(k.name))
-                 .forEach(builder::add);
-        return builder.build();
-    }
-
-    /**
-     * Update (or insert) new keyspace definition
-     *
-     * @param ksm The metadata about keyspace
-     */
-    public void setKeyspaceMetadata(KeyspaceMetadata ksm)
-    {
-        assert ksm != null;
-
-        keyspaces.put(ksm.name, ksm);
-        Keyspace keyspace = getKeyspaceInstance(ksm.name);
-        if (keyspace != null)
-            keyspace.setMetadata(ksm);
-    }
-
-    /* ColumnFamily query/control methods */
-
-    /**
-     * @param cfId The identifier of the ColumnFamily to lookup
-     * @return The (ksname,cfname) pair for the given id, or null if it has been dropped.
-     */
-    public Pair<String,String> getCF(UUID cfId)
-    {
-        return cfIdMap.inverse().get(cfId);
-    }
-
-    /**
-     * @param ksAndCFName The identifier of the ColumnFamily to lookup
-     * @return true if the KS and CF pair is a known one, false otherwise.
-     */
-    public boolean hasCF(Pair<String, String> ksAndCFName)
-    {
-        return cfIdMap.containsKey(ksAndCFName);
-    }
-
-    /**
-     * Lookup keyspace/ColumnFamily identifier
-     *
-     * @param ksName The keyspace name
-     * @param cfName The ColumnFamily name
-     *
-     * @return The id for the given (ksname,cfname) pair, or null if it has been dropped.
-     */
-    public UUID getId(String ksName, String cfName)
-    {
-        return cfIdMap.get(Pair.create(ksName, cfName));
-    }
-
-    /**
-     * Load individual ColumnFamily Definition to the schema
-     * (to make ColumnFamily lookup faster)
-     *
-     * @param cfm The ColumnFamily definition to load
-     */
-    public void load(CFMetaData cfm)
-    {
-        Pair<String, String> key = Pair.create(cfm.ksName, cfm.cfName);
-
-        if (cfIdMap.containsKey(key))
-            throw new RuntimeException(String.format("Attempting to load already loaded table %s.%s", cfm.ksName, cfm.cfName));
-
-        logger.debug("Adding {} to cfIdMap", cfm);
-        cfIdMap.put(key, cfm.cfId);
-    }
-
-    /**
-     * Load individual View Definition to the schema
-     * (to make View lookup faster)
-     *
-     * @param view The View definition to load
-     */
-    public void load(ViewDefinition view)
-    {
-        CFMetaData cfm = view.metadata;
-        Pair<String, String> key = Pair.create(cfm.ksName, cfm.cfName);
-
-        if (cfIdMap.containsKey(key))
-            throw new RuntimeException(String.format("Attempting to load already loaded view %s.%s", cfm.ksName, cfm.cfName));
-
-        logger.debug("Adding {} to cfIdMap", cfm);
-        cfIdMap.put(key, cfm.cfId);
-    }
-
-    /**
-     * Used for ColumnFamily data eviction out from the schema
-     *
-     * @param cfm The ColumnFamily Definition to evict
-     */
-    public void unload(CFMetaData cfm)
-    {
-        cfIdMap.remove(Pair.create(cfm.ksName, cfm.cfName));
-    }
-
-    /**
-     * Used for View eviction from the schema
-     *
-     * @param view The view definition to evict
-     */
-    private void unload(ViewDefinition view)
-    {
-        cfIdMap.remove(Pair.create(view.ksName, view.viewName));
-    }
-
-    /* Function helpers */
-
-    /**
-     * Get all function overloads with the specified name
-     *
-     * @param name fully qualified function name
-     * @return an empty list if the keyspace or the function name are not found;
-     *         a non-empty collection of {@link Function} otherwise
-     */
-    public Collection<Function> getFunctions(FunctionName name)
-    {
-        if (!name.hasKeyspace())
-            throw new IllegalArgumentException(String.format("Function name must be fully quallified: got %s", name));
-
-        KeyspaceMetadata ksm = getKSMetaData(name.keyspace);
-        return ksm == null
-             ? Collections.emptyList()
-             : ksm.functions.get(name);
-    }
-
-    /**
-     * Find the function with the specified name
-     *
-     * @param name fully qualified function name
-     * @param argTypes function argument types
-     * @return an empty {@link Optional} if the keyspace or the function name are not found;
-     *         a non-empty optional of {@link Function} otherwise
-     */
-    public Optional<Function> findFunction(FunctionName name, List<AbstractType<?>> argTypes)
-    {
-        if (!name.hasKeyspace())
-            throw new IllegalArgumentException(String.format("Function name must be fully quallified: got %s", name));
-
-        KeyspaceMetadata ksm = getKSMetaData(name.keyspace);
-        return ksm == null
-             ? Optional.empty()
-             : ksm.functions.find(name, argTypes);
-    }
-
-    /* Version control */
-
-    /**
-     * @return current schema version
-     */
-    public UUID getVersion()
-    {
-        return version;
-    }
-
-    /**
-     * 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 void updateVersion()
-    {
-        version = SchemaKeyspace.calculateSchemaDigest();
-        SystemKeyspace.updateSchemaVersion(version);
-    }
-
-    /*
-     * Like updateVersion, but also announces via gossip
-     */
-    public void updateVersionAndAnnounce()
-    {
-        updateVersion();
-        MigrationManager.passiveAnnounce(version);
-    }
-
-    /**
-     * Clear all KS/CF metadata and reset version.
-     */
-    public synchronized void clear()
-    {
-        for (String keyspaceName : getNonSystemKeyspaces())
-        {
-            KeyspaceMetadata ksm = getKSMetaData(keyspaceName);
-            ksm.tables.forEach(this::unload);
-            ksm.views.forEach(this::unload);
-            clearKeyspaceMetadata(ksm);
-        }
-
-        updateVersionAndAnnounce();
-    }
-
-    public void addKeyspace(KeyspaceMetadata ksm)
-    {
-        assert getKSMetaData(ksm.name) == null;
-        load(ksm);
-
-        Keyspace.open(ksm.name);
-        MigrationManager.instance.notifyCreateKeyspace(ksm);
-    }
-
-    public void updateKeyspace(String ksName, KeyspaceParams newParams)
-    {
-        KeyspaceMetadata ksm = update(ksName, ks -> ks.withSwapped(newParams));
-        MigrationManager.instance.notifyUpdateKeyspace(ksm);
-    }
-
-    public void dropKeyspace(String ksName)
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(ksName);
-        String snapshotName = Keyspace.getTimestampedSnapshotNameWithPrefix(ksName, ColumnFamilyStore.SNAPSHOT_DROP_PREFIX);
-
-        CompactionManager.instance.interruptCompactionFor(ksm.tablesAndViews(), true);
-
-        Keyspace keyspace = Keyspace.open(ksm.name);
-
-        // remove all cfs from the keyspace instance.
-        List<UUID> droppedCfs = new ArrayList<>();
-        for (CFMetaData cfm : ksm.tablesAndViews())
-        {
-            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfm.cfName);
-
-            unload(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);
-        clearKeyspaceMetadata(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;
-
-        // Make sure the keyspace is initialized
-        // and init the new CF before switching the KSM to the new one
-        // to avoid races as in CASSANDRA-10761
-        Keyspace.open(cfm.ksName).initCf(cfm, true);
-        // Update the keyspaces map with the updated metadata
-        update(cfm.ksName, ks -> ks.withSwapped(ks.tables.with(cfm)));
-        // Update the table ID <-> table name map (cfIdMap)
-        load(cfm);
-        MigrationManager.instance.notifyCreateColumnFamily(cfm);
-    }
-
-    public void updateTable(CFMetaData table)
-    {
-        CFMetaData current = getCFMetaData(table.ksName, table.cfName);
-        assert current != null;
-        boolean changeAffectsStatements = current.apply(table);
-
-        Keyspace keyspace = Keyspace.open(current.ksName);
-        keyspace.getColumnFamilyStore(current.cfName).reload();
-        MigrationManager.instance.notifyUpdateColumnFamily(current, changeAffectsStatements);
-    }
-
-    public void dropTable(String ksName, String tableName)
-    {
-        KeyspaceMetadata oldKsm = getKSMetaData(ksName);
-        assert oldKsm != null;
-        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(tableName);
-        assert cfs != null;
-
-        // make sure all the indexes are dropped, or else.
-        cfs.indexManager.markAllIndexesRemoved();
-
-        // reinitialize the keyspace.
-        CFMetaData cfm = oldKsm.tables.get(tableName).get();
-        KeyspaceMetadata newKsm = oldKsm.withSwapped(oldKsm.tables.without(tableName));
-
-        unload(cfm);
-        setKeyspaceMetadata(newKsm);
-
-        CompactionManager.instance.interruptCompactionFor(Collections.singleton(cfm), true);
-
-        if (DatabaseDescriptor.isAutoSnapshot())
-            cfs.snapshot(Keyspace.getTimestampedSnapshotNameWithPrefix(cfs.name, ColumnFamilyStore.SNAPSHOT_DROP_PREFIX));
-        Keyspace.open(ksName).dropCf(cfm.cfId);
-        MigrationManager.instance.notifyDropColumnFamily(cfm);
-
-        CommitLog.instance.forceRecycleAllSegments(Collections.singleton(cfm.cfId));
-    }
-
-    public void addView(ViewDefinition view)
-    {
-        assert getCFMetaData(view.ksName, view.viewName) == null;
-
-        Keyspace keyspace = Keyspace.open(view.ksName);
-
-        // Make sure the keyspace is initialized and initialize the table.
-        keyspace.initCf(view.metadata, true);
-        // Update the keyspaces map with the updated metadata
-        update(view.ksName, ks -> ks.withSwapped(ks.views.with(view)));
-        // Update the table ID <-> table name map (cfIdMap)
-        load(view);
-
-        keyspace.viewManager.reload();
-        MigrationManager.instance.notifyCreateView(view);
-    }
-
-    public void updateView(ViewDefinition view)
-    {
-        ViewDefinition current = getKSMetaData(view.ksName).views.get(view.viewName).get();
-        boolean changeAffectsStatements = current.metadata.apply(view.metadata);
-
-        Keyspace keyspace = Keyspace.open(current.ksName);
-        keyspace.getColumnFamilyStore(current.viewName).reload();
-        Keyspace.open(current.ksName).viewManager.update(current.viewName);
-        MigrationManager.instance.notifyUpdateView(current, changeAffectsStatements);
-    }
-
-    public void dropView(String ksName, String viewName)
-    {
-        KeyspaceMetadata oldKsm = getKSMetaData(ksName);
-        assert oldKsm != null;
-        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(viewName);
-        assert cfs != null;
-
-        // make sure all the indexes are dropped, or else.
-        cfs.indexManager.markAllIndexesRemoved();
-
-        // reinitialize the keyspace.
-        ViewDefinition view = oldKsm.views.get(viewName).get();
-        KeyspaceMetadata newKsm = oldKsm.withSwapped(oldKsm.views.without(viewName));
-
-        unload(view);
-        setKeyspaceMetadata(newKsm);
-
-        CompactionManager.instance.interruptCompactionFor(Collections.singleton(view.metadata), true);
-
-        if (DatabaseDescriptor.isAutoSnapshot())
-            cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name));
-        Keyspace.open(ksName).dropCf(view.metadata.cfId);
-        Keyspace.open(ksName).viewManager.reload();
-        MigrationManager.instance.notifyDropView(view);
-
-        CommitLog.instance.forceRecycleAllSegments(Collections.singleton(view.metadata.cfId));
-    }
-
-    public void addType(UserType ut)
-    {
-        update(ut.keyspace, ks -> ks.withSwapped(ks.types.with(ut)));
-        MigrationManager.instance.notifyCreateUserType(ut);
-    }
-
-    public void updateType(UserType ut)
-    {
-        update(ut.keyspace, ks -> ks.withSwapped(ks.types.without(ut.name).with(ut)));
-        MigrationManager.instance.notifyUpdateUserType(ut);
-    }
-
-    public void dropType(UserType ut)
-    {
-        update(ut.keyspace, ks -> ks.withSwapped(ks.types.without(ut.name)));
-        MigrationManager.instance.notifyDropUserType(ut);
-    }
-
-    public void addFunction(UDFunction udf)
-    {
-        update(udf.name().keyspace, ks -> ks.withSwapped(ks.functions.with(udf)));
-        MigrationManager.instance.notifyCreateFunction(udf);
-    }
-
-    public void updateFunction(UDFunction udf)
-    {
-        update(udf.name().keyspace, ks -> ks.withSwapped(ks.functions.without(udf.name(), udf.argTypes()).with(udf)));
-        MigrationManager.instance.notifyUpdateFunction(udf);
-    }
-
-    public void dropFunction(UDFunction udf)
-    {
-        update(udf.name().keyspace, ks -> ks.withSwapped(ks.functions.without(udf.name(), udf.argTypes())));
-        MigrationManager.instance.notifyDropFunction(udf);
-    }
-
-    public void addAggregate(UDAggregate uda)
-    {
-        update(uda.name().keyspace, ks -> ks.withSwapped(ks.functions.with(uda)));
-        MigrationManager.instance.notifyCreateAggregate(uda);
-    }
-
-    public void updateAggregate(UDAggregate uda)
-    {
-        update(uda.name().keyspace, ks -> ks.withSwapped(ks.functions.without(uda.name(), uda.argTypes()).with(uda)));
-        MigrationManager.instance.notifyUpdateAggregate(uda);
-    }
-
-    public void dropAggregate(UDAggregate uda)
-    {
-        update(uda.name().keyspace, ks -> ks.withSwapped(ks.functions.without(uda.name(), uda.argTypes())));
-        MigrationManager.instance.notifyDropAggregate(uda);
-    }
-
-    private synchronized KeyspaceMetadata update(String keyspaceName, java.util.function.Function<KeyspaceMetadata, KeyspaceMetadata> transformation)
-    {
-        KeyspaceMetadata current = getKSMetaData(keyspaceName);
-        if (current == null)
-            throw new IllegalStateException(String.format("Keyspace %s doesn't exist", keyspaceName));
-
-        KeyspaceMetadata transformed = transformation.apply(current);
-        setKeyspaceMetadata(transformed);
-
-        return transformed;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/config/SchemaConstants.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/SchemaConstants.java b/src/java/org/apache/cassandra/config/SchemaConstants.java
deleted file mode 100644
index 2416d6b..0000000
--- a/src/java/org/apache/cassandra/config/SchemaConstants.java
+++ /dev/null
@@ -1,74 +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.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.Set;
-import java.util.UUID;
-
-import com.google.common.collect.ImmutableSet;
-
-public final class SchemaConstants
-{
-    public static final String SYSTEM_KEYSPACE_NAME = "system";
-    public static final String SCHEMA_KEYSPACE_NAME = "system_schema";
-
-    public static final String TRACE_KEYSPACE_NAME = "system_traces";
-    public static final String AUTH_KEYSPACE_NAME = "system_auth";
-    public static final String DISTRIBUTED_KEYSPACE_NAME = "system_distributed";
-
-    /* system keyspace names (the ones with LocalStrategy replication strategy) */
-    public static final Set<String> SYSTEM_KEYSPACE_NAMES = ImmutableSet.of(SYSTEM_KEYSPACE_NAME, SCHEMA_KEYSPACE_NAME);
-
-    /* replicate system keyspace names (the ones with a "true" replication strategy) */
-    public static final Set<String> REPLICATED_SYSTEM_KEYSPACE_NAMES = ImmutableSet.of(TRACE_KEYSPACE_NAME,
-                                                                                       AUTH_KEYSPACE_NAME,
-                                                                                       DISTRIBUTED_KEYSPACE_NAME);
-    /**
-     * longest permissible KS or CF name.  Our main concern is that filename not be more than 255 characters;
-     * the filename will contain both the KS and CF names. Since non-schema-name components only take up
-     * ~64 characters, we could allow longer names than this, but on Windows, the entire path should be not greater than
-     * 255 characters, so a lower limit here helps avoid problems.  See CASSANDRA-4110.
-     */
-    public static final int NAME_LENGTH = 48;
-
-    // 59adb24e-f3cd-3e02-97f0-5b395827453f
-    public static final UUID emptyVersion;
-
-    static
-    {
-        try
-        {
-            emptyVersion = UUID.nameUUIDFromBytes(MessageDigest.getInstance("MD5").digest());
-        }
-        catch (NoSuchAlgorithmException e)
-        {
-            throw new AssertionError();
-        }
-    }
-
-    /**
-     * @return whether or not the keyspace is a really system one (w/ LocalStrategy, unmodifiable, hardcoded)
-     */
-    public static boolean isSystemKeyspace(String keyspaceName)
-    {
-        return SYSTEM_KEYSPACE_NAMES.contains(keyspaceName.toLowerCase());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/config/ViewDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ViewDefinition.java b/src/java/org/apache/cassandra/config/ViewDefinition.java
deleted file mode 100644
index 77cbcc9..0000000
--- a/src/java/org/apache/cassandra/config/ViewDefinition.java
+++ /dev/null
@@ -1,166 +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.util.List;
-import java.util.Objects;
-import java.util.UUID;
-import java.util.stream.Collectors;
-
-import org.antlr.runtime.*;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.statements.SelectStatement;
-import org.apache.cassandra.db.view.View;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-
-public class ViewDefinition
-{
-    public final String ksName;
-    public final String viewName;
-    public final UUID baseTableId;
-    public final String baseTableName;
-    public final boolean includeAllColumns;
-    public final CFMetaData metadata;
-
-    public SelectStatement.RawStatement select;
-    public String whereClause;
-
-    public ViewDefinition(ViewDefinition def)
-    {
-        this(def.ksName, def.viewName, def.baseTableId, def.baseTableName, def.includeAllColumns, def.select, def.whereClause, def.metadata);
-    }
-
-    /**
-     * @param viewName          Name of the view
-     * @param baseTableId       Internal ID of the table which this view is based off of
-     * @param includeAllColumns Whether to include all columns or not
-     */
-    public ViewDefinition(String ksName, String viewName, UUID baseTableId, String baseTableName, boolean includeAllColumns, SelectStatement.RawStatement select, String whereClause, CFMetaData metadata)
-    {
-        this.ksName = ksName;
-        this.viewName = viewName;
-        this.baseTableId = baseTableId;
-        this.baseTableName = baseTableName;
-        this.includeAllColumns = includeAllColumns;
-        this.select = select;
-        this.whereClause = whereClause;
-        this.metadata = metadata;
-    }
-
-    /**
-     * @return true if the view specified by this definition will include the column, false otherwise
-     */
-    public boolean includes(ColumnIdentifier column)
-    {
-        return metadata.getColumnDefinition(column) != null;
-    }
-
-    public ViewDefinition copy()
-    {
-        return new ViewDefinition(ksName, viewName, baseTableId, baseTableName, includeAllColumns, select, whereClause, metadata.copy());
-    }
-
-    public CFMetaData baseTableMetadata()
-    {
-        return Schema.instance.getCFMetaData(baseTableId);
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o)
-            return true;
-
-        if (!(o instanceof ViewDefinition))
-            return false;
-
-        ViewDefinition other = (ViewDefinition) o;
-        return Objects.equals(ksName, other.ksName)
-               && Objects.equals(viewName, other.viewName)
-               && Objects.equals(baseTableId, other.baseTableId)
-               && Objects.equals(includeAllColumns, other.includeAllColumns)
-               && Objects.equals(whereClause, other.whereClause)
-               && Objects.equals(metadata, other.metadata);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return new HashCodeBuilder(29, 1597)
-               .append(ksName)
-               .append(viewName)
-               .append(baseTableId)
-               .append(includeAllColumns)
-               .append(whereClause)
-               .append(metadata)
-               .toHashCode();
-    }
-
-    @Override
-    public String toString()
-    {
-        return new ToStringBuilder(this)
-               .append("ksName", ksName)
-               .append("viewName", viewName)
-               .append("baseTableId", baseTableId)
-               .append("baseTableName", baseTableName)
-               .append("includeAllColumns", includeAllColumns)
-               .append("whereClause", whereClause)
-               .append("metadata", metadata)
-               .toString();
-    }
-
-    /**
-     * Replace the column 'from' with 'to' in this materialized view definition's partition,
-     * clustering, or included columns.
-     * @param from the existing column
-     * @param to the new column
-     */
-    public void renameColumn(ColumnIdentifier from, ColumnIdentifier to)
-    {
-        metadata.renameColumn(from, to);
-
-        // convert whereClause to Relations, rename ids in Relations, then convert back to whereClause
-        List<Relation> relations = whereClauseToRelations(whereClause);
-        ColumnDefinition.Raw fromRaw = ColumnDefinition.Raw.forQuoted(from.toString());
-        ColumnDefinition.Raw toRaw = ColumnDefinition.Raw.forQuoted(to.toString());
-        List<Relation> newRelations = relations.stream()
-                .map(r -> r.renameIdentifier(fromRaw, toRaw))
-                .collect(Collectors.toList());
-
-        this.whereClause = View.relationsToWhereClause(newRelations);
-        String rawSelect = View.buildSelectStatement(baseTableName, metadata.allColumns(), whereClause);
-        this.select = (SelectStatement.RawStatement) QueryProcessor.parseStatement(rawSelect);
-    }
-
-    private static List<Relation> whereClauseToRelations(String whereClause)
-    {
-        try
-        {
-            List<Relation> relations = CQLFragmentParser.parseAnyUnhandled(CqlParser::whereClause, whereClause).build().relations;
-
-            return relations;
-        }
-        catch (RecognitionException | SyntaxException exc)
-        {
-            throw new RuntimeException("Unexpected error parsing materialized view's where clause while handling column rename: ", exc);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/CQL3Type.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java
index 7e375bf..d387a25 100644
--- a/src/java/org/apache/cassandra/cql3/CQL3Type.java
+++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java
@@ -24,7 +24,7 @@ import java.util.List;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.db.marshal.CollectionType.Kind;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -524,7 +524,7 @@ public interface CQL3Type
 
         public CQL3Type prepare(String keyspace)
         {
-            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
+            KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(keyspace);
             if (ksm == null)
                 throw new ConfigurationException(String.format("Keyspace %s doesn't exist", keyspace));
             return prepare(keyspace, ksm.types);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index 70f7590..7436c90 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -218,7 +218,6 @@ public class ColumnIdentifier implements IMeasurableMemory, Comparable<ColumnIde
         return ByteBufferUtil.compareUnsigned(this.bytes, that.bytes);
     }
 
-    @VisibleForTesting
     public static String maybeQuote(String text)
     {
         if (UNQUOTED_IDENTIFIER.matcher(text).matches() && !ReservedKeywords.isReserved(text))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/Constants.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Constants.java b/src/java/org/apache/cassandra/cql3/Constants.java
index 734d1d6..6dce3a3 100644
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -424,7 +424,7 @@ public abstract class Constants
 
     public static class Setter extends Operation
     {
-        public Setter(ColumnDefinition column, Term t)
+        public Setter(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -441,7 +441,7 @@ public abstract class Constants
 
     public static class Adder extends Operation
     {
-        public Adder(ColumnDefinition column, Term t)
+        public Adder(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -461,7 +461,7 @@ public abstract class Constants
 
     public static class Substracter extends Operation
     {
-        public Substracter(ColumnDefinition column, Term t)
+        public Substracter(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -486,7 +486,7 @@ public abstract class Constants
     // duplicating this further
     public static class Deleter extends Operation
     {
-        public Deleter(ColumnDefinition column)
+        public Deleter(ColumnMetadata column)
         {
             super(column, null);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/Json.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Json.java b/src/java/org/apache/cassandra/cql3/Json.java
index 2e67a1e..f0aee63 100644
--- a/src/java/org/apache/cassandra/cql3/Json.java
+++ b/src/java/org/apache/cassandra/cql3/Json.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.cql3;
 import java.io.IOException;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UTF8Type;
@@ -59,7 +59,7 @@ public class Json
 
     public interface Raw
     {
-        public Prepared prepareAndCollectMarkers(CFMetaData metadata, Collection<ColumnDefinition> receivers, VariableSpecifications boundNames);
+        public Prepared prepareAndCollectMarkers(TableMetadata metadata, Collection<ColumnMetadata> receivers, VariableSpecifications boundNames);
     }
 
     /**
@@ -75,7 +75,7 @@ public class Json
             this.text = text;
         }
 
-        public Prepared prepareAndCollectMarkers(CFMetaData metadata, Collection<ColumnDefinition> receivers, VariableSpecifications boundNames)
+        public Prepared prepareAndCollectMarkers(TableMetadata metadata, Collection<ColumnMetadata> receivers, VariableSpecifications boundNames)
         {
             return new PreparedLiteral(parseJson(text, receivers));
         }
@@ -94,15 +94,15 @@ public class Json
             this.bindIndex = bindIndex;
         }
 
-        public Prepared prepareAndCollectMarkers(CFMetaData metadata, Collection<ColumnDefinition> receivers, VariableSpecifications boundNames)
+        public Prepared prepareAndCollectMarkers(TableMetadata metadata, Collection<ColumnMetadata> receivers, VariableSpecifications boundNames)
         {
             boundNames.add(bindIndex, makeReceiver(metadata));
             return new PreparedMarker(bindIndex, receivers);
         }
 
-        private ColumnSpecification makeReceiver(CFMetaData metadata)
+        private ColumnSpecification makeReceiver(TableMetadata metadata)
         {
-            return new ColumnSpecification(metadata.ksName, metadata.cfName, JSON_COLUMN_ID, UTF8Type.instance);
+            return new ColumnSpecification(metadata.keyspace, metadata.name, JSON_COLUMN_ID, UTF8Type.instance);
         }
     }
 
@@ -111,7 +111,7 @@ public class Json
      */
     public static abstract class Prepared
     {
-        public abstract Term.Raw getRawTermForColumn(ColumnDefinition def, boolean defaultUnset);
+        public abstract Term.Raw getRawTermForColumn(ColumnMetadata def, boolean defaultUnset);
     }
 
     /**
@@ -126,7 +126,7 @@ public class Json
             this.columnMap = columnMap;
         }
 
-        public Term.Raw getRawTermForColumn(ColumnDefinition def, boolean defaultUnset)
+        public Term.Raw getRawTermForColumn(ColumnMetadata def, boolean defaultUnset)
         {
             Term value = columnMap.get(def.name);
             return value == null
@@ -141,15 +141,15 @@ public class Json
     private static class PreparedMarker extends Prepared
     {
         private final int bindIndex;
-        private final Collection<ColumnDefinition> columns;
+        private final Collection<ColumnMetadata> columns;
 
-        public PreparedMarker(int bindIndex, Collection<ColumnDefinition> columns)
+        public PreparedMarker(int bindIndex, Collection<ColumnMetadata> columns)
         {
             this.bindIndex = bindIndex;
             this.columns = columns;
         }
 
-        public RawDelayedColumnValue getRawTermForColumn(ColumnDefinition def, boolean defaultUnset)
+        public RawDelayedColumnValue getRawTermForColumn(ColumnMetadata def, boolean defaultUnset)
         {
             return new RawDelayedColumnValue(this, def, defaultUnset);
         }
@@ -199,10 +199,10 @@ public class Json
     private static class RawDelayedColumnValue extends Term.Raw
     {
         private final PreparedMarker marker;
-        private final ColumnDefinition column;
+        private final ColumnMetadata column;
         private final boolean defaultUnset;
 
-        public RawDelayedColumnValue(PreparedMarker prepared, ColumnDefinition column, boolean defaultUnset)
+        public RawDelayedColumnValue(PreparedMarker prepared, ColumnMetadata column, boolean defaultUnset)
         {
             this.marker = prepared;
             this.column = column;
@@ -238,10 +238,10 @@ public class Json
     private static class DelayedColumnValue extends Term.NonTerminal
     {
         private final PreparedMarker marker;
-        private final ColumnDefinition column;
+        private final ColumnMetadata column;
         private final boolean defaultUnset;
 
-        public DelayedColumnValue(PreparedMarker prepared, ColumnDefinition column, boolean defaultUnset)
+        public DelayedColumnValue(PreparedMarker prepared, ColumnMetadata column, boolean defaultUnset)
         {
             this.marker = prepared;
             this.column = column;
@@ -278,7 +278,7 @@ public class Json
     /**
      * Given a JSON string, return a map of columns to their values for the insert.
      */
-    public static Map<ColumnIdentifier, Term> parseJson(String jsonString, Collection<ColumnDefinition> expectedReceivers)
+    public static Map<ColumnIdentifier, Term> parseJson(String jsonString, Collection<ColumnMetadata> expectedReceivers)
     {
         try
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/Lists.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java
index eb4b685..48fe54f 100644
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@ -28,7 +28,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
@@ -347,7 +347,7 @@ public abstract class Lists
 
     public static class Setter extends Operation
     {
-        public Setter(ColumnDefinition column, Term t)
+        public Setter(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -365,7 +365,7 @@ public abstract class Lists
         }
     }
 
-    private static int existingSize(Row row, ColumnDefinition column)
+    private static int existingSize(Row row, ColumnMetadata column)
     {
         if (row == null)
             return 0;
@@ -378,7 +378,7 @@ public abstract class Lists
     {
         private final Term idx;
 
-        public SetterByIndex(ColumnDefinition column, Term idx, Term t)
+        public SetterByIndex(ColumnMetadata column, Term idx, Term t)
         {
             super(column, t);
             this.idx = idx;
@@ -428,7 +428,7 @@ public abstract class Lists
 
     public static class Appender extends Operation
     {
-        public Appender(ColumnDefinition column, Term t)
+        public Appender(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -440,7 +440,7 @@ public abstract class Lists
             doAppend(value, column, params);
         }
 
-        static void doAppend(Term.Terminal value, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
+        static void doAppend(Term.Terminal value, ColumnMetadata column, UpdateParameters params) throws InvalidRequestException
         {
             if (column.type.isMultiCell())
             {
@@ -468,7 +468,7 @@ public abstract class Lists
 
     public static class Prepender extends Operation
     {
-        public Prepender(ColumnDefinition column, Term t)
+        public Prepender(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -494,7 +494,7 @@ public abstract class Lists
 
     public static class Discarder extends Operation
     {
-        public Discarder(ColumnDefinition column, Term t)
+        public Discarder(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -532,7 +532,7 @@ public abstract class Lists
 
     public static class DiscarderByIndex extends Operation
     {
-        public DiscarderByIndex(ColumnDefinition column, Term idx)
+        public DiscarderByIndex(ColumnMetadata column, Term idx)
         {
             super(column, idx);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/Maps.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Maps.java b/src/java/org/apache/cassandra/cql3/Maps.java
index c98b829..e02169e 100644
--- a/src/java/org/apache/cassandra/cql3/Maps.java
+++ b/src/java/org/apache/cassandra/cql3/Maps.java
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.stream.Collectors;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.rows.*;
@@ -328,7 +328,7 @@ public abstract class Maps
 
     public static class Setter extends Operation
     {
-        public Setter(ColumnDefinition column, Term t)
+        public Setter(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -350,7 +350,7 @@ public abstract class Maps
     {
         private final Term k;
 
-        public SetterByKey(ColumnDefinition column, Term k, Term t)
+        public SetterByKey(ColumnMetadata column, Term k, Term t)
         {
             super(column, t);
             this.k = k;
@@ -388,7 +388,7 @@ public abstract class Maps
 
     public static class Putter extends Operation
     {
-        public Putter(ColumnDefinition column, Term t)
+        public Putter(ColumnMetadata column, Term t)
         {
             super(column, t);
         }
@@ -401,7 +401,7 @@ public abstract class Maps
                 doPut(value, column, params);
         }
 
-        static void doPut(Term.Terminal value, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
+        static void doPut(Term.Terminal value, ColumnMetadata column, UpdateParameters params) throws InvalidRequestException
         {
             if (column.type.isMultiCell())
             {
@@ -425,7 +425,7 @@ public abstract class Maps
 
     public static class DiscarderByKey extends Operation
     {
-        public DiscarderByKey(ColumnDefinition column, Term k)
+        public DiscarderByKey(ColumnMetadata column, Term k)
         {
             super(column, k);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
index 4ddfabb..411af07 100644
--- a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@ -21,8 +21,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.stream.Collectors;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.Term.MultiColumnRaw;
 import org.apache.cassandra.cql3.Term.Raw;
 import org.apache.cassandra.cql3.restrictions.MultiColumnRestriction;
@@ -46,7 +46,7 @@ import static org.apache.cassandra.cql3.statements.RequestValidations.invalidReq
  */
 public class MultiColumnRelation extends Relation
 {
-    private final List<ColumnDefinition.Raw> entities;
+    private final List<ColumnMetadata.Raw> entities;
 
     /** A Tuples.Literal or Tuples.Raw marker */
     private final Term.MultiColumnRaw valuesOrMarker;
@@ -56,7 +56,7 @@ public class MultiColumnRelation extends Relation
 
     private final Tuples.INRaw inMarker;
 
-    private MultiColumnRelation(List<ColumnDefinition.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
+    private MultiColumnRelation(List<ColumnMetadata.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker, List<? extends Term.MultiColumnRaw> inValues, Tuples.INRaw inMarker)
     {
         this.entities = entities;
         this.relationType = relationType;
@@ -76,7 +76,7 @@ public class MultiColumnRelation extends Relation
      * @param valuesOrMarker a Tuples.Literal instance or a Tuples.Raw marker
      * @return a new <code>MultiColumnRelation</code> instance
      */
-    public static MultiColumnRelation createNonInRelation(List<ColumnDefinition.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker)
+    public static MultiColumnRelation createNonInRelation(List<ColumnMetadata.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker)
     {
         assert relationType != Operator.IN;
         return new MultiColumnRelation(entities, relationType, valuesOrMarker, null, null);
@@ -89,7 +89,7 @@ public class MultiColumnRelation extends Relation
      * @param inValues a list of Tuples.Literal instances or a Tuples.Raw markers
      * @return a new <code>MultiColumnRelation</code> instance
      */
-    public static MultiColumnRelation createInRelation(List<ColumnDefinition.Raw> entities, List<? extends Term.MultiColumnRaw> inValues)
+    public static MultiColumnRelation createInRelation(List<ColumnMetadata.Raw> entities, List<? extends Term.MultiColumnRaw> inValues)
     {
         return new MultiColumnRelation(entities, Operator.IN, null, inValues, null);
     }
@@ -101,12 +101,12 @@ public class MultiColumnRelation extends Relation
      * @param inMarker a single IN marker
      * @return a new <code>MultiColumnRelation</code> instance
      */
-    public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnDefinition.Raw> entities, Tuples.INRaw inMarker)
+    public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnMetadata.Raw> entities, Tuples.INRaw inMarker)
     {
         return new MultiColumnRelation(entities, Operator.IN, null, null, inMarker);
     }
 
-    public List<ColumnDefinition.Raw> getEntities()
+    public List<ColumnMetadata.Raw> getEntities()
     {
         return entities;
     }
@@ -133,23 +133,21 @@ public class MultiColumnRelation extends Relation
     }
 
     @Override
-    protected Restriction newEQRestriction(CFMetaData cfm,
-                                           VariableSpecifications boundNames) throws InvalidRequestException
+    protected Restriction newEQRestriction(TableMetadata table, VariableSpecifications boundNames)
     {
-        List<ColumnDefinition> receivers = receivers(cfm);
-        Term term = toTerm(receivers, getValue(), cfm.ksName, boundNames);
+        List<ColumnMetadata> receivers = receivers(table);
+        Term term = toTerm(receivers, getValue(), table.keyspace, boundNames);
         return new MultiColumnRestriction.EQRestriction(receivers, term);
     }
 
     @Override
-    protected Restriction newINRestriction(CFMetaData cfm,
-                                           VariableSpecifications boundNames) throws InvalidRequestException
+    protected Restriction newINRestriction(TableMetadata table, VariableSpecifications boundNames)
     {
-        List<ColumnDefinition> receivers = receivers(cfm);
-        List<Term> terms = toTerms(receivers, inValues, cfm.ksName, boundNames);
+        List<ColumnMetadata> receivers = receivers(table);
+        List<Term> terms = toTerms(receivers, inValues, table.keyspace, boundNames);
         if (terms == null)
         {
-            Term term = toTerm(receivers, getValue(), cfm.ksName, boundNames);
+            Term term = toTerm(receivers, getValue(), table.keyspace, boundNames);
             return new MultiColumnRestriction.InRestrictionWithMarker(receivers, (AbstractMarker) term);
         }
 
@@ -160,34 +158,28 @@ public class MultiColumnRelation extends Relation
     }
 
     @Override
-    protected Restriction newSliceRestriction(CFMetaData cfm,
-                                              VariableSpecifications boundNames,
-                                              Bound bound,
-                                              boolean inclusive) throws InvalidRequestException
+    protected Restriction newSliceRestriction(TableMetadata table, VariableSpecifications boundNames, Bound bound, boolean inclusive)
     {
-        List<ColumnDefinition> receivers = receivers(cfm);
-        Term term = toTerm(receivers(cfm), getValue(), cfm.ksName, boundNames);
+        List<ColumnMetadata> receivers = receivers(table);
+        Term term = toTerm(receivers(table), getValue(), table.keyspace, boundNames);
         return new MultiColumnRestriction.SliceRestriction(receivers, bound, inclusive, term);
     }
 
     @Override
-    protected Restriction newContainsRestriction(CFMetaData cfm,
-                                                 VariableSpecifications boundNames,
-                                                 boolean isKey) throws InvalidRequestException
+    protected Restriction newContainsRestriction(TableMetadata table, VariableSpecifications boundNames, boolean isKey)
     {
         throw invalidRequest("%s cannot be used for multi-column relations", operator());
     }
 
     @Override
-    protected Restriction newIsNotRestriction(CFMetaData cfm,
-                                              VariableSpecifications boundNames) throws InvalidRequestException
+    protected Restriction newIsNotRestriction(TableMetadata table, VariableSpecifications boundNames)
     {
         // this is currently disallowed by the grammar
         throw new AssertionError(String.format("%s cannot be used for multi-column relations", operator()));
     }
 
     @Override
-    protected Restriction newLikeRestriction(CFMetaData cfm, VariableSpecifications boundNames, Operator operator) throws InvalidRequestException
+    protected Restriction newLikeRestriction(TableMetadata table, VariableSpecifications boundNames, Operator operator)
     {
         throw invalidRequest("%s cannot be used for multi-column relations", operator());
     }
@@ -203,13 +195,13 @@ public class MultiColumnRelation extends Relation
         return term;
     }
 
-    protected List<ColumnDefinition> receivers(CFMetaData cfm) throws InvalidRequestException
+    protected List<ColumnMetadata> receivers(TableMetadata table) throws InvalidRequestException
     {
-        List<ColumnDefinition> names = new ArrayList<>(getEntities().size());
+        List<ColumnMetadata> names = new ArrayList<>(getEntities().size());
         int previousPosition = -1;
-        for (ColumnDefinition.Raw raw : getEntities())
+        for (ColumnMetadata.Raw raw : getEntities())
         {
-            ColumnDefinition def = raw.prepare(cfm);
+            ColumnMetadata def = raw.prepare(table);
             checkTrue(def.isClusteringColumn(), "Multi-column relations can only be applied to clustering columns but was applied to: %s", def.name);
             checkFalse(names.contains(def), "Column \"%s\" appeared twice in a relation: %s", def.name, this);
 
@@ -223,12 +215,12 @@ public class MultiColumnRelation extends Relation
         return names;
     }
 
-    public Relation renameIdentifier(ColumnDefinition.Raw from, ColumnDefinition.Raw to)
+    public Relation renameIdentifier(ColumnMetadata.Raw from, ColumnMetadata.Raw to)
     {
         if (!entities.contains(from))
             return this;
 
-        List<ColumnDefinition.Raw> newEntities = entities.stream().map(e -> e.equals(from) ? to : e).collect(Collectors.toList());
+        List<ColumnMetadata.Raw> newEntities = entities.stream().map(e -> e.equals(from) ? to : e).collect(Collectors.toList());
         return new MultiColumnRelation(newEntities, operator(), valuesOrMarker, inValues, inMarker);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/Operation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Operation.java b/src/java/org/apache/cassandra/cql3/Operation.java
index 85a1eb2..8db9306 100644
--- a/src/java/org/apache/cassandra/cql3/Operation.java
+++ b/src/java/org/apache/cassandra/cql3/Operation.java
@@ -19,12 +19,12 @@ package org.apache.cassandra.cql3;
 
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 
 /**
  * An UPDATE or DELETE operation.
@@ -43,13 +43,13 @@ import org.apache.cassandra.exceptions.InvalidRequestException;
 public abstract class Operation
 {
     // the column the operation applies to
-    public final ColumnDefinition column;
+    public final ColumnMetadata column;
 
     // Term involved in the operation. In theory this should not be here since some operation
     // may require none of more than one term, but most need 1 so it simplify things a bit.
     protected final Term t;
 
-    protected Operation(ColumnDefinition column, Term t)
+    protected Operation(ColumnMetadata column, Term t)
     {
         assert column != null;
         this.column = column;
@@ -109,10 +109,12 @@ public abstract class Operation
          * It returns an Operation which can be though as post-preparation well-typed
          * Operation.
          *
+         *
+         * @param metadata
          * @param receiver the column this operation applies to.
          * @return the prepared update operation.
          */
-        public Operation prepare(CFMetaData cfm, ColumnDefinition receiver) throws InvalidRequestException;
+        public Operation prepare(TableMetadata metadata, ColumnMetadata receiver) throws InvalidRequestException;
 
         /**
          * @return whether this operation can be applied alongside the {@code
@@ -133,7 +135,7 @@ public abstract class Operation
         /**
          * The name of the column affected by this delete operation.
          */
-        public ColumnDefinition.Raw affectedColumn();
+        public ColumnMetadata.Raw affectedColumn();
 
         /**
          * This method validates the operation (i.e. validate it is well typed)
@@ -144,9 +146,10 @@ public abstract class Operation
          * Operation.
          *
          * @param receiver the "column" this operation applies to.
+         * @param metadata
          * @return the prepared delete operation.
          */
-        public Operation prepare(String keyspace, ColumnDefinition receiver, CFMetaData cfm) throws InvalidRequestException;
+        public Operation prepare(String keyspace, ColumnMetadata receiver, TableMetadata metadata) throws InvalidRequestException;
     }
 
     public static class SetValue implements RawUpdate
@@ -158,9 +161,9 @@ public abstract class Operation
             this.value = value;
         }
 
-        public Operation prepare(CFMetaData cfm, ColumnDefinition receiver) throws InvalidRequestException
+        public Operation prepare(TableMetadata metadata, ColumnMetadata receiver) throws InvalidRequestException
         {
-            Term v = value.prepare(cfm.ksName, receiver);
+            Term v = value.prepare(metadata.keyspace, receiver);
 
             if (receiver.type instanceof CounterColumnType)
                 throw new InvalidRequestException(String.format("Cannot set the value of counter column %s (counters can only be incremented/decremented, not set)", receiver.name));
@@ -210,7 +213,7 @@ public abstract class Operation
             this.value = value;
         }
 
-        public Operation prepare(CFMetaData cfm, ColumnDefinition receiver) throws InvalidRequestException
+        public Operation prepare(TableMetadata metadata, ColumnMetadata receiver) throws InvalidRequestException
         {
             if (!(receiver.type instanceof CollectionType))
                 throw new InvalidRequestException(String.format("Invalid operation (%s) for non collection column %s", toString(receiver), receiver.name));
@@ -220,14 +223,14 @@ public abstract class Operation
             switch (((CollectionType)receiver.type).kind)
             {
                 case LIST:
-                    Term idx = selector.prepare(cfm.ksName, Lists.indexSpecOf(receiver));
-                    Term lval = value.prepare(cfm.ksName, Lists.valueSpecOf(receiver));
+                    Term idx = selector.prepare(metadata.keyspace, Lists.indexSpecOf(receiver));
+                    Term lval = value.prepare(metadata.keyspace, Lists.valueSpecOf(receiver));
                     return new Lists.SetterByIndex(receiver, idx, lval);
                 case SET:
                     throw new InvalidRequestException(String.format("Invalid operation (%s) for set column %s", toString(receiver), receiver.name));
                 case MAP:
-                    Term key = selector.prepare(cfm.ksName, Maps.keySpecOf(receiver));
-                    Term mval = value.prepare(cfm.ksName, Maps.valueSpecOf(receiver));
+                    Term key = selector.prepare(metadata.keyspace, Maps.keySpecOf(receiver));
+                    Term mval = value.prepare(metadata.keyspace, Maps.valueSpecOf(receiver));
                     return new Maps.SetterByKey(receiver, key, mval);
             }
             throw new AssertionError();
@@ -257,7 +260,7 @@ public abstract class Operation
             this.value = value;
         }
 
-        public Operation prepare(CFMetaData cfm, ColumnDefinition receiver) throws InvalidRequestException
+        public Operation prepare(TableMetadata metadata, ColumnMetadata receiver) throws InvalidRequestException
         {
             if (!receiver.type.isUDT())
                 throw new InvalidRequestException(String.format("Invalid operation (%s) for non-UDT column %s", toString(receiver), receiver.name));
@@ -268,7 +271,7 @@ public abstract class Operation
             if (fieldPosition == -1)
                 throw new InvalidRequestException(String.format("UDT column %s does not have a field named %s", receiver.name, field));
 
-            Term val = value.prepare(cfm.ksName, UserTypes.fieldSpecOf(receiver, fieldPosition));
+            Term val = value.prepare(metadata.keyspace, UserTypes.fieldSpecOf(receiver, fieldPosition));
             return new UserTypes.SetterByField(receiver, field, val);
         }
 
@@ -295,9 +298,9 @@ public abstract class Operation
             this.value = value;
         }
 
-        public Operation prepare(CFMetaData cfm, ColumnDefinition receiver) throws InvalidRequestException
+        public Operation prepare(TableMetadata metadata, ColumnMetadata receiver) throws InvalidRequestException
         {
-            Term v = value.prepare(cfm.ksName, receiver);
+            Term v = value.prepare(metadata.keyspace, receiver);
 
             if (!(receiver.type instanceof CollectionType))
             {
@@ -340,13 +343,13 @@ public abstract class Operation
             this.value = value;
         }
 
-        public Operation prepare(CFMetaData cfm, ColumnDefinition receiver) throws InvalidRequestException
+        public Operation prepare(TableMetadata metadata, ColumnMetadata receiver) throws InvalidRequestException
         {
             if (!(receiver.type instanceof CollectionType))
             {
                 if (!(receiver.type instanceof CounterColumnType))
                     throw new InvalidRequestException(String.format("Invalid operation (%s) for non counter column %s", toString(receiver), receiver.name));
-                return new Constants.Substracter(receiver, value.prepare(cfm.ksName, receiver));
+                return new Constants.Substracter(receiver, value.prepare(metadata.keyspace, receiver));
             }
             else if (!(receiver.type.isMultiCell()))
                 throw new InvalidRequestException(String.format("Invalid operation (%s) for frozen collection column %s", toString(receiver), receiver.name));
@@ -354,16 +357,16 @@ public abstract class Operation
             switch (((CollectionType)receiver.type).kind)
             {
                 case LIST:
-                    return new Lists.Discarder(receiver, value.prepare(cfm.ksName, receiver));
+                    return new Lists.Discarder(receiver, value.prepare(metadata.keyspace, receiver));
                 case SET:
-                    return new Sets.Discarder(receiver, value.prepare(cfm.ksName, receiver));
+                    return new Sets.Discarder(receiver, value.prepare(metadata.keyspace, receiver));
                 case MAP:
                     // The value for a map subtraction is actually a set
                     ColumnSpecification vr = new ColumnSpecification(receiver.ksName,
                                                                      receiver.cfName,
                                                                      receiver.name,
                                                                      SetType.getInstance(((MapType)receiver.type).getKeysType(), false));
-                    return new Sets.Discarder(receiver, value.prepare(cfm.ksName, vr));
+                    return new Sets.Discarder(receiver, value.prepare(metadata.keyspace, vr));
             }
             throw new AssertionError();
         }
@@ -388,9 +391,9 @@ public abstract class Operation
             this.value = value;
         }
 
-        public Operation prepare(CFMetaData cfm, ColumnDefinition receiver) throws InvalidRequestException
+        public Operation prepare(TableMetadata metadata, ColumnMetadata receiver) throws InvalidRequestException
         {
-            Term v = value.prepare(cfm.ksName, receiver);
+            Term v = value.prepare(metadata.keyspace, receiver);
 
             if (!(receiver.type instanceof ListType))
                 throw new InvalidRequestException(String.format("Invalid operation (%s) for non list column %s", toString(receiver), receiver.name));
@@ -413,19 +416,19 @@ public abstract class Operation
 
     public static class ColumnDeletion implements RawDeletion
     {
-        private final ColumnDefinition.Raw id;
+        private final ColumnMetadata.Raw id;
 
-        public ColumnDeletion(ColumnDefinition.Raw id)
+        public ColumnDeletion(ColumnMetadata.Raw id)
         {
             this.id = id;
         }
 
-        public ColumnDefinition.Raw affectedColumn()
+        public ColumnMetadata.Raw affectedColumn()
         {
             return id;
         }
 
-        public Operation prepare(String keyspace, ColumnDefinition receiver, CFMetaData cfm) throws InvalidRequestException
+        public Operation prepare(String keyspace, ColumnMetadata receiver, TableMetadata metadata) throws InvalidRequestException
         {
             // No validation, deleting a column is always "well typed"
             return new Constants.Deleter(receiver);
@@ -434,21 +437,21 @@ public abstract class Operation
 
     public static class ElementDeletion implements RawDeletion
     {
-        private final ColumnDefinition.Raw id;
+        private final ColumnMetadata.Raw id;
         private final Term.Raw element;
 
-        public ElementDeletion(ColumnDefinition.Raw id, Term.Raw element)
+        public ElementDeletion(ColumnMetadata.Raw id, Term.Raw element)
         {
             this.id = id;
             this.element = element;
         }
 
-        public ColumnDefinition.Raw affectedColumn()
+        public ColumnMetadata.Raw affectedColumn()
         {
             return id;
         }
 
-        public Operation prepare(String keyspace, ColumnDefinition receiver, CFMetaData cfm) throws InvalidRequestException
+        public Operation prepare(String keyspace, ColumnMetadata receiver, TableMetadata metadata) throws InvalidRequestException
         {
             if (!(receiver.type.isCollection()))
                 throw new InvalidRequestException(String.format("Invalid deletion operation for non collection column %s", receiver.name));
@@ -473,21 +476,21 @@ public abstract class Operation
 
     public static class FieldDeletion implements RawDeletion
     {
-        private final ColumnDefinition.Raw id;
+        private final ColumnMetadata.Raw id;
         private final FieldIdentifier field;
 
-        public FieldDeletion(ColumnDefinition.Raw id, FieldIdentifier field)
+        public FieldDeletion(ColumnMetadata.Raw id, FieldIdentifier field)
         {
             this.id = id;
             this.field = field;
         }
 
-        public ColumnDefinition.Raw affectedColumn()
+        public ColumnMetadata.Raw affectedColumn()
         {
             return id;
         }
 
-        public Operation prepare(String keyspace, ColumnDefinition receiver, CFMetaData cfm) throws InvalidRequestException
+        public Operation prepare(String keyspace, ColumnMetadata receiver, TableMetadata metadata) throws InvalidRequestException
         {
             if (!receiver.type.isUDT())
                 throw new InvalidRequestException(String.format("Invalid field deletion operation for non-UDT column %s", receiver.name));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/QueryOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryOptions.java b/src/java/org/apache/cassandra/cql3/QueryOptions.java
index 57d5eac..afe20d7 100644
--- a/src/java/org/apache/cassandra/cql3/QueryOptions.java
+++ b/src/java/org/apache/cassandra/cql3/QueryOptions.java
@@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableList;
 
 import io.netty.buffer.ByteBuf;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -101,7 +101,7 @@ public abstract class QueryOptions
      * @return the value correspong to column {@code columnName} in the (JSON) bind value at index {@code bindIndex}. This may return null if the
      * JSON value has no value for this column.
      */
-    public Term getJsonColumnValue(int bindIndex, ColumnIdentifier columnName, Collection<ColumnDefinition> expectedReceivers) throws InvalidRequestException
+    public Term getJsonColumnValue(int bindIndex, ColumnIdentifier columnName, Collection<ColumnMetadata> expectedReceivers) throws InvalidRequestException
     {
         if (jsonValuesCache == null)
             jsonValuesCache = new ArrayList<>(Collections.<Map<ColumnIdentifier, Term>>nCopies(getValues().size(), null));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index a100878..cf0e777 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -39,8 +39,9 @@ import org.slf4j.LoggerFactory;
 import org.antlr.runtime.*;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaChangeListener;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.cql3.statements.*;
@@ -163,7 +164,7 @@ public class QueryProcessor implements QueryHandler
 
     private QueryProcessor()
     {
-        MigrationManager.instance.register(new MigrationSubscriber());
+        Schema.instance.registerListener(new StatementInvalidatingListener());
     }
 
     public ParsedStatement.Prepared getPrepared(MD5Digest id)
@@ -559,7 +560,7 @@ public class QueryProcessor implements QueryHandler
         internalStatements.clear();
     }
 
-    private static class MigrationSubscriber extends MigrationListener
+    private static class StatementInvalidatingListener extends SchemaChangeListener
     {
         private static void removeInvalidPreparedStatements(String ksName, String cfName)
         {
@@ -659,18 +660,18 @@ public class QueryProcessor implements QueryHandler
         {
             // in case there are other overloads, we have to remove all overloads since argument type
             // matching may change (due to type casting)
-            if (Schema.instance.getKSMetaData(ksName).functions.get(new FunctionName(ksName, functionName)).size() > 1)
+            if (Schema.instance.getKeyspaceMetadata(ksName).functions.get(new FunctionName(ksName, functionName)).size() > 1)
                 removeInvalidPreparedStatementsForFunction(ksName, functionName);
         }
 
-        public void onUpdateColumnFamily(String ksName, String cfName, boolean affectsStatements)
+        public void onAlterTable(String ksName, String cfName, boolean affectsStatements)
         {
             logger.trace("Column definitions for {}.{} changed, invalidating related prepared statements", ksName, cfName);
             if (affectsStatements)
                 removeInvalidPreparedStatements(ksName, cfName);
         }
 
-        public void onUpdateFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+        public void onAlterFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
         {
             // Updating a function may imply we've changed the body of the function, so we need to invalid statements so that
             // the new definition is picked (the function is resolved at preparation time).
@@ -679,7 +680,7 @@ public class QueryProcessor implements QueryHandler
             removeInvalidPreparedStatementsForFunction(ksName, functionName);
         }
 
-        public void onUpdateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+        public void onAlterAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
         {
             // Updating a function may imply we've changed the body of the function, so we need to invalid statements so that
             // the new definition is picked (the function is resolved at preparation time).
@@ -694,7 +695,7 @@ public class QueryProcessor implements QueryHandler
             removeInvalidPreparedStatements(ksName, null);
         }
 
-        public void onDropColumnFamily(String ksName, String cfName)
+        public void onDropTable(String ksName, String cfName)
         {
             logger.trace("Table {}.{} was dropped, invalidating related prepared statements", ksName, cfName);
             removeInvalidPreparedStatements(ksName, cfName);


[37/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
Make TableMetadata immutable, optimize Schema

patch by Aleksey Yeschenko; reviewed by Sylvain Lebresne for
CASSANDRA-9425


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

Branch: refs/heads/trunk
Commit: af3fe39dcabd9ef77a00309ce6741268423206df
Parents: 3580f6c
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Thu Nov 10 01:16:59 2016 +0000
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri Jan 27 22:17:46 2017 +0000

----------------------------------------------------------------------
 CHANGES.txt                                     |    2 +
 .../apache/cassandra/triggers/AuditTrigger.java |   12 +-
 src/antlr/Cql.g                                 |    8 +-
 src/antlr/Parser.g                              |   60 +-
 .../org/apache/cassandra/auth/AuthKeyspace.java |   87 +-
 .../cassandra/auth/AuthMigrationListener.java   |   53 -
 .../auth/AuthSchemaChangeListener.java          |   53 +
 .../cassandra/auth/CassandraAuthorizer.java     |   12 +-
 .../cassandra/auth/CassandraRoleManager.java    |   12 +-
 .../org/apache/cassandra/auth/DataResource.java |    4 +-
 .../apache/cassandra/auth/FunctionResource.java |    2 +-
 .../cassandra/auth/PasswordAuthenticator.java   |    8 +-
 .../cassandra/batchlog/BatchlogManager.java     |    9 +-
 .../apache/cassandra/cache/AutoSavingCache.java |   29 +-
 .../org/apache/cassandra/cache/CacheKey.java    |   24 +-
 .../apache/cassandra/cache/CounterCacheKey.java |   50 +-
 .../org/apache/cassandra/cache/KeyCacheKey.java |   23 +-
 .../org/apache/cassandra/cache/OHCProvider.java |   27 +-
 .../org/apache/cassandra/cache/RowCacheKey.java |   35 +-
 .../org/apache/cassandra/config/CFMetaData.java | 1362 ------------------
 .../cassandra/config/ColumnDefinition.java      |  623 --------
 .../cassandra/config/ReadRepairDecision.java    |   23 -
 .../org/apache/cassandra/config/Schema.java     |  776 ----------
 .../cassandra/config/SchemaConstants.java       |   74 -
 .../apache/cassandra/config/ViewDefinition.java |  166 ---
 .../org/apache/cassandra/cql3/CQL3Type.java     |    4 +-
 .../apache/cassandra/cql3/ColumnIdentifier.java |    1 -
 .../org/apache/cassandra/cql3/Constants.java    |   10 +-
 src/java/org/apache/cassandra/cql3/Json.java    |   34 +-
 src/java/org/apache/cassandra/cql3/Lists.java   |   18 +-
 src/java/org/apache/cassandra/cql3/Maps.java    |   12 +-
 .../cassandra/cql3/MultiColumnRelation.java     |   62 +-
 .../org/apache/cassandra/cql3/Operation.java    |   77 +-
 .../org/apache/cassandra/cql3/QueryOptions.java |    4 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |   19 +-
 .../org/apache/cassandra/cql3/Relation.java     |   65 +-
 src/java/org/apache/cassandra/cql3/Sets.java    |   12 +-
 .../cassandra/cql3/SingleColumnRelation.java    |   64 +-
 .../apache/cassandra/cql3/TokenRelation.java    |   66 +-
 .../apache/cassandra/cql3/UntypedResultSet.java |   21 +-
 .../apache/cassandra/cql3/UpdateParameters.java |   28 +-
 .../org/apache/cassandra/cql3/UserTypes.java    |    8 +-
 .../org/apache/cassandra/cql3/Validation.java   |   58 +-
 .../cassandra/cql3/VariableSpecifications.java  |   24 +-
 .../cql3/conditions/AbstractConditions.java     |    4 +-
 .../cql3/conditions/ColumnCondition.java        |   48 +-
 .../cql3/conditions/ColumnConditions.java       |    6 +-
 .../cassandra/cql3/conditions/Conditions.java   |    4 +-
 .../cql3/functions/AbstractFunction.java        |   11 +
 .../cassandra/cql3/functions/FunctionName.java  |    2 +-
 .../cql3/functions/FunctionResolver.java        |    4 +-
 .../cassandra/cql3/functions/OperationFcts.java |    2 +-
 .../cassandra/cql3/functions/TokenFct.java      |   22 +-
 .../cassandra/cql3/functions/UDFunction.java    |   38 +-
 .../ClusteringColumnRestrictions.java           |   16 +-
 .../restrictions/CustomIndexExpression.java     |   20 +-
 .../cql3/restrictions/IndexRestrictions.java    |   14 +-
 .../restrictions/MultiColumnRestriction.java    |   50 +-
 .../restrictions/PartitionKeyRestrictions.java  |   10 +-
 .../PartitionKeySingleRestrictionSet.java       |   12 +-
 .../cql3/restrictions/Restriction.java          |    8 +-
 .../cql3/restrictions/RestrictionSet.java       |   42 +-
 .../restrictions/RestrictionSetWrapper.java     |   10 +-
 .../cql3/restrictions/Restrictions.java         |    4 +-
 .../restrictions/SingleColumnRestriction.java   |   34 +-
 .../restrictions/StatementRestrictions.java     |  118 +-
 .../cassandra/cql3/restrictions/TermSlice.java  |    4 +-
 .../cql3/restrictions/TokenFilter.java          |   20 +-
 .../cql3/restrictions/TokenRestriction.java     |   42 +-
 .../selection/AbstractFunctionSelector.java     |    4 +-
 .../cql3/selection/CollectionFactory.java       |    4 +-
 .../cassandra/cql3/selection/ListSelector.java  |    2 -
 .../cassandra/cql3/selection/MapSelector.java   |    4 +-
 .../cassandra/cql3/selection/RawSelector.java   |   16 +-
 .../cassandra/cql3/selection/Selectable.java    |  146 +-
 .../cassandra/cql3/selection/Selection.java     |   82 +-
 .../cql3/selection/SelectionColumnMapping.java  |   14 +-
 .../cql3/selection/SelectionColumns.java        |    4 +-
 .../cassandra/cql3/selection/Selector.java      |   10 +-
 .../cql3/selection/SelectorFactories.java       |   20 +-
 .../cassandra/cql3/selection/SetSelector.java   |    2 -
 .../cql3/selection/SimpleSelector.java          |    4 +-
 .../cassandra/cql3/selection/TermSelector.java  |    4 +-
 .../cql3/selection/UserTypeSelector.java        |    4 +-
 .../cql3/selection/WritetimeOrTTLSelector.java  |    4 +-
 .../cql3/statements/AlterKeyspaceStatement.java |   10 +-
 .../cql3/statements/AlterTableStatement.java    |  171 +--
 .../statements/AlterTableStatementColumn.java   |   12 +-
 .../cql3/statements/AlterTypeStatement.java     |  132 +-
 .../cql3/statements/AlterViewStatement.java     |   24 +-
 .../cql3/statements/BatchStatement.java         |   65 +-
 .../apache/cassandra/cql3/statements/Bound.java |    8 +-
 .../cql3/statements/CQL3CasRequest.java         |   36 +-
 .../statements/CreateAggregateStatement.java    |    8 +-
 .../statements/CreateFunctionStatement.java     |    9 +-
 .../cql3/statements/CreateIndexStatement.java   |   49 +-
 .../statements/CreateKeyspaceStatement.java     |    7 +-
 .../cql3/statements/CreateTableStatement.java   |   60 +-
 .../cql3/statements/CreateTriggerStatement.java |   24 +-
 .../cql3/statements/CreateTypeStatement.java    |    8 +-
 .../cql3/statements/CreateViewStatement.java    |   94 +-
 .../cql3/statements/DeleteStatement.java        |   26 +-
 .../cql3/statements/DropAggregateStatement.java |    9 +-
 .../cql3/statements/DropFunctionStatement.java  |   11 +-
 .../cql3/statements/DropIndexStatement.java     |   35 +-
 .../cql3/statements/DropKeyspaceStatement.java  |    6 +-
 .../cql3/statements/DropTableStatement.java     |   24 +-
 .../cql3/statements/DropTriggerStatement.java   |   23 +-
 .../cql3/statements/DropTypeStatement.java      |   16 +-
 .../cql3/statements/DropViewStatement.java      |   26 +-
 .../cassandra/cql3/statements/IndexTarget.java  |   56 +-
 .../cql3/statements/KeyspaceAttributes.java     |   10 +
 .../statements/ListPermissionsStatement.java    |    2 +-
 .../cql3/statements/ListRolesStatement.java     |    2 +-
 .../cql3/statements/ListUsersStatement.java     |    2 +-
 .../cql3/statements/ModificationStatement.java  |  167 ++-
 .../PermissionsManagementStatement.java         |    2 +-
 .../cql3/statements/SelectStatement.java        |  152 +-
 .../cql3/statements/TableAttributes.java        |    7 +-
 .../cql3/statements/TruncateStatement.java      |   10 +-
 .../cql3/statements/UpdateStatement.java        |   70 +-
 .../cql3/statements/UpdatesCollector.java       |   27 +-
 .../db/AbstractReadCommandBuilder.java          |   44 +-
 .../org/apache/cassandra/db/Clustering.java     |   16 +-
 .../cassandra/db/ClusteringBoundOrBoundary.java |    4 +-
 .../cassandra/db/ClusteringComparator.java      |    4 +-
 .../apache/cassandra/db/ClusteringPrefix.java   |   20 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |  212 +--
 .../db/ColumnFamilyStoreCQLHelper.java          |  146 +-
 .../cassandra/db/ColumnFamilyStoreMBean.java    |    5 +-
 src/java/org/apache/cassandra/db/Columns.java   |  124 +-
 .../org/apache/cassandra/db/CompactTables.java  |   16 +-
 .../apache/cassandra/db/ConsistencyLevel.java   |    8 +-
 .../apache/cassandra/db/CounterMutation.java    |   13 +-
 src/java/org/apache/cassandra/db/DataRange.java |   26 +-
 .../db/DefinitionsUpdateVerbHandler.java        |   55 -
 .../org/apache/cassandra/db/Directories.java    |   43 +-
 .../org/apache/cassandra/db/EmptyIterators.java |   38 +-
 src/java/org/apache/cassandra/db/IMutation.java |    4 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   91 +-
 src/java/org/apache/cassandra/db/Memtable.java  |   51 +-
 .../db/MigrationRequestVerbHandler.java         |   48 -
 .../org/apache/cassandra/db/MultiCBuilder.java  |   12 +-
 src/java/org/apache/cassandra/db/Mutation.java  |   67 +-
 .../apache/cassandra/db/PartitionColumns.java   |  199 ---
 .../cassandra/db/PartitionRangeReadCommand.java |   21 +-
 .../org/apache/cassandra/db/ReadCommand.java    |   57 +-
 .../cassandra/db/ReadExecutionController.java   |   16 +-
 src/java/org/apache/cassandra/db/ReadQuery.java |    2 +-
 .../org/apache/cassandra/db/ReadResponse.java   |    4 +-
 .../cassandra/db/RegularAndStaticColumns.java   |  194 +++
 .../cassandra/db/SchemaCheckVerbHandler.java    |   42 -
 .../cassandra/db/SerializationHeader.java       |   61 +-
 .../org/apache/cassandra/db/SimpleBuilders.java |   53 +-
 .../db/SinglePartitionReadCommand.java          |   68 +-
 .../cassandra/db/SizeEstimatesRecorder.java     |   16 +-
 src/java/org/apache/cassandra/db/Slice.java     |    6 -
 src/java/org/apache/cassandra/db/Slices.java    |   18 +-
 .../org/apache/cassandra/db/StorageHook.java    |   11 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |  537 +++----
 .../cassandra/db/UnfilteredDeserializer.java    |   16 +-
 .../cassandra/db/UnknownColumnException.java    |   51 -
 .../db/UnknownColumnFamilyException.java        |   33 -
 .../columniterator/AbstractSSTableIterator.java |   16 +-
 .../db/columniterator/SSTableIterator.java      |    2 +-
 .../columniterator/SSTableReversedIterator.java |   14 +-
 .../AbstractCommitLogSegmentManager.java        |   29 +-
 .../cassandra/db/commitlog/CommitLog.java       |   15 +-
 .../cassandra/db/commitlog/CommitLogReader.java |   15 +-
 .../db/commitlog/CommitLogReplayer.java         |   54 +-
 .../db/commitlog/CommitLogSegment.java          |   57 +-
 .../cassandra/db/compaction/CompactionInfo.java |   26 +-
 .../db/compaction/CompactionIterator.java       |   18 +-
 .../db/compaction/CompactionManager.java        |   32 +-
 .../compaction/CompactionStrategyManager.java   |   29 +-
 .../compaction/LeveledCompactionStrategy.java   |   12 +-
 .../cassandra/db/compaction/Scrubber.java       |   16 +-
 .../cassandra/db/compaction/Upgrader.java       |    4 +-
 .../cassandra/db/compaction/Verifier.java       |    4 +-
 .../writers/DefaultCompactionWriter.java        |    4 +-
 .../writers/MajorLeveledCompactionWriter.java   |    4 +-
 .../writers/MaxSSTableSizeWriter.java           |    4 +-
 .../SplittingSizeTieredCompactionWriter.java    |    4 +-
 .../filter/AbstractClusteringIndexFilter.java   |   10 +-
 .../db/filter/ClusteringIndexFilter.java        |   18 +-
 .../db/filter/ClusteringIndexNamesFilter.java   |   30 +-
 .../db/filter/ClusteringIndexSliceFilter.java   |   10 +-
 .../cassandra/db/filter/ColumnFilter.java       |   82 +-
 .../cassandra/db/filter/ColumnSubselection.java |   32 +-
 .../apache/cassandra/db/filter/DataLimits.java  |    2 +-
 .../apache/cassandra/db/filter/RowFilter.java   |   69 +-
 .../filter/TombstoneOverwhelmingException.java  |    8 +-
 .../db/lifecycle/LifecycleTransaction.java      |    7 +-
 .../cassandra/db/lifecycle/LogTransaction.java  |    4 +-
 .../cassandra/db/marshal/CompositeType.java     |    6 +
 .../apache/cassandra/db/marshal/UserType.java   |    5 +
 .../db/partitions/AbstractBTreePartition.java   |   52 +-
 .../db/partitions/AtomicBTreePartition.java     |   19 +-
 .../db/partitions/CachedBTreePartition.java     |   22 +-
 .../db/partitions/FilteredPartition.java        |   10 +-
 .../db/partitions/ImmutableBTreePartition.java  |   32 +-
 .../cassandra/db/partitions/Partition.java      |    6 +-
 .../db/partitions/PartitionUpdate.java          |   89 +-
 .../SingletonUnfilteredPartitionIterator.java   |    4 +-
 .../partitions/UnfilteredPartitionIterator.java |    4 +-
 .../UnfilteredPartitionIterators.java           |   15 +-
 .../apache/cassandra/db/rows/AbstractCell.java  |    6 +-
 .../db/rows/AbstractRangeTombstoneMarker.java   |    8 +-
 .../apache/cassandra/db/rows/AbstractRow.java   |   10 +-
 .../db/rows/AbstractUnfilteredRowIterator.java  |   14 +-
 .../org/apache/cassandra/db/rows/BTreeRow.java  |   53 +-
 .../cassandra/db/rows/BaseRowIterator.java      |    8 +-
 .../apache/cassandra/db/rows/BufferCell.java    |   20 +-
 src/java/org/apache/cassandra/db/rows/Cell.java |   13 +-
 .../org/apache/cassandra/db/rows/Cells.java     |    6 +-
 .../apache/cassandra/db/rows/ColumnData.java    |    8 +-
 .../cassandra/db/rows/ComplexColumnData.java    |   14 +-
 .../LazilyInitializedUnfilteredRowIterator.java |    6 +-
 .../apache/cassandra/db/rows/NativeCell.java    |    6 +-
 .../db/rows/RangeTombstoneBoundMarker.java      |    4 +-
 .../db/rows/RangeTombstoneBoundaryMarker.java   |    4 +-
 src/java/org/apache/cassandra/db/rows/Row.java  |   26 +-
 .../db/rows/RowAndDeletionMergeIterator.java    |    5 +-
 .../cassandra/db/rows/RowDiffListener.java      |    4 +-
 .../apache/cassandra/db/rows/RowIterators.java  |   10 +-
 src/java/org/apache/cassandra/db/rows/Rows.java |   13 +-
 .../cassandra/db/rows/SerializationHelper.java  |   25 +-
 .../apache/cassandra/db/rows/Unfiltered.java    |   10 +-
 .../rows/UnfilteredRowIteratorSerializer.java   |   12 +-
 .../UnfilteredRowIteratorWithLowerBound.java    |   22 +-
 .../db/rows/UnfilteredRowIterators.java         |   33 +-
 .../cassandra/db/rows/UnfilteredSerializer.java |   22 +-
 .../cassandra/db/rows/WithOnlyQueriedData.java  |    4 +-
 .../db/rows/WrappingUnfilteredRowIterator.java  |    6 +-
 .../apache/cassandra/db/transform/BaseRows.java |    8 +-
 .../cassandra/db/transform/Transformation.java  |    4 +-
 .../db/transform/UnfilteredPartitions.java      |    4 +-
 .../cassandra/db/transform/UnfilteredRows.java  |   12 +-
 .../apache/cassandra/db/view/TableViews.java    |   35 +-
 src/java/org/apache/cassandra/db/view/View.java |   54 +-
 .../apache/cassandra/db/view/ViewBuilder.java   |   12 +-
 .../apache/cassandra/db/view/ViewManager.java   |   47 +-
 .../cassandra/db/view/ViewUpdateGenerator.java  |   37 +-
 .../org/apache/cassandra/dht/BootStrapper.java  |    2 +-
 .../cassandra/dht/ByteOrderedPartitioner.java   |    8 +-
 .../dht/OrderPreservingPartitioner.java         |    7 +-
 .../exceptions/UnknownIndexException.java       |   39 +
 .../exceptions/UnknownTableException.java       |   33 +
 .../hadoop/cql3/CqlBulkRecordWriter.java        |    7 +-
 .../cassandra/hadoop/cql3/CqlInputFormat.java   |    2 +-
 src/java/org/apache/cassandra/hints/Hint.java   |    7 +-
 .../org/apache/cassandra/hints/HintMessage.java |   11 +-
 .../org/apache/cassandra/hints/HintsReader.java |    7 +-
 src/java/org/apache/cassandra/index/Index.java  |   10 +-
 .../cassandra/index/SecondaryIndexManager.java  |   39 +-
 .../apache/cassandra/index/TargetParser.java    |   18 +-
 .../index/internal/CassandraIndex.java          |   79 +-
 .../index/internal/CassandraIndexFunctions.java |   45 +-
 .../index/internal/CassandraIndexSearcher.java  |    7 +-
 .../internal/CollatedViewIndexBuilder.java      |    2 +-
 .../composites/ClusteringColumnIndex.java       |    2 +-
 .../composites/CollectionEntryIndex.java        |    4 +-
 .../internal/composites/CollectionKeyIndex.java |    4 +-
 .../composites/CollectionKeyIndexBase.java      |    2 +-
 .../composites/CollectionValueIndex.java        |    8 +-
 .../internal/composites/CompositesSearcher.java |    8 +-
 .../internal/composites/PartitionKeyIndex.java  |    4 +-
 .../index/internal/keys/KeysIndex.java          |   15 +-
 .../index/internal/keys/KeysSearcher.java       |    6 +-
 .../apache/cassandra/index/sasi/SASIIndex.java  |   37 +-
 .../cassandra/index/sasi/SASIIndexBuilder.java  |   18 +-
 .../cassandra/index/sasi/conf/ColumnIndex.java  |   10 +-
 .../cassandra/index/sasi/conf/IndexMode.java    |    6 +-
 .../index/sasi/disk/PerSSTableIndexWriter.java  |   10 +-
 .../index/sasi/memory/TrieMemIndex.java         |   10 +-
 .../cassandra/index/sasi/plan/Expression.java   |    4 +-
 .../cassandra/index/sasi/plan/Operation.java    |   21 +-
 .../index/sasi/plan/QueryController.java        |   17 +-
 .../cassandra/index/sasi/plan/QueryPlan.java    |    4 +-
 .../io/sstable/AbstractSSTableSimpleWriter.java |   17 +-
 .../cassandra/io/sstable/CQLSSTableWriter.java  |   48 +-
 .../io/sstable/IndexSummaryManager.java         |    9 +-
 .../io/sstable/IndexSummaryRedistribution.java  |   20 +-
 .../cassandra/io/sstable/KeyIterator.java       |    7 +-
 .../io/sstable/ReducingKeyIterator.java         |    2 +-
 .../apache/cassandra/io/sstable/SSTable.java    |   17 +-
 .../io/sstable/SSTableIdentityIterator.java     |   20 +-
 .../cassandra/io/sstable/SSTableLoader.java     |    8 +-
 .../io/sstable/SSTableMultiWriter.java          |    4 +-
 .../io/sstable/SSTableSimpleIterator.java       |   21 +-
 .../io/sstable/SSTableSimpleUnsortedWriter.java |    8 +-
 .../io/sstable/SSTableSimpleWriter.java         |    6 +-
 .../cassandra/io/sstable/SSTableTxnWriter.java  |   13 +-
 .../io/sstable/SimpleSSTableMultiWriter.java    |   13 +-
 .../sstable/format/RangeAwareSSTableWriter.java |    6 +-
 .../io/sstable/format/SSTableFormat.java        |    5 +-
 .../io/sstable/format/SSTableReader.java        |  118 +-
 .../io/sstable/format/SSTableWriter.java        |   27 +-
 .../io/sstable/format/big/BigFormat.java        |    9 +-
 .../io/sstable/format/big/BigTableReader.java   |   25 +-
 .../io/sstable/format/big/BigTableScanner.java  |   19 +-
 .../io/sstable/format/big/BigTableWriter.java   |   20 +-
 .../cassandra/metrics/CompactionMetrics.java    |   20 +-
 .../apache/cassandra/metrics/TableMetrics.java  |    6 +-
 .../cassandra/net/IncomingTcpConnection.java    |    7 +-
 .../apache/cassandra/net/MessagingService.java  |    6 +-
 .../repair/RepairMessageVerbHandler.java        |   13 +-
 .../apache/cassandra/repair/RepairRunnable.java |    2 +-
 .../repair/SystemDistributedKeyspace.java       |  131 +-
 .../repair/messages/PrepareMessage.java         |   47 +-
 .../apache/cassandra/schema/ColumnMetadata.java |  617 ++++++++
 .../cassandra/schema/CompactionParams.java      |    2 +-
 .../apache/cassandra/schema/DroppedColumn.java  |   59 +
 .../org/apache/cassandra/schema/Functions.java  |   27 +-
 .../apache/cassandra/schema/IndexMetadata.java  |   53 +-
 .../org/apache/cassandra/schema/Indexes.java    |   55 +-
 .../cassandra/schema/KeyspaceMetadata.java      |   53 +-
 .../org/apache/cassandra/schema/Keyspaces.java  |   46 +
 .../cassandra/schema/MigrationManager.java      |  493 +++++++
 .../apache/cassandra/schema/MigrationTask.java  |  113 ++
 .../org/apache/cassandra/schema/Schema.java     |  804 +++++++++++
 .../cassandra/schema/SchemaChangeListener.java  |  102 ++
 .../cassandra/schema/SchemaConstants.java       |   82 ++
 .../apache/cassandra/schema/SchemaKeyspace.java |  808 +++++------
 .../cassandra/schema/SchemaPullVerbHandler.java |   50 +
 .../cassandra/schema/SchemaPushVerbHandler.java |   47 +
 .../schema/SchemaVersionVerbHandler.java        |   46 +
 .../org/apache/cassandra/schema/TableId.java    |  118 ++
 .../apache/cassandra/schema/TableMetadata.java  |  956 ++++++++++++
 .../cassandra/schema/TableMetadataRef.java      |   78 +
 .../apache/cassandra/schema/TableParams.java    |    5 +
 .../org/apache/cassandra/schema/Tables.java     |   89 +-
 .../org/apache/cassandra/schema/Triggers.java   |   17 +
 .../cassandra/schema/UnknownIndexException.java |   39 -
 .../apache/cassandra/schema/ViewMetadata.java   |  197 +++
 src/java/org/apache/cassandra/schema/Views.java |   66 +-
 .../cassandra/service/AbstractReadExecutor.java |   23 +-
 .../cassandra/service/ActiveRepairService.java  |   97 +-
 .../apache/cassandra/service/CacheService.java  |   60 +-
 .../cassandra/service/CassandraDaemon.java      |    9 +-
 .../apache/cassandra/service/ClientState.java   |   22 +-
 .../apache/cassandra/service/DataResolver.java  |   22 +-
 .../cassandra/service/MigrationListener.java    |  102 --
 .../cassandra/service/MigrationManager.java     |  626 --------
 .../apache/cassandra/service/MigrationTask.java |  115 --
 .../service/PendingRangeCalculatorService.java  |    2 +-
 .../apache/cassandra/service/ReadCallback.java  |    4 +-
 .../cassandra/service/ReadRepairDecision.java   |   23 +
 .../apache/cassandra/service/StartupChecks.java |    9 +-
 .../apache/cassandra/service/StorageProxy.java  |   38 +-
 .../cassandra/service/StorageService.java       |   53 +-
 .../service/pager/AbstractQueryPager.java       |    6 +-
 .../service/pager/AggregationQueryPager.java    |    6 +-
 .../cassandra/service/pager/PagingState.java    |   12 +-
 .../service/pager/PartitionRangeQueryPager.java |    2 +-
 .../apache/cassandra/service/paxos/Commit.java  |    6 +-
 .../cassandra/service/paxos/PaxosState.java     |   12 +-
 .../service/paxos/PrepareCallback.java          |    7 +-
 .../cassandra/streaming/StreamReader.java       |   45 +-
 .../cassandra/streaming/StreamReceiveTask.java  |   27 +-
 .../cassandra/streaming/StreamSession.java      |   37 +-
 .../cassandra/streaming/StreamSummary.java      |   22 +-
 .../apache/cassandra/streaming/StreamTask.java  |   10 +-
 .../cassandra/streaming/StreamTransferTask.java |    7 +-
 .../compress/CompressedStreamReader.java        |   13 +-
 .../management/StreamSummaryCompositeData.java  |    8 +-
 .../streaming/messages/FileMessageHeader.java   |   27 +-
 .../streaming/messages/OutgoingFileMessage.java |    4 +-
 .../streaming/messages/ReceivedMessage.java     |   16 +-
 .../apache/cassandra/tools/JsonTransformer.java |   27 +-
 .../cassandra/tools/SSTableExpiredBlockers.java |   13 +-
 .../apache/cassandra/tools/SSTableExport.java   |   27 +-
 .../cassandra/tools/SSTableLevelResetter.java   |    4 +-
 .../cassandra/tools/SSTableOfflineRelevel.java  |    4 +-
 .../cassandra/tools/StandaloneSSTableUtil.java  |    9 +-
 .../cassandra/tools/StandaloneScrubber.java     |    4 +-
 .../cassandra/tools/StandaloneSplitter.java     |    2 +-
 .../cassandra/tools/StandaloneUpgrader.java     |    6 +-
 .../cassandra/tools/StandaloneVerifier.java     |    4 +-
 .../cassandra/tools/nodetool/Cleanup.java       |    2 +-
 .../apache/cassandra/tools/nodetool/Repair.java |    2 +-
 .../apache/cassandra/tracing/TraceKeyspace.java |   74 +-
 .../org/apache/cassandra/transport/Server.java  |   24 +-
 .../cassandra/triggers/TriggerExecutor.java     |   21 +-
 .../utils/NativeSSTableLoaderClient.java        |   78 +-
 .../apache/cassandra/cql3/CorruptionTest.java   |    2 +-
 .../db/commitlog/CommitLogStressTest.java       |    5 +-
 .../db/compaction/LongCompactionsTest.java      |   10 +-
 .../LongLeveledCompactionStrategyTest.java      |    6 +-
 .../cassandra/hints/HintsWriteThenReadTest.java |    6 +-
 .../cassandra/streaming/LongStreamingTest.java  |   13 +-
 .../test/microbench/CompactionBench.java        |   35 -
 .../test/microbench/MutationBench.java          |   26 +-
 .../test/microbench/ReadWriteTest.java          |   31 -
 test/unit/org/apache/cassandra/MockSchema.java  |  187 ---
 .../unit/org/apache/cassandra/SchemaLoader.java |  788 +++++-----
 .../org/apache/cassandra/UpdateBuilder.java     |    7 +-
 test/unit/org/apache/cassandra/Util.java        |   75 +-
 .../apache/cassandra/batchlog/BatchTest.java    |    6 +-
 .../cassandra/batchlog/BatchlogManagerTest.java |   22 +-
 .../cassandra/cache/AutoSavingCacheTest.java    |   17 +-
 .../cassandra/cache/CacheProviderTest.java      |   80 +-
 .../apache/cassandra/config/CFMetaDataTest.java |  193 ---
 .../config/DatabaseDescriptorTest.java          |   49 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |    9 +-
 .../apache/cassandra/cql3/KeyCacheCqlTest.java  |   14 +-
 .../apache/cassandra/cql3/OutOfSpaceTest.java   |    6 +-
 .../cassandra/cql3/PreparedStatementsTest.java  |    2 +-
 .../cassandra/cql3/PstmtPersistenceTest.java    |    2 +-
 .../cassandra/cql3/ViewFilteringTest.java       |    3 -
 .../apache/cassandra/cql3/ViewSchemaTest.java   |   18 +-
 .../org/apache/cassandra/cql3/ViewTest.java     |    8 +-
 .../cql3/conditions/ColumnConditionTest.java    |   20 +-
 .../ClusteringColumnRestrictionsTest.java       |  593 ++++----
 .../selection/SelectionColumnMappingTest.java   |   41 +-
 .../validation/entities/SecondaryIndexTest.java |   28 +-
 .../cql3/validation/entities/UFAuthTest.java    |    2 +-
 .../cql3/validation/entities/UFJavaTest.java    |    3 +-
 .../cql3/validation/entities/UFTest.java        |    6 +-
 .../miscellaneous/CrcCheckChanceTest.java       |    2 +-
 .../validation/operations/AggregationTest.java  |    8 +-
 .../cql3/validation/operations/AlterTest.java   |   10 +-
 .../cql3/validation/operations/CreateTest.java  |   15 +-
 .../operations/DropRecreateAndRestoreTest.java  |    8 +-
 .../operations/InsertUpdateIfConditionTest.java |    2 +-
 test/unit/org/apache/cassandra/db/CellTest.java |   55 +-
 .../org/apache/cassandra/db/CleanupTest.java    |    6 +-
 .../cassandra/db/ColumnFamilyMetricTest.java    |    6 +-
 .../db/ColumnFamilyStoreCQLHelperTest.java      |  371 +++--
 .../cassandra/db/ColumnFamilyStoreTest.java     |   50 +-
 .../org/apache/cassandra/db/ColumnsTest.java    |   85 +-
 .../apache/cassandra/db/CounterCacheTest.java   |   63 +-
 .../apache/cassandra/db/CounterCellTest.java    |   12 +-
 .../cassandra/db/CounterMutationTest.java       |   46 +-
 .../cassandra/db/DeletePartitionTest.java       |    8 +-
 .../apache/cassandra/db/DirectoriesTest.java    |   95 +-
 .../org/apache/cassandra/db/KeyCacheTest.java   |    4 +-
 .../org/apache/cassandra/db/KeyspaceTest.java   |   34 +-
 .../org/apache/cassandra/db/NameSortTest.java   |    6 +-
 .../org/apache/cassandra/db/NativeCellTest.java |   18 +-
 .../cassandra/db/PartitionRangeReadTest.java    |   16 +-
 .../org/apache/cassandra/db/PartitionTest.java  |   20 +-
 .../apache/cassandra/db/RangeTombstoneTest.java |  134 +-
 .../apache/cassandra/db/ReadCommandTest.java    |   70 +-
 .../apache/cassandra/db/ReadMessageTest.java    |   59 +-
 .../db/RecoveryManagerFlushedTest.java          |    4 +-
 .../db/RecoveryManagerMissingHeaderTest.java    |    4 +-
 .../cassandra/db/RecoveryManagerTest.java       |   49 +-
 .../db/RecoveryManagerTruncateTest.java         |    2 +-
 .../org/apache/cassandra/db/RowCacheTest.java   |   13 +-
 .../apache/cassandra/db/RowIndexEntryTest.java  |   22 +-
 test/unit/org/apache/cassandra/db/RowTest.java  |   68 +-
 .../apache/cassandra/db/RowUpdateBuilder.java   |   40 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |   50 +-
 .../apache/cassandra/db/SecondaryIndexTest.java |  115 +-
 .../db/SinglePartitionSliceCommandTest.java     |   53 +-
 .../apache/cassandra/db/SystemKeyspaceTest.java |    2 +-
 .../apache/cassandra/db/TransformerTest.java    |   28 +-
 .../org/apache/cassandra/db/VerifyTest.java     |   41 +-
 .../db/commitlog/CommitLogReaderTest.java       |   18 +-
 .../CommitLogSegmentManagerCDCTest.java         |    6 +-
 .../commitlog/CommitLogSegmentManagerTest.java  |    2 +-
 .../cassandra/db/commitlog/CommitLogTest.java   |   65 +-
 .../db/commitlog/CommitLogUpgradeTest.java      |   37 +-
 .../db/commitlog/CommitLogUpgradeTestMaker.java |   10 +-
 .../db/commitlog/SnapshotDeletingTest.java      |    4 +-
 .../db/compaction/AntiCompactionTest.java       |   19 +-
 .../compaction/BlacklistingCompactionsTest.java |    6 +-
 .../db/compaction/CompactionControllerTest.java |   52 +-
 .../db/compaction/CompactionIteratorTest.java   |    9 +-
 .../db/compaction/CompactionsPurgeTest.java     |   70 +-
 .../db/compaction/CompactionsTest.java          |   15 +-
 .../DateTieredCompactionStrategyTest.java       |   12 +-
 .../LeveledCompactionStrategyTest.java          |   14 +-
 .../db/compaction/OneCompactionTest.java        |    2 +-
 .../SizeTieredCompactionStrategyTest.java       |    2 +-
 .../cassandra/db/compaction/TTLExpiryTest.java  |   71 +-
 .../TimeWindowCompactionStrategyTest.java       |   10 +-
 .../cassandra/db/lifecycle/HelpersTest.java     |    2 +-
 .../db/lifecycle/LifecycleTransactionTest.java  |    4 +-
 .../db/lifecycle/LogTransactionTest.java        |   81 +-
 .../db/lifecycle/RealTransactionsTest.java      |    8 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |    2 +-
 .../apache/cassandra/db/lifecycle/ViewTest.java |    2 +-
 .../cassandra/db/marshal/CompositeTypeTest.java |   15 +-
 .../db/marshal/DynamicCompositeTypeTest.java    |   26 +-
 .../partition/PartitionImplementationTest.java  |   82 +-
 .../db/partition/PartitionUpdateTest.java       |    6 +-
 .../rows/RowAndDeletionMergeIteratorTest.java   |   35 +-
 .../apache/cassandra/db/rows/RowBuilder.java    |    6 +-
 .../org/apache/cassandra/db/rows/RowsTest.java  |   66 +-
 .../rows/UnfilteredRowIteratorsMergeTest.java   |   16 +-
 .../db/rows/UnfilteredRowsGenerator.java        |   10 +-
 .../apache/cassandra/db/view/ViewUtilsTest.java |    8 +-
 .../apache/cassandra/dht/BootStrapperTest.java  |    2 +-
 .../apache/cassandra/dht/KeyCollisionTest.java  |    4 +-
 .../apache/cassandra/dht/LengthPartitioner.java |    8 +-
 .../apache/cassandra/hints/AlteredHints.java    |    7 +-
 .../apache/cassandra/hints/HintMessageTest.java |    6 +-
 .../org/apache/cassandra/hints/HintTest.java    |   49 +-
 .../apache/cassandra/hints/HintsBufferTest.java |    6 +-
 .../cassandra/hints/HintsServiceTest.java       |   12 +-
 .../apache/cassandra/hints/HintsTestUtil.java   |   16 +-
 .../apache/cassandra/index/CustomIndexTest.java |   84 +-
 .../org/apache/cassandra/index/StubIndex.java   |   10 +-
 .../index/internal/CassandraIndexTest.java      |   38 +-
 .../index/internal/CustomCassandraIndex.java    |   39 +-
 .../cassandra/index/sasi/SASIIndexTest.java     |  100 +-
 .../sasi/disk/PerSSTableIndexWriterTest.java    |   26 +-
 .../index/sasi/plan/OperationTest.java          |   67 +-
 .../io/compress/CQLCompressionTest.java         |   10 +-
 .../io/sstable/BigTableWriterTest.java          |    4 +-
 .../io/sstable/CQLSSTableWriterTest.java        |    6 +-
 .../io/sstable/IndexSummaryManagerTest.java     |  113 +-
 .../sstable/SSTableCorruptionDetectionTest.java |   26 +-
 .../cassandra/io/sstable/SSTableLoaderTest.java |   13 +-
 .../io/sstable/SSTableMetadataTest.java         |   32 +-
 .../cassandra/io/sstable/SSTableReaderTest.java |   30 +-
 .../io/sstable/SSTableRewriterTest.java         |    8 +-
 .../io/sstable/SSTableScannerTest.java          |   21 +-
 .../cassandra/io/sstable/SSTableUtils.java      |   14 +-
 .../cassandra/io/sstable/SSTableWriterTest.java |   16 +-
 .../io/sstable/SSTableWriterTestBase.java       |    2 +-
 .../format/SSTableFlushObserverTest.java        |   30 +-
 .../metadata/MetadataSerializerTest.java        |    5 +-
 .../ReplicationStrategyEndpointCacheTest.java   |    3 +-
 .../cassandra/locator/SimpleStrategyTest.java   |    4 +-
 .../cassandra/metrics/BatchMetricsTest.java     |    2 +-
 .../cassandra/metrics/CQLMetricsTest.java       |    2 +-
 .../cassandra/net/WriteCallbackInfoTest.java    |    6 +-
 .../apache/cassandra/repair/ValidatorTest.java  |    9 +-
 .../RepairMessageSerializationsTest.java        |    3 +-
 .../org/apache/cassandra/schema/DefsTest.java   |  545 -------
 .../cassandra/schema/MigrationManagerTest.java  |  539 +++++++
 .../org/apache/cassandra/schema/MockSchema.java |  187 +++
 .../cassandra/schema/SchemaKeyspaceTest.java    |   46 +-
 .../org/apache/cassandra/schema/SchemaTest.java |   79 +
 .../apache/cassandra/schema/ValidationTest.java |  126 ++
 .../service/ActiveRepairServiceTest.java        |   30 +-
 .../cassandra/service/DataResolverTest.java     |   62 +-
 .../cassandra/service/JoinTokenRingTest.java    |    2 +-
 .../service/LeaveAndBootstrapTest.java          |    4 +-
 .../org/apache/cassandra/service/MoveTest.java  |   20 +-
 .../cassandra/service/PaxosStateTest.java       |    6 +-
 .../cassandra/service/QueryPagerTest.java       |   23 +-
 .../cassandra/service/StartupChecksTest.java    |    2 +-
 .../service/StorageServiceServerTest.java       |   20 +-
 .../cassandra/streaming/SessionInfoTest.java    |    7 +-
 .../streaming/StreamTransferTaskTest.java       |    8 +-
 .../streaming/StreamingTransferTest.java        |   54 +-
 .../org/apache/cassandra/tools/ToolsTester.java |    4 +-
 .../cassandra/triggers/TriggerExecutorTest.java |  118 +-
 .../cassandra/triggers/TriggersSchemaTest.java  |   89 +-
 .../apache/cassandra/triggers/TriggersTest.java |    7 +-
 .../io/sstable/StressCQLSSTableWriter.java      |   44 +-
 .../apache/cassandra/stress/StressProfile.java  |   60 +-
 .../stress/settings/OptionCompaction.java       |    7 +-
 557 files changed, 12829 insertions(+), 13197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e695bb0..3c5bfc2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Make TableMetadata immutable, optimize Schema (CASSANDRA-9425)
  * Refactor ColumnCondition (CASSANDRA-12981)
  * Parallelize streaming of different keyspaces (CASSANDRA-4663)
  * Improved compactions metrics (CASSANDRA-13015)
@@ -30,6 +31,7 @@
  * Add support for arithmetic operators (CASSANDRA-11935)
  * Tables in system_distributed should not use gcgs of 0 (CASSANDRA-12954)
 
+
 3.10
  * Fix secondary index queries regression (CASSANDRA-13013)
  * Add duration type to the protocol V5 (CASSANDRA-12850)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/examples/triggers/src/org/apache/cassandra/triggers/AuditTrigger.java
----------------------------------------------------------------------
diff --git a/examples/triggers/src/org/apache/cassandra/triggers/AuditTrigger.java b/examples/triggers/src/org/apache/cassandra/triggers/AuditTrigger.java
index 1efbf13..2528ec7 100644
--- a/examples/triggers/src/org/apache/cassandra/triggers/AuditTrigger.java
+++ b/examples/triggers/src/org/apache/cassandra/triggers/AuditTrigger.java
@@ -22,8 +22,8 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Properties;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -40,13 +40,13 @@ public class AuditTrigger implements ITrigger
         String auditKeyspace = properties.getProperty("keyspace");
         String auditTable = properties.getProperty("table");
 
-        CFMetaData metadata = Schema.instance.getCFMetaData(auditKeyspace, auditTable);
+        TableMetadata metadata = Schema.instance.getTableMetadata(auditKeyspace, auditTable);
         PartitionUpdate.SimpleBuilder audit = PartitionUpdate.simpleBuilder(metadata, UUIDGen.getTimeUUID());
 
         audit.row()
-             .add("keyspace_name", update.metadata().ksName)
-             .add("table_name", update.metadata().cfName)
-             .add("primary_key", update.metadata().getKeyValidator().getString(update.partitionKey().getKey()));
+             .add("keyspace_name", update.metadata().keyspace)
+             .add("table_name", update.metadata().table)
+             .add("primary_key", update.metadata().partitionKeyType.getString(update.partitionKey().getKey()));
 
         return Collections.singletonList(audit.buildAsMutation());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/antlr/Cql.g
----------------------------------------------------------------------
diff --git a/src/antlr/Cql.g b/src/antlr/Cql.g
index 8b26426..17a11aa 100644
--- a/src/antlr/Cql.g
+++ b/src/antlr/Cql.g
@@ -40,17 +40,17 @@ import Parser,Lexer;
     import java.util.Set;
 
     import org.apache.cassandra.auth.*;
-    import org.apache.cassandra.config.ColumnDefinition;
     import org.apache.cassandra.cql3.*;
+    import org.apache.cassandra.cql3.conditions.*;
+    import org.apache.cassandra.cql3.functions.*;
     import org.apache.cassandra.cql3.restrictions.CustomIndexExpression;
-    import org.apache.cassandra.cql3.statements.*;
     import org.apache.cassandra.cql3.selection.*;
-    import org.apache.cassandra.cql3.functions.*;
-    import org.apache.cassandra.cql3.conditions.*;
+    import org.apache.cassandra.cql3.statements.*;
     import org.apache.cassandra.db.marshal.CollectionType;
     import org.apache.cassandra.exceptions.ConfigurationException;
     import org.apache.cassandra.exceptions.InvalidRequestException;
     import org.apache.cassandra.exceptions.SyntaxException;
+    import org.apache.cassandra.schema.ColumnMetadata;
     import org.apache.cassandra.utils.Pair;
 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/antlr/Parser.g
----------------------------------------------------------------------
diff --git a/src/antlr/Parser.g b/src/antlr/Parser.g
index 5fd6851..6cc5f1b 100644
--- a/src/antlr/Parser.g
+++ b/src/antlr/Parser.g
@@ -133,9 +133,9 @@ options {
         return res;
     }
 
-    public void addRawUpdate(List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, Operation.RawUpdate update)
+    public void addRawUpdate(List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> operations, ColumnMetadata.Raw key, Operation.RawUpdate update)
     {
-        for (Pair<ColumnDefinition.Raw, Operation.RawUpdate> p : operations)
+        for (Pair<ColumnMetadata.Raw, Operation.RawUpdate> p : operations)
         {
             if (p.left.equals(key) && !p.right.isCompatibleWith(update))
                 addRecognitionError("Multiple incompatible setting of column " + key);
@@ -262,8 +262,8 @@ selectStatement returns [SelectStatement.RawStatement expr]
     @init {
         Term.Raw limit = null;
         Term.Raw perPartitionLimit = null;
-        Map<ColumnDefinition.Raw, Boolean> orderings = new LinkedHashMap<>();
-        List<ColumnDefinition.Raw> groups = new ArrayList<>();
+        Map<ColumnMetadata.Raw, Boolean> orderings = new LinkedHashMap<>();
+        List<ColumnMetadata.Raw> groups = new ArrayList<>();
         boolean allowFiltering = false;
         boolean isJson = false;
     }
@@ -414,8 +414,8 @@ selectionFunctionArgs returns [List<Selectable.Raw> a]
 
 sident returns [Selectable.Raw id]
     : t=IDENT              { $id = Selectable.RawIdentifier.forUnquoted($t.text); }
-    | t=QUOTED_NAME        { $id = ColumnDefinition.RawIdentifier.forQuoted($t.text); }
-    | k=unreserved_keyword { $id = ColumnDefinition.RawIdentifier.forUnquoted(k); }
+    | t=QUOTED_NAME        { $id = ColumnMetadata.RawIdentifier.forQuoted($t.text); }
+    | k=unreserved_keyword { $id = ColumnMetadata.RawIdentifier.forUnquoted(k); }
     ;
 
 whereClause returns [WhereClause.Builder clause]
@@ -433,14 +433,14 @@ customIndexExpression [WhereClause.Builder clause]
     : 'expr(' idxName[name] ',' t=term ')' { clause.add(new CustomIndexExpression(name, t));}
     ;
 
-orderByClause[Map<ColumnDefinition.Raw, Boolean> orderings]
+orderByClause[Map<ColumnMetadata.Raw, Boolean> orderings]
     @init{
         boolean reversed = false;
     }
     : c=cident (K_ASC | K_DESC { reversed = true; })? { orderings.put(c, reversed); }
     ;
 
-groupByClause[List<ColumnDefinition.Raw> groups]
+groupByClause[List<ColumnMetadata.Raw> groups]
     : c=cident { groups.add(c); }
     ;
 
@@ -459,7 +459,7 @@ insertStatement returns [ModificationStatement.Parsed expr]
 normalInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsert expr]
     @init {
         Attributes.Raw attrs = new Attributes.Raw();
-        List<ColumnDefinition.Raw> columnNames  = new ArrayList<>();
+        List<ColumnMetadata.Raw> columnNames  = new ArrayList<>();
         List<Term.Raw> values = new ArrayList<>();
         boolean ifNotExists = false;
     }
@@ -513,7 +513,7 @@ usingClauseObjective[Attributes.Raw attrs]
 updateStatement returns [UpdateStatement.ParsedUpdate expr]
     @init {
         Attributes.Raw attrs = new Attributes.Raw();
-        List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations = new ArrayList<>();
+        List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> operations = new ArrayList<>();
         boolean ifExists = false;
     }
     : K_UPDATE cf=columnFamilyName
@@ -526,13 +526,13 @@ updateStatement returns [UpdateStatement.ParsedUpdate expr]
                                                    attrs,
                                                    operations,
                                                    wclause.build(),
-                                                   conditions == null ? Collections.<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>emptyList() : conditions,
+                                                   conditions == null ? Collections.<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>>emptyList() : conditions,
                                                    ifExists);
      }
     ;
 
-updateConditions returns [List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions]
-    @init { conditions = new ArrayList<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>(); }
+updateConditions returns [List<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>> conditions]
+    @init { conditions = new ArrayList<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>>(); }
     : columnCondition[conditions] ( K_AND columnCondition[conditions] )*
     ;
 
@@ -560,7 +560,7 @@ deleteStatement returns [DeleteStatement.Parsed expr]
                                              attrs,
                                              columnDeletions,
                                              wclause.build(),
-                                             conditions == null ? Collections.<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>emptyList() : conditions,
+                                             conditions == null ? Collections.<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>>emptyList() : conditions,
                                              ifExists);
       }
     ;
@@ -837,8 +837,8 @@ indexIdent [List<IndexTarget.Raw> targets]
 createMaterializedViewStatement returns [CreateViewStatement expr]
     @init {
         boolean ifNotExists = false;
-        List<ColumnDefinition.Raw> partitionKeys = new ArrayList<>();
-        List<ColumnDefinition.Raw> compositeKeys = new ArrayList<>();
+        List<ColumnMetadata.Raw> partitionKeys = new ArrayList<>();
+        List<ColumnMetadata.Raw> compositeKeys = new ArrayList<>();
     }
     : K_CREATE K_MATERIALIZED K_VIEW (K_IF K_NOT K_EXISTS { ifNotExists = true; })? cf=columnFamilyName K_AS
         K_SELECT sclause=selectors K_FROM basecf=columnFamilyName
@@ -895,7 +895,7 @@ alterTableStatement returns [AlterTableStatement expr]
     @init {
         AlterTableStatement.Type type = null;
         TableAttributes attrs = new TableAttributes();
-        Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames = new HashMap<ColumnDefinition.Raw, ColumnDefinition.Raw>();
+        Map<ColumnMetadata.Raw, ColumnMetadata.Raw> renames = new HashMap<ColumnMetadata.Raw, ColumnMetadata.Raw>();
         List<AlterTableStatementColumn> colNameList = new ArrayList<AlterTableStatementColumn>();
         Long deleteTimestamp = null;
     }
@@ -1268,10 +1268,10 @@ userPassword[RoleOptions opts]
 // Column Identifiers.  These need to be treated differently from other
 // identifiers because the underlying comparator is not necessarily text. See
 // CASSANDRA-8178 for details.
-cident returns [ColumnDefinition.Raw id]
-    : t=IDENT              { $id = ColumnDefinition.Raw.forUnquoted($t.text); }
-    | t=QUOTED_NAME        { $id = ColumnDefinition.Raw.forQuoted($t.text); }
-    | k=unreserved_keyword { $id = ColumnDefinition.Raw.forUnquoted(k); }
+cident returns [ColumnMetadata.Raw id]
+    : t=IDENT              { $id = ColumnMetadata.Raw.forUnquoted($t.text); }
+    | t=QUOTED_NAME        { $id = ColumnMetadata.Raw.forQuoted($t.text); }
+    | k=unreserved_keyword { $id = ColumnMetadata.Raw.forUnquoted(k); }
     ;
 
 // Column identifiers where the comparator is known to be text
@@ -1480,18 +1480,18 @@ simpleTerm returns [Term.Raw term]
     | '(' c=comparatorType ')' t=simpleTerm   { $term = new TypeCast(c, t); }
     ;
 
-columnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations]
+columnOperation[List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> operations]
     : key=cident columnOperationDifferentiator[operations, key]
     ;
 
-columnOperationDifferentiator[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
+columnOperationDifferentiator[List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> operations, ColumnMetadata.Raw key]
     : '=' normalColumnOperation[operations, key]
     | shorthandColumnOperation[operations, key]
     | '[' k=term ']' collectionColumnOperation[operations, key, k]
     | '.' field=fident udtColumnOperation[operations, key, field]
     ;
 
-normalColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
+normalColumnOperation[List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> operations, ColumnMetadata.Raw key]
     : t=term ('+' c=cident )?
       {
           if (c == null)
@@ -1521,28 +1521,28 @@ normalColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> oper
       }
     ;
 
-shorthandColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key]
+shorthandColumnOperation[List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> operations, ColumnMetadata.Raw key]
     : sig=('+=' | '-=') t=term
       {
           addRawUpdate(operations, key, $sig.text.equals("+=") ? new Operation.Addition(t) : new Operation.Substraction(t));
       }
     ;
 
-collectionColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, Term.Raw k]
+collectionColumnOperation[List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> operations, ColumnMetadata.Raw key, Term.Raw k]
     : '=' t=term
       {
           addRawUpdate(operations, key, new Operation.SetElement(k, t));
       }
     ;
 
-udtColumnOperation[List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> operations, ColumnDefinition.Raw key, FieldIdentifier field]
+udtColumnOperation[List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> operations, ColumnMetadata.Raw key, FieldIdentifier field]
     : '=' t=term
       {
           addRawUpdate(operations, key, new Operation.SetField(field, t));
       }
     ;
 
-columnCondition[List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions]
+columnCondition[List<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>> conditions]
     // Note: we'll reject duplicates later
     : key=cident
         ( op=relationType t=term { conditions.add(Pair.create(key, ColumnCondition.Raw.simpleCondition(t, op))); }
@@ -1634,8 +1634,8 @@ inMarker returns [AbstractMarker.INRaw marker]
     | ':' name=noncol_ident { $marker = newINBindVariables(name); }
     ;
 
-tupleOfIdentifiers returns [List<ColumnDefinition.Raw> ids]
-    @init { $ids = new ArrayList<ColumnDefinition.Raw>(); }
+tupleOfIdentifiers returns [List<ColumnMetadata.Raw> ids]
+    @init { $ids = new ArrayList<ColumnMetadata.Raw>(); }
     : '(' n1=cident { $ids.add(n1); } (',' ni=cident { $ids.add(ni); })* ')'
     ;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/auth/AuthKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthKeyspace.java b/src/java/org/apache/cassandra/auth/AuthKeyspace.java
index a5add9a..a7079dc 100644
--- a/src/java/org/apache/cassandra/auth/AuthKeyspace.java
+++ b/src/java/org/apache/cassandra/auth/AuthKeyspace.java
@@ -19,12 +19,16 @@ package org.apache.cassandra.auth;
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.Tables;
 
+import static java.lang.String.format;
+
 public final class AuthKeyspace
 {
     private AuthKeyspace()
@@ -38,52 +42,57 @@ public final class AuthKeyspace
 
     public static final long SUPERUSER_SETUP_DELAY = Long.getLong("cassandra.superuser_setup_delay_ms", 10000);
 
-    private static final CFMetaData Roles =
-        compile(ROLES,
-                "role definitions",
-                "CREATE TABLE %s ("
-                + "role text,"
-                + "is_superuser boolean,"
-                + "can_login boolean,"
-                + "salted_hash text,"
-                + "member_of set<text>,"
-                + "PRIMARY KEY(role))");
+    private static final TableMetadata Roles =
+        parse(ROLES,
+              "role definitions",
+              "CREATE TABLE %s ("
+              + "role text,"
+              + "is_superuser boolean,"
+              + "can_login boolean,"
+              + "salted_hash text,"
+              + "member_of set<text>,"
+              + "PRIMARY KEY(role))");
 
-    private static final CFMetaData RoleMembers =
-        compile(ROLE_MEMBERS,
-                "role memberships lookup table",
-                "CREATE TABLE %s ("
-                + "role text,"
-                + "member text,"
-                + "PRIMARY KEY(role, member))");
+    private static final TableMetadata RoleMembers =
+        parse(ROLE_MEMBERS,
+              "role memberships lookup table",
+              "CREATE TABLE %s ("
+              + "role text,"
+              + "member text,"
+              + "PRIMARY KEY(role, member))");
 
-    private static final CFMetaData RolePermissions =
-        compile(ROLE_PERMISSIONS,
-                "permissions granted to db roles",
-                "CREATE TABLE %s ("
-                + "role text,"
-                + "resource text,"
-                + "permissions set<text>,"
-                + "PRIMARY KEY(role, resource))");
+    private static final TableMetadata RolePermissions =
+        parse(ROLE_PERMISSIONS,
+              "permissions granted to db roles",
+              "CREATE TABLE %s ("
+              + "role text,"
+              + "resource text,"
+              + "permissions set<text>,"
+              + "PRIMARY KEY(role, resource))");
 
-    private static final CFMetaData ResourceRoleIndex =
-        compile(RESOURCE_ROLE_INDEX,
-                "index of db roles with permissions granted on a resource",
-                "CREATE TABLE %s ("
-                + "resource text,"
-                + "role text,"
-                + "PRIMARY KEY(resource, role))");
+    private static final TableMetadata ResourceRoleIndex =
+        parse(RESOURCE_ROLE_INDEX,
+              "index of db roles with permissions granted on a resource",
+              "CREATE TABLE %s ("
+              + "resource text,"
+              + "role text,"
+              + "PRIMARY KEY(resource, role))");
 
 
-    private static CFMetaData compile(String name, String description, String schema)
+    private static TableMetadata parse(String name, String description, String cql)
     {
-        return CFMetaData.compile(String.format(schema, name), SchemaConstants.AUTH_KEYSPACE_NAME)
-                         .comment(description)
-                         .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(90));
+        return CreateTableStatement.parse(format(cql, name), SchemaConstants.AUTH_KEYSPACE_NAME)
+                                   .id(TableId.forSystemTable(SchemaConstants.AUTH_KEYSPACE_NAME, name))
+                                   .comment(description)
+                                   .dcLocalReadRepairChance(0.0)
+                                   .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(90))
+                                   .build();
     }
 
     public static KeyspaceMetadata metadata()
     {
-        return KeyspaceMetadata.create(SchemaConstants.AUTH_KEYSPACE_NAME, KeyspaceParams.simple(1), Tables.of(Roles, RoleMembers, RolePermissions, ResourceRoleIndex));
+        return KeyspaceMetadata.create(SchemaConstants.AUTH_KEYSPACE_NAME,
+                                       KeyspaceParams.simple(1),
+                                       Tables.of(Roles, RoleMembers, RolePermissions, ResourceRoleIndex));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/auth/AuthMigrationListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthMigrationListener.java b/src/java/org/apache/cassandra/auth/AuthMigrationListener.java
deleted file mode 100644
index 64fe7c6..0000000
--- a/src/java/org/apache/cassandra/auth/AuthMigrationListener.java
+++ /dev/null
@@ -1,53 +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.auth;
-
-import java.util.List;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.service.MigrationListener;
-
-/**
- * MigrationListener implementation that cleans up permissions on dropped resources.
- */
-public class AuthMigrationListener extends MigrationListener
-{
-    public void onDropKeyspace(String ksName)
-    {
-        DatabaseDescriptor.getAuthorizer().revokeAllOn(DataResource.keyspace(ksName));
-        DatabaseDescriptor.getAuthorizer().revokeAllOn(FunctionResource.keyspace(ksName));
-    }
-
-    public void onDropColumnFamily(String ksName, String cfName)
-    {
-        DatabaseDescriptor.getAuthorizer().revokeAllOn(DataResource.table(ksName, cfName));
-    }
-
-    public void onDropFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
-    {
-        DatabaseDescriptor.getAuthorizer()
-                          .revokeAllOn(FunctionResource.function(ksName, functionName, argTypes));
-    }
-
-    public void onDropAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
-    {
-        DatabaseDescriptor.getAuthorizer()
-                          .revokeAllOn(FunctionResource.function(ksName, aggregateName, argTypes));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/auth/AuthSchemaChangeListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/AuthSchemaChangeListener.java b/src/java/org/apache/cassandra/auth/AuthSchemaChangeListener.java
new file mode 100644
index 0000000..6c21d7b
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/AuthSchemaChangeListener.java
@@ -0,0 +1,53 @@
+/*
+ * 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.auth;
+
+import java.util.List;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.SchemaChangeListener;
+
+/**
+ * SchemaChangeListener implementation that cleans up permissions on dropped resources.
+ */
+public class AuthSchemaChangeListener extends SchemaChangeListener
+{
+    public void onDropKeyspace(String ksName)
+    {
+        DatabaseDescriptor.getAuthorizer().revokeAllOn(DataResource.keyspace(ksName));
+        DatabaseDescriptor.getAuthorizer().revokeAllOn(FunctionResource.keyspace(ksName));
+    }
+
+    public void onDropTable(String ksName, String cfName)
+    {
+        DatabaseDescriptor.getAuthorizer().revokeAllOn(DataResource.table(ksName, cfName));
+    }
+
+    public void onDropFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+    {
+        DatabaseDescriptor.getAuthorizer()
+                          .revokeAllOn(FunctionResource.function(ksName, functionName, argTypes));
+    }
+
+    public void onDropAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+    {
+        DatabaseDescriptor.getAuthorizer()
+                          .revokeAllOn(FunctionResource.function(ksName, aggregateName, argTypes));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
index 7f44eef..045a38a 100644
--- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
@@ -30,8 +30,8 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.ModificationStatement;
@@ -218,7 +218,7 @@ public class CassandraAuthorizer implements IAuthorizer
         SelectStatement statement;
         // If it exists, read from the legacy user permissions table to handle the case where the cluster
         // is being upgraded and so is running with mixed versions of the authz schema
-        if (Schema.instance.getCFMetaData(SchemaConstants.AUTH_KEYSPACE_NAME, USER_PERMISSIONS) == null)
+        if (Schema.instance.getTableMetadata(SchemaConstants.AUTH_KEYSPACE_NAME, USER_PERMISSIONS) == null)
             statement = authorizeRoleStatement;
         else
         {
@@ -305,7 +305,7 @@ public class CassandraAuthorizer implements IAuthorizer
         Set<PermissionDetails> details = new HashSet<>();
         // If it exists, try the legacy user permissions table first. This is to handle the case
         // where the cluster is being upgraded and so is running with mixed versions of the perms table
-        boolean useLegacyTable = Schema.instance.getCFMetaData(SchemaConstants.AUTH_KEYSPACE_NAME, USER_PERMISSIONS) != null;
+        boolean useLegacyTable = Schema.instance.getTableMetadata(SchemaConstants.AUTH_KEYSPACE_NAME, USER_PERMISSIONS) != null;
         String entityColumnName = useLegacyTable ? USERNAME : ROLE;
         for (UntypedResultSet.Row row : process(buildListQuery(resource, role, useLegacyTable)))
         {
@@ -370,7 +370,7 @@ public class CassandraAuthorizer implements IAuthorizer
 
         // If old user permissions table exists, migrate the legacy authz data to the new table
         // The delay is to give the node a chance to see its peers before attempting the conversion
-        if (Schema.instance.getCFMetaData(SchemaConstants.AUTH_KEYSPACE_NAME, "permissions") != null)
+        if (Schema.instance.getTableMetadata(SchemaConstants.AUTH_KEYSPACE_NAME, "permissions") != null)
         {
             legacyAuthorizeRoleStatement = prepare(USERNAME, USER_PERMISSIONS);
 
@@ -404,7 +404,7 @@ public class CassandraAuthorizer implements IAuthorizer
     {
         try
         {
-            if (Schema.instance.getCFMetaData("system_auth", "permissions") != null)
+            if (Schema.instance.getTableMetadata("system_auth", "permissions") != null)
             {
                 logger.info("Converting legacy permissions data");
                 CQLStatement insertStatement =

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
index d371df3..bd1d176 100644
--- a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -32,8 +32,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.ConsistencyLevel;
@@ -162,7 +162,7 @@ public class CassandraRoleManager implements IRoleManager
         // it, so we can continue to use the old tables while the cluster is upgraded.
         // Otherwise, we may need to create a default superuser role to enable others
         // to be added.
-        if (Schema.instance.getCFMetaData(SchemaConstants.AUTH_KEYSPACE_NAME, "users") != null)
+        if (Schema.instance.getTableMetadata(SchemaConstants.AUTH_KEYSPACE_NAME, "users") != null)
         {
             legacySelectUserStatement = prepareLegacySelectUserStatement();
 
@@ -399,7 +399,7 @@ public class CassandraRoleManager implements IRoleManager
         try
         {
             // read old data at QUORUM as it may contain the data for the default superuser
-            if (Schema.instance.getCFMetaData("system_auth", "users") != null)
+            if (Schema.instance.getTableMetadata("system_auth", "users") != null)
             {
                 logger.info("Converting legacy users");
                 UntypedResultSet users = QueryProcessor.process("SELECT * FROM system_auth.users",
@@ -414,7 +414,7 @@ public class CassandraRoleManager implements IRoleManager
                 logger.info("Completed conversion of legacy users");
             }
 
-            if (Schema.instance.getCFMetaData("system_auth", "credentials") != null)
+            if (Schema.instance.getTableMetadata("system_auth", "credentials") != null)
             {
                 logger.info("Migrating legacy credentials data to new system table");
                 UntypedResultSet credentials = QueryProcessor.process("SELECT * FROM system_auth.credentials",
@@ -489,7 +489,7 @@ public class CassandraRoleManager implements IRoleManager
             // If it exists, try the legacy users table in case the cluster
             // is in the process of being upgraded and so is running with mixed
             // versions of the authn schema.
-            if (Schema.instance.getCFMetaData(SchemaConstants.AUTH_KEYSPACE_NAME, "users") == null)
+            if (Schema.instance.getTableMetadata(SchemaConstants.AUTH_KEYSPACE_NAME, "users") == null)
                 return getRoleFromTable(name, loadRoleStatement, ROW_TO_ROLE);
             else
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/auth/DataResource.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/DataResource.java b/src/java/org/apache/cassandra/auth/DataResource.java
index 0aa24db..c3f5b32 100644
--- a/src/java/org/apache/cassandra/auth/DataResource.java
+++ b/src/java/org/apache/cassandra/auth/DataResource.java
@@ -23,7 +23,7 @@ import com.google.common.base.Objects;
 import com.google.common.collect.Sets;
 import org.apache.commons.lang3.StringUtils;
 
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 
 /**
  * The primary type of resource in Cassandra.
@@ -211,7 +211,7 @@ public class DataResource implements IResource
             case KEYSPACE:
                 return Schema.instance.getKeyspaces().contains(keyspace);
             case TABLE:
-                return Schema.instance.getCFMetaData(keyspace, table) != null;
+                return Schema.instance.getTableMetadata(keyspace, table) != null;
         }
         throw new AssertionError();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/auth/FunctionResource.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/FunctionResource.java b/src/java/org/apache/cassandra/auth/FunctionResource.java
index 01a4de5..f9f51d5 100644
--- a/src/java/org/apache/cassandra/auth/FunctionResource.java
+++ b/src/java/org/apache/cassandra/auth/FunctionResource.java
@@ -28,7 +28,7 @@ import com.google.common.base.Splitter;
 import com.google.common.collect.Sets;
 import org.apache.commons.lang3.StringUtils;
 
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.functions.FunctionName;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
index b1b0536..74ef851 100644
--- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
@@ -30,8 +30,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -133,7 +133,7 @@ public class PasswordAuthenticator implements IAuthenticator
      */
     private SelectStatement authenticationStatement()
     {
-        if (Schema.instance.getCFMetaData(SchemaConstants.AUTH_KEYSPACE_NAME, LEGACY_CREDENTIALS_TABLE) == null)
+        if (Schema.instance.getTableMetadata(SchemaConstants.AUTH_KEYSPACE_NAME, LEGACY_CREDENTIALS_TABLE) == null)
             return authenticateStatement;
         else
         {
@@ -164,7 +164,7 @@ public class PasswordAuthenticator implements IAuthenticator
                                      AuthKeyspace.ROLES);
         authenticateStatement = prepare(query);
 
-        if (Schema.instance.getCFMetaData(SchemaConstants.AUTH_KEYSPACE_NAME, LEGACY_CREDENTIALS_TABLE) != null)
+        if (Schema.instance.getTableMetadata(SchemaConstants.AUTH_KEYSPACE_NAME, LEGACY_CREDENTIALS_TABLE) != null)
             prepareLegacyAuthenticateStatement();
 
         cache = new CredentialsCache(this);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
index d23103c..321fca6 100644
--- a/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/batchlog/BatchlogManager.java
@@ -34,7 +34,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.BytesType;
@@ -51,6 +51,7 @@ import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.WriteResponseHandler;
 import org.apache.cassandra.utils.FBUtilities;
@@ -364,9 +365,9 @@ public class BatchlogManager implements BatchlogManagerMBean
         // truncated.
         private void addMutation(Mutation mutation)
         {
-            for (UUID cfId : mutation.getColumnFamilyIds())
-                if (writtenAt <= SystemKeyspace.getTruncatedAt(cfId))
-                    mutation = mutation.without(cfId);
+            for (TableId tableId : mutation.getTableIds())
+                if (writtenAt <= SystemKeyspace.getTruncatedAt(tableId))
+                    mutation = mutation.without(tableId);
 
             if (!mutation.isEmpty())
                 mutations.add(mutation);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cache/AutoSavingCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
index b7616b9..1f06266 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -34,10 +34,11 @@ import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 
 import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
@@ -79,8 +80,10 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
      * Sticking with "d" is fine for 3.0 since it has never been released or used by another version
      *
      * "e" introduced with CASSANDRA-11206, omits IndexInfo from key-cache, stores offset into index-file
+     *
+     * "f" introduced with CASSANDRA-9425, changes "keyspace.table.index" in cache keys to TableMetadata.id+TableMetadata.indexName
      */
-    private static final String CURRENT_VERSION = "e";
+    private static final String CURRENT_VERSION = "f";
 
     private static volatile IStreamFactory streamFactory = new IStreamFactory()
     {
@@ -208,13 +211,17 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
                 ArrayDeque<Future<Pair<K, V>>> futures = new ArrayDeque<Future<Pair<K, V>>>();
                 while (in.available() > 0)
                 {
-                    //ksname and cfname are serialized by the serializers in CacheService
+                    //tableId and indexName are serialized by the serializers in CacheService
                     //That is delegated there because there are serializer specific conditions
                     //where a cache key is skipped and not written
-                    String ksname = in.readUTF();
-                    String cfname = in.readUTF();
+                    TableId tableId = TableId.deserialize(in);
+                    String indexName = in.readUTF();
+                    if (indexName.isEmpty())
+                        indexName = null;
 
-                    ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreIncludingIndexes(Pair.create(ksname, cfname));
+                    ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(tableId);
+                    if (indexName != null && cfs != null)
+                        cfs = cfs.indexManager.getIndexByName(indexName).getBackingTable().orElse(null);
 
                     Future<Pair<K, V>> entryFuture = cacheLoader.deserialize(in, cfs);
                     // Key cache entry can return null, if the SSTable doesn't exist.
@@ -308,7 +315,7 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
             else
                 type = OperationType.UNKNOWN;
 
-            info = new CompactionInfo(CFMetaData.createFake(SchemaConstants.SYSTEM_KEYSPACE_NAME, cacheType.toString()),
+            info = new CompactionInfo(TableMetadata.minimal(SchemaConstants.SYSTEM_KEYSPACE_NAME, cacheType.toString()),
                                       type,
                                       0,
                                       keysEstimate,
@@ -359,9 +366,11 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
                 {
                     K key = keyIterator.next();
 
-                    ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreIncludingIndexes(key.ksAndCFName);
+                    ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(key.tableId);
                     if (cfs == null)
                         continue; // the table or 2i has been dropped.
+                    if (key.indexName != null)
+                        cfs = cfs.indexManager.getIndexByName(key.indexName).getBackingTable().orElse(null);
 
                     cacheLoader.serialize(key, writer, cfs);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cache/CacheKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/CacheKey.java b/src/java/org/apache/cassandra/cache/CacheKey.java
index 0e82990..ccab672 100644
--- a/src/java/org/apache/cassandra/cache/CacheKey.java
+++ b/src/java/org/apache/cassandra/cache/CacheKey.java
@@ -17,14 +17,30 @@
  */
 package org.apache.cassandra.cache;
 
-import org.apache.cassandra.utils.Pair;
+import java.util.Objects;
+
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 
 public abstract class CacheKey implements IMeasurableMemory
 {
-    public final Pair<String, String> ksAndCFName;
+    public final TableId tableId;
+    public final String indexName;
+
+    protected CacheKey(TableId tableId, String indexName)
+    {
+        this.tableId = tableId;
+        this.indexName = indexName;
+    }
+
+    public CacheKey(TableMetadata metadata)
+    {
+        this(metadata.id, metadata.indexName().orElse(null));
+    }
 
-    public CacheKey(Pair<String, String> ksAndCFName)
+    public boolean sameTable(TableMetadata tableMetadata)
     {
-        this.ksAndCFName = ksAndCFName;
+        return tableId.equals(tableMetadata.id)
+               && Objects.equals(indexName, tableMetadata.indexName().orElse(null));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cache/CounterCacheKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/CounterCacheKey.java b/src/java/org/apache/cassandra/cache/CounterCacheKey.java
index 67dd15f..2d123d4 100644
--- a/src/java/org/apache/cassandra/cache/CounterCacheKey.java
+++ b/src/java/org/apache/cassandra/cache/CounterCacheKey.java
@@ -20,47 +20,55 @@ package org.apache.cassandra.cache;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
 import org.apache.cassandra.db.filter.ClusteringIndexNamesFilter;
 import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.db.rows.RowIterator;
 import org.apache.cassandra.db.rows.UnfilteredRowIterators;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.utils.*;
 
 public final class CounterCacheKey extends CacheKey
 {
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new CounterCacheKey(null, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBuffer.allocate(1)));
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new CounterCacheKey(TableMetadata.builder("ks", "tab")
+                                                                                                .addPartitionKeyColumn("pk", UTF8Type.instance)
+                                                                                                .build(), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBuffer.allocate(1)));
 
     private final byte[] partitionKey;
     private final byte[] cellName;
 
-    private CounterCacheKey(Pair<String, String> ksAndCFName, byte[] partitionKey, byte[] cellName)
+    private CounterCacheKey(TableMetadata tableMetadata, byte[] partitionKey, byte[] cellName)
     {
-        super(ksAndCFName);
+        super(tableMetadata);
         this.partitionKey = partitionKey;
         this.cellName = cellName;
     }
 
-    private CounterCacheKey(Pair<String, String> ksAndCFName, ByteBuffer partitionKey, ByteBuffer cellName)
+    private CounterCacheKey(TableMetadata tableMetadata, ByteBuffer partitionKey, ByteBuffer cellName)
     {
-        this(ksAndCFName, ByteBufferUtil.getArray(partitionKey), ByteBufferUtil.getArray(cellName));
+        this(tableMetadata, ByteBufferUtil.getArray(partitionKey), ByteBufferUtil.getArray(cellName));
     }
 
-    public static CounterCacheKey create(Pair<String, String> ksAndCFName, ByteBuffer partitionKey, Clustering clustering, ColumnDefinition c, CellPath path)
+    public static CounterCacheKey create(TableMetadata tableMetadata, ByteBuffer partitionKey, Clustering clustering, ColumnMetadata c, CellPath path)
     {
-        return new CounterCacheKey(ksAndCFName, partitionKey, makeCellName(clustering, c, path));
+        return new CounterCacheKey(tableMetadata, partitionKey, makeCellName(clustering, c, path));
     }
 
-    private static ByteBuffer makeCellName(Clustering clustering, ColumnDefinition c, CellPath path)
+    private static ByteBuffer makeCellName(Clustering clustering, ColumnMetadata c, CellPath path)
     {
         int cs = clustering.size();
         ByteBuffer[] values = new ByteBuffer[cs + 1 + (path == null ? 0 : path.size())];
@@ -87,8 +95,8 @@ public final class CounterCacheKey extends CacheKey
      */
     public ByteBuffer readCounterValue(ColumnFamilyStore cfs)
     {
-        CFMetaData metadata = cfs.metadata;
-        assert metadata.ksAndCFName.equals(ksAndCFName);
+        TableMetadata metadata = cfs.metadata();
+        assert metadata.id.equals(tableId) && Objects.equals(metadata.indexName().orElse(null), indexName);
 
         DecoratedKey key = cfs.decorateKey(partitionKey());
 
@@ -97,7 +105,7 @@ public final class CounterCacheKey extends CacheKey
         assert buffers.size() >= clusteringSize + 1; // See makeCellName above
 
         Clustering clustering = Clustering.make(buffers.subList(0, clusteringSize).toArray(new ByteBuffer[clusteringSize]));
-        ColumnDefinition column = metadata.getColumnDefinition(buffers.get(clusteringSize));
+        ColumnMetadata column = metadata.getColumn(buffers.get(clusteringSize));
         // This can theoretically happen if a column is dropped after the cache is saved and we
         // try to load it. Not point if failing in any case, just skip the value.
         if (column == null)
@@ -134,10 +142,10 @@ public final class CounterCacheKey extends CacheKey
         ByteBufferUtil.writeWithLength(cellName, out);
     }
 
-    public static CounterCacheKey read(Pair<String, String> ksAndCFName, DataInputPlus in)
+    public static CounterCacheKey read(TableMetadata tableMetadata, DataInputPlus in)
     throws IOException
     {
-        return new CounterCacheKey(ksAndCFName,
+        return new CounterCacheKey(tableMetadata,
                                    ByteBufferUtil.readBytesWithLength(in),
                                    ByteBufferUtil.readBytesWithLength(in));
     }
@@ -152,8 +160,9 @@ public final class CounterCacheKey extends CacheKey
     @Override
     public String toString()
     {
-        return String.format("CounterCacheKey(%s, %s, %s)",
-                             ksAndCFName,
+        TableMetadataRef tableRef = Schema.instance.getTableMetadataRef(tableId);
+        return String.format("CounterCacheKey(%s, %s, %s, %s)",
+                             tableRef, indexName,
                              ByteBufferUtil.bytesToHex(ByteBuffer.wrap(partitionKey)),
                              ByteBufferUtil.bytesToHex(ByteBuffer.wrap(cellName)));
     }
@@ -161,7 +170,7 @@ public final class CounterCacheKey extends CacheKey
     @Override
     public int hashCode()
     {
-        return Arrays.deepHashCode(new Object[]{ksAndCFName, partitionKey, cellName});
+        return Arrays.deepHashCode(new Object[]{tableId, indexName, partitionKey, cellName});
     }
 
     @Override
@@ -175,7 +184,8 @@ public final class CounterCacheKey extends CacheKey
 
         CounterCacheKey cck = (CounterCacheKey) o;
 
-        return ksAndCFName.equals(cck.ksAndCFName)
+        return tableId.equals(cck.tableId)
+            && Objects.equals(indexName, cck.indexName)
             && Arrays.equals(partitionKey, cck.partitionKey)
             && Arrays.equals(cellName, cck.cellName);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cache/KeyCacheKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/KeyCacheKey.java b/src/java/org/apache/cassandra/cache/KeyCacheKey.java
index 222622c..3728cae 100644
--- a/src/java/org/apache/cassandra/cache/KeyCacheKey.java
+++ b/src/java/org/apache/cassandra/cache/KeyCacheKey.java
@@ -19,26 +19,31 @@ package org.apache.cassandra.cache;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.Objects;
+import java.util.UUID;
 
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
-import org.apache.cassandra.utils.Pair;
 
 public class KeyCacheKey extends CacheKey
 {
     public final Descriptor desc;
 
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new KeyCacheKey(null, null, ByteBufferUtil.EMPTY_BYTE_BUFFER));
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new KeyCacheKey(TableMetadata.builder("ks", "tab")
+                                                                                            .addPartitionKeyColumn("pk", UTF8Type.instance)
+                                                                                            .build(), null, ByteBufferUtil.EMPTY_BYTE_BUFFER));
 
     // keeping an array instead of a ByteBuffer lowers the overhead of the key cache working set,
     // without extra copies on lookup since client-provided key ByteBuffers will be array-backed already
     public final byte[] key;
 
-    public KeyCacheKey(Pair<String, String> ksAndCFName, Descriptor desc, ByteBuffer key)
+    public KeyCacheKey(TableMetadata tableMetadata, Descriptor desc, ByteBuffer key)
     {
-
-        super(ksAndCFName);
+        super(tableMetadata);
         this.desc = desc;
         this.key = ByteBufferUtil.getArray(key);
         assert this.key != null;
@@ -62,13 +67,17 @@ public class KeyCacheKey extends CacheKey
 
         KeyCacheKey that = (KeyCacheKey) o;
 
-        return ksAndCFName.equals(that.ksAndCFName) && desc.equals(that.desc) && Arrays.equals(key, that.key);
+        return tableId.equals(that.tableId)
+               && Objects.equals(indexName, that.indexName)
+               && desc.equals(that.desc)
+               && Arrays.equals(key, that.key);
     }
 
     @Override
     public int hashCode()
     {
-        int result = ksAndCFName.hashCode();
+        int result = tableId.hashCode();
+        result = 31 * result + Objects.hashCode(indexName);
         result = 31 * result + desc.hashCode();
         result = 31 * result + Arrays.hashCode(key);
         return result;


[29/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ClusteringPrefix.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ClusteringPrefix.java b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
index 340e237..0c67b82 100644
--- a/src/java/org/apache/cassandra/db/ClusteringPrefix.java
+++ b/src/java/org/apache/cassandra/db/ClusteringPrefix.java
@@ -24,10 +24,12 @@ import java.util.*;
 
 import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
@@ -235,8 +237,24 @@ public interface ClusteringPrefix extends IMeasurableMemory, Clusterable
      * @param metadata the metadata for the table the clustering prefix is of.
      * @return a human-readable string representation fo this prefix.
      */
-    public String toString(CFMetaData metadata);
+    public String toString(TableMetadata metadata);
 
+    /*
+     * TODO: we should stop using Clustering for partition keys. Maybe we can add
+     * a few methods to DecoratedKey so we don't have to (note that while using a Clustering
+     * allows to use buildBound(), it's actually used for partition keys only when every restriction
+     * is an equal, so we could easily create a specific method for keys for that.
+     */
+    default ByteBuffer serializeAsPartitionKey()
+    {
+        if (size() == 1)
+            return get(0);
+
+        ByteBuffer[] values = new ByteBuffer[size()];
+        for (int i = 0; i < size(); i++)
+            values[i] = get(i);
+        return CompositeType.build(values);
+    }
     /**
      * The values of this prefix as an array.
      * <p>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index f712935..4aa2d3e 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -21,6 +21,8 @@ import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.lang.management.ManagementFactory;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
 import java.util.*;
@@ -65,7 +67,6 @@ import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.sstable.format.*;
-import org.apache.cassandra.io.sstable.format.big.BigFormat;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.metrics.TableMetrics;
@@ -142,6 +143,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                                                                                           "internal");
 
     private static final ExecutorService [] perDiskflushExecutors = new ExecutorService[DatabaseDescriptor.getAllDataFileLocations().length];
+
     static
     {
         for (int i = 0; i < DatabaseDescriptor.getAllDataFileLocations().length; i++)
@@ -208,7 +210,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public final Keyspace keyspace;
     public final String name;
-    public final CFMetaData metadata;
+    public final TableMetadataRef metadata;
     private final String mbeanName;
     @Deprecated
     private final String oldMBeanName;
@@ -261,15 +263,15 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         // only update these runtime-modifiable settings if they have not been modified.
         if (!minCompactionThreshold.isModified())
             for (ColumnFamilyStore cfs : concatWithIndexes())
-                cfs.minCompactionThreshold = new DefaultValue(metadata.params.compaction.minCompactionThreshold());
+                cfs.minCompactionThreshold = new DefaultValue(metadata().params.compaction.minCompactionThreshold());
         if (!maxCompactionThreshold.isModified())
             for (ColumnFamilyStore cfs : concatWithIndexes())
-                cfs.maxCompactionThreshold = new DefaultValue(metadata.params.compaction.maxCompactionThreshold());
+                cfs.maxCompactionThreshold = new DefaultValue(metadata().params.compaction.maxCompactionThreshold());
         if (!crcCheckChance.isModified())
             for (ColumnFamilyStore cfs : concatWithIndexes())
-                cfs.crcCheckChance = new DefaultValue(metadata.params.crcCheckChance);
+                cfs.crcCheckChance = new DefaultValue(metadata().params.crcCheckChance);
 
-        compactionStrategyManager.maybeReload(metadata);
+        compactionStrategyManager.maybeReload(metadata());
         directories = compactionStrategyManager.getDirectories();
 
         scheduleFlush();
@@ -278,13 +280,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         // If the CF comparator has changed, we need to change the memtable,
         // because the old one still aliases the previous comparator.
-        if (data.getView().getCurrentMemtable().initialComparator != metadata.comparator)
+        if (data.getView().getCurrentMemtable().initialComparator != metadata().comparator)
             switchMemtable();
     }
 
     void scheduleFlush()
     {
-        int period = metadata.params.memtableFlushPeriodInMs;
+        int period = metadata().params.memtableFlushPeriodInMs;
         if (period > 0)
         {
             logger.trace("scheduling flush in {} ms", period);
@@ -329,9 +331,9 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         };
     }
 
-    public void setCompactionParametersJson(String options)
+    public Map<String, String> getCompactionParameters()
     {
-        setCompactionParameters(FBUtilities.fromJsonMap(options));
+        return compactionStrategyManager.getCompactionParams().asMap();
     }
 
     public String getCompactionParametersJson()
@@ -355,22 +357,28 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    public Map<String, String> getCompactionParameters()
+    public void setCompactionParametersJson(String options)
     {
-        return compactionStrategyManager.getCompactionParams().asMap();
+        setCompactionParameters(FBUtilities.fromJsonMap(options));
     }
 
     public Map<String,String> getCompressionParameters()
     {
-        return metadata.params.compression.asMap();
+        return metadata().params.compression.asMap();
+    }
+
+    public String getCompressionParametersJson()
+    {
+        return FBUtilities.json(getCompressionParameters());
     }
 
     public void setCompressionParameters(Map<String,String> opts)
     {
         try
         {
-            metadata.compression(CompressionParams.fromMap(opts));
-            metadata.params.compression.validate();
+            CompressionParams params = CompressionParams.fromMap(opts);
+            params.validate();
+            throw new UnsupportedOperationException(); // TODO FIXME CASSANDRA-12949
         }
         catch (ConfigurationException e)
         {
@@ -378,15 +386,20 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
+    public void setCompressionParametersJson(String options)
+    {
+        setCompressionParameters(FBUtilities.fromJsonMap(options));
+    }
+
     @VisibleForTesting
     public ColumnFamilyStore(Keyspace keyspace,
-                              String columnFamilyName,
-                              int generation,
-                              CFMetaData metadata,
-                              Directories directories,
-                              boolean loadSSTables,
-                              boolean registerBookeeping,
-                              boolean offline)
+                             String columnFamilyName,
+                             int generation,
+                             TableMetadataRef metadata,
+                             Directories directories,
+                             boolean loadSSTables,
+                             boolean registerBookeeping,
+                             boolean offline)
     {
         assert directories != null;
         assert metadata != null : "null metadata for " + keyspace + ":" + columnFamilyName;
@@ -394,11 +407,11 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         this.keyspace = keyspace;
         this.metadata = metadata;
         name = columnFamilyName;
-        minCompactionThreshold = new DefaultValue<>(metadata.params.compaction.minCompactionThreshold());
-        maxCompactionThreshold = new DefaultValue<>(metadata.params.compaction.maxCompactionThreshold());
-        crcCheckChance = new DefaultValue<>(metadata.params.crcCheckChance);
+        minCompactionThreshold = new DefaultValue<>(metadata.get().params.compaction.minCompactionThreshold());
+        maxCompactionThreshold = new DefaultValue<>(metadata.get().params.compaction.maxCompactionThreshold());
+        crcCheckChance = new DefaultValue<>(metadata.get().params.crcCheckChance);
         indexManager = new SecondaryIndexManager(this);
-        viewManager = keyspace.viewManager.forTable(metadata);
+        viewManager = keyspace.viewManager.forTable(metadata.id);
         metric = new TableMetrics(this);
         fileIndexGenerator.set(generation);
         sampleLatencyNanos = DatabaseDescriptor.getReadRpcTimeout() / 2;
@@ -426,7 +439,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         if (offline)
             this.directories = directories;
         else
-            this.directories = new Directories(metadata, Directories.dataDirectories);
+            this.directories = new Directories(metadata.get(), Directories.dataDirectories);
 
 
         // compaction strategy should be created after the CFS has been prepared
@@ -442,7 +455,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
 
         // create the private ColumnFamilyStores for the secondary column indexes
-        for (IndexMetadata info : metadata.getIndexes())
+        for (IndexMetadata info : metadata.get().indexes)
             indexManager.addIndex(info);
 
         if (registerBookeeping)
@@ -467,12 +480,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             {
                 throw new RuntimeException(e);
             }
-            logger.trace("retryPolicy for {} is {}", name, this.metadata.params.speculativeRetry);
+            logger.trace("retryPolicy for {} is {}", name, this.metadata.get().params.speculativeRetry);
             latencyCalculator = ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(new Runnable()
             {
                 public void run()
                 {
-                    SpeculativeRetryParam retryPolicy = ColumnFamilyStore.this.metadata.params.speculativeRetry;
+                    SpeculativeRetryParam retryPolicy = ColumnFamilyStore.this.metadata.get().params.speculativeRetry;
                     switch (retryPolicy.kind())
                     {
                         case PERCENTILE:
@@ -497,6 +510,11 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
+    public TableMetadata metadata()
+    {
+        return metadata.get();
+    }
+
     public Directories getDirectories()
     {
         return directories;
@@ -504,7 +522,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public SSTableMultiWriter createSSTableMultiWriter(Descriptor descriptor, long keyCount, long repairedAt, int sstableLevel, SerializationHeader header, LifecycleTransaction txn)
     {
-        MetadataCollector collector = new MetadataCollector(metadata.comparator).sstableLevel(sstableLevel);
+        MetadataCollector collector = new MetadataCollector(metadata().comparator).sstableLevel(sstableLevel);
         return createSSTableMultiWriter(descriptor, keyCount, repairedAt, collector, header, txn);
     }
 
@@ -545,7 +563,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         latencyCalculator.cancel(false);
         compactionStrategyManager.shutdown();
-        SystemKeyspace.removeTruncationRecord(metadata.cfId);
+        SystemKeyspace.removeTruncationRecord(metadata.id);
 
         data.dropSSTables();
         LifecycleTransaction.waitForDeletions();
@@ -578,24 +596,24 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     }
 
 
-    public static ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace, CFMetaData metadata, boolean loadSSTables)
+    public static ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace, TableMetadataRef metadata, boolean loadSSTables)
     {
-        return createColumnFamilyStore(keyspace, metadata.cfName, metadata, loadSSTables);
+        return createColumnFamilyStore(keyspace, metadata.name, metadata, loadSSTables);
     }
 
     public static synchronized ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace,
                                                                          String columnFamily,
-                                                                         CFMetaData metadata,
+                                                                         TableMetadataRef metadata,
                                                                          boolean loadSSTables)
     {
-        Directories directories = new Directories(metadata, initialDirectories);
+        Directories directories = new Directories(metadata.get(), initialDirectories);
         return createColumnFamilyStore(keyspace, columnFamily, metadata, directories, loadSSTables, true, false);
     }
 
     /** This is only directly used by offline tools */
     public static synchronized ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace,
                                                                          String columnFamily,
-                                                                         CFMetaData metadata,
+                                                                         TableMetadataRef metadata,
                                                                          Directories directories,
                                                                          boolean loadSSTables,
                                                                          boolean registerBookkeeping,
@@ -622,7 +640,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * Removes unnecessary files from the cf directory at startup: these include temp files, orphans, zero-length files
      * and compacted sstables. Files that cannot be recognized will be ignored.
      */
-    public static void  scrubDataDirectories(CFMetaData metadata) throws StartupException
+    public static void  scrubDataDirectories(TableMetadata metadata) throws StartupException
     {
         Directories directories = new Directories(metadata, initialDirectories);
         Set<File> cleanedDirectories = new HashSet<>();
@@ -632,15 +650,15 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         directories.removeTemporaryDirectories();
 
-        logger.trace("Removing temporary or obsoleted files from unfinished operations for table {}", metadata.cfName);
+        logger.trace("Removing temporary or obsoleted files from unfinished operations for table {}", metadata.name);
         if (!LifecycleTransaction.removeUnfinishedLeftovers(metadata))
             throw new StartupException(StartupException.ERR_WRONG_DISK_STATE,
-                                       String.format("Cannot remove temporary or obsoleted files for %s.%s due to a problem with transaction " +
+                                       String.format("Cannot remove temporary or obsoleted files for %s due to a problem with transaction " +
                                                      "log files. Please check records with problems in the log messages above and fix them. " +
                                                      "Refer to the 3.0 upgrading instructions in NEWS.txt " +
-                                                     "for a description of transaction log files.", metadata.ksName, metadata.cfName));
+                                                     "for a description of transaction log files.", metadata.toString()));
 
-        logger.trace("Further extra check for orphan sstable files for {}", metadata.cfName);
+        logger.trace("Further extra check for orphan sstable files for {}", metadata.name);
         for (Map.Entry<Descriptor,Set<Component>> sstableFiles : directories.sstableLister(Directories.OnTxnErr.IGNORE).list().entrySet())
         {
             Descriptor desc = sstableFiles.getKey();
@@ -670,7 +688,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
 
         // cleanup incomplete saved caches
-        Pattern tmpCacheFilePattern = Pattern.compile(metadata.ksName + "-" + metadata.cfName + "-(Key|Row)Cache.*\\.tmp$");
+        Pattern tmpCacheFilePattern = Pattern.compile(metadata.keyspace + "-" + metadata.name + "-(Key|Row)Cache.*\\.tmp$");
         File dir = new File(DatabaseDescriptor.getSavedCachesLocation());
 
         if (dir.exists())
@@ -683,10 +701,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
 
         // also clean out any index leftovers.
-        for (IndexMetadata index : metadata.getIndexes())
+        for (IndexMetadata index : metadata.indexes)
             if (!index.isCustom())
             {
-                CFMetaData indexMetadata = CassandraIndex.indexCfsMetadata(metadata, index);
+                TableMetadata indexMetadata = CassandraIndex.indexCfsMetadata(metadata, index);
                 scrubDataDirectories(indexMetadata);
             }
     }
@@ -790,7 +808,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public void rebuildSecondaryIndex(String idxName)
     {
-        rebuildSecondaryIndex(keyspace.getName(), metadata.cfName, idxName);
+        rebuildSecondaryIndex(keyspace.getName(), metadata.name, idxName);
     }
 
     public static void rebuildSecondaryIndex(String ksName, String cfName, String... idxNames)
@@ -807,6 +825,20 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
+    public AbstractCompactionStrategy createCompactionStrategyInstance(CompactionParams compactionParams)
+    {
+        try
+        {
+            Constructor<? extends AbstractCompactionStrategy> constructor =
+                compactionParams.klass().getConstructor(ColumnFamilyStore.class, Map.class);
+            return constructor.newInstance(this, compactionParams.options());
+        }
+        catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException | InstantiationException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
     @Deprecated
     public String getColumnFamilyName()
     {
@@ -994,7 +1026,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             {
                 Memtable memtable = memtables.get(0);
                 commitLogUpperBound = memtable.getCommitLogUpperBound();
-                CommitLog.instance.discardCompletedSegments(metadata.cfId, memtable.getCommitLogLowerBound(), commitLogUpperBound);
+                CommitLog.instance.discardCompletedSegments(metadata.id, memtable.getCommitLogLowerBound(), commitLogUpperBound);
             }
 
             metric.pendingFlushes.dec();
@@ -1314,7 +1346,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             DecoratedKey key = update.partitionKey();
             invalidateCachedPartition(key);
             metric.samplers.get(Sampler.WRITES).addSample(key.getKey(), key.hashCode(), 1);
-            StorageHook.instance.reportWrite(metadata.cfId, update);
+            StorageHook.instance.reportWrite(metadata.id, update);
             metric.writeLatency.addNano(System.nanoTime() - start);
             // CASSANDRA-11117 - certain resolution paths on memtable put can result in very
             // large time deltas, either through a variety of sentinel timestamps (used for empty values, ensuring
@@ -1607,7 +1639,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         // what we're caching. Wen doing that, we should be careful about expiring cells: we should count
         // something expired that wasn't when the partition was cached, or we could decide that the whole
         // partition is cached when it's not. This is why we use CachedPartition#cachedLiveRows.
-        if (cached.cachedLiveRows() < metadata.params.caching.rowsPerPartitionToCache())
+        if (cached.cachedLiveRows() < metadata().params.caching.rowsPerPartitionToCache())
             return true;
 
         // If the whole partition isn't cached, then we must guarantee that the filter cannot select data that
@@ -1619,7 +1651,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public int gcBefore(int nowInSec)
     {
-        return nowInSec - metadata.params.gcGraceSeconds;
+        return nowInSec - metadata().params.gcGraceSeconds;
     }
 
     @SuppressWarnings("resource")
@@ -1664,7 +1696,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public List<String> getSSTablesForKey(String key, boolean hexFormat)
     {
-        ByteBuffer keyBuffer = hexFormat ? ByteBufferUtil.hexToBytes(key) : metadata.getKeyValidator().fromString(key);
+        ByteBuffer keyBuffer = hexFormat ? ByteBufferUtil.hexToBytes(key) : metadata().partitionKeyType.fromString(key);
         DecoratedKey dk = decorateKey(keyBuffer);
         try (OpOrder.Group op = readOrdering.start())
         {
@@ -1699,7 +1731,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     ByteBufferUtil.bytesToHex(key), // raw
                     counter.getCount(),  // count
                     counter.getError(),  // error
-                    metadata.getKeyValidator().getString(key) })); // string
+                    metadata().partitionKeyType.getString(key) })); // string
         }
         return new CompositeDataSupport(SAMPLING_RESULT, SAMPLER_NAMES, new Object[]{
                 samplerResults.cardinality, result});
@@ -1724,18 +1756,18 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         {
             RowCacheKey key = keyIter.next();
             DecoratedKey dk = decorateKey(ByteBuffer.wrap(key.key));
-            if (key.ksAndCFName.equals(metadata.ksAndCFName) && !Range.isInRanges(dk.getToken(), ranges))
+            if (key.sameTable(metadata()) && !Range.isInRanges(dk.getToken(), ranges))
                 invalidateCachedPartition(dk);
         }
 
-        if (metadata.isCounter())
+        if (metadata().isCounter())
         {
             for (Iterator<CounterCacheKey> keyIter = CacheService.instance.counterCache.keyIterator();
                  keyIter.hasNext(); )
             {
                 CounterCacheKey key = keyIter.next();
                 DecoratedKey dk = decorateKey(key.partitionKey());
-                if (key.ksAndCFName.equals(metadata.ksAndCFName) && !Range.isInRanges(dk.getToken(), ranges))
+                if (key.sameTable(metadata()) && !Range.isInRanges(dk.getToken(), ranges))
                     CacheService.instance.counterCache.remove(key);
             }
         }
@@ -1743,7 +1775,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public ClusteringComparator getComparator()
     {
-        return metadata.comparator;
+        return metadata().comparator;
     }
 
     public void snapshotWithoutFlush(String snapshotName)
@@ -1774,7 +1806,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 }
 
                 writeSnapshotManifest(filesJSONArr, snapshotName);
-                if (!SchemaConstants.SYSTEM_KEYSPACE_NAMES.contains(metadata.ksName) && !SchemaConstants.REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(metadata.ksName))
+                if (!SchemaConstants.SYSTEM_KEYSPACE_NAMES.contains(metadata.keyspace) && !SchemaConstants.REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(metadata.keyspace))
                     writeSnapshotSchema(snapshotName);
             }
         }
@@ -1816,7 +1848,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
             try (PrintStream out = new PrintStream(schemaFile))
             {
-                for (String s: ColumnFamilyStoreCQLHelper.dumpReCreateStatements(metadata))
+                for (String s: ColumnFamilyStoreCQLHelper.dumpReCreateStatements(metadata()))
                     out.println(s);
             }
         }
@@ -1974,16 +2006,16 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         if (!isRowCacheEnabled())
             return null;
-        IRowCacheEntry cached = CacheService.instance.rowCache.getInternal(new RowCacheKey(metadata.ksAndCFName, key));
+        IRowCacheEntry cached = CacheService.instance.rowCache.getInternal(new RowCacheKey(metadata(), key));
         return cached == null || cached instanceof RowCacheSentinel ? null : (CachedPartition)cached;
     }
 
     private void invalidateCaches()
     {
-        CacheService.instance.invalidateKeyCacheForCf(metadata.ksAndCFName);
-        CacheService.instance.invalidateRowCacheForCf(metadata.ksAndCFName);
-        if (metadata.isCounter())
-            CacheService.instance.invalidateCounterCacheForCf(metadata.ksAndCFName);
+        CacheService.instance.invalidateKeyCacheForCf(metadata());
+        CacheService.instance.invalidateRowCacheForCf(metadata());
+        if (metadata().isCounter())
+            CacheService.instance.invalidateCounterCacheForCf(metadata());
     }
 
     public int invalidateRowCache(Collection<Bounds<Token>> boundsToInvalidate)
@@ -1994,7 +2026,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         {
             RowCacheKey key = keyIter.next();
             DecoratedKey dk = decorateKey(ByteBuffer.wrap(key.key));
-            if (key.ksAndCFName.equals(metadata.ksAndCFName) && Bounds.isInBounds(dk.getToken(), boundsToInvalidate))
+            if (key.sameTable(metadata()) && Bounds.isInBounds(dk.getToken(), boundsToInvalidate))
             {
                 invalidateCachedPartition(dk);
                 invalidatedKeys++;
@@ -2011,7 +2043,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         {
             CounterCacheKey key = keyIter.next();
             DecoratedKey dk = decorateKey(key.partitionKey());
-            if (key.ksAndCFName.equals(metadata.ksAndCFName) && Bounds.isInBounds(dk.getToken(), boundsToInvalidate))
+            if (key.sameTable(metadata()) && Bounds.isInBounds(dk.getToken(), boundsToInvalidate))
             {
                 CacheService.instance.counterCache.remove(key);
                 invalidatedKeys++;
@@ -2025,7 +2057,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     public boolean containsCachedParition(DecoratedKey key)
     {
-        return CacheService.instance.rowCache.getCapacity() != 0 && CacheService.instance.rowCache.containsKey(new RowCacheKey(metadata.ksAndCFName, key));
+        return CacheService.instance.rowCache.getCapacity() != 0 && CacheService.instance.rowCache.containsKey(new RowCacheKey(metadata(), key));
     }
 
     public void invalidateCachedPartition(RowCacheKey key)
@@ -2038,21 +2070,21 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         if (!isRowCacheEnabled())
             return;
 
-        invalidateCachedPartition(new RowCacheKey(metadata.ksAndCFName, key));
+        invalidateCachedPartition(new RowCacheKey(metadata(), key));
     }
 
-    public ClockAndCount getCachedCounter(ByteBuffer partitionKey, Clustering clustering, ColumnDefinition column, CellPath path)
+    public ClockAndCount getCachedCounter(ByteBuffer partitionKey, Clustering clustering, ColumnMetadata column, CellPath path)
     {
         if (CacheService.instance.counterCache.getCapacity() == 0L) // counter cache disabled.
             return null;
-        return CacheService.instance.counterCache.get(CounterCacheKey.create(metadata.ksAndCFName, partitionKey, clustering, column, path));
+        return CacheService.instance.counterCache.get(CounterCacheKey.create(metadata(), partitionKey, clustering, column, path));
     }
 
-    public void putCachedCounter(ByteBuffer partitionKey, Clustering clustering, ColumnDefinition column, CellPath path, ClockAndCount clockAndCount)
+    public void putCachedCounter(ByteBuffer partitionKey, Clustering clustering, ColumnMetadata column, CellPath path, ClockAndCount clockAndCount)
     {
         if (CacheService.instance.counterCache.getCapacity() == 0L) // counter cache disabled.
             return;
-        CacheService.instance.counterCache.put(CounterCacheKey.create(metadata.ksAndCFName, partitionKey, clustering, column, path), clockAndCount);
+        CacheService.instance.counterCache.put(CounterCacheKey.create(metadata(), partitionKey, clustering, column, path), clockAndCount);
     }
 
     public void forceMajorCompaction() throws InterruptedException, ExecutionException
@@ -2219,7 +2251,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         // and so we only run one major compaction at a time
         synchronized (this)
         {
-            logger.trace("Cancelling in-progress compactions for {}", metadata.cfName);
+            logger.trace("Cancelling in-progress compactions for {}", metadata.name);
 
             Iterable<ColumnFamilyStore> selfWithAuxiliaryCfs = interruptViews
                                                                ? Iterables.concat(concatWithIndexes(), viewManager.allViewsCfs())
@@ -2238,7 +2270,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 {
                     if (!cfs.getTracker().getCompacting().isEmpty())
                     {
-                        logger.warn("Unable to cancel in-progress compactions for {}.  Perhaps there is an unusually large row in progress somewhere, or the system is simply overloaded.", metadata.cfName);
+                        logger.warn("Unable to cancel in-progress compactions for {}.  Perhaps there is an unusually large row in progress somewhere, or the system is simply overloaded.", metadata.name);
                         return null;
                     }
                 }
@@ -2439,18 +2471,18 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public IPartitioner getPartitioner()
     {
-        return metadata.partitioner;
+        return metadata().partitioner;
     }
 
     public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return metadata.decorateKey(key);
+        return getPartitioner().decorateKey(key);
     }
 
     /** true if this CFS contains secondary index data */
     public boolean isIndex()
     {
-        return metadata.isIndex();
+        return metadata().isIndex();
     }
 
     public Iterable<ColumnFamilyStore> concatWithIndexes()
@@ -2521,19 +2553,19 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     public boolean isRowCacheEnabled()
     {
 
-        boolean retval = metadata.params.caching.cacheRows() && CacheService.instance.rowCache.getCapacity() > 0;
+        boolean retval = metadata().params.caching.cacheRows() && CacheService.instance.rowCache.getCapacity() > 0;
         assert(!retval || !isIndex());
         return retval;
     }
 
     public boolean isCounterCacheEnabled()
     {
-        return metadata.isCounter() && CacheService.instance.counterCache.getCapacity() > 0;
+        return metadata().isCounter() && CacheService.instance.counterCache.getCapacity() > 0;
     }
 
     public boolean isKeyCacheEnabled()
     {
-        return metadata.params.caching.cacheKeys() && CacheService.instance.keyCache.getCapacity() > 0;
+        return metadata().params.caching.cacheKeys() && CacheService.instance.keyCache.getCapacity() > 0;
     }
 
     /**
@@ -2568,7 +2600,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         for (SSTableReader sstable : getSSTables(SSTableSet.LIVE))
         {
-            allDroppable += sstable.getDroppableTombstonesBefore(localTime - sstable.metadata.params.gcGraceSeconds);
+            allDroppable += sstable.getDroppableTombstonesBefore(localTime - metadata().params.gcGraceSeconds);
             allColumns += sstable.getEstimatedColumnCount().mean() * sstable.getEstimatedColumnCount().count();
         }
         return allColumns > 0 ? allDroppable / allColumns : 0;
@@ -2586,20 +2618,22 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     }
 
     /**
-     * Returns a ColumnFamilyStore by cfId if it exists, null otherwise
+     * Returns a ColumnFamilyStore by id if it exists, null otherwise
      * Differently from others, this method does not throw exception if the table does not exist.
      */
-    public static ColumnFamilyStore getIfExists(UUID cfId)
+    public static ColumnFamilyStore getIfExists(TableId id)
     {
-        Pair<String, String> kscf = Schema.instance.getCF(cfId);
-        if (kscf == null)
+        TableMetadata metadata = Schema.instance.getTableMetadata(id);
+        if (metadata == null)
             return null;
 
-        Keyspace keyspace = Keyspace.open(kscf.left);
+        Keyspace keyspace = Keyspace.open(metadata.keyspace);
         if (keyspace == null)
             return null;
 
-        return keyspace.getColumnFamilyStore(cfId);
+        return keyspace.hasColumnFamilyStore(id)
+             ? keyspace.getColumnFamilyStore(id)
+             : null;
     }
 
     /**
@@ -2615,10 +2649,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         if (keyspace == null)
             return null;
 
-        UUID id = Schema.instance.getId(ksName, cfName);
-        if (id == null)
+        TableMetadata table = Schema.instance.getTableMetadata(ksName, cfName);
+        if (table == null)
             return null;
 
-        return keyspace.getColumnFamilyStore(id);
+        return keyspace.getColumnFamilyStore(table.id);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ColumnFamilyStoreCQLHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStoreCQLHelper.java b/src/java/org/apache/cassandra/db/ColumnFamilyStoreCQLHelper.java
index 54c8117..d79fa0f 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStoreCQLHelper.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStoreCQLHelper.java
@@ -26,25 +26,23 @@ import java.util.function.*;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.statements.*;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.schema.*;
 import org.apache.cassandra.utils.*;
 
 /**
- * Helper methods to represent CFMetadata and related objects in CQL format
+ * Helper methods to represent TableMetadata and related objects in CQL format
  */
 public class ColumnFamilyStoreCQLHelper
 {
-    public static List<String> dumpReCreateStatements(CFMetaData metadata)
+    public static List<String> dumpReCreateStatements(TableMetadata metadata)
     {
         List<String> l = new ArrayList<>();
         // Types come first, as table can't be created without them
         l.addAll(ColumnFamilyStoreCQLHelper.getUserTypesAsCQL(metadata));
         // Record re-create schema statements
-        l.add(ColumnFamilyStoreCQLHelper.getCFMetadataAsCQL(metadata, true));
+        l.add(ColumnFamilyStoreCQLHelper.getTableMetadataAsCQL(metadata, true));
         // Dropped columns (and re-additions)
         l.addAll(ColumnFamilyStoreCQLHelper.getDroppedColumnsAsCQL(metadata));
         // Indexes applied as last, since otherwise they may interfere with column drops / re-additions
@@ -52,35 +50,35 @@ public class ColumnFamilyStoreCQLHelper
         return l;
     }
 
-    private static List<ColumnDefinition> getClusteringColumns(CFMetaData metadata)
+    private static List<ColumnMetadata> getClusteringColumns(TableMetadata metadata)
     {
-        List<ColumnDefinition> cds = new ArrayList<>(metadata.clusteringColumns().size());
+        List<ColumnMetadata> cds = new ArrayList<>(metadata.clusteringColumns().size());
 
         if (!metadata.isStaticCompactTable())
-            for (ColumnDefinition cd : metadata.clusteringColumns())
+            for (ColumnMetadata cd : metadata.clusteringColumns())
                 cds.add(cd);
 
         return cds;
     }
 
-    private static List<ColumnDefinition> getPartitionColumns(CFMetaData metadata)
+    private static List<ColumnMetadata> getPartitionColumns(TableMetadata metadata)
     {
-        List<ColumnDefinition> cds = new ArrayList<>(metadata.partitionColumns().size());
+        List<ColumnMetadata> cds = new ArrayList<>(metadata.regularAndStaticColumns().size());
 
-        for (ColumnDefinition cd : metadata.partitionColumns().statics)
+        for (ColumnMetadata cd : metadata.staticColumns())
             cds.add(cd);
 
         if (metadata.isDense())
         {
             // remove an empty type
-            for (ColumnDefinition cd : metadata.partitionColumns().withoutStatics())
+            for (ColumnMetadata cd : metadata.regularColumns())
                 if (!cd.type.equals(EmptyType.instance))
                     cds.add(cd);
         }
         // "regular" columns are not exposed for static compact tables
         else if (!metadata.isStaticCompactTable())
         {
-            for (ColumnDefinition cd : metadata.partitionColumns().withoutStatics())
+            for (ColumnMetadata cd : metadata.regularColumns())
                 cds.add(cd);
         }
 
@@ -91,28 +89,27 @@ public class ColumnFamilyStoreCQLHelper
      * Build a CQL String representation of Column Family Metadata
      */
     @VisibleForTesting
-    public static String getCFMetadataAsCQL(CFMetaData metadata, boolean includeDroppedColumns)
+    public static String getTableMetadataAsCQL(TableMetadata metadata, boolean includeDroppedColumns)
     {
         StringBuilder sb = new StringBuilder();
         if (!isCqlCompatible(metadata))
         {
-            sb.append(String.format("/*\nWarning: Table %s.%s omitted because it has constructs not compatible with CQL (was created via legacy API).\n",
-                                    metadata.ksName,
-                                    metadata.cfName));
+            sb.append(String.format("/*\nWarning: Table %s omitted because it has constructs not compatible with CQL (was created via legacy API).\n",
+                                    metadata.toString()));
             sb.append("\nApproximate structure, for reference:");
             sb.append("\n(this should not be used to reproduce this schema)\n\n");
         }
 
         sb.append("CREATE TABLE IF NOT EXISTS ");
-        sb.append(quoteIdentifier(metadata.ksName)).append('.').append(quoteIdentifier(metadata.cfName)).append(" (");
+        sb.append(metadata.toString()).append(" (");
 
-        List<ColumnDefinition> partitionKeyColumns = metadata.partitionKeyColumns();
-        List<ColumnDefinition> clusteringColumns = getClusteringColumns(metadata);
-        List<ColumnDefinition> partitionColumns = getPartitionColumns(metadata);
+        List<ColumnMetadata> partitionKeyColumns = metadata.partitionKeyColumns();
+        List<ColumnMetadata> clusteringColumns = getClusteringColumns(metadata);
+        List<ColumnMetadata> partitionColumns = getPartitionColumns(metadata);
 
         Consumer<StringBuilder> cdCommaAppender = commaAppender("\n\t");
         sb.append("\n\t");
-        for (ColumnDefinition cfd: partitionKeyColumns)
+        for (ColumnMetadata cfd: partitionKeyColumns)
         {
             cdCommaAppender.accept(sb);
             sb.append(toCQL(cfd));
@@ -120,13 +117,13 @@ public class ColumnFamilyStoreCQLHelper
                 sb.append(" PRIMARY KEY");
         }
 
-        for (ColumnDefinition cfd: clusteringColumns)
+        for (ColumnMetadata cfd: clusteringColumns)
         {
             cdCommaAppender.accept(sb);
             sb.append(toCQL(cfd));
         }
 
-        for (ColumnDefinition cfd: partitionColumns)
+        for (ColumnMetadata cfd: partitionColumns)
         {
             cdCommaAppender.accept(sb);
             sb.append(toCQL(cfd, metadata.isStaticCompactTable()));
@@ -134,16 +131,16 @@ public class ColumnFamilyStoreCQLHelper
 
         if (includeDroppedColumns)
         {
-            for (Map.Entry<ByteBuffer, CFMetaData.DroppedColumn> entry: metadata.getDroppedColumns().entrySet())
+            for (Map.Entry<ByteBuffer, DroppedColumn> entry: metadata.droppedColumns.entrySet())
             {
-                if (metadata.getColumnDefinition(entry.getKey()) != null)
+                if (metadata.getColumn(entry.getKey()) != null)
                     continue;
 
-                CFMetaData.DroppedColumn droppedColumn = entry.getValue();
+                DroppedColumn droppedColumn = entry.getValue();
                 cdCommaAppender.accept(sb);
-                sb.append(quoteIdentifier(droppedColumn.name));
+                sb.append(droppedColumn.column.name.toCQLString());
                 sb.append(' ');
-                sb.append(droppedColumn.type.asCQL3Type().toString());
+                sb.append(droppedColumn.column.type.asCQL3Type().toString());
             }
         }
 
@@ -154,27 +151,27 @@ public class ColumnFamilyStoreCQLHelper
             {
                 sb.append("(");
                 Consumer<StringBuilder> pkCommaAppender = commaAppender(" ");
-                for (ColumnDefinition cfd : partitionKeyColumns)
+                for (ColumnMetadata cfd : partitionKeyColumns)
                 {
                     pkCommaAppender.accept(sb);
-                    sb.append(quoteIdentifier(cfd.name.toString()));
+                    sb.append(cfd.name.toCQLString());
                 }
                 sb.append(")");
             }
             else
             {
-                sb.append(quoteIdentifier(partitionKeyColumns.get(0).name.toString()));
+                sb.append(partitionKeyColumns.get(0).name.toCQLString());
             }
 
-            for (ColumnDefinition cfd : metadata.clusteringColumns())
-                sb.append(", ").append(quoteIdentifier(cfd.name.toString()));
+            for (ColumnMetadata cfd : metadata.clusteringColumns())
+                sb.append(", ").append(cfd.name.toCQLString());
 
             sb.append(')');
         }
         sb.append(")\n\t");
         sb.append("WITH ");
 
-        sb.append("ID = ").append(metadata.cfId).append("\n\tAND ");
+        sb.append("ID = ").append(metadata.id).append("\n\tAND ");
 
         if (metadata.isCompactTable())
             sb.append("COMPACT STORAGE\n\tAND ");
@@ -184,10 +181,10 @@ public class ColumnFamilyStoreCQLHelper
             sb.append("CLUSTERING ORDER BY (");
 
             Consumer<StringBuilder> cOrderCommaAppender = commaAppender(" ");
-            for (ColumnDefinition cd : clusteringColumns)
+            for (ColumnMetadata cd : clusteringColumns)
             {
                 cOrderCommaAppender.accept(sb);
-                sb.append(quoteIdentifier(cd.name.toString())).append(' ').append(cd.clusteringOrder().toString());
+                sb.append(cd.name.toCQLString()).append(' ').append(cd.clusteringOrder().toString());
             }
             sb.append(")\n\tAND ");
         }
@@ -209,11 +206,11 @@ public class ColumnFamilyStoreCQLHelper
      * to the outermost.
      */
     @VisibleForTesting
-    public static List<String> getUserTypesAsCQL(CFMetaData metadata)
+    public static List<String> getUserTypesAsCQL(TableMetadata metadata)
     {
         List<AbstractType> types = new ArrayList<>();
         Set<AbstractType> typeSet = new HashSet<>();
-        for (ColumnDefinition cd: Iterables.concat(metadata.partitionKeyColumns(), metadata.clusteringColumns(), metadata.partitionColumns()))
+        for (ColumnMetadata cd: Iterables.concat(metadata.partitionKeyColumns(), metadata.clusteringColumns(), metadata.regularAndStaticColumns()))
         {
             AbstractType type = cd.type;
             if (type.isUDT())
@@ -232,16 +229,16 @@ public class ColumnFamilyStoreCQLHelper
      * If the column was dropped once, but is now re-created `ADD` will be appended accordingly.
      */
     @VisibleForTesting
-    public static List<String> getDroppedColumnsAsCQL(CFMetaData metadata)
+    public static List<String> getDroppedColumnsAsCQL(TableMetadata metadata)
     {
         List<String> droppedColumns = new ArrayList<>();
 
-        for (Map.Entry<ByteBuffer, CFMetaData.DroppedColumn> entry: metadata.getDroppedColumns().entrySet())
+        for (Map.Entry<ByteBuffer, DroppedColumn> entry: metadata.droppedColumns.entrySet())
         {
-            CFMetaData.DroppedColumn column = entry.getValue();
-            droppedColumns.add(toCQLDrop(metadata.ksName, metadata.cfName, column));
-            if (metadata.getColumnDefinition(entry.getKey()) != null)
-                droppedColumns.add(toCQLAdd(metadata.ksName, metadata.cfName, metadata.getColumnDefinition(entry.getKey())));
+            DroppedColumn column = entry.getValue();
+            droppedColumns.add(toCQLDrop(metadata, column));
+            if (metadata.getColumn(entry.getKey()) != null)
+                droppedColumns.add(toCQLAdd(metadata, metadata.getColumn(entry.getKey())));
         }
 
         return droppedColumns;
@@ -251,15 +248,15 @@ public class ColumnFamilyStoreCQLHelper
      * Build a CQL String representation of Indexes on columns in the given Column Family
      */
     @VisibleForTesting
-    public static List<String> getIndexesAsCQL(CFMetaData metadata)
+    public static List<String> getIndexesAsCQL(TableMetadata metadata)
     {
         List<String> indexes = new ArrayList<>();
-        for (IndexMetadata indexMetadata: metadata.getIndexes())
-            indexes.add(toCQL(metadata.ksName, metadata.cfName, indexMetadata));
+        for (IndexMetadata indexMetadata: metadata.indexes)
+            indexes.add(toCQL(metadata, indexMetadata));
         return indexes;
     }
 
-    private static String toCQL(String keyspace, String cf, IndexMetadata indexMetadata)
+    private static String toCQL(TableMetadata baseTable, IndexMetadata indexMetadata)
     {
         if (indexMetadata.isCustom())
         {
@@ -269,29 +266,25 @@ public class ColumnFamilyStoreCQLHelper
                     options.put(k, v);
             });
 
-            return String.format("CREATE CUSTOM INDEX %s ON %s.%s (%s) USING '%s'%s;",
-                                 quoteIdentifier(indexMetadata.name),
-                                 quoteIdentifier(keyspace),
-                                 quoteIdentifier(cf),
+            return String.format("CREATE CUSTOM INDEX %s ON %s (%s) USING '%s'%s;",
+                                 indexMetadata.toCQLString(),
+                                 baseTable.toString(),
                                  indexMetadata.options.get(IndexTarget.TARGET_OPTION_NAME),
                                  indexMetadata.options.get(IndexTarget.CUSTOM_INDEX_OPTION_NAME),
                                  options.isEmpty() ? "" : " WITH OPTIONS " + toCQL(options));
         }
         else
         {
-            return String.format("CREATE INDEX %s ON %s.%s (%s);",
-                                 quoteIdentifier(indexMetadata.name),
-                                 quoteIdentifier(keyspace),
-                                 quoteIdentifier(cf),
+            return String.format("CREATE INDEX %s ON %s (%s);",
+                                 indexMetadata.toCQLString(),
+                                 baseTable.toString(),
                                  indexMetadata.options.get(IndexTarget.TARGET_OPTION_NAME));
         }
     }
     private static String toCQL(UserType userType)
     {
         StringBuilder sb = new StringBuilder();
-        sb.append(String.format("CREATE TYPE %s.%s(",
-                                quoteIdentifier(userType.keyspace),
-                                quoteIdentifier(userType.getNameAsString())));
+        sb.append("CREATE TYPE ").append(userType.toCQLString()).append(" (");
 
         Consumer<StringBuilder> commaAppender = commaAppender(" ");
         for (int i = 0; i < userType.size(); i++)
@@ -356,35 +349,33 @@ public class ColumnFamilyStoreCQLHelper
         return builder.toString();
     }
 
-    private static String toCQL(ColumnDefinition cd)
+    private static String toCQL(ColumnMetadata cd)
     {
         return toCQL(cd, false);
     }
 
-    private static String toCQL(ColumnDefinition cd, boolean isStaticCompactTable)
+    private static String toCQL(ColumnMetadata cd, boolean isStaticCompactTable)
     {
         return String.format("%s %s%s",
-                             quoteIdentifier(cd.name.toString()),
+                             cd.name.toCQLString(),
                              cd.type.asCQL3Type().toString(),
                              cd.isStatic() && !isStaticCompactTable ? " static" : "");
     }
 
-    private static String toCQLAdd(String keyspace, String cf, ColumnDefinition cd)
+    private static String toCQLAdd(TableMetadata table, ColumnMetadata cd)
     {
-        return String.format("ALTER TABLE %s.%s ADD %s %s%s;",
-                             quoteIdentifier(keyspace),
-                             quoteIdentifier(cf),
-                             quoteIdentifier(cd.name.toString()),
+        return String.format("ALTER TABLE %s ADD %s %s%s;",
+                             table.toString(),
+                             cd.name.toCQLString(),
                              cd.type.asCQL3Type().toString(),
                              cd.isStatic() ? " static" : "");
     }
 
-    private static String toCQLDrop(String keyspace, String cf, CFMetaData.DroppedColumn droppedColumn)
+    private static String toCQLDrop(TableMetadata table, DroppedColumn droppedColumn)
     {
-        return String.format("ALTER TABLE %s.%s DROP %s USING TIMESTAMP %s;",
-                             quoteIdentifier(keyspace),
-                             quoteIdentifier(cf),
-                             quoteIdentifier(droppedColumn.name),
+        return String.format("ALTER TABLE %s DROP %s USING TIMESTAMP %s;",
+                             table.toString(),
+                             droppedColumn.column.name.toCQLString(),
                              droppedColumn.droppedTime);
     }
 
@@ -419,21 +410,16 @@ public class ColumnFamilyStoreCQLHelper
         };
     }
 
-    private static String quoteIdentifier(String id)
-    {
-        return ColumnIdentifier.maybeQuote(id);
-    }
-
     /**
      * Whether or not the given metadata is compatible / representable with CQL Language
      */
-    public static boolean isCqlCompatible(CFMetaData metaData)
+    public static boolean isCqlCompatible(TableMetadata metaData)
     {
         if (metaData.isSuper())
             return false;
 
         if (metaData.isCompactTable()
-            && metaData.partitionColumns().withoutStatics().size() > 1
+            && metaData.regularColumns().size() > 1
             && metaData.clusteringColumns().size() >= 1)
             return false;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
index d788e2e..e361ffe 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
@@ -102,11 +102,14 @@ public interface ColumnFamilyStoreMBean
      */
     public Map<String,String> getCompressionParameters();
 
+    public String getCompressionParametersJson();
+
     /**
-     * Set the compression parameters
+     * Set the compression parameters locally for this node
      * @param opts map of string names to values
      */
     public void setCompressionParameters(Map<String,String> opts);
+    public void setCompressionParametersJson(String options);
 
     /**
      * Set new crc check chance

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/Columns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Columns.java b/src/java/org/apache/cassandra/db/Columns.java
index e9e3abf..f38856f 100644
--- a/src/java/org/apache/cassandra/db/Columns.java
+++ b/src/java/org/apache/cassandra/db/Columns.java
@@ -28,8 +28,8 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterators;
 
 import net.nicoulaj.compilecommand.annotations.DontInline;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.marshal.SetType;
 import org.apache.cassandra.db.marshal.UTF8Type;
@@ -46,19 +46,19 @@ import org.apache.cassandra.utils.btree.UpdateFunction;
  * An immutable and sorted list of (non-PK) columns for a given table.
  * <p>
  * Note that in practice, it will either store only static columns, or only regular ones. When
- * we need both type of columns, we use a {@link PartitionColumns} object.
+ * we need both type of columns, we use a {@link RegularAndStaticColumns} object.
  */
-public class Columns extends AbstractCollection<ColumnDefinition> implements Collection<ColumnDefinition>
+public class Columns extends AbstractCollection<ColumnMetadata> implements Collection<ColumnMetadata>
 {
     public static final Serializer serializer = new Serializer();
     public static final Columns NONE = new Columns(BTree.empty(), 0);
-    public static final ColumnDefinition FIRST_COMPLEX =
-        new ColumnDefinition("",
-                             "",
-                             ColumnIdentifier.getInterned(ByteBufferUtil.EMPTY_BYTE_BUFFER, UTF8Type.instance),
-                             SetType.getInstance(UTF8Type.instance, true),
-                             ColumnDefinition.NO_POSITION,
-                             ColumnDefinition.Kind.REGULAR);
+    private static final ColumnMetadata FIRST_COMPLEX =
+        new ColumnMetadata("",
+                           "",
+                           ColumnIdentifier.getInterned(ByteBufferUtil.EMPTY_BYTE_BUFFER, UTF8Type.instance),
+                           SetType.getInstance(UTF8Type.instance, true),
+                           ColumnMetadata.NO_POSITION,
+                           ColumnMetadata.Kind.REGULAR);
 
     private final Object[] columns;
     private final int complexIdx; // Index of the first complex column
@@ -82,7 +82,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      *
      * @return the newly created {@code Columns} containing only {@code c}.
      */
-    public static Columns of(ColumnDefinition c)
+    public static Columns of(ColumnMetadata c)
     {
         return new Columns(BTree.singleton(c), c.isComplex() ? 0 : 1);
     }
@@ -93,9 +93,9 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      * @param s the set from which to create the new {@code Columns}.
      * @return the newly created {@code Columns} containing the columns from {@code s}.
      */
-    public static Columns from(Collection<ColumnDefinition> s)
+    public static Columns from(Collection<ColumnMetadata> s)
     {
-        Object[] tree = BTree.<ColumnDefinition>builder(Comparator.naturalOrder()).addAll(s).build();
+        Object[] tree = BTree.<ColumnMetadata>builder(Comparator.naturalOrder()).addAll(s).build();
         return new Columns(tree, findFirstComplexIdx(tree));
     }
 
@@ -103,7 +103,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
     {
         // have fast path for common no-complex case
         int size = BTree.size(tree);
-        if (!BTree.isEmpty(tree) && BTree.<ColumnDefinition>findByIndex(tree, size - 1).isSimple())
+        if (!BTree.isEmpty(tree) && BTree.<ColumnMetadata>findByIndex(tree, size - 1).isSimple())
             return size;
         return BTree.ceilIndex(tree, Comparator.naturalOrder(), FIRST_COMPLEX);
     }
@@ -176,7 +176,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      *
      * @return the {@code i}th simple column in this object.
      */
-    public ColumnDefinition getSimple(int i)
+    public ColumnMetadata getSimple(int i)
     {
         return BTree.findByIndex(columns, i);
     }
@@ -189,7 +189,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      *
      * @return the {@code i}th complex column in this object.
      */
-    public ColumnDefinition getComplex(int i)
+    public ColumnMetadata getComplex(int i)
     {
         return BTree.findByIndex(columns, complexIdx + i);
     }
@@ -203,7 +203,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      * @return the index for simple column {@code c} if it is contains in this
      * object
      */
-    public int simpleIdx(ColumnDefinition c)
+    public int simpleIdx(ColumnMetadata c)
     {
         return BTree.findIndex(columns, Comparator.naturalOrder(), c);
     }
@@ -217,7 +217,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      * @return the index for complex column {@code c} if it is contains in this
      * object
      */
-    public int complexIdx(ColumnDefinition c)
+    public int complexIdx(ColumnMetadata c)
     {
         return BTree.findIndex(columns, Comparator.naturalOrder(), c) - complexIdx;
     }
@@ -229,7 +229,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      *
      * @return whether {@code c} is contained by this object.
      */
-    public boolean contains(ColumnDefinition c)
+    public boolean contains(ColumnMetadata c)
     {
         return BTree.findIndex(columns, Comparator.naturalOrder(), c) >= 0;
     }
@@ -251,8 +251,8 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
         if (this == NONE)
             return other;
 
-        Object[] tree = BTree.<ColumnDefinition>merge(this.columns, other.columns, Comparator.naturalOrder(),
-                                                      UpdateFunction.noOp());
+        Object[] tree = BTree.<ColumnMetadata>merge(this.columns, other.columns, Comparator.naturalOrder(),
+                                                    UpdateFunction.noOp());
         if (tree == this.columns)
             return this;
         if (tree == other.columns)
@@ -275,9 +275,9 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
         if (other.size() > this.size())
             return false;
 
-        BTreeSearchIterator<ColumnDefinition, ColumnDefinition> iter = BTree.slice(columns, Comparator.naturalOrder(), BTree.Dir.ASC);
+        BTreeSearchIterator<ColumnMetadata, ColumnMetadata> iter = BTree.slice(columns, Comparator.naturalOrder(), BTree.Dir.ASC);
         for (Object def : other)
-            if (iter.next((ColumnDefinition) def) == null)
+            if (iter.next((ColumnMetadata) def) == null)
                 return false;
         return true;
     }
@@ -287,7 +287,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      *
      * @return an iterator over the simple columns of this object.
      */
-    public Iterator<ColumnDefinition> simpleColumns()
+    public Iterator<ColumnMetadata> simpleColumns()
     {
         return BTree.iterator(columns, 0, complexIdx - 1, BTree.Dir.ASC);
     }
@@ -297,7 +297,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      *
      * @return an iterator over the complex columns of this object.
      */
-    public Iterator<ColumnDefinition> complexColumns()
+    public Iterator<ColumnMetadata> complexColumns()
     {
         return BTree.iterator(columns, complexIdx, BTree.size(columns) - 1, BTree.Dir.ASC);
     }
@@ -307,9 +307,9 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      *
      * @return an iterator over all the columns of this object.
      */
-    public BTreeSearchIterator<ColumnDefinition, ColumnDefinition> iterator()
+    public BTreeSearchIterator<ColumnMetadata, ColumnMetadata> iterator()
     {
-        return BTree.<ColumnDefinition, ColumnDefinition>slice(columns, Comparator.naturalOrder(), BTree.Dir.ASC);
+        return BTree.<ColumnMetadata, ColumnMetadata>slice(columns, Comparator.naturalOrder(), BTree.Dir.ASC);
     }
 
     /**
@@ -319,11 +319,11 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      *
      * @return an iterator returning columns in alphabetical order.
      */
-    public Iterator<ColumnDefinition> selectOrderIterator()
+    public Iterator<ColumnMetadata> selectOrderIterator()
     {
         // In wildcard selection, we want to return all columns in alphabetical order,
         // irregarding of whether they are complex or not
-        return Iterators.<ColumnDefinition>
+        return Iterators.<ColumnMetadata>
                          mergeSorted(ImmutableList.of(simpleColumns(), complexColumns()),
                                      (s, c) ->
                                      {
@@ -340,12 +340,12 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      * @return newly allocated columns containing all the columns of {@code this} expect
      * for {@code column}.
      */
-    public Columns without(ColumnDefinition column)
+    public Columns without(ColumnMetadata column)
     {
         if (!contains(column))
             return this;
 
-        Object[] newColumns = BTreeRemoval.<ColumnDefinition>remove(columns, Comparator.naturalOrder(), column);
+        Object[] newColumns = BTreeRemoval.<ColumnMetadata>remove(columns, Comparator.naturalOrder(), column);
         return new Columns(newColumns);
     }
 
@@ -355,15 +355,15 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      *
      * @return a predicate to test the inclusion of sorted columns in this object.
      */
-    public Predicate<ColumnDefinition> inOrderInclusionTester()
+    public Predicate<ColumnMetadata> inOrderInclusionTester()
     {
-        SearchIterator<ColumnDefinition, ColumnDefinition> iter = BTree.slice(columns, Comparator.naturalOrder(), BTree.Dir.ASC);
+        SearchIterator<ColumnMetadata, ColumnMetadata> iter = BTree.slice(columns, Comparator.naturalOrder(), BTree.Dir.ASC);
         return column -> iter.next(column) != null;
     }
 
     public void digest(MessageDigest digest)
     {
-        for (ColumnDefinition c : this)
+        for (ColumnMetadata c : this)
             digest.update(c.name.bytes.duplicate());
     }
 
@@ -372,7 +372,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
      * @param function
      * @param reversed
      */
-    public void apply(Consumer<ColumnDefinition> function, boolean reversed)
+    public void apply(Consumer<ColumnMetadata> function, boolean reversed)
     {
         BTree.apply(columns, function, reversed);
     }
@@ -400,7 +400,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
     {
         StringBuilder sb = new StringBuilder("[");
         boolean first = true;
-        for (ColumnDefinition def : this)
+        for (ColumnMetadata def : this)
         {
             if (first) first = false; else sb.append(" ");
             sb.append(def.name);
@@ -413,33 +413,33 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
         public void serialize(Columns columns, DataOutputPlus out) throws IOException
         {
             out.writeUnsignedVInt(columns.size());
-            for (ColumnDefinition column : columns)
+            for (ColumnMetadata column : columns)
                 ByteBufferUtil.writeWithVIntLength(column.name.bytes, out);
         }
 
         public long serializedSize(Columns columns)
         {
             long size = TypeSizes.sizeofUnsignedVInt(columns.size());
-            for (ColumnDefinition column : columns)
+            for (ColumnMetadata column : columns)
                 size += ByteBufferUtil.serializedSizeWithVIntLength(column.name.bytes);
             return size;
         }
 
-        public Columns deserialize(DataInputPlus in, CFMetaData metadata) throws IOException
+        public Columns deserialize(DataInputPlus in, TableMetadata metadata) throws IOException
         {
             int length = (int)in.readUnsignedVInt();
-            BTree.Builder<ColumnDefinition> builder = BTree.builder(Comparator.naturalOrder());
+            BTree.Builder<ColumnMetadata> builder = BTree.builder(Comparator.naturalOrder());
             builder.auto(false);
             for (int i = 0; i < length; i++)
             {
                 ByteBuffer name = ByteBufferUtil.readWithVIntLength(in);
-                ColumnDefinition column = metadata.getColumnDefinition(name);
+                ColumnMetadata column = metadata.getColumn(name);
                 if (column == null)
                 {
                     // If we don't find the definition, it could be we have data for a dropped column, and we shouldn't
-                    // fail deserialization because of that. So we grab a "fake" ColumnDefinition that ensure proper
+                    // fail deserialization because of that. So we grab a "fake" ColumnMetadata that ensure proper
                     // deserialization. The column will be ignore later on anyway.
-                    column = metadata.getDroppedColumnDefinition(name);
+                    column = metadata.getDroppedColumn(name);
                     if (column == null)
                         throw new RuntimeException("Unknown column " + UTF8Type.instance.getString(name) + " during deserialization");
                 }
@@ -452,7 +452,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
          * If both ends have a pre-shared superset of the columns we are serializing, we can send them much
          * more efficiently. Both ends must provide the identically same set of columns.
          */
-        public void serializeSubset(Collection<ColumnDefinition> columns, Columns superset, DataOutputPlus out) throws IOException
+        public void serializeSubset(Collection<ColumnMetadata> columns, Columns superset, DataOutputPlus out) throws IOException
         {
             /**
              * We weight this towards small sets, and sets where the majority of items are present, since
@@ -482,7 +482,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
             }
         }
 
-        public long serializedSubsetSize(Collection<ColumnDefinition> columns, Columns superset)
+        public long serializedSubsetSize(Collection<ColumnMetadata> columns, Columns superset)
         {
             int columnCount = columns.size();
             int supersetCount = superset.size();
@@ -513,9 +513,9 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
             }
             else
             {
-                BTree.Builder<ColumnDefinition> builder = BTree.builder(Comparator.naturalOrder());
+                BTree.Builder<ColumnMetadata> builder = BTree.builder(Comparator.naturalOrder());
                 int firstComplexIdx = 0;
-                for (ColumnDefinition column : superset)
+                for (ColumnMetadata column : superset)
                 {
                     if ((encoded & 1) == 0)
                     {
@@ -531,13 +531,13 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
 
         // encodes a 1 bit for every *missing* column, on the assumption presence is more common,
         // and because this is consistent with encoding 0 to represent all present
-        private static long encodeBitmap(Collection<ColumnDefinition> columns, Columns superset, int supersetCount)
+        private static long encodeBitmap(Collection<ColumnMetadata> columns, Columns superset, int supersetCount)
         {
             long bitmap = 0L;
-            BTreeSearchIterator<ColumnDefinition, ColumnDefinition> iter = superset.iterator();
+            BTreeSearchIterator<ColumnMetadata, ColumnMetadata> iter = superset.iterator();
             // the index we would encounter next if all columns are present
             int expectIndex = 0;
-            for (ColumnDefinition column : columns)
+            for (ColumnMetadata column : columns)
             {
                 if (iter.next(column) == null)
                     throw new IllegalStateException(columns + " is not a subset of " + superset);
@@ -556,15 +556,15 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
         }
 
         @DontInline
-        private void serializeLargeSubset(Collection<ColumnDefinition> columns, int columnCount, Columns superset, int supersetCount, DataOutputPlus out) throws IOException
+        private void serializeLargeSubset(Collection<ColumnMetadata> columns, int columnCount, Columns superset, int supersetCount, DataOutputPlus out) throws IOException
         {
             // write flag indicating we're in lengthy mode
             out.writeUnsignedVInt(supersetCount - columnCount);
-            BTreeSearchIterator<ColumnDefinition, ColumnDefinition> iter = superset.iterator();
+            BTreeSearchIterator<ColumnMetadata, ColumnMetadata> iter = superset.iterator();
             if (columnCount < supersetCount / 2)
             {
                 // write present columns
-                for (ColumnDefinition column : columns)
+                for (ColumnMetadata column : columns)
                 {
                     if (iter.next(column) == null)
                         throw new IllegalStateException();
@@ -575,7 +575,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
             {
                 // write missing columns
                 int prev = -1;
-                for (ColumnDefinition column : columns)
+                for (ColumnMetadata column : columns)
                 {
                     if (iter.next(column) == null)
                         throw new IllegalStateException();
@@ -594,7 +594,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
             int supersetCount = superset.size();
             int columnCount = supersetCount - delta;
 
-            BTree.Builder<ColumnDefinition> builder = BTree.builder(Comparator.naturalOrder());
+            BTree.Builder<ColumnMetadata> builder = BTree.builder(Comparator.naturalOrder());
             if (columnCount < supersetCount / 2)
             {
                 for (int i = 0 ; i < columnCount ; i++)
@@ -605,7 +605,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
             }
             else
             {
-                Iterator<ColumnDefinition> iter = superset.iterator();
+                Iterator<ColumnMetadata> iter = superset.iterator();
                 int idx = 0;
                 int skipped = 0;
                 while (true)
@@ -613,7 +613,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
                     int nextMissingIndex = skipped < delta ? (int)in.readUnsignedVInt() : supersetCount;
                     while (idx < nextMissingIndex)
                     {
-                        ColumnDefinition def = iter.next();
+                        ColumnMetadata def = iter.next();
                         builder.add(def);
                         idx++;
                     }
@@ -628,15 +628,15 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
         }
 
         @DontInline
-        private int serializeLargeSubsetSize(Collection<ColumnDefinition> columns, int columnCount, Columns superset, int supersetCount)
+        private int serializeLargeSubsetSize(Collection<ColumnMetadata> columns, int columnCount, Columns superset, int supersetCount)
         {
             // write flag indicating we're in lengthy mode
             int size = TypeSizes.sizeofUnsignedVInt(supersetCount - columnCount);
-            BTreeSearchIterator<ColumnDefinition, ColumnDefinition> iter = superset.iterator();
+            BTreeSearchIterator<ColumnMetadata, ColumnMetadata> iter = superset.iterator();
             if (columnCount < supersetCount / 2)
             {
                 // write present columns
-                for (ColumnDefinition column : columns)
+                for (ColumnMetadata column : columns)
                 {
                     if (iter.next(column) == null)
                         throw new IllegalStateException();
@@ -647,7 +647,7 @@ public class Columns extends AbstractCollection<ColumnDefinition> implements Col
             {
                 // write missing columns
                 int prev = -1;
-                for (ColumnDefinition column : columns)
+                for (ColumnMetadata column : columns)
                 {
                     if (iter.next(column) == null)
                         throw new IllegalStateException();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/CompactTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CompactTables.java b/src/java/org/apache/cassandra/db/CompactTables.java
index 31e482c..29993a2 100644
--- a/src/java/org/apache/cassandra/db/CompactTables.java
+++ b/src/java/org/apache/cassandra/db/CompactTables.java
@@ -20,9 +20,9 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
@@ -79,11 +79,11 @@ public abstract class CompactTables
 
     private CompactTables() {}
 
-    public static ColumnDefinition getCompactValueColumn(PartitionColumns columns, boolean isSuper)
+    public static ColumnMetadata getCompactValueColumn(RegularAndStaticColumns columns, boolean isSuper)
     {
         if (isSuper)
         {
-            for (ColumnDefinition column : columns.regulars)
+            for (ColumnMetadata column : columns.regulars)
                 if (column.name.bytes.equals(SUPER_COLUMN_MAP_COLUMN))
                     return column;
             throw new AssertionError("Invalid super column table definition, no 'dynamic' map column");
@@ -92,14 +92,14 @@ public abstract class CompactTables
         return columns.regulars.getSimple(0);
     }
 
-    public static boolean hasEmptyCompactValue(CFMetaData metadata)
+    public static boolean hasEmptyCompactValue(TableMetadata metadata)
     {
-        return metadata.compactValueColumn().type instanceof EmptyType;
+        return metadata.compactValueColumn.type instanceof EmptyType;
     }
 
-    public static boolean isSuperColumnMapColumn(ColumnDefinition column)
+    public static boolean isSuperColumnMapColumn(ColumnMetadata column)
     {
-        return column.kind == ColumnDefinition.Kind.REGULAR && column.name.bytes.equals(SUPER_COLUMN_MAP_COLUMN);
+        return column.kind == ColumnMetadata.Kind.REGULAR && column.name.bytes.equals(SUPER_COLUMN_MAP_COLUMN);
     }
 
     public static DefaultNames defaultNameGenerator(Set<String> usedNames)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ConsistencyLevel.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
index ab4243f..2214d8d 100644
--- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java
+++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
@@ -28,9 +28,9 @@ import com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.ReadRepairDecision;
+import org.apache.cassandra.service.ReadRepairDecision;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnavailableException;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
@@ -381,10 +381,10 @@ public enum ConsistencyLevel
         return this == SERIAL || this == LOCAL_SERIAL;
     }
 
-    public void validateCounterForWrite(CFMetaData metadata) throws InvalidRequestException
+    public void validateCounterForWrite(TableMetadata metadata) throws InvalidRequestException
     {
         if (this == ConsistencyLevel.ANY)
-            throw new InvalidRequestException("Consistency level ANY is not yet supported for counter table " + metadata.cfName);
+            throw new InvalidRequestException("Consistency level ANY is not yet supported for counter table " + metadata.name);
 
         if (isSerialConsistency())
             throw new InvalidRequestException("Counter operations are inherently non-serializable");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/CounterMutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java
index 4e4a30d..0f1ad06 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -40,6 +40,7 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.*;
@@ -65,9 +66,9 @@ public class CounterMutation implements IMutation
         return mutation.getKeyspaceName();
     }
 
-    public Collection<UUID> getColumnFamilyIds()
+    public Collection<TableId> getTableIds()
     {
-        return mutation.getColumnFamilyIds();
+        return mutation.getTableIds();
     }
 
     public Collection<PartitionUpdate> getPartitionUpdates()
@@ -175,7 +176,7 @@ public class CounterMutation implements IMutation
                         {
                             public Object apply(final ColumnData data)
                             {
-                                return Objects.hashCode(update.metadata().cfId, key(), row.clustering(), data.column());
+                                return Objects.hashCode(update.metadata().id, key(), row.clustering(), data.column());
                             }
                         }));
                     }
@@ -186,7 +187,7 @@ public class CounterMutation implements IMutation
 
     private PartitionUpdate processModifications(PartitionUpdate changes)
     {
-        ColumnFamilyStore cfs = Keyspace.open(getKeyspaceName()).getColumnFamilyStore(changes.metadata().cfId);
+        ColumnFamilyStore cfs = Keyspace.open(getKeyspaceName()).getColumnFamilyStore(changes.metadata().id);
 
         List<PartitionUpdate.CounterMark> marks = changes.collectCounterMarks();
 
@@ -239,7 +240,7 @@ public class CounterMutation implements IMutation
     private void updateWithCurrentValuesFromCFS(List<PartitionUpdate.CounterMark> marks, ColumnFamilyStore cfs)
     {
         ColumnFilter.Builder builder = ColumnFilter.selectionBuilder();
-        BTreeSet.Builder<Clustering> names = BTreeSet.builder(cfs.metadata.comparator);
+        BTreeSet.Builder<Clustering> names = BTreeSet.builder(cfs.metadata().comparator);
         for (PartitionUpdate.CounterMark mark : marks)
         {
             if (mark.clustering() != Clustering.STATIC_CLUSTERING)
@@ -252,7 +253,7 @@ public class CounterMutation implements IMutation
 
         int nowInSec = FBUtilities.nowInSeconds();
         ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names.build(), false);
-        SinglePartitionReadCommand cmd = SinglePartitionReadCommand.create(cfs.metadata, nowInSec, key(), builder.build(), filter);
+        SinglePartitionReadCommand cmd = SinglePartitionReadCommand.create(cfs.metadata(), nowInSec, key(), builder.build(), filter);
         PeekingIterator<PartitionUpdate.CounterMark> markIter = Iterators.peekingIterator(marks.iterator());
         try (ReadExecutionController controller = cmd.executionController();
              RowIterator partition = UnfilteredRowIterators.filter(cmd.queryMemtableAndDisk(cfs, controller), nowInSec))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/DataRange.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DataRange.java b/src/java/org/apache/cassandra/db/DataRange.java
index ffe041e..2983a57 100644
--- a/src/java/org/apache/cassandra/db/DataRange.java
+++ b/src/java/org/apache/cassandra/db/DataRange.java
@@ -19,8 +19,8 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CompositeType;
@@ -236,12 +236,12 @@ public class DataRange
         return new DataRange(range, clusteringIndexFilter);
     }
 
-    public String toString(CFMetaData metadata)
+    public String toString(TableMetadata metadata)
     {
-        return String.format("range=%s pfilter=%s", keyRange.getString(metadata.getKeyValidator()), clusteringIndexFilter.toString(metadata));
+        return String.format("range=%s pfilter=%s", keyRange.getString(metadata.partitionKeyType), clusteringIndexFilter.toString(metadata));
     }
 
-    public String toCQLString(CFMetaData metadata)
+    public String toCQLString(TableMetadata metadata)
     {
         if (isUnrestricted())
             return "UNRESTRICTED";
@@ -269,15 +269,15 @@ public class DataRange
         return sb.toString();
     }
 
-    private void appendClause(PartitionPosition pos, StringBuilder sb, CFMetaData metadata, boolean isStart, boolean isInclusive)
+    private void appendClause(PartitionPosition pos, StringBuilder sb, TableMetadata metadata, boolean isStart, boolean isInclusive)
     {
         sb.append("token(");
-        sb.append(ColumnDefinition.toCQLString(metadata.partitionKeyColumns()));
+        sb.append(ColumnMetadata.toCQLString(metadata.partitionKeyColumns()));
         sb.append(") ").append(getOperator(isStart, isInclusive)).append(" ");
         if (pos instanceof DecoratedKey)
         {
             sb.append("token(");
-            appendKeyString(sb, metadata.getKeyValidator(), ((DecoratedKey)pos).getKey());
+            appendKeyString(sb, metadata.partitionKeyType, ((DecoratedKey)pos).getKey());
             sb.append(")");
         }
         else
@@ -380,10 +380,10 @@ public class DataRange
         }
 
         @Override
-        public String toString(CFMetaData metadata)
+        public String toString(TableMetadata metadata)
         {
             return String.format("range=%s (paging) pfilter=%s lastReturned=%s (%s)",
-                                 keyRange.getString(metadata.getKeyValidator()),
+                                 keyRange.getString(metadata.partitionKeyType),
                                  clusteringIndexFilter.toString(metadata),
                                  lastReturned.toString(metadata),
                                  inclusive ? "included" : "excluded");
@@ -392,7 +392,7 @@ public class DataRange
 
     public static class Serializer
     {
-        public void serialize(DataRange range, DataOutputPlus out, int version, CFMetaData metadata) throws IOException
+        public void serialize(DataRange range, DataOutputPlus out, int version, TableMetadata metadata) throws IOException
         {
             AbstractBounds.rowPositionSerializer.serialize(range.keyRange, out, version);
             ClusteringIndexFilter.serializer.serialize(range.clusteringIndexFilter, out, version);
@@ -405,7 +405,7 @@ public class DataRange
             }
         }
 
-        public DataRange deserialize(DataInputPlus in, int version, CFMetaData metadata) throws IOException
+        public DataRange deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException
         {
             AbstractBounds<PartitionPosition> range = AbstractBounds.rowPositionSerializer.deserialize(in, metadata.partitioner, version);
             ClusteringIndexFilter filter = ClusteringIndexFilter.serializer.deserialize(in, version, metadata);
@@ -422,7 +422,7 @@ public class DataRange
             }
         }
 
-        public long serializedSize(DataRange range, int version, CFMetaData metadata)
+        public long serializedSize(DataRange range, int version, TableMetadata metadata)
         {
             long size = AbstractBounds.rowPositionSerializer.serializedSize(range.keyRange, version)
                       + ClusteringIndexFilter.serializer.serializedSize(range.clusteringIndexFilter, version)


[11/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictionsTest.java b/test/unit/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictionsTest.java
index 83c00d0..35adff3 100644
--- a/test/unit/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/restrictions/ClusteringColumnRestrictionsTest.java
@@ -24,8 +24,8 @@ import com.google.common.collect.Iterables;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.Term.MultiItemTerminal;
@@ -52,9 +52,9 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithNoRestrictions()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC);
 
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
         assertEquals(1, bounds.size());
@@ -71,12 +71,12 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithOneEqRestrictionsAndOneClusteringColumn()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC);
 
         ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
-        Restriction eq = newSingleEq(cfMetaData, 0, clustering_0);
+        Restriction eq = newSingleEq(tableMetadata, 0, clustering_0);
 
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(eq);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -94,12 +94,12 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithOneEqRestrictionsAndTwoClusteringColumns()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC);
 
         ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
-        Restriction eq = newSingleEq(cfMetaData, 0, clustering_0);
+        Restriction eq = newSingleEq(tableMetadata, 0, clustering_0);
 
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(eq);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -121,11 +121,11 @@ public class ClusteringColumnRestrictionsTest
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
         ByteBuffer value3 = ByteBufferUtil.bytes(3);
 
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC);
 
-        Restriction in = newSingleIN(cfMetaData, 0, value1, value2, value3);
+        Restriction in = newSingleIN(tableMetadata, 0, value1, value2, value3);
 
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(in);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -147,13 +147,13 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithSliceRestrictionsAndOneClusteringColumn()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
 
-        Restriction slice = newSingleSlice(cfMetaData, 0, Bound.START, false, value1);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newSingleSlice(tableMetadata, 0, Bound.START, false, value1);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -164,8 +164,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEmptyEnd(get(bounds, 0));
 
-        slice = newSingleSlice(cfMetaData, 0, Bound.START, true, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 0, Bound.START, true, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -176,8 +176,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEmptyEnd(get(bounds, 0));
 
-        slice = newSingleSlice(cfMetaData, 0, Bound.END, true, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 0, Bound.END, true, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -188,8 +188,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), true, value1);
 
-        slice = newSingleSlice(cfMetaData, 0, Bound.END, false, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 0, Bound.END, false, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -200,9 +200,9 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), false, value1);
 
-        slice = newSingleSlice(cfMetaData, 0, Bound.START, false, value1);
-        Restriction slice2 = newSingleSlice(cfMetaData, 0, Bound.END, false, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 0, Bound.START, false, value1);
+        Restriction slice2 = newSingleSlice(tableMetadata, 0, Bound.END, false, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -213,9 +213,9 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), false, value2);
 
-        slice = newSingleSlice(cfMetaData, 0, Bound.START, true, value1);
-        slice2 = newSingleSlice(cfMetaData, 0, Bound.END, true, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 0, Bound.START, true, value1);
+        slice2 = newSingleSlice(tableMetadata, 0, Bound.END, true, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -233,13 +233,13 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithSliceRestrictionsAndOneDescendingClusteringColumn()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.DESC, Sort.DESC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.DESC, Sort.DESC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
 
-        Restriction slice = newSingleSlice(cfMetaData, 0, Bound.START, false, value1);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newSingleSlice(tableMetadata, 0, Bound.START, false, value1);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -250,8 +250,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), false, value1);
 
-        slice = newSingleSlice(cfMetaData, 0, Bound.START, true, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 0, Bound.START, true, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -262,8 +262,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), true, value1);
 
-        slice = newSingleSlice(cfMetaData, 0, Bound.END, true, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 0, Bound.END, true, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -274,8 +274,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEmptyEnd(get(bounds, 0));
 
-        slice = newSingleSlice(cfMetaData, 0, Bound.END, false, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 0, Bound.END, false, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -286,9 +286,9 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEmptyEnd(get(bounds, 0));
 
-        slice = newSingleSlice(cfMetaData, 0, Bound.START, false, value1);
-        Restriction slice2 = newSingleSlice(cfMetaData, 0, Bound.END, false, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 0, Bound.START, false, value1);
+        Restriction slice2 = newSingleSlice(tableMetadata, 0, Bound.END, false, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -299,9 +299,9 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), false, value1);
 
-        slice = newSingleSlice(cfMetaData, 0, Bound.START, true, value1);
-        slice2 = newSingleSlice(cfMetaData, 0, Bound.END, true, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 0, Bound.START, true, value1);
+        slice2 = newSingleSlice(tableMetadata, 0, Bound.END, true, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -319,14 +319,14 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithEqAndInRestrictions()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
         ByteBuffer value3 = ByteBufferUtil.bytes(3);
-        Restriction eq = newSingleEq(cfMetaData, 0, value1);
-        Restriction in = newSingleIN(cfMetaData, 1, value1, value2, value3);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction eq = newSingleEq(tableMetadata, 0, value1);
+        Restriction in = newSingleIN(tableMetadata, 1, value1, value2, value3);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(eq).mergeWith(in);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -348,16 +348,16 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithEqAndSliceRestrictions()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
         ByteBuffer value3 = ByteBufferUtil.bytes(3);
 
-        Restriction eq = newSingleEq(cfMetaData, 0, value3);
+        Restriction eq = newSingleEq(tableMetadata, 0, value3);
 
-        Restriction slice = newSingleSlice(cfMetaData, 1, Bound.START, false, value1);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newSingleSlice(tableMetadata, 1, Bound.START, false, value1);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(eq).mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -368,8 +368,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), true, value3);
 
-        slice = newSingleSlice(cfMetaData, 1, Bound.START, true, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 1, Bound.START, true, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(eq).mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -380,8 +380,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), true, value3);
 
-        slice = newSingleSlice(cfMetaData, 1, Bound.END, true, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 1, Bound.END, true, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(eq).mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -392,8 +392,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), true, value3, value1);
 
-        slice = newSingleSlice(cfMetaData, 1, Bound.END, false, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 1, Bound.END, false, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(eq).mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -404,9 +404,9 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), false, value3, value1);
 
-        slice = newSingleSlice(cfMetaData, 1, Bound.START, false, value1);
-        Restriction slice2 = newSingleSlice(cfMetaData, 1, Bound.END, false, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 1, Bound.START, false, value1);
+        Restriction slice2 = newSingleSlice(tableMetadata, 1, Bound.END, false, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(eq).mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -417,9 +417,9 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), false, value3, value2);
 
-        slice = newSingleSlice(cfMetaData, 1, Bound.START, true, value1);
-        slice2 = newSingleSlice(cfMetaData, 1, Bound.END, true, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newSingleSlice(tableMetadata, 1, Bound.START, true, value1);
+        slice2 = newSingleSlice(tableMetadata, 1, Bound.END, true, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(eq).mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -437,12 +437,12 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiEqRestrictions()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        Restriction eq = newMultiEq(cfMetaData, 0, value1, value2);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction eq = newMultiEq(tableMetadata, 0, value1, value2);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(eq);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -460,13 +460,13 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiInRestrictions()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
         ByteBuffer value3 = ByteBufferUtil.bytes(3);
-        Restriction in = newMultiIN(cfMetaData, 0, asList(value1, value2), asList(value2, value3));
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction in = newMultiIN(tableMetadata, 0, asList(value1, value2), asList(value2, value3));
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(in);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -486,14 +486,14 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiSliceRestrictionsWithOneClusteringColumn()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC);
 
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
 
-        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -504,8 +504,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEmptyEnd(get(bounds, 0));
 
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -516,8 +516,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEmptyEnd(get(bounds, 0));
 
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, true, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -528,8 +528,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), true, value1);
 
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, false, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -540,9 +540,9 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), false, value1);
 
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
-        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1);
+        Restriction slice2 = newMultiSlice(tableMetadata, 0, Bound.END, false, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -553,9 +553,9 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), false, value2);
 
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1);
-        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1);
+        slice2 = newMultiSlice(tableMetadata, 0, Bound.END, true, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -574,13 +574,13 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiSliceRestrictionsWithOneDescendingClusteringColumn()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.DESC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.DESC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
 
-        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -591,8 +591,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), false, value1);
 
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -603,8 +603,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), true, value1);
 
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, true, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -615,8 +615,8 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEmptyEnd(get(bounds, 0));
 
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, false, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -627,9 +627,9 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEmptyEnd(get(bounds, 0));
 
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
-        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1);
+        Restriction slice2 = newMultiSlice(tableMetadata, 0, Bound.END, false, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -640,9 +640,9 @@ public class ClusteringColumnRestrictionsTest
         assertEquals(1, bounds.size());
         assertEndBound(get(bounds, 0), false, value1);
 
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1);
-        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1);
+        slice2 = newMultiSlice(tableMetadata, 0, Bound.END, true, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -660,14 +660,14 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiSliceRestrictionsWithTwoClusteringColumn()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
 
         // (clustering_0, clustering1) > (1, 2)
-        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -679,8 +679,8 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 0));
 
         // (clustering_0, clustering1) >= (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -692,8 +692,8 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 0));
 
         // (clustering_0, clustering1) <= (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, true, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -705,8 +705,8 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), true, value1, value2);
 
         // (clustering_0, clustering1) < (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, false, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -718,9 +718,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), false, value1, value2);
 
         // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
-        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2);
+        Restriction slice2 = newMultiSlice(tableMetadata, 0, Bound.END, false, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -732,9 +732,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), false, value2);
 
         // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
-        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2);
+        slice2 = newMultiSlice(tableMetadata, 0, Bound.END, true, value2, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -752,14 +752,14 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiSliceRestrictionsWithTwoDescendingClusteringColumns()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.DESC, Sort.DESC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.DESC, Sort.DESC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
 
         // (clustering_0, clustering1) > (1, 2)
-        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -771,8 +771,8 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), false, value1, value2);
 
         // (clustering_0, clustering1) >= (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -784,8 +784,8 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), true, value1, value2);
 
         // (clustering_0, clustering1) <= (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, true, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -797,8 +797,8 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 0));
 
         // (clustering_0, clustering1) < (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, false, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -811,9 +811,9 @@ public class ClusteringColumnRestrictionsTest
 
 
         // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
-        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2);
+        Restriction slice2 = newMultiSlice(tableMetadata, 0, Bound.END, false, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -825,9 +825,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), false, value1, value2);
 
         // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
-        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2);
+        slice2 = newMultiSlice(tableMetadata, 0, Bound.END, true, value2, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -846,14 +846,14 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiSliceRestrictionsWithOneDescendingAndOneAscendingClusteringColumns()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.DESC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.DESC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
 
         // (clustering_0, clustering1) > (1, 2)
-        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -867,8 +867,8 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), true, value1);
 
         // (clustering_0, clustering1) >= (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -882,8 +882,8 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), true, value1);
 
         // (clustering_0, clustering1) <= (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, true, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -897,8 +897,8 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 1));
 
         // (clustering_0, clustering1) < (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, false, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -912,9 +912,9 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 1));
 
         // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
-        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2);
+        Restriction slice2 = newMultiSlice(tableMetadata, 0, Bound.END, false, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -928,9 +928,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), true, value1);
 
         // (clustering_0) > (1) AND (clustering_0, clustering1) < (2, 1)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
-        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1);
+        slice2 = newMultiSlice(tableMetadata, 0, Bound.END, false, value2, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -944,9 +944,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), false, value1);
 
         // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
-        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2);
+        slice2 = newMultiSlice(tableMetadata, 0, Bound.END, true, value2, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -969,14 +969,14 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiSliceRestrictionsWithOneAscendingAndOneDescendingClusteringColumns()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.DESC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.DESC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
 
         // (clustering_0, clustering1) > (1, 2)
-        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -990,8 +990,8 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 1));
 
         // (clustering_0, clustering1) >= (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1005,8 +1005,8 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 1));
 
         // (clustering_0, clustering1) <= (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, true, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1020,8 +1020,8 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), true, value1);
 
         // (clustering_0, clustering1) < (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, false, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1035,9 +1035,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), true, value1);
 
         // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
-        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2);
+        Restriction slice2 = newMultiSlice(tableMetadata, 0, Bound.END, false, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1051,9 +1051,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), false, value2);
 
         // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
-        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2);
+        slice2 = newMultiSlice(tableMetadata, 0, Bound.END, true, value2, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1076,7 +1076,7 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiSliceRestrictionsWithTwoAscendingAndTwoDescendingClusteringColumns()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.DESC, Sort.DESC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC, Sort.DESC, Sort.DESC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
@@ -1084,8 +1084,8 @@ public class ClusteringColumnRestrictionsTest
         ByteBuffer value4 = ByteBufferUtil.bytes(4);
 
         // (clustering_0, clustering1, clustering_2, clustering_3) > (1, 2, 3, 4)
-        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2, value3, value4);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2, value3, value4);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1099,9 +1099,9 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 1));
 
         // clustering_0 = 1 AND (clustering_1, clustering_2, clustering_3) > (2, 3, 4)
-        Restriction eq = newSingleEq(cfMetaData, 0, value1);
-        slice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction eq = newSingleEq(tableMetadata, 0, value1);
+        slice = newMultiSlice(tableMetadata, 1, Bound.START, false, value2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
         restrictions = restrictions.mergeWith(eq);
 
@@ -1116,9 +1116,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), true, value1);
 
         // clustering_0 IN (1, 2) AND (clustering_1, clustering_2, clustering_3) > (2, 3, 4)
-        Restriction in = newSingleIN(cfMetaData, 0, value1, value2);
-        slice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction in = newSingleIN(tableMetadata, 0, value1, value2);
+        slice = newMultiSlice(tableMetadata, 1, Bound.START, false, value2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
         restrictions = restrictions.mergeWith(in);
 
@@ -1137,8 +1137,8 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 3), true, value2);
 
         // (clustering_0, clustering1) >= (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1150,8 +1150,8 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 0));
 
         // (clustering_0, clustering1, clustering_2, clustering_3) >= (1, 2, 3, 4)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1165,8 +1165,8 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 1));
 
         // (clustering_0, clustering1, clustering_2, clustering_3) <= (1, 2, 3, 4)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, true, value1, value2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1180,8 +1180,8 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), true, value1, value2);
 
         // (clustering_0, clustering1, clustering_2, clustering_3) < (1, 2, 3, 4)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, false, value1, value2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1195,9 +1195,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), true, value1, value2);
 
         // (clustering_0, clustering1, clustering_2, clustering_3) > (1, 2, 3, 4) AND (clustering_0, clustering_1) < (2, 3)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2, value3, value4);
-        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2, value3);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2, value3, value4);
+        Restriction slice2 = newMultiSlice(tableMetadata, 0, Bound.END, false, value2, value3);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1211,9 +1211,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), false, value2, value3);
 
         // (clustering_0, clustering1, clustering_2, clustering_3) >= (1, 2, 3, 4) AND (clustering_0, clustering1, clustering_2, clustering_3) <= (4, 3, 2, 1)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2, value3, value4);
-        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value4, value3, value2, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2, value3, value4);
+        slice2 = newMultiSlice(tableMetadata, 0, Bound.END, true, value4, value3, value2, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1236,7 +1236,7 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiSliceRestrictionsWithAscendingDescendingColumnMix()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.DESC, Sort.ASC, Sort.DESC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.DESC, Sort.ASC, Sort.DESC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
@@ -1244,8 +1244,8 @@ public class ClusteringColumnRestrictionsTest
         ByteBuffer value4 = ByteBufferUtil.bytes(4);
 
         // (clustering_0, clustering1, clustering_2, clustering_3) > (1, 2, 3, 4)
-        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2, value3, value4);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2, value3, value4);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1264,9 +1264,9 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 3));
 
         // clustering_0 = 1 AND (clustering_1, clustering_2, clustering_3) > (2, 3, 4)
-        Restriction eq = newSingleEq(cfMetaData, 0, value1);
-        slice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction eq = newSingleEq(tableMetadata, 0, value1);
+        slice = newMultiSlice(tableMetadata, 1, Bound.START, false, value2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
         restrictions = restrictions.mergeWith(eq);
 
@@ -1283,8 +1283,8 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 2), true, value1, value2);
 
         // (clustering_0, clustering1) >= (1, 2)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1298,8 +1298,8 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 1));
 
         // (clustering_0, clustering1, clustering_2, clustering_3) >= (1, 2, 3, 4)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1317,8 +1317,8 @@ public class ClusteringColumnRestrictionsTest
         assertEmptyEnd(get(bounds, 3));
 
         // (clustering_0, clustering1, clustering_2, clustering_3) <= (1, 2, 3, 4)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, true, value1, value2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1336,8 +1336,8 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 3), true, value1);
 
         // (clustering_0, clustering1, clustering_2, clustering_3) < (1, 2, 3, 4)
-        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.END, false, value1, value2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1355,9 +1355,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 3), true, value1);
 
         // (clustering_0, clustering1, clustering_2, clustering_3) > (1, 2, 3, 4) AND (clustering_0, clustering_1) < (2, 3)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2, value3, value4);
-        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2, value3);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, false, value1, value2, value3, value4);
+        Restriction slice2 = newMultiSlice(tableMetadata, 0, Bound.END, false, value2, value3);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1377,9 +1377,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 4), true, value2);
 
         // (clustering_0, clustering1, clustering_2, clustering_3) >= (1, 2, 3, 4) AND (clustering_0, clustering1, clustering_2, clustering_3) <= (4, 3, 2, 1)
-        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2, value3, value4);
-        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value4, value3, value2, value1);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        slice = newMultiSlice(tableMetadata, 0, Bound.START, true, value1, value2, value3, value4);
+        slice2 = newMultiSlice(tableMetadata, 0, Bound.END, true, value4, value3, value2, value1);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1409,7 +1409,7 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithSingleEqAndMultiEqRestrictions()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC, Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
@@ -1417,9 +1417,9 @@ public class ClusteringColumnRestrictionsTest
         ByteBuffer value4 = ByteBufferUtil.bytes(4);
 
         // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3)
-        Restriction singleEq = newSingleEq(cfMetaData, 0, value1);
-        Restriction multiEq = newMultiEq(cfMetaData, 1, value2, value3);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction singleEq = newSingleEq(tableMetadata, 0, value1);
+        Restriction multiEq = newMultiEq(tableMetadata, 1, value2, value3);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(singleEq).mergeWith(multiEq);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1431,10 +1431,10 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), true, value1, value2, value3);
 
         // clustering_0 = 1 AND clustering_1 = 2 AND (clustering_2, clustering_3) = (3, 4)
-        singleEq = newSingleEq(cfMetaData, 0, value1);
-        Restriction singleEq2 = newSingleEq(cfMetaData, 1, value2);
-        multiEq = newMultiEq(cfMetaData, 2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        singleEq = newSingleEq(tableMetadata, 0, value1);
+        Restriction singleEq2 = newSingleEq(tableMetadata, 1, value2);
+        multiEq = newMultiEq(tableMetadata, 2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(singleEq).mergeWith(singleEq2).mergeWith(multiEq);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1446,9 +1446,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), true, value1, value2, value3, value4);
 
         // (clustering_0, clustering_1) = (1, 2) AND clustering_2 = 3
-        singleEq = newSingleEq(cfMetaData, 2, value3);
-        multiEq = newMultiEq(cfMetaData, 0, value1, value2);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        singleEq = newSingleEq(tableMetadata, 2, value3);
+        multiEq = newMultiEq(tableMetadata, 0, value1, value2);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(singleEq).mergeWith(multiEq);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1460,10 +1460,10 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), true, value1, value2, value3);
 
         // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3) AND clustering_3 = 4
-        singleEq = newSingleEq(cfMetaData, 0, value1);
-        singleEq2 = newSingleEq(cfMetaData, 3, value4);
-        multiEq = newMultiEq(cfMetaData, 1, value2, value3);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        singleEq = newSingleEq(tableMetadata, 0, value1);
+        singleEq2 = newSingleEq(tableMetadata, 3, value4);
+        multiEq = newMultiEq(tableMetadata, 1, value2, value3);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(singleEq).mergeWith(multiEq).mergeWith(singleEq2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1481,7 +1481,7 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithSingleEqAndMultiINRestrictions()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC, Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
@@ -1490,9 +1490,9 @@ public class ClusteringColumnRestrictionsTest
         ByteBuffer value5 = ByteBufferUtil.bytes(5);
 
         // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
-        Restriction singleEq = newSingleEq(cfMetaData, 0, value1);
-        Restriction multiIN = newMultiIN(cfMetaData, 1, asList(value2, value3), asList(value4, value5));
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction singleEq = newSingleEq(tableMetadata, 0, value1);
+        Restriction multiIN = newMultiIN(tableMetadata, 1, asList(value2, value3), asList(value4, value5));
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(singleEq).mergeWith(multiIN);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1506,9 +1506,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), true, value1, value4, value5);
 
         // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3))
-        singleEq = newSingleEq(cfMetaData, 0, value1);
-        multiIN = newMultiIN(cfMetaData, 1, asList(value2, value3));
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        singleEq = newSingleEq(tableMetadata, 0, value1);
+        multiIN = newMultiIN(tableMetadata, 1, asList(value2, value3));
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(multiIN).mergeWith(singleEq);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1520,10 +1520,10 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), true, value1, value2, value3);
 
         // clustering_0 = 1 AND clustering_1 = 5 AND (clustering_2, clustering_3) IN ((2, 3), (4, 5))
-        singleEq = newSingleEq(cfMetaData, 0, value1);
-        Restriction singleEq2 = newSingleEq(cfMetaData, 1, value5);
-        multiIN = newMultiIN(cfMetaData, 2, asList(value2, value3), asList(value4, value5));
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        singleEq = newSingleEq(tableMetadata, 0, value1);
+        Restriction singleEq2 = newSingleEq(tableMetadata, 1, value5);
+        multiIN = newMultiIN(tableMetadata, 2, asList(value2, value3), asList(value4, value5));
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(singleEq).mergeWith(multiIN).mergeWith(singleEq2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1544,7 +1544,7 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithSingleEqAndSliceRestrictions()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
@@ -1553,9 +1553,9 @@ public class ClusteringColumnRestrictionsTest
         ByteBuffer value5 = ByteBufferUtil.bytes(5);
 
         // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3)
-        Restriction singleEq = newSingleEq(cfMetaData, 0, value1);
-        Restriction multiSlice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction singleEq = newSingleEq(tableMetadata, 0, value1);
+        Restriction multiSlice = newMultiSlice(tableMetadata, 1, Bound.START, false, value2, value3);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(singleEq).mergeWith(multiSlice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1567,10 +1567,10 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), true, value1);
 
         // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3) AND (clustering_1) < (4)
-        singleEq = newSingleEq(cfMetaData, 0, value1);
-        multiSlice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3);
-        Restriction multiSlice2 = newMultiSlice(cfMetaData, 1, Bound.END, false, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        singleEq = newSingleEq(tableMetadata, 0, value1);
+        multiSlice = newMultiSlice(tableMetadata, 1, Bound.START, false, value2, value3);
+        Restriction multiSlice2 = newMultiSlice(tableMetadata, 1, Bound.END, false, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(multiSlice2).mergeWith(singleEq).mergeWith(multiSlice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1582,10 +1582,10 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), false, value1, value4);
 
         // clustering_0 = 1 AND (clustering_1, clustering_2) => (2, 3) AND (clustering_1, clustering_2) <= (4, 5)
-        singleEq = newSingleEq(cfMetaData, 0, value1);
-        multiSlice = newMultiSlice(cfMetaData, 1, Bound.START, true, value2, value3);
-        multiSlice2 = newMultiSlice(cfMetaData, 1, Bound.END, true, value4, value5);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        singleEq = newSingleEq(tableMetadata, 0, value1);
+        multiSlice = newMultiSlice(tableMetadata, 1, Bound.START, true, value2, value3);
+        multiSlice2 = newMultiSlice(tableMetadata, 1, Bound.END, true, value4, value5);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(multiSlice2).mergeWith(singleEq).mergeWith(multiSlice);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1604,16 +1604,16 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithMultiEqAndSingleSliceRestrictions()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
         ByteBuffer value3 = ByteBufferUtil.bytes(3);
 
         // (clustering_0, clustering_1) = (1, 2) AND clustering_2 > 3
-        Restriction multiEq = newMultiEq(cfMetaData, 0, value1, value2);
-        Restriction singleSlice = newSingleSlice(cfMetaData, 2, Bound.START, false, value3);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction multiEq = newMultiEq(tableMetadata, 0, value1, value2);
+        Restriction singleSlice = newSingleSlice(tableMetadata, 2, Bound.START, false, value3);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(multiEq).mergeWith(singleSlice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1628,7 +1628,7 @@ public class ClusteringColumnRestrictionsTest
     @Test
     public void testBoundsAsClusteringWithSeveralMultiColumnRestrictions()
     {
-        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.ASC, Sort.ASC);
+        TableMetadata tableMetadata = newTableMetadata(Sort.ASC, Sort.ASC, Sort.ASC, Sort.ASC);
 
         ByteBuffer value1 = ByteBufferUtil.bytes(1);
         ByteBuffer value2 = ByteBufferUtil.bytes(2);
@@ -1637,9 +1637,9 @@ public class ClusteringColumnRestrictionsTest
         ByteBuffer value5 = ByteBufferUtil.bytes(5);
 
         // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) > (3, 4)
-        Restriction multiEq = newMultiEq(cfMetaData, 0, value1, value2);
-        Restriction multiSlice = newMultiSlice(cfMetaData, 2, Bound.START, false, value3, value4);
-        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        Restriction multiEq = newMultiEq(tableMetadata, 0, value1, value2);
+        Restriction multiSlice = newMultiSlice(tableMetadata, 2, Bound.START, false, value3, value4);
+        ClusteringColumnRestrictions restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(multiEq).mergeWith(multiSlice);
 
         SortedSet<ClusteringBound> bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1651,9 +1651,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 0), true, value1, value2);
 
         // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) IN ((3, 4), (4, 5))
-        multiEq = newMultiEq(cfMetaData, 0, value1, value2);
-        Restriction multiIN = newMultiIN(cfMetaData, 2, asList(value3, value4), asList(value4, value5));
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        multiEq = newMultiEq(tableMetadata, 0, value1, value2);
+        Restriction multiIN = newMultiIN(tableMetadata, 2, asList(value3, value4), asList(value4, value5));
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(multiEq).mergeWith(multiIN);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1667,9 +1667,9 @@ public class ClusteringColumnRestrictionsTest
         assertEndBound(get(bounds, 1), true, value1, value2, value4, value5);
 
         // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) = (3, 4)
-        multiEq = newMultiEq(cfMetaData, 0, value1, value2);
-        Restriction multiEq2 = newMultiEq(cfMetaData, 2, value3, value4);
-        restrictions = new ClusteringColumnRestrictions(cfMetaData);
+        multiEq = newMultiEq(tableMetadata, 0, value1, value2);
+        Restriction multiEq2 = newMultiEq(tableMetadata, 2, value3, value4);
+        restrictions = new ClusteringColumnRestrictions(tableMetadata);
         restrictions = restrictions.mergeWith(multiEq).mergeWith(multiEq2);
 
         bounds = restrictions.boundsAsClustering(Bound.START, QueryOptions.DEFAULT);
@@ -1741,21 +1741,16 @@ public class ClusteringColumnRestrictionsTest
         }
     }
 
-    /**
-     * Creates a new <code>CFMetaData</code> instance.
-     *
-     * @param numberOfClusteringColumns the number of clustering column
-     * @return a new <code>CFMetaData</code> instance
-     */
-    private static CFMetaData newCFMetaData(Sort... sorts)
+    private static TableMetadata newTableMetadata(Sort... sorts)
     {
         List<AbstractType<?>> types = new ArrayList<>();
 
         for (Sort sort : sorts)
             types.add(sort == Sort.ASC ? Int32Type.instance : ReversedType.getInstance(Int32Type.instance));
 
-        CFMetaData.Builder builder = CFMetaData.Builder.create("keyspace", "test")
-                                                       .addPartitionKey("partition_key", Int32Type.instance);
+        TableMetadata.Builder builder =
+            TableMetadata.builder("keyspace", "test")
+                         .addPartitionKeyColumn("partition_key", Int32Type.instance);
 
         for (int i = 0; i < sorts.length; i++)
             builder.addClusteringColumn("clustering_" + i, types.get(i));
@@ -1766,116 +1761,116 @@ public class ClusteringColumnRestrictionsTest
     /**
      * Creates a new <code>SingleColumnRestriction.EQ</code> instance for the specified clustering column.
      *
-     * @param cfMetaData the column family meta data
+     * @param tableMetadata the column family meta data
      * @param index the clustering column index
      * @param value the equality value
      * @return a new <code>SingleColumnRestriction.EQ</code> instance for the specified clustering column
      */
-    private static Restriction newSingleEq(CFMetaData cfMetaData, int index, ByteBuffer value)
+    private static Restriction newSingleEq(TableMetadata tableMetadata, int index, ByteBuffer value)
     {
-        ColumnDefinition columnDef = getClusteringColumnDefinition(cfMetaData, index);
+        ColumnMetadata columnDef = getClusteringColumnDefinition(tableMetadata, index);
         return new SingleColumnRestriction.EQRestriction(columnDef, toTerm(value));
     }
 
     /**
      * Creates a new <code>MultiColumnRestriction.EQ</code> instance for the specified clustering column.
      *
-     * @param cfMetaData the column family meta data
-     * @param index the clustering column index
-     * @param value the equality value
+     * @param tableMetadata the column family meta data
+     * @param firstIndex the clustering column index
+     * @param values the equality value
      * @return a new <code>MultiColumnRestriction.EQ</code> instance for the specified clustering column
      */
-    private static Restriction newMultiEq(CFMetaData cfMetaData, int firstIndex, ByteBuffer... values)
+    private static Restriction newMultiEq(TableMetadata tableMetadata, int firstIndex, ByteBuffer... values)
     {
-        List<ColumnDefinition> columnDefinitions = new ArrayList<>();
+        List<ColumnMetadata> columnMetadatas = new ArrayList<>();
         for (int i = 0; i < values.length; i++)
         {
-            columnDefinitions.add(getClusteringColumnDefinition(cfMetaData, firstIndex + i));
+            columnMetadatas.add(getClusteringColumnDefinition(tableMetadata, firstIndex + i));
         }
-        return new MultiColumnRestriction.EQRestriction(columnDefinitions, toMultiItemTerminal(values));
+        return new MultiColumnRestriction.EQRestriction(columnMetadatas, toMultiItemTerminal(values));
     }
 
     /**
      * Creates a new <code>MultiColumnRestriction.IN</code> instance for the specified clustering column.
      *
-     * @param cfMetaData the column family meta data
+     * @param tableMetadata the column family meta data
      * @param firstIndex the index of the first clustering column
      * @param values the in values
      * @return a new <code>MultiColumnRestriction.IN</code> instance for the specified clustering column
      */
     @SafeVarargs
-    private static Restriction newMultiIN(CFMetaData cfMetaData, int firstIndex, List<ByteBuffer>... values)
+    private static Restriction newMultiIN(TableMetadata tableMetadata, int firstIndex, List<ByteBuffer>... values)
     {
-        List<ColumnDefinition> columnDefinitions = new ArrayList<>();
+        List<ColumnMetadata> columnMetadatas = new ArrayList<>();
         List<Term> terms = new ArrayList<>();
         for (int i = 0; i < values.length; i++)
         {
-            columnDefinitions.add(getClusteringColumnDefinition(cfMetaData, firstIndex + i));
+            columnMetadatas.add(getClusteringColumnDefinition(tableMetadata, firstIndex + i));
             terms.add(toMultiItemTerminal(values[i].toArray(new ByteBuffer[0])));
         }
-        return new MultiColumnRestriction.InRestrictionWithValues(columnDefinitions, terms);
+        return new MultiColumnRestriction.InRestrictionWithValues(columnMetadatas, terms);
     }
 
     /**
      * Creates a new <code>SingleColumnRestriction.IN</code> instance for the specified clustering column.
      *
-     * @param cfMetaData the column family meta data
+     * @param tableMetadata the column family meta data
      * @param index the clustering column index
      * @param values the in values
      * @return a new <code>SingleColumnRestriction.IN</code> instance for the specified clustering column
      */
-    private static Restriction newSingleIN(CFMetaData cfMetaData, int index, ByteBuffer... values)
+    private static Restriction newSingleIN(TableMetadata tableMetadata, int index, ByteBuffer... values)
     {
-        ColumnDefinition columnDef = getClusteringColumnDefinition(cfMetaData, index);
+        ColumnMetadata columnDef = getClusteringColumnDefinition(tableMetadata, index);
         return new SingleColumnRestriction.InRestrictionWithValues(columnDef, toTerms(values));
     }
 
     /**
-     * Returns the clustering <code>ColumnDefinition</code> for the specified position.
+     * Returns the clustering <code>ColumnMetadata</code> for the specified position.
      *
-     * @param cfMetaData the column family meta data
+     * @param tableMetadata the column family meta data
      * @param index the clustering column index
-     * @return the clustering <code>ColumnDefinition</code> for the specified position.
+     * @return the clustering <code>ColumnMetadata</code> for the specified position.
      */
-    private static ColumnDefinition getClusteringColumnDefinition(CFMetaData cfMetaData, int index)
+    private static ColumnMetadata getClusteringColumnDefinition(TableMetadata tableMetadata, int index)
     {
-        return cfMetaData.clusteringColumns().get(index);
+        return tableMetadata.clusteringColumns().get(index);
     }
 
     /**
      * Creates a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column.
      *
-     * @param cfMetaData the column family meta data
+     * @param tableMetadata the column family meta data
      * @param index the clustering column index
      * @param bound the slice bound
      * @param inclusive <code>true</code> if the bound is inclusive
      * @param value the bound value
      * @return a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column
      */
-    private static Restriction newSingleSlice(CFMetaData cfMetaData, int index, Bound bound, boolean inclusive, ByteBuffer value)
+    private static Restriction newSingleSlice(TableMetadata tableMetadata, int index, Bound bound, boolean inclusive, ByteBuffer value)
     {
-        ColumnDefinition columnDef = getClusteringColumnDefinition(cfMetaData, index);
+        ColumnMetadata columnDef = getClusteringColumnDefinition(tableMetadata, index);
         return new SingleColumnRestriction.SliceRestriction(columnDef, bound, inclusive, toTerm(value));
     }
 
     /**
      * Creates a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column.
      *
-     * @param cfMetaData the column family meta data
-     * @param index the clustering column index
+     * @param tableMetadata the column family meta data
+     * @param firstIndex the clustering column index
      * @param bound the slice bound
      * @param inclusive <code>true</code> if the bound is inclusive
-     * @param value the bound value
+     * @param values the bound value
      * @return a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column
      */
-    private static Restriction newMultiSlice(CFMetaData cfMetaData, int firstIndex, Bound bound, boolean inclusive, ByteBuffer... values)
+    private static Restriction newMultiSlice(TableMetadata tableMetadata, int firstIndex, Bound bound, boolean inclusive, ByteBuffer... values)
     {
-        List<ColumnDefinition> columnDefinitions = new ArrayList<>();
+        List<ColumnMetadata> columnMetadatas = new ArrayList<>();
         for (int i = 0; i < values.length; i++)
         {
-            columnDefinitions.add(getClusteringColumnDefinition(cfMetaData, i + firstIndex));
+            columnMetadatas.add(getClusteringColumnDefinition(tableMetadata, i + firstIndex));
         }
-        return new MultiColumnRestriction.SliceRestriction(columnDefinitions, bound, inclusive, toMultiItemTerminal(values));
+        return new MultiColumnRestriction.SliceRestriction(columnMetadatas, bound, inclusive, toMultiItemTerminal(values));
     }
 
     /**


[32/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java
index 401442f..078438b 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@ -25,8 +25,6 @@ import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.Clustering;
@@ -37,6 +35,8 @@ import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -45,28 +45,22 @@ public abstract class Selection
     /**
      * A predicate that returns <code>true</code> for static columns.
      */
-    private static final Predicate<ColumnDefinition> STATIC_COLUMN_FILTER = new Predicate<ColumnDefinition>()
-    {
-        public boolean apply(ColumnDefinition def)
-        {
-            return def.isStatic();
-        }
-    };
+    private static final Predicate<ColumnMetadata> STATIC_COLUMN_FILTER = (column) -> column.isStatic();
 
-    private final CFMetaData cfm;
-    private final List<ColumnDefinition> columns;
+    private final TableMetadata table;
+    private final List<ColumnMetadata> columns;
     private final SelectionColumnMapping columnMapping;
     private final ResultSet.ResultMetadata metadata;
     private final boolean collectTimestamps;
     private final boolean collectTTLs;
 
-    protected Selection(CFMetaData cfm,
-                        List<ColumnDefinition> columns,
+    protected Selection(TableMetadata table,
+                        List<ColumnMetadata> columns,
                         SelectionColumnMapping columnMapping,
                         boolean collectTimestamps,
                         boolean collectTTLs)
     {
-        this.cfm = cfm;
+        this.table = table;
         this.columns = columns;
         this.columnMapping = columnMapping;
         this.metadata = new ResultSet.ResultMetadata(columnMapping.getColumnSpecifications());
@@ -86,7 +80,7 @@ public abstract class Selection
      */
     public boolean containsStaticColumns()
     {
-        if (!cfm.hasStaticColumns())
+        if (!table.hasStaticColumns())
             return false;
 
         if (isWildcard())
@@ -107,7 +101,7 @@ public abstract class Selection
         if (isWildcard())
             return false;
 
-        for (ColumnDefinition def : getColumns())
+        for (ColumnMetadata def : getColumns())
         {
             if (!def.isPartitionKey() && !def.isStatic())
                 return false;
@@ -126,19 +120,19 @@ public abstract class Selection
         return new ResultSet.ResultMetadata(Arrays.asList(jsonSpec));
     }
 
-    public static Selection wildcard(CFMetaData cfm)
+    public static Selection wildcard(TableMetadata table)
     {
-        List<ColumnDefinition> all = new ArrayList<>(cfm.allColumns().size());
-        Iterators.addAll(all, cfm.allColumnsInSelectOrder());
-        return new SimpleSelection(cfm, all, true);
+        List<ColumnMetadata> all = new ArrayList<>(table.columns().size());
+        Iterators.addAll(all, table.allColumnsInSelectOrder());
+        return new SimpleSelection(table, all, true);
     }
 
-    public static Selection forColumns(CFMetaData cfm, List<ColumnDefinition> columns)
+    public static Selection forColumns(TableMetadata table, List<ColumnMetadata> columns)
     {
-        return new SimpleSelection(cfm, columns, false);
+        return new SimpleSelection(table, columns, false);
     }
 
-    public int addColumnForOrdering(ColumnDefinition c)
+    public int addColumnForOrdering(ColumnMetadata c)
     {
         columns.add(c);
         metadata.addNonSerializedColumn(c);
@@ -159,17 +153,17 @@ public abstract class Selection
         return false;
     }
 
-    public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> rawSelectors, VariableSpecifications boundNames, boolean hasGroupBy)
+    public static Selection fromSelectors(TableMetadata table, List<RawSelector> rawSelectors, VariableSpecifications boundNames, boolean hasGroupBy)
     {
-        List<ColumnDefinition> defs = new ArrayList<>();
+        List<ColumnMetadata> defs = new ArrayList<>();
 
         SelectorFactories factories =
-                SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, cfm), null, cfm, defs, boundNames);
-        SelectionColumnMapping mapping = collectColumnMappings(cfm, rawSelectors, factories);
+                SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, table), null, table, defs, boundNames);
+        SelectionColumnMapping mapping = collectColumnMappings(table, rawSelectors, factories);
 
         return (processesSelection(rawSelectors) || rawSelectors.size() != defs.size() || hasGroupBy)
-               ? new SelectionWithProcessing(cfm, defs, mapping, factories)
-               : new SimpleSelection(cfm, defs, mapping, false);
+               ? new SelectionWithProcessing(table, defs, mapping, factories)
+               : new SimpleSelection(table, defs, mapping, false);
     }
 
     /**
@@ -177,7 +171,7 @@ public abstract class Selection
      * @param c the column
      * @return the index of the specified column within the resultset or -1
      */
-    public int getResultSetIndex(ColumnDefinition c)
+    public int getResultSetIndex(ColumnMetadata c)
     {
         return getColumnIndex(c);
     }
@@ -187,7 +181,7 @@ public abstract class Selection
      * @param c the column
      * @return the index of the specified column or -1
      */
-    protected final int getColumnIndex(ColumnDefinition c)
+    protected final int getColumnIndex(ColumnMetadata c)
     {
         for (int i = 0, m = columns.size(); i < m; i++)
             if (columns.get(i).name.equals(c.name))
@@ -195,7 +189,7 @@ public abstract class Selection
         return -1;
     }
 
-    private static SelectionColumnMapping collectColumnMappings(CFMetaData cfm,
+    private static SelectionColumnMapping collectColumnMappings(TableMetadata table,
                                                                 List<RawSelector> rawSelectors,
                                                                 SelectorFactories factories)
     {
@@ -203,7 +197,7 @@ public abstract class Selection
         Iterator<RawSelector> iter = rawSelectors.iterator();
         for (Selector.Factory factory : factories)
         {
-            ColumnSpecification colSpec = factory.getColumnSpecification(cfm);
+            ColumnSpecification colSpec = factory.getColumnSpecification(table);
             ColumnIdentifier alias = iter.next().alias;
             factory.addColumnMapping(selectionColumns,
                                      alias == null ? colSpec : colSpec.withAlias(alias));
@@ -216,7 +210,7 @@ public abstract class Selection
     /**
      * @return the list of CQL3 columns value this SelectionClause needs.
      */
-    public List<ColumnDefinition> getColumns()
+    public List<ColumnMetadata> getColumns()
     {
         return columns;
     }
@@ -442,13 +436,13 @@ public abstract class Selection
     {
         private final boolean isWildcard;
 
-        public SimpleSelection(CFMetaData cfm, List<ColumnDefinition> columns, boolean isWildcard)
+        public SimpleSelection(TableMetadata table, List<ColumnMetadata> columns, boolean isWildcard)
         {
-            this(cfm, columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
+            this(table, columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
         }
 
-        public SimpleSelection(CFMetaData cfm,
-                               List<ColumnDefinition> columns,
+        public SimpleSelection(TableMetadata table,
+                               List<ColumnMetadata> columns,
                                SelectionColumnMapping metadata,
                                boolean isWildcard)
         {
@@ -457,7 +451,7 @@ public abstract class Selection
              * could filter those duplicate out of columns. But since we're very unlikely to
              * get much duplicate in practice, it's more efficient not to bother.
              */
-            super(cfm, columns, metadata, false, false);
+            super(table, columns, metadata, false, false);
             this.isWildcard = isWildcard;
         }
 
@@ -505,12 +499,12 @@ public abstract class Selection
     {
         private final SelectorFactories factories;
 
-        public SelectionWithProcessing(CFMetaData cfm,
-                                       List<ColumnDefinition> columns,
+        public SelectionWithProcessing(TableMetadata table,
+                                       List<ColumnMetadata> columns,
                                        SelectionColumnMapping metadata,
                                        SelectorFactories factories) throws InvalidRequestException
         {
-            super(cfm,
+            super(table,
                   columns,
                   metadata,
                   factories.containsWritetimeSelectorFactory(),
@@ -526,7 +520,7 @@ public abstract class Selection
         }
 
         @Override
-        public int getResultSetIndex(ColumnDefinition c)
+        public int getResultSetIndex(ColumnMetadata c)
         {
             int index = getColumnIndex(c);
 
@@ -541,7 +535,7 @@ public abstract class Selection
         }
 
         @Override
-        public int addColumnForOrdering(ColumnDefinition c)
+        public int addColumnForOrdering(ColumnMetadata c)
         {
             int index = super.addColumnForOrdering(c);
             factories.addSelectorForOrdering(c, index);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java b/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
index 5072066..cd04d94 100644
--- a/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
@@ -26,7 +26,7 @@ import java.util.stream.Collectors;
 import com.google.common.base.Objects;
 import com.google.common.collect.*;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.ColumnSpecification;
 
 /**
@@ -38,7 +38,7 @@ import org.apache.cassandra.cql3.ColumnSpecification;
 public class SelectionColumnMapping implements SelectionColumns
 {
     private final ArrayList<ColumnSpecification> columnSpecifications;
-    private final HashMultimap<ColumnSpecification, ColumnDefinition> columnMappings;
+    private final HashMultimap<ColumnSpecification, ColumnMetadata> columnMappings;
 
     private SelectionColumnMapping()
     {
@@ -51,15 +51,15 @@ public class SelectionColumnMapping implements SelectionColumns
         return new SelectionColumnMapping();
     }
 
-    protected static SelectionColumnMapping simpleMapping(Iterable<ColumnDefinition> columnDefinitions)
+    protected static SelectionColumnMapping simpleMapping(Iterable<ColumnMetadata> columnDefinitions)
     {
         SelectionColumnMapping mapping = new SelectionColumnMapping();
-        for (ColumnDefinition def: columnDefinitions)
+        for (ColumnMetadata def: columnDefinitions)
             mapping.addMapping(def, def);
         return mapping;
     }
 
-    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, ColumnDefinition column)
+    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, ColumnMetadata column)
     {
         columnSpecifications.add(colSpec);
         // functions without arguments do not map to any column, so don't
@@ -69,7 +69,7 @@ public class SelectionColumnMapping implements SelectionColumns
         return this;
     }
 
-    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<ColumnDefinition> columns)
+    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<ColumnMetadata> columns)
     {
         columnSpecifications.add(colSpec);
         columnMappings.putAll(colSpec, columns);
@@ -83,7 +83,7 @@ public class SelectionColumnMapping implements SelectionColumns
         return Lists.newArrayList(columnSpecifications);
     }
 
-    public Multimap<ColumnSpecification, ColumnDefinition> getMappings()
+    public Multimap<ColumnSpecification, ColumnMetadata> getMappings()
     {
         return Multimaps.unmodifiableMultimap(columnMappings);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java b/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
index 151a2f3..f4a8593 100644
--- a/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
@@ -24,7 +24,7 @@ import java.util.List;
 
 import com.google.common.collect.Multimap;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.ColumnSpecification;
 
 /**
@@ -34,5 +34,5 @@ import org.apache.cassandra.cql3.ColumnSpecification;
 public interface SelectionColumns
 {
     List<ColumnSpecification> getColumnSpecifications();
-    Multimap<ColumnSpecification, ColumnDefinition> getMappings();
+    Multimap<ColumnSpecification, ColumnMetadata> getMappings();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/Selector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selector.java b/src/java/org/apache/cassandra/cql3/selection/Selector.java
index 6f83dfc..420af9c 100644
--- a/src/java/org/apache/cassandra/cql3/selection/Selector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/Selector.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.cql3.selection;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -51,13 +51,13 @@ public abstract class Selector
          * Returns the column specification corresponding to the output value of the selector instances created by
          * this factory.
          *
-         * @param cfm the column family meta data
+         * @param table the table meta data
          * @return a column specification
          */
-        public final ColumnSpecification getColumnSpecification(CFMetaData cfm)
+        public final ColumnSpecification getColumnSpecification(TableMetadata table)
         {
-            return new ColumnSpecification(cfm.ksName,
-                                           cfm.cfName,
+            return new ColumnSpecification(table.keyspace,
+                                           table.name,
                                            new ColumnIdentifier(getColumnName(), true), // note that the name is not necessarily
                                                                                         // a true column name so we shouldn't intern it
                                            getReturnType());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
index 41bf193..25a1059 100644
--- a/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
@@ -21,8 +21,8 @@ import java.util.*;
 
 import com.google.common.collect.Lists;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.VariableSpecifications;
 import org.apache.cassandra.cql3.functions.Function;
@@ -63,7 +63,7 @@ final class SelectorFactories implements Iterable<Selector.Factory>
      * is any such expectations, or {@code null} otherwise. This will be {@code null} when called on
      * the top-level selectables, but may not be for selectable nested within a function for instance
      * (as the argument selectable will be expected to be of the type expected by the function).
-     * @param cfm the Column Family Definition
+     * @param table the table Definition
      * @param defs the collector parameter for the column definitions
      * @param boundNames the collector for the specification of bound markers in the selection
      * @return a new <code>SelectorFactories</code> instance
@@ -71,18 +71,18 @@ final class SelectorFactories implements Iterable<Selector.Factory>
      */
     public static SelectorFactories createFactoriesAndCollectColumnDefinitions(List<Selectable> selectables,
                                                                                List<AbstractType<?>> expectedTypes,
-                                                                               CFMetaData cfm,
-                                                                               List<ColumnDefinition> defs,
+                                                                               TableMetadata table,
+                                                                               List<ColumnMetadata> defs,
                                                                                VariableSpecifications boundNames)
                                                                                throws InvalidRequestException
     {
-        return new SelectorFactories(selectables, expectedTypes, cfm, defs, boundNames);
+        return new SelectorFactories(selectables, expectedTypes, table, defs, boundNames);
     }
 
     private SelectorFactories(List<Selectable> selectables,
                               List<AbstractType<?>> expectedTypes,
-                              CFMetaData cfm,
-                              List<ColumnDefinition> defs,
+                              TableMetadata table,
+                              List<ColumnMetadata> defs,
                               VariableSpecifications boundNames)
                               throws InvalidRequestException
     {
@@ -92,7 +92,7 @@ final class SelectorFactories implements Iterable<Selector.Factory>
         {
             Selectable selectable = selectables.get(i);
             AbstractType<?> expectedType = expectedTypes == null ? null : expectedTypes.get(i);
-            Factory factory = selectable.newSelectorFactory(cfm, expectedType, defs, boundNames);
+            Factory factory = selectable.newSelectorFactory(table, expectedType, defs, boundNames);
             containsWritetimeFactory |= factory.isWritetimeSelectorFactory();
             containsTTLFactory |= factory.isTTLSelectorFactory();
             if (factory.isAggregateSelectorFactory())
@@ -122,7 +122,7 @@ final class SelectorFactories implements Iterable<Selector.Factory>
      * @param def the column that is needed for ordering
      * @param index the index of the column definition in the Selection's list of columns
      */
-    public void addSelectorForOrdering(ColumnDefinition def, int index)
+    public void addSelectorForOrdering(ColumnMetadata def, int index)
     {
         factories.add(SimpleSelector.newFactory(def, index));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/SetSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SetSelector.java b/src/java/org/apache/cassandra/cql3/selection/SetSelector.java
index 34de078..2ee086e 100644
--- a/src/java/org/apache/cassandra/cql3/selection/SetSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SetSelector.java
@@ -22,8 +22,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.Sets;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java b/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
index 8d5a305..cbd65a9 100644
--- a/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.cql3.selection;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
@@ -35,7 +35,7 @@ public final class SimpleSelector extends Selector
     private ByteBuffer current;
     private boolean isSet;
 
-    public static Factory newFactory(final ColumnDefinition def, final int idx)
+    public static Factory newFactory(final ColumnMetadata def, final int idx)
     {
         return new Factory()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/TermSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/TermSelector.java b/src/java/org/apache/cassandra/cql3/selection/TermSelector.java
index 2b0e975..bdb4953 100644
--- a/src/java/org/apache/cassandra/cql3/selection/TermSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/TermSelector.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.cql3.selection;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.Term;
@@ -54,7 +54,7 @@ public class TermSelector extends Selector
 
             protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultColumn)
             {
-               mapping.addMapping(resultColumn, (ColumnDefinition)null);
+               mapping.addMapping(resultColumn, (ColumnMetadata)null);
             }
 
             public Selector newInstance(QueryOptions options)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java b/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java
index 3c298b5..7600e1d 100644
--- a/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/UserTypeSelector.java
@@ -23,7 +23,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.FieldIdentifier;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -76,7 +76,7 @@ final class UserTypeSelector extends Selector
 
                 if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
                     // add a null mapping for cases where the collection is empty
-                    mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+                    mapping.addMapping(resultsColumn, (ColumnMetadata)null);
                 else
                     // collate the mapped columns from the child factories & add those
                     mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java b/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
index 939f8c2..1e38337 100644
--- a/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
+++ b/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.cql3.selection;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
@@ -37,7 +37,7 @@ final class WritetimeOrTTLSelector extends Selector
     private ByteBuffer current;
     private boolean isSet;
 
-    public static Factory newFactory(final ColumnDefinition def, final int idx, final boolean isWritetime)
+    public static Factory newFactory(final ColumnMetadata def, final int idx, final boolean isWritetime)
     {
         return new Factory()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
index f3df6f2..68700f2 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
@@ -18,14 +18,14 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.locator.LocalStrategy;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -54,7 +54,7 @@ public class AlterKeyspaceStatement extends SchemaAlteringStatement
 
     public void validate(ClientState state) throws RequestValidationException
     {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(name);
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name);
         if (ksm == null)
             throw new InvalidRequestException("Unknown keyspace " + name);
         if (SchemaConstants.isSystemKeyspace(ksm.name))
@@ -79,7 +79,7 @@ public class AlterKeyspaceStatement extends SchemaAlteringStatement
 
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
     {
-        KeyspaceMetadata oldKsm = Schema.instance.getKSMetaData(name);
+        KeyspaceMetadata oldKsm = Schema.instance.getKeyspaceMetadata(name);
         // In the (very) unlikely case the keyspace was dropped since validate()
         if (oldKsm == null)
             throw new InvalidRequestException("Unknown keyspace " + name);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index f2f0698..35459de 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -23,7 +23,6 @@ import java.util.stream.Collectors;
 import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -31,11 +30,16 @@ import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.DroppedColumn;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.schema.Indexes;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TableParams;
+import org.apache.cassandra.schema.ViewMetadata;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -48,7 +52,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
     public final Type oType;
     private final TableAttributes attrs;
-    private final Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames;
+    private final Map<ColumnMetadata.Raw, ColumnMetadata.Raw> renames;
     private final List<AlterTableStatementColumn> colNameList;
     private final Long deleteTimestamp;
 
@@ -56,7 +60,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                                Type type,
                                List<AlterTableStatementColumn> colDataList,
                                TableAttributes attrs,
-                               Map<ColumnDefinition.Raw, ColumnDefinition.Raw> renames,
+                               Map<ColumnMetadata.Raw, ColumnMetadata.Raw> renames,
                                Long deleteTimestamp)
     {
         super(name);
@@ -79,32 +83,33 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
     {
-        CFMetaData meta = Validation.validateColumnFamily(keyspace(), columnFamily());
-        if (meta.isView())
+        TableMetadata current = Schema.instance.validateTable(keyspace(), columnFamily());
+        if (current.isView())
             throw new InvalidRequestException("Cannot use ALTER TABLE on Materialized View");
 
-        CFMetaData cfm = meta.copy();
+        TableMetadata.Builder builder = current.unbuild();
+
         ColumnIdentifier columnName = null;
-        ColumnDefinition def = null;
+        ColumnMetadata def = null;
         CQL3Type.Raw dataType = null;
         boolean isStatic = false;
         CQL3Type validator = null;
 
-        List<ViewDefinition> viewUpdates = null;
-        Iterable<ViewDefinition> views = View.findAll(keyspace(), columnFamily());
+        List<ViewMetadata> viewUpdates = new ArrayList<>();
+        Iterable<ViewMetadata> views = View.findAll(keyspace(), columnFamily());
 
         switch (oType)
         {
             case ALTER:
                 throw new InvalidRequestException("Altering of types is not allowed");
             case ADD:
-                if (cfm.isDense())
+                if (current.isDense())
                     throw new InvalidRequestException("Cannot add new column to a COMPACT STORAGE table");
 
                 for (AlterTableStatementColumn colData : colNameList)
                 {
-                    columnName = colData.getColumnName().getIdentifier(cfm);
-                    def = cfm.getColumnDefinition(columnName);
+                    columnName = colData.getColumnName().getIdentifier(current);
+                    def = builder.getColumn(columnName);
                     dataType = colData.getColumnType();
                     assert dataType != null;
                     isStatic = colData.getStaticType();
@@ -113,9 +118,9 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
                     if (isStatic)
                     {
-                        if (!cfm.isCompound())
+                        if (!current.isCompound())
                             throw new InvalidRequestException("Static columns are not allowed in COMPACT STORAGE tables");
-                        if (cfm.clusteringColumns().isEmpty())
+                        if (current.clusteringColumns().isEmpty())
                             throw new InvalidRequestException("Static columns are only useful (and thus allowed) if the table has at least one clustering column");
                     }
 
@@ -132,64 +137,54 @@ public class AlterTableStatement extends SchemaAlteringStatement
                     }
 
                     // Cannot re-add a dropped counter column. See #7831.
-                    if (meta.isCounter() && meta.getDroppedColumns().containsKey(columnName.bytes))
+                    if (current.isCounter() && current.getDroppedColumn(columnName.bytes) != null)
                         throw new InvalidRequestException(String.format("Cannot re-add previously dropped counter column %s", columnName));
 
                     AbstractType<?> type = validator.getType();
                     if (type.isCollection() && type.isMultiCell())
                     {
-                        if (!cfm.isCompound())
+                        if (!current.isCompound())
                             throw new InvalidRequestException("Cannot use non-frozen collections in COMPACT STORAGE tables");
-                        if (cfm.isSuper())
+                        if (current.isSuper())
                             throw new InvalidRequestException("Cannot use non-frozen collections with super column families");
 
                         // If there used to be a non-frozen collection column with the same name (that has been dropped),
                         // we could still have some data using the old type, and so we can't allow adding a collection
                         // with the same name unless the types are compatible (see #6276).
-                        CFMetaData.DroppedColumn dropped = cfm.getDroppedColumns().get(columnName.bytes);
-                        if (dropped != null && dropped.type instanceof CollectionType
-                            && dropped.type.isMultiCell() && !type.isCompatibleWith(dropped.type))
+                        DroppedColumn dropped = current.droppedColumns.get(columnName.bytes);
+                        if (dropped != null && dropped.column.type instanceof CollectionType
+                            && dropped.column.type.isMultiCell() && !type.isCompatibleWith(dropped.column.type))
                         {
                             String message =
                                 String.format("Cannot add a collection with the name %s because a collection with the same name"
                                               + " and a different type (%s) has already been used in the past",
                                               columnName,
-                                              dropped.type.asCQL3Type());
+                                              dropped.column.type.asCQL3Type());
                             throw new InvalidRequestException(message);
                         }
                     }
 
-                    cfm.addColumnDefinition(isStatic
-                                            ? ColumnDefinition.staticDef(cfm, columnName.bytes, type)
-                                            : ColumnDefinition.regularDef(cfm, columnName.bytes, type));
+                    builder.addColumn(isStatic
+                                    ? ColumnMetadata.staticColumn(current, columnName.bytes, type)
+                                    : ColumnMetadata.regularColumn(current, columnName.bytes, type));
 
                     // Adding a column to a table which has an include all view requires the column to be added to the view
                     // as well
                     if (!isStatic)
-                    {
-                        for (ViewDefinition view : views)
-                        {
+                        for (ViewMetadata view : views)
                             if (view.includeAllColumns)
-                            {
-                                ViewDefinition viewCopy = view.copy();
-                                viewCopy.metadata.addColumnDefinition(ColumnDefinition.regularDef(viewCopy.metadata, columnName.bytes, type));
-                                if (viewUpdates == null)
-                                    viewUpdates = new ArrayList<>();
-                                viewUpdates.add(viewCopy);
-                            }
-                        }
-                    }
+                                viewUpdates.add(view.withAddedRegularColumn(ColumnMetadata.regularColumn(view.metadata, columnName.bytes, type)));
+
                 }
                 break;
-
             case DROP:
-                if (!cfm.isCQLTable())
+                if (!current.isCQLTable())
                     throw new InvalidRequestException("Cannot drop columns from a non-CQL3 table");
 
                 for (AlterTableStatementColumn colData : colNameList)
                 {
-                    columnName = colData.getColumnName().getIdentifier(cfm);
-                    def = cfm.getColumnDefinition(columnName);
+                    columnName = colData.getColumnName().getIdentifier(current);
+                    def = builder.getColumn(columnName);
 
                     if (def == null)
                         throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
@@ -201,53 +196,45 @@ public class AlterTableStatement extends SchemaAlteringStatement
                               throw new InvalidRequestException(String.format("Cannot drop PRIMARY KEY part %s", columnName));
                          case REGULAR:
                          case STATIC:
-                              ColumnDefinition toDelete = null;
-                              for (ColumnDefinition columnDef : cfm.partitionColumns())
-                              {
-                                   if (columnDef.name.equals(columnName))
-                                   {
-                                       toDelete = columnDef;
-                                       break;
-                                   }
-                               }
-                             assert toDelete != null;
-                             cfm.removeColumnDefinition(toDelete);
-                             cfm.recordColumnDrop(toDelete, deleteTimestamp  == null ? queryState.getTimestamp() : deleteTimestamp);
+                             builder.removeRegularOrStaticColumn(def.name);
+                             builder.recordColumnDrop(def, deleteTimestamp  == null ? queryState.getTimestamp() : deleteTimestamp);
                              break;
                     }
 
                     // If the dropped column is required by any secondary indexes
                     // we reject the operation, as the indexes must be dropped first
-                    Indexes allIndexes = cfm.getIndexes();
+                    Indexes allIndexes = current.indexes;
                     if (!allIndexes.isEmpty())
                     {
-                        ColumnFamilyStore store = Keyspace.openAndGetStore(cfm);
+                        ColumnFamilyStore store = Keyspace.openAndGetStore(current);
                         Set<IndexMetadata> dependentIndexes = store.indexManager.getDependentIndexes(def);
                         if (!dependentIndexes.isEmpty())
+                        {
                             throw new InvalidRequestException(String.format("Cannot drop column %s because it has " +
                                                                             "dependent secondary indexes (%s)",
                                                                             def,
                                                                             dependentIndexes.stream()
                                                                                             .map(i -> i.name)
                                                                                             .collect(Collectors.joining(","))));
+                        }
                     }
 
                     // If a column is dropped which is included in a view, we don't allow the drop to take place.
                     boolean rejectAlter = false;
-                    StringBuilder builder = new StringBuilder();
-                    for (ViewDefinition view : views)
+                    StringBuilder viewNames = new StringBuilder();
+                    for (ViewMetadata view : views)
                     {
                         if (!view.includes(columnName)) continue;
                         if (rejectAlter)
-                            builder.append(',');
+                            viewNames.append(',');
                         rejectAlter = true;
-                        builder.append(view.viewName);
+                        viewNames.append(view.name);
                     }
                     if (rejectAlter)
                         throw new InvalidRequestException(String.format("Cannot drop column %s, depended on by materialized views (%s.{%s})",
                                                                         columnName.toString(),
                                                                         keyspace(),
-                                                                        builder.toString()));
+                                                                        viewNames.toString()));
                 }
                 break;
             case OPTS:
@@ -255,7 +242,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                     throw new InvalidRequestException("ALTER TABLE WITH invoked, but no parameters found");
                 attrs.validate();
 
-                TableParams params = attrs.asAlteredTableParams(cfm.params);
+                TableParams params = attrs.asAlteredTableParams(current.params);
 
                 if (!Iterables.isEmpty(views) && params.gcGraceSeconds == 0)
                 {
@@ -266,44 +253,62 @@ public class AlterTableStatement extends SchemaAlteringStatement
                                                       "before being replayed.");
                 }
 
-                if (meta.isCounter() && params.defaultTimeToLive > 0)
+                if (current.isCounter() && params.defaultTimeToLive > 0)
                     throw new InvalidRequestException("Cannot set default_time_to_live on a table with counters");
 
-                cfm.params(params);
+                builder.params(params);
 
                 break;
             case RENAME:
-                for (Map.Entry<ColumnDefinition.Raw, ColumnDefinition.Raw> entry : renames.entrySet())
+                for (Map.Entry<ColumnMetadata.Raw, ColumnMetadata.Raw> entry : renames.entrySet())
                 {
-                    ColumnIdentifier from = entry.getKey().getIdentifier(cfm);
-                    ColumnIdentifier to = entry.getValue().getIdentifier(cfm);
-                    cfm.renameColumn(from, to);
+                    ColumnIdentifier from = entry.getKey().getIdentifier(current);
+                    ColumnIdentifier to = entry.getValue().getIdentifier(current);
 
-                    // If the view includes a renamed column, it must be renamed in the view table and the definition.
-                    for (ViewDefinition view : views)
+                    def = current.getColumn(from);
+                    if (def == null)
+                        throw new InvalidRequestException(String.format("Cannot rename unknown column %s in table %s", from, current.name));
+
+                    if (current.getColumn(to) != null)
+                        throw new InvalidRequestException(String.format("Cannot rename column %s to %s in table %s; another column of that name already exist", from, to, current.name));
+
+                    if (!def.isPrimaryKeyColumn())
+                        throw new InvalidRequestException(String.format("Cannot rename non PRIMARY KEY part %s", from));
+
+                    if (!current.indexes.isEmpty())
                     {
-                        if (!view.includes(from)) continue;
+                        ColumnFamilyStore store = Keyspace.openAndGetStore(current);
+                        Set<IndexMetadata> dependentIndexes = store.indexManager.getDependentIndexes(def);
+                        if (!dependentIndexes.isEmpty())
+                            throw new InvalidRequestException(String.format("Cannot rename column %s because it has " +
+                                                                            "dependent secondary indexes (%s)",
+                                                                            from,
+                                                                            dependentIndexes.stream()
+                                                                                            .map(i -> i.name)
+                                                                                            .collect(Collectors.joining(","))));
+                    }
+
+                    builder.renamePrimaryKeyColumn(from, to);
 
-                        ViewDefinition viewCopy = view.copy();
-                        ColumnIdentifier viewFrom = entry.getKey().getIdentifier(viewCopy.metadata);
-                        ColumnIdentifier viewTo = entry.getValue().getIdentifier(viewCopy.metadata);
-                        viewCopy.renameColumn(viewFrom, viewTo);
+                    // If the view includes a renamed column, it must be renamed in the view table and the definition.
+                    for (ViewMetadata view : views)
+                    {
+                        if (!view.includes(from))
+                            continue;
 
-                        if (viewUpdates == null)
-                            viewUpdates = new ArrayList<>();
-                        viewUpdates.add(viewCopy);
+                        ColumnIdentifier viewFrom = entry.getKey().getIdentifier(view.metadata);
+                        ColumnIdentifier viewTo = entry.getValue().getIdentifier(view.metadata);
+                        viewUpdates.add(view.renamePrimaryKeyColumn(viewFrom, viewTo));
                     }
                 }
                 break;
         }
 
-        MigrationManager.announceColumnFamilyUpdate(cfm, isLocalOnly);
+        // FIXME: Should really be a single announce for the table and views.
+        MigrationManager.announceTableUpdate(builder.build(), isLocalOnly);
+        for (ViewMetadata viewUpdate : viewUpdates)
+            MigrationManager.announceViewUpdate(viewUpdate, isLocalOnly);
 
-        if (viewUpdates != null)
-        {
-            for (ViewDefinition viewUpdate : viewUpdates)
-                MigrationManager.announceViewUpdate(viewUpdate, isLocalOnly);
-        }
         return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
index 7dea565..c1ba0d3 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatementColumn.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.CQL3Type;
 
 /**
@@ -28,10 +28,10 @@ import org.apache.cassandra.cql3.CQL3Type;
 public class AlterTableStatementColumn
 {
     private final CQL3Type.Raw dataType;
-    private final ColumnDefinition.Raw colName;
+    private final ColumnMetadata.Raw colName;
     private final Boolean isStatic;
 
-    public AlterTableStatementColumn(ColumnDefinition.Raw colName, CQL3Type.Raw dataType, boolean isStatic)
+    public AlterTableStatementColumn(ColumnMetadata.Raw colName, CQL3Type.Raw dataType, boolean isStatic)
     {
         assert colName != null;
         this.dataType = dataType; // will be null when dropping columns, and never null otherwise (for ADD and ALTER).
@@ -39,12 +39,12 @@ public class AlterTableStatementColumn
         this.isStatic = isStatic;
     }
 
-    public AlterTableStatementColumn(ColumnDefinition.Raw colName, CQL3Type.Raw dataType)
+    public AlterTableStatementColumn(ColumnMetadata.Raw colName, CQL3Type.Raw dataType)
     {
         this(colName, dataType, false);
     }
 
-    public AlterTableStatementColumn(ColumnDefinition.Raw colName)
+    public AlterTableStatementColumn(ColumnMetadata.Raw colName)
     {
         this(colName, null, false);
     }
@@ -54,7 +54,7 @@ public class AlterTableStatementColumn
         return dataType;
     }
 
-    public ColumnDefinition.Raw getColumnName()
+    public ColumnMetadata.Raw getColumnName()
     {
         return colName;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
index 71d19fa..614b482 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
@@ -17,17 +17,16 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -86,7 +85,7 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
 
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
     {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(name.getKeyspace());
+        KeyspaceMetadata ksm = Schema.instance.getKeyspaceMetadata(name.getKeyspace());
         if (ksm == null)
             throw new InvalidRequestException(String.format("Cannot alter type in unknown keyspace %s", name.getKeyspace()));
 
@@ -100,134 +99,9 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
         // but we also need to find all existing user types and CF using it and change them.
         MigrationManager.announceTypeUpdate(updated, isLocalOnly);
 
-        for (CFMetaData cfm : ksm.tables)
-        {
-            CFMetaData copy = cfm.copy();
-            boolean modified = false;
-            for (ColumnDefinition def : copy.allColumns())
-                modified |= updateDefinition(copy, def, toUpdate.keyspace, toUpdate.name, updated);
-            if (modified)
-                MigrationManager.announceColumnFamilyUpdate(copy, isLocalOnly);
-        }
-
-        for (ViewDefinition view : ksm.views)
-        {
-            ViewDefinition copy = view.copy();
-            boolean modified = false;
-            for (ColumnDefinition def : copy.metadata.allColumns())
-                modified |= updateDefinition(copy.metadata, def, toUpdate.keyspace, toUpdate.name, updated);
-            if (modified)
-                MigrationManager.announceViewUpdate(copy, isLocalOnly);
-        }
-
-        // Other user types potentially using the updated type
-        for (UserType ut : ksm.types)
-        {
-            // Re-updating the type we've just updated would be harmless but useless so we avoid it.
-            // Besides, we use the occasion to drop the old version of the type if it's a type rename
-            if (ut.keyspace.equals(toUpdate.keyspace) && ut.name.equals(toUpdate.name))
-            {
-                if (!ut.keyspace.equals(updated.keyspace) || !ut.name.equals(updated.name))
-                    MigrationManager.announceTypeDrop(ut);
-                continue;
-            }
-            AbstractType<?> upd = updateWith(ut, toUpdate.keyspace, toUpdate.name, updated);
-            if (upd != null)
-                MigrationManager.announceTypeUpdate((UserType) upd, isLocalOnly);
-        }
         return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TYPE, keyspace(), name.getStringTypeName());
     }
 
-    private boolean updateDefinition(CFMetaData cfm, ColumnDefinition def, String keyspace, ByteBuffer toReplace, UserType updated)
-    {
-        AbstractType<?> t = updateWith(def.type, keyspace, toReplace, updated);
-        if (t == null)
-            return false;
-
-        // We need to update this validator ...
-        cfm.addOrReplaceColumnDefinition(def.withNewType(t));
-        return true;
-    }
-
-    // Update the provided type were all instance of a given userType is replaced by a new version
-    // Note that this methods reaches inside other UserType, CompositeType and CollectionType.
-    private static AbstractType<?> updateWith(AbstractType<?> type, String keyspace, ByteBuffer toReplace, UserType updated)
-    {
-        if (type instanceof UserType)
-        {
-            UserType ut = (UserType)type;
-
-            // If it's directly the type we've updated, then just use the new one.
-            if (keyspace.equals(ut.keyspace) && toReplace.equals(ut.name))
-                return type.isMultiCell() ? updated : updated.freeze();
-
-            // Otherwise, check for nesting
-            List<AbstractType<?>> updatedTypes = updateTypes(ut.fieldTypes(), keyspace, toReplace, updated);
-            return updatedTypes == null ? null : new UserType(ut.keyspace, ut.name, new ArrayList<>(ut.fieldNames()), updatedTypes, type.isMultiCell());
-        }
-        else if (type instanceof TupleType)
-        {
-            TupleType tt = (TupleType)type;
-            List<AbstractType<?>> updatedTypes = updateTypes(tt.allTypes(), keyspace, toReplace, updated);
-            return updatedTypes == null ? null : new TupleType(updatedTypes);
-        }
-        else if (type instanceof CompositeType)
-        {
-            CompositeType ct = (CompositeType)type;
-            List<AbstractType<?>> updatedTypes = updateTypes(ct.types, keyspace, toReplace, updated);
-            return updatedTypes == null ? null : CompositeType.getInstance(updatedTypes);
-        }
-        else if (type instanceof CollectionType)
-        {
-            if (type instanceof ListType)
-            {
-                AbstractType<?> t = updateWith(((ListType)type).getElementsType(), keyspace, toReplace, updated);
-                if (t == null)
-                    return null;
-                return ListType.getInstance(t, type.isMultiCell());
-            }
-            else if (type instanceof SetType)
-            {
-                AbstractType<?> t = updateWith(((SetType)type).getElementsType(), keyspace, toReplace, updated);
-                if (t == null)
-                    return null;
-                return SetType.getInstance(t, type.isMultiCell());
-            }
-            else
-            {
-                assert type instanceof MapType;
-                MapType mt = (MapType)type;
-                AbstractType<?> k = updateWith(mt.getKeysType(), keyspace, toReplace, updated);
-                AbstractType<?> v = updateWith(mt.getValuesType(), keyspace, toReplace, updated);
-                if (k == null && v == null)
-                    return null;
-                return MapType.getInstance(k == null ? mt.getKeysType() : k, v == null ? mt.getValuesType() : v, type.isMultiCell());
-            }
-        }
-        else
-        {
-            return null;
-        }
-    }
-
-    private static List<AbstractType<?>> updateTypes(List<AbstractType<?>> toUpdate, String keyspace, ByteBuffer toReplace, UserType updated)
-    {
-        // But this can also be nested.
-        List<AbstractType<?>> updatedTypes = null;
-        for (int i = 0; i < toUpdate.size(); i++)
-        {
-            AbstractType<?> t = updateWith(toUpdate.get(i), keyspace, toReplace, updated);
-            if (t == null)
-                continue;
-
-            if (updatedTypes == null)
-                updatedTypes = new ArrayList<>(toUpdate);
-
-            updatedTypes.set(i, t);
-        }
-        return updatedTypes;
-    }
-
     protected void checkTypeNotUsedByAggregate(KeyspaceMetadata ksm)
     {
         ksm.functions.udas().filter(aggregate -> aggregate.initialCondition() != null && aggregate.stateType().referencesUserType(name.getStringTypeName()))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java
index e73d9bf..fbfc54c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java
@@ -18,18 +18,18 @@
 package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.cql3.Validation;
 import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.MigrationManager;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.schema.TableParams;
+import org.apache.cassandra.schema.ViewMetadata;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -45,9 +45,9 @@ public class AlterViewStatement extends SchemaAlteringStatement
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
     {
-        CFMetaData baseTable = View.findBaseTable(keyspace(), columnFamily());
+        TableMetadataRef baseTable = View.findBaseTable(keyspace(), columnFamily());
         if (baseTable != null)
-            state.hasColumnFamilyAccess(keyspace(), baseTable.cfName, Permission.ALTER);
+            state.hasColumnFamilyAccess(keyspace(), baseTable.name, Permission.ALTER);
     }
 
     public void validate(ClientState state)
@@ -57,18 +57,18 @@ public class AlterViewStatement extends SchemaAlteringStatement
 
     public Event.SchemaChange announceMigration(QueryState queryState, boolean isLocalOnly) throws RequestValidationException
     {
-        CFMetaData meta = Validation.validateColumnFamily(keyspace(), columnFamily());
+        TableMetadata meta = Schema.instance.validateTable(keyspace(), columnFamily());
         if (!meta.isView())
             throw new InvalidRequestException("Cannot use ALTER MATERIALIZED VIEW on Table");
 
-        ViewDefinition viewCopy = Schema.instance.getView(keyspace(), columnFamily()).copy();
+        ViewMetadata current = Schema.instance.getView(keyspace(), columnFamily());
 
         if (attrs == null)
             throw new InvalidRequestException("ALTER MATERIALIZED VIEW WITH invoked, but no parameters found");
 
         attrs.validate();
 
-        TableParams params = attrs.asAlteredTableParams(viewCopy.metadata.params);
+        TableParams params = attrs.asAlteredTableParams(current.metadata.params);
         if (params.gcGraceSeconds == 0)
         {
             throw new InvalidRequestException("Cannot alter gc_grace_seconds of a materialized view to 0, since this " +
@@ -83,9 +83,9 @@ public class AlterViewStatement extends SchemaAlteringStatement
                                               "the corresponding data in the parent table.");
         }
 
-        viewCopy.metadata.params(params);
+        ViewMetadata updated = current.copy(current.metadata.unbuild().params(params).build());
 
-        MigrationManager.announceViewUpdate(viewCopy, isLocalOnly);
+        MigrationManager.announceViewUpdate(updated, isLocalOnly);
         return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 25e3e22..e181968 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -27,8 +27,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.helpers.MessageFormatter;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.*;
@@ -60,9 +61,9 @@ public class BatchStatement implements CQLStatement
     private final List<ModificationStatement> statements;
 
     // Columns modified for each table (keyed by the table ID)
-    private final Map<UUID, PartitionColumns> updatedColumns;
+    private final Map<TableId, RegularAndStaticColumns> updatedColumns;
     // Columns on which there is conditions. Note that if there is any, then the batch can only be on a single partition (and thus table).
-    private final PartitionColumns conditionColumns;
+    private final RegularAndStaticColumns conditionColumns;
 
     private final boolean updatesRegularRows;
     private final boolean updatesStaticRow;
@@ -98,13 +99,13 @@ public class BatchStatement implements CQLStatement
 
         boolean hasConditions = false;
         MultiTableColumnsBuilder regularBuilder = new MultiTableColumnsBuilder();
-        PartitionColumns.Builder conditionBuilder = PartitionColumns.builder();
+        RegularAndStaticColumns.Builder conditionBuilder = RegularAndStaticColumns.builder();
         boolean updateRegular = false;
         boolean updateStatic = false;
 
         for (ModificationStatement stmt : statements)
         {
-            regularBuilder.addAll(stmt.cfm, stmt.updatedColumns());
+            regularBuilder.addAll(stmt.metadata(), stmt.updatedColumns());
             updateRegular |= stmt.updatesRegularRows();
             if (stmt.hasConditions())
             {
@@ -227,11 +228,11 @@ public class BatchStatement implements CQLStatement
         for (int i = 0; i < statements.size(); i++)
         {
             ModificationStatement statement = statements.get(i);
-            if (isLogged() && statement.cfm.params.gcGraceSeconds == 0)
+            if (isLogged() && statement.metadata().params.gcGraceSeconds == 0)
             {
                 if (tablesWithZeroGcGs == null)
                     tablesWithZeroGcGs = new HashSet<>();
-                tablesWithZeroGcGs.add(String.format("%s.%s", statement.cfm.ksName, statement.cfm.cfName));
+                tablesWithZeroGcGs.add(statement.metadata.toString());
             }
             QueryOptions statementOptions = options.forStatement(i);
             long timestamp = attrs.getTimestamp(now, statementOptions);
@@ -278,7 +279,7 @@ public class BatchStatement implements CQLStatement
             for (IMutation mutation : mutations)
             {
                 for (PartitionUpdate update : mutation.getPartitionUpdates())
-                    tableNames.add(String.format("%s.%s", update.metadata().ksName, update.metadata().cfName));
+                    tableNames.add(update.metadata().toString());
             }
 
             long failThreshold = DatabaseDescriptor.getBatchSizeFailThreshold();
@@ -314,7 +315,7 @@ public class BatchStatement implements CQLStatement
                 {
                     keySet.add(update.partitionKey());
 
-                    tableNames.add(String.format("%s.%s", update.metadata().ksName, update.metadata().cfName));
+                    tableNames.add(update.metadata().toString());
                 }
             }
 
@@ -385,12 +386,12 @@ public class BatchStatement implements CQLStatement
     private ResultMessage executeWithConditions(BatchQueryOptions options, QueryState state, long queryStartNanoTime)
     throws RequestExecutionException, RequestValidationException
     {
-        Pair<CQL3CasRequest, Set<ColumnDefinition>> p = makeCasRequest(options, state);
+        Pair<CQL3CasRequest, Set<ColumnMetadata>> p = makeCasRequest(options, state);
         CQL3CasRequest casRequest = p.left;
-        Set<ColumnDefinition> columnsWithConditions = p.right;
+        Set<ColumnMetadata> columnsWithConditions = p.right;
 
-        String ksName = casRequest.cfm.ksName;
-        String tableName = casRequest.cfm.cfName;
+        String ksName = casRequest.metadata.keyspace;
+        String tableName = casRequest.metadata.name;
 
         try (RowIterator result = StorageProxy.cas(ksName,
                                                    tableName,
@@ -411,12 +412,12 @@ public class BatchStatement implements CQLStatement
         }
     }
 
-    private Pair<CQL3CasRequest,Set<ColumnDefinition>> makeCasRequest(BatchQueryOptions options, QueryState state)
+    private Pair<CQL3CasRequest,Set<ColumnMetadata>> makeCasRequest(BatchQueryOptions options, QueryState state)
     {
         long now = state.getTimestamp();
         DecoratedKey key = null;
         CQL3CasRequest casRequest = null;
-        Set<ColumnDefinition> columnsWithConditions = new LinkedHashSet<>();
+        Set<ColumnMetadata> columnsWithConditions = new LinkedHashSet<>();
 
         for (int i = 0; i < statements.size(); i++)
         {
@@ -428,8 +429,8 @@ public class BatchStatement implements CQLStatement
                 throw new IllegalArgumentException("Batch with conditions cannot span multiple partitions (you cannot use IN on the partition key)");
             if (key == null)
             {
-                key = statement.cfm.decorateKey(pks.get(0));
-                casRequest = new CQL3CasRequest(statement.cfm, key, true, conditionColumns, updatesRegularRows, updatesStaticRow);
+                key = statement.metadata().partitioner.decorateKey(pks.get(0));
+                casRequest = new CQL3CasRequest(statement.metadata(), key, true, conditionColumns, updatesRegularRows, updatesStaticRow);
             }
             else if (!key.getKey().equals(pks.get(0)))
             {
@@ -475,12 +476,12 @@ public class BatchStatement implements CQLStatement
 
     private ResultMessage executeInternalWithConditions(BatchQueryOptions options, QueryState state) throws RequestExecutionException, RequestValidationException
     {
-        Pair<CQL3CasRequest, Set<ColumnDefinition>> p = makeCasRequest(options, state);
+        Pair<CQL3CasRequest, Set<ColumnMetadata>> p = makeCasRequest(options, state);
         CQL3CasRequest request = p.left;
-        Set<ColumnDefinition> columnsWithConditions = p.right;
+        Set<ColumnMetadata> columnsWithConditions = p.right;
 
-        String ksName = request.cfm.ksName;
-        String tableName = request.cfm.cfName;
+        String ksName = request.metadata.keyspace;
+        String tableName = request.metadata.name;
 
         try (RowIterator result = ModificationStatement.casInternal(request, state))
         {
@@ -544,10 +545,10 @@ public class BatchStatement implements CQLStatement
             BatchStatement batchStatement = new BatchStatement(boundNames.size(), type, statements, prepAttrs);
             batchStatement.validate();
 
-            // Use the CFMetadata of the first statement for partition key bind indexes.  If the statements affect
+            // Use the TableMetadata of the first statement for partition key bind indexes.  If the statements affect
             // multiple tables, we won't send partition key bind indexes.
             short[] partitionKeyBindIndexes = (haveMultipleCFs || batchStatement.statements.isEmpty())? null
-                                                              : boundNames.getPartitionKeyBindIndexes(batchStatement.statements.get(0).cfm);
+                                                              : boundNames.getPartitionKeyBindIndexes(batchStatement.statements.get(0).metadata());
 
             return new ParsedStatement.Prepared(batchStatement, boundNames, partitionKeyBindIndexes);
         }
@@ -555,23 +556,23 @@ public class BatchStatement implements CQLStatement
 
     private static class MultiTableColumnsBuilder
     {
-        private final Map<UUID, PartitionColumns.Builder> perTableBuilders = new HashMap<>();
+        private final Map<TableId, RegularAndStaticColumns.Builder> perTableBuilders = new HashMap<>();
 
-        public void addAll(CFMetaData table, PartitionColumns columns)
+        public void addAll(TableMetadata table, RegularAndStaticColumns columns)
         {
-            PartitionColumns.Builder builder = perTableBuilders.get(table.cfId);
+            RegularAndStaticColumns.Builder builder = perTableBuilders.get(table.id);
             if (builder == null)
             {
-                builder = PartitionColumns.builder();
-                perTableBuilders.put(table.cfId, builder);
+                builder = RegularAndStaticColumns.builder();
+                perTableBuilders.put(table.id, builder);
             }
             builder.addAll(columns);
         }
 
-        public Map<UUID, PartitionColumns> build()
+        public Map<TableId, RegularAndStaticColumns> build()
         {
-            Map<UUID, PartitionColumns> m = Maps.newHashMapWithExpectedSize(perTableBuilders.size());
-            for (Map.Entry<UUID, PartitionColumns.Builder> p : perTableBuilders.entrySet())
+            Map<TableId, RegularAndStaticColumns> m = Maps.newHashMapWithExpectedSize(perTableBuilders.size());
+            for (Map.Entry<TableId, RegularAndStaticColumns.Builder> p : perTableBuilders.entrySet())
                 m.put(p.getKey(), p.getValue().build());
             return m;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/Bound.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Bound.java b/src/java/org/apache/cassandra/cql3/statements/Bound.java
index 824743c..7682ac5 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Bound.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Bound.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 
 public enum Bound
 {
@@ -33,12 +33,12 @@ public enum Bound
     /**
      * Reverses the bound if the column type is a reversed one.
      *
-     * @param columnDefinition the column definition
+     * @param columnMetadata the column definition
      * @return the bound reversed if the column type was a reversed one or the original bound
      */
-    public Bound reverseIfNeeded(ColumnDefinition columnDefinition)
+    public Bound reverseIfNeeded(ColumnMetadata columnMetadata)
     {
-        return columnDefinition.isReversedType() ? reverse() : this;
+        return columnMetadata.isReversedType() ? reverse() : this;
     }
 
     public Bound reverse()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
index 4cf2baf..c450160 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
@@ -22,7 +22,7 @@ import java.util.*;
 
 import com.google.common.collect.*;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.conditions.ColumnCondition;
 import org.apache.cassandra.db.*;
@@ -40,10 +40,10 @@ import org.apache.cassandra.utils.Pair;
  */
 public class CQL3CasRequest implements CASRequest
 {
-    public final CFMetaData cfm;
+    public final TableMetadata metadata;
     public final DecoratedKey key;
     public final boolean isBatch;
-    private final PartitionColumns conditionColumns;
+    private final RegularAndStaticColumns conditionColumns;
     private final boolean updatesRegularRows;
     private final boolean updatesStaticRow;
     private boolean hasExists; // whether we have an exist or if not exist condition
@@ -58,16 +58,16 @@ public class CQL3CasRequest implements CASRequest
 
     private final List<RowUpdate> updates = new ArrayList<>();
 
-    public CQL3CasRequest(CFMetaData cfm,
+    public CQL3CasRequest(TableMetadata metadata,
                           DecoratedKey key,
                           boolean isBatch,
-                          PartitionColumns conditionColumns,
+                          RegularAndStaticColumns conditionColumns,
                           boolean updatesRegularRows,
                           boolean updatesStaticRow)
     {
-        this.cfm = cfm;
+        this.metadata = metadata;
         this.key = key;
-        this.conditions = new TreeMap<>(cfm.comparator);
+        this.conditions = new TreeMap<>(metadata.comparator);
         this.isBatch = isBatch;
         this.conditionColumns = conditionColumns;
         this.updatesRegularRows = updatesRegularRows;
@@ -156,7 +156,7 @@ public class CQL3CasRequest implements CASRequest
         }
     }
 
-    private PartitionColumns columnsToRead()
+    private RegularAndStaticColumns columnsToRead()
     {
         // If all our conditions are columns conditions (IF x = ?), then it's enough to query
         // the columns from the conditions. If we have a IF EXISTS or IF NOT EXISTS however,
@@ -167,10 +167,10 @@ public class CQL3CasRequest implements CASRequest
         // case.
         if (hasExists)
         {
-            PartitionColumns allColumns = cfm.partitionColumns();
+            RegularAndStaticColumns allColumns = metadata.regularAndStaticColumns();
             Columns statics = updatesStaticRow ? allColumns.statics : Columns.NONE;
             Columns regulars = updatesRegularRows ? allColumns.regulars : Columns.NONE;
-            return new PartitionColumns(statics, regulars);
+            return new RegularAndStaticColumns(statics, regulars);
         }
         return conditionColumns;
     }
@@ -180,12 +180,12 @@ public class CQL3CasRequest implements CASRequest
         assert staticConditions != null || !conditions.isEmpty();
 
         // Fetch all columns, but query only the selected ones
-        ColumnFilter columnFilter = ColumnFilter.selection(cfm, columnsToRead());
+        ColumnFilter columnFilter = ColumnFilter.selection(metadata, columnsToRead());
 
         // With only a static condition, we still want to make the distinction between a non-existing partition and one
         // that exists (has some live data) but has not static content. So we query the first live row of the partition.
         if (conditions.isEmpty())
-            return SinglePartitionReadCommand.create(cfm,
+            return SinglePartitionReadCommand.create(metadata,
                                                      nowInSec,
                                                      columnFilter,
                                                      RowFilter.NONE,
@@ -194,7 +194,7 @@ public class CQL3CasRequest implements CASRequest
                                                      new ClusteringIndexSliceFilter(Slices.ALL, false));
 
         ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(conditions.navigableKeySet(), false);
-        return SinglePartitionReadCommand.create(cfm, nowInSec, key, columnFilter, filter);
+        return SinglePartitionReadCommand.create(metadata, nowInSec, key, columnFilter, filter);
     }
 
     /**
@@ -219,9 +219,9 @@ public class CQL3CasRequest implements CASRequest
         return true;
     }
 
-    private PartitionColumns updatedColumns()
+    private RegularAndStaticColumns updatedColumns()
     {
-        PartitionColumns.Builder builder = PartitionColumns.builder();
+        RegularAndStaticColumns.Builder builder = RegularAndStaticColumns.builder();
         for (RowUpdate upd : updates)
             builder.addAll(upd.stmt.updatedColumns());
         return builder.build();
@@ -229,11 +229,11 @@ public class CQL3CasRequest implements CASRequest
 
     public PartitionUpdate makeUpdates(FilteredPartition current) throws InvalidRequestException
     {
-        PartitionUpdate update = new PartitionUpdate(cfm, key, updatedColumns(), conditions.size());
+        PartitionUpdate update = new PartitionUpdate(metadata, key, updatedColumns(), conditions.size());
         for (RowUpdate upd : updates)
             upd.applyUpdates(current, update);
 
-        Keyspace.openAndGetStore(cfm).indexManager.validate(update);
+        Keyspace.openAndGetStore(metadata).indexManager.validate(update);
         return update;
     }
 
@@ -261,7 +261,7 @@ public class CQL3CasRequest implements CASRequest
         public void applyUpdates(FilteredPartition current, PartitionUpdate updates) throws InvalidRequestException
         {
             Map<DecoratedKey, Partition> map = stmt.requiresRead() ? Collections.<DecoratedKey, Partition>singletonMap(key, current) : null;
-            UpdateParameters params = new UpdateParameters(cfm, updates.columns(), options, timestamp, stmt.getTimeToLive(options), map);
+            UpdateParameters params = new UpdateParameters(metadata, updates.columns(), options, timestamp, stmt.getTimeToLive(options), map);
             stmt.addUpdateForKey(updates, clustering, params);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
index c887255..ade2752 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
@@ -25,14 +25,14 @@ import java.util.List;
 
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.transport.ProtocolVersion;
@@ -160,7 +160,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
         if (!functionName.hasKeyspace())
             throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
 
-        Validation.validateKeyspaceNotSystem(functionName.keyspace);
+        Schema.validateKeyspaceNotSystem(functionName.keyspace);
 
         stateFunc = new FunctionName(functionName.keyspace, stateFunc.name);
         if (finalFunc != null)
@@ -203,7 +203,7 @@ public final class CreateAggregateStatement extends SchemaAlteringStatement
         if (ifNotExists && orReplace)
             throw new InvalidRequestException("Cannot use both 'OR REPLACE' and 'IF NOT EXISTS' directives");
 
-        if (Schema.instance.getKSMetaData(functionName.keyspace) == null)
+        if (Schema.instance.getKeyspaceMetadata(functionName.keyspace) == null)
             throw new InvalidRequestException(String.format("Cannot add aggregate '%s' to non existing keyspace '%s'.", functionName.name, functionName.keyspace));
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
index 96b9721..2e1f78c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
@@ -23,16 +23,15 @@ import java.util.List;
 
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.Validation;
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.schema.Functions;
 import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 
@@ -98,7 +97,7 @@ public final class CreateFunctionStatement extends SchemaAlteringStatement
         if (!functionName.hasKeyspace())
             throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
 
-        Validation.validateKeyspaceNotSystem(functionName.keyspace);
+        Schema.validateKeyspaceNotSystem(functionName.keyspace);
     }
 
     protected void grantPermissionsToCreator(QueryState state)
@@ -134,7 +133,7 @@ public final class CreateFunctionStatement extends SchemaAlteringStatement
         if (ifNotExists && orReplace)
             throw new InvalidRequestException("Cannot use both 'OR REPLACE' and 'IF NOT EXISTS' directives");
 
-        if (Schema.instance.getKSMetaData(functionName.keyspace) == null)
+        if (Schema.instance.getKeyspaceMetadata(functionName.keyspace) == null)
             throw new InvalidRequestException(String.format("Cannot add function '%s' to non existing keyspace '%s'.", functionName.name, functionName.keyspace));
     }
 


[18/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index ee0974f..6716652 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -23,7 +23,6 @@ import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.MapDifference;
@@ -32,7 +31,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.*;
-import org.apache.cassandra.config.ColumnDefinition.ClusteringOrder;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.schema.ColumnMetadata.ClusteringOrder;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.*;
 import org.apache.cassandra.cql3.statements.SelectStatement;
@@ -42,19 +42,17 @@ import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.view.View;
-import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
 
 import static java.lang.String.format;
 
 import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toSet;
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
-import static org.apache.cassandra.schema.CQLTypeParser.parse;
 
 /**
  * system_schema.* tables and methods for manipulating them.
@@ -83,163 +81,167 @@ public final class SchemaKeyspace
     public static final List<String> ALL =
         ImmutableList.of(KEYSPACES, TABLES, COLUMNS, DROPPED_COLUMNS, TRIGGERS, VIEWS, TYPES, FUNCTIONS, AGGREGATES, INDEXES);
 
-    private static final CFMetaData Keyspaces =
-        compile(KEYSPACES,
-                "keyspace definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "durable_writes boolean,"
-                + "replication frozen<map<text, text>>,"
-                + "PRIMARY KEY ((keyspace_name)))");
-
-    private static final CFMetaData Tables =
-        compile(TABLES,
-                "table definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "table_name text,"
-                + "bloom_filter_fp_chance double,"
-                + "caching frozen<map<text, text>>,"
-                + "comment text,"
-                + "compaction frozen<map<text, text>>,"
-                + "compression frozen<map<text, text>>,"
-                + "crc_check_chance double,"
-                + "dclocal_read_repair_chance double,"
-                + "default_time_to_live int,"
-                + "extensions frozen<map<text, blob>>,"
-                + "flags frozen<set<text>>," // SUPER, COUNTER, DENSE, COMPOUND
-                + "gc_grace_seconds int,"
-                + "id uuid,"
-                + "max_index_interval int,"
-                + "memtable_flush_period_in_ms int,"
-                + "min_index_interval int,"
-                + "read_repair_chance double,"
-                + "speculative_retry text,"
-                + "cdc boolean,"
-                + "PRIMARY KEY ((keyspace_name), table_name))");
-
-    private static final CFMetaData Columns =
-        compile(COLUMNS,
-                "column definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "table_name text,"
-                + "column_name text,"
-                + "clustering_order text,"
-                + "column_name_bytes blob,"
-                + "kind text,"
-                + "position int,"
-                + "type text,"
-                + "PRIMARY KEY ((keyspace_name), table_name, column_name))");
-
-    private static final CFMetaData DroppedColumns =
-        compile(DROPPED_COLUMNS,
-                "dropped column registry",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "table_name text,"
-                + "column_name text,"
-                + "dropped_time timestamp,"
-                + "type text,"
-                + "kind text,"
-                + "PRIMARY KEY ((keyspace_name), table_name, column_name))");
-
-    private static final CFMetaData Triggers =
-        compile(TRIGGERS,
-                "trigger definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "table_name text,"
-                + "trigger_name text,"
-                + "options frozen<map<text, text>>,"
-                + "PRIMARY KEY ((keyspace_name), table_name, trigger_name))");
-
-    private static final CFMetaData Views =
-        compile(VIEWS,
-                "view definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "view_name text,"
-                + "base_table_id uuid,"
-                + "base_table_name text,"
-                + "where_clause text,"
-                + "bloom_filter_fp_chance double,"
-                + "caching frozen<map<text, text>>,"
-                + "comment text,"
-                + "compaction frozen<map<text, text>>,"
-                + "compression frozen<map<text, text>>,"
-                + "crc_check_chance double,"
-                + "dclocal_read_repair_chance double,"
-                + "default_time_to_live int,"
-                + "extensions frozen<map<text, blob>>,"
-                + "gc_grace_seconds int,"
-                + "id uuid,"
-                + "include_all_columns boolean,"
-                + "max_index_interval int,"
-                + "memtable_flush_period_in_ms int,"
-                + "min_index_interval int,"
-                + "read_repair_chance double,"
-                + "speculative_retry text,"
-                + "cdc boolean,"
-                + "PRIMARY KEY ((keyspace_name), view_name))");
-
-    private static final CFMetaData Indexes =
-        compile(INDEXES,
-                "secondary index definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "table_name text,"
-                + "index_name text,"
-                + "kind text,"
-                + "options frozen<map<text, text>>,"
-                + "PRIMARY KEY ((keyspace_name), table_name, index_name))");
-
-    private static final CFMetaData Types =
-        compile(TYPES,
-                "user defined type definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "type_name text,"
-                + "field_names frozen<list<text>>,"
-                + "field_types frozen<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,"
-                + "argument_types frozen<list<text>>,"
-                + "argument_names frozen<list<text>>,"
-                + "body text,"
-                + "language text,"
-                + "return_type text,"
-                + "called_on_null_input boolean,"
-                + "PRIMARY KEY ((keyspace_name), function_name, argument_types))");
-
-    private static final CFMetaData Aggregates =
-        compile(AGGREGATES,
-                "user defined aggregate definitions",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "aggregate_name text,"
-                + "argument_types frozen<list<text>>,"
-                + "final_func text,"
-                + "initcond text,"
-                + "return_type text,"
-                + "state_func text,"
-                + "state_type text,"
-                + "PRIMARY KEY ((keyspace_name), aggregate_name, argument_types))");
-
-    public static final List<CFMetaData> ALL_TABLE_METADATA =
+    private static final TableMetadata Keyspaces =
+        parse(KEYSPACES,
+              "keyspace definitions",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "durable_writes boolean,"
+              + "replication frozen<map<text, text>>,"
+              + "PRIMARY KEY ((keyspace_name)))");
+
+    private static final TableMetadata Tables =
+        parse(TABLES,
+              "table definitions",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "table_name text,"
+              + "bloom_filter_fp_chance double,"
+              + "caching frozen<map<text, text>>,"
+              + "comment text,"
+              + "compaction frozen<map<text, text>>,"
+              + "compression frozen<map<text, text>>,"
+              + "crc_check_chance double,"
+              + "dclocal_read_repair_chance double,"
+              + "default_time_to_live int,"
+              + "extensions frozen<map<text, blob>>,"
+              + "flags frozen<set<text>>," // SUPER, COUNTER, DENSE, COMPOUND
+              + "gc_grace_seconds int,"
+              + "id uuid,"
+              + "max_index_interval int,"
+              + "memtable_flush_period_in_ms int,"
+              + "min_index_interval int,"
+              + "read_repair_chance double,"
+              + "speculative_retry text,"
+              + "cdc boolean,"
+              + "PRIMARY KEY ((keyspace_name), table_name))");
+
+    private static final TableMetadata Columns =
+        parse(COLUMNS,
+              "column definitions",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "table_name text,"
+              + "column_name text,"
+              + "clustering_order text,"
+              + "column_name_bytes blob,"
+              + "kind text,"
+              + "position int,"
+              + "type text,"
+              + "PRIMARY KEY ((keyspace_name), table_name, column_name))");
+
+    private static final TableMetadata DroppedColumns =
+        parse(DROPPED_COLUMNS,
+              "dropped column registry",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "table_name text,"
+              + "column_name text,"
+              + "dropped_time timestamp,"
+              + "type text,"
+              + "kind text,"
+              + "PRIMARY KEY ((keyspace_name), table_name, column_name))");
+
+    private static final TableMetadata Triggers =
+        parse(TRIGGERS,
+              "trigger definitions",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "table_name text,"
+              + "trigger_name text,"
+              + "options frozen<map<text, text>>,"
+              + "PRIMARY KEY ((keyspace_name), table_name, trigger_name))");
+
+    private static final TableMetadata Views =
+        parse(VIEWS,
+              "view definitions",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "view_name text,"
+              + "base_table_id uuid,"
+              + "base_table_name text,"
+              + "where_clause text,"
+              + "bloom_filter_fp_chance double,"
+              + "caching frozen<map<text, text>>,"
+              + "comment text,"
+              + "compaction frozen<map<text, text>>,"
+              + "compression frozen<map<text, text>>,"
+              + "crc_check_chance double,"
+              + "dclocal_read_repair_chance double,"
+              + "default_time_to_live int,"
+              + "extensions frozen<map<text, blob>>,"
+              + "gc_grace_seconds int,"
+              + "id uuid,"
+              + "include_all_columns boolean,"
+              + "max_index_interval int,"
+              + "memtable_flush_period_in_ms int,"
+              + "min_index_interval int,"
+              + "read_repair_chance double,"
+              + "speculative_retry text,"
+              + "cdc boolean,"
+              + "PRIMARY KEY ((keyspace_name), view_name))");
+
+    private static final TableMetadata Indexes =
+        parse(INDEXES,
+              "secondary index definitions",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "table_name text,"
+              + "index_name text,"
+              + "kind text,"
+              + "options frozen<map<text, text>>,"
+              + "PRIMARY KEY ((keyspace_name), table_name, index_name))");
+
+    private static final TableMetadata Types =
+        parse(TYPES,
+              "user defined type definitions",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "type_name text,"
+              + "field_names frozen<list<text>>,"
+              + "field_types frozen<list<text>>,"
+              + "PRIMARY KEY ((keyspace_name), type_name))");
+
+    private static final TableMetadata Functions =
+        parse(FUNCTIONS,
+              "user defined function definitions",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "function_name text,"
+              + "argument_types frozen<list<text>>,"
+              + "argument_names frozen<list<text>>,"
+              + "body text,"
+              + "language text,"
+              + "return_type text,"
+              + "called_on_null_input boolean,"
+              + "PRIMARY KEY ((keyspace_name), function_name, argument_types))");
+
+    private static final TableMetadata Aggregates =
+        parse(AGGREGATES,
+              "user defined aggregate definitions",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "aggregate_name text,"
+              + "argument_types frozen<list<text>>,"
+              + "final_func text,"
+              + "initcond text,"
+              + "return_type text,"
+              + "state_func text,"
+              + "state_type text,"
+              + "PRIMARY KEY ((keyspace_name), aggregate_name, argument_types))");
+
+    private static final List<TableMetadata> ALL_TABLE_METADATA =
         ImmutableList.of(Keyspaces, Tables, Columns, Triggers, DroppedColumns, Views, Types, Functions, Aggregates, Indexes);
 
-    private static CFMetaData compile(String name, String description, String schema)
+    private static TableMetadata parse(String name, String description, String cql)
     {
-        return CFMetaData.compile(String.format(schema, name), SchemaConstants.SCHEMA_KEYSPACE_NAME)
-                         .comment(description)
-                         .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(7));
+        return CreateTableStatement.parse(format(cql, name), SchemaConstants.SCHEMA_KEYSPACE_NAME)
+                                   .id(TableId.forSystemTable(SchemaConstants.SCHEMA_KEYSPACE_NAME, name))
+                                   .dcLocalReadRepairChance(0.0)
+                                   .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(7))
+                                   .memtableFlushPeriod((int) TimeUnit.HOURS.toMillis(1))
+                                   .comment(description)
+                                   .build();
     }
 
     public static KeyspaceMetadata metadata()
@@ -252,8 +254,8 @@ public final class SchemaKeyspace
      */
     public static void saveSystemKeyspacesSchema()
     {
-        KeyspaceMetadata system = Schema.instance.getKSMetaData(SchemaConstants.SYSTEM_KEYSPACE_NAME);
-        KeyspaceMetadata schema = Schema.instance.getKSMetaData(SchemaConstants.SCHEMA_KEYSPACE_NAME);
+        KeyspaceMetadata system = Schema.instance.getKeyspaceMetadata(SchemaConstants.SYSTEM_KEYSPACE_NAME);
+        KeyspaceMetadata schema = Schema.instance.getKeyspaceMetadata(SchemaConstants.SCHEMA_KEYSPACE_NAME);
 
         long timestamp = FBUtilities.timestampMicros();
 
@@ -285,7 +287,7 @@ public final class SchemaKeyspace
      * 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()
+    static UUID calculateSchemaDigest()
     {
         MessageDigest digest;
         try
@@ -337,10 +339,10 @@ public final class SchemaKeyspace
     private static ReadCommand getReadCommandForTableSchema(String schemaTableName)
     {
         ColumnFamilyStore cfs = getSchemaCFS(schemaTableName);
-        return PartitionRangeReadCommand.allDataRead(cfs.metadata, FBUtilities.nowInSeconds());
+        return PartitionRangeReadCommand.allDataRead(cfs.metadata(), FBUtilities.nowInSeconds());
     }
 
-    public static Collection<Mutation> convertSchemaToMutations()
+    static Collection<Mutation> convertSchemaToMutations()
     {
         Map<DecoratedKey, Mutation> mutationMap = new HashMap<>();
 
@@ -364,12 +366,7 @@ public final class SchemaKeyspace
                         continue;
 
                     DecoratedKey key = partition.partitionKey();
-                    Mutation mutation = mutationMap.get(key);
-                    if (mutation == null)
-                    {
-                        mutation = new Mutation(SchemaConstants.SCHEMA_KEYSPACE_NAME, key);
-                        mutationMap.put(key, mutation);
-                    }
+                    Mutation mutation = mutationMap.computeIfAbsent(key, k -> new Mutation(SchemaConstants.SCHEMA_KEYSPACE_NAME, key));
 
                     mutation.add(makeUpdateForSchema(partition, cmd.columnFilter()));
                 }
@@ -387,13 +384,13 @@ public final class SchemaKeyspace
         // This method is used during schema migration tasks, and if cdc is disabled, we want to force excluding the
         // 'cdc' column from the TABLES schema table because it is problematic if received by older nodes (see #12236
         // and #12697). Otherwise though, we just simply "buffer" the content of the partition into a PartitionUpdate.
-        if (DatabaseDescriptor.isCDCEnabled() || !partition.metadata().cfName.equals(TABLES))
+        if (DatabaseDescriptor.isCDCEnabled() || !partition.metadata().name.equals(TABLES))
             return PartitionUpdate.fromIterator(partition, filter);
 
         // We want to skip the 'cdc' column. A simple solution for that is based on the fact that
         // 'PartitionUpdate.fromIterator()' will ignore any columns that are marked as 'fetched' but not 'queried'.
         ColumnFilter.Builder builder = ColumnFilter.allRegularColumnsBuilder(partition.metadata());
-        for (ColumnDefinition column : filter.fetchedColumns())
+        for (ColumnMetadata column : filter.fetchedColumns())
         {
             if (!column.name.toString().equals("cdc"))
                 builder.add(column);
@@ -411,14 +408,15 @@ public final class SchemaKeyspace
      * Schema entities to mutations
      */
 
-    private static DecoratedKey decorate(CFMetaData metadata, Object value)
+    @SuppressWarnings("unchecked")
+    private static DecoratedKey decorate(TableMetadata metadata, Object value)
     {
-        return metadata.decorateKey(((AbstractType)metadata.getKeyValidator()).decompose(value));
+        return metadata.partitioner.decorateKey(((AbstractType)metadata.partitionKeyType).decompose(value));
     }
 
-    public static Mutation.SimpleBuilder makeCreateKeyspaceMutation(String name, KeyspaceParams params, long timestamp)
+    static Mutation.SimpleBuilder makeCreateKeyspaceMutation(String name, KeyspaceParams params, long timestamp)
     {
-        Mutation.SimpleBuilder builder = Mutation.simpleBuilder(Keyspaces.ksName, decorate(Keyspaces, name))
+        Mutation.SimpleBuilder builder = Mutation.simpleBuilder(Keyspaces.keyspace, decorate(Keyspaces, name))
                                                  .timestamp(timestamp);
 
         builder.update(Keyspaces)
@@ -429,7 +427,7 @@ public final class SchemaKeyspace
         return builder;
     }
 
-    public static Mutation.SimpleBuilder makeCreateKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
+    static Mutation.SimpleBuilder makeCreateKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
     {
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
@@ -442,18 +440,18 @@ public final class SchemaKeyspace
         return builder;
     }
 
-    public static Mutation.SimpleBuilder makeDropKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
+    static Mutation.SimpleBuilder makeDropKeyspaceMutation(KeyspaceMetadata keyspace, long timestamp)
     {
         Mutation.SimpleBuilder builder = Mutation.simpleBuilder(SchemaConstants.SCHEMA_KEYSPACE_NAME, decorate(Keyspaces, keyspace.name))
                                                  .timestamp(timestamp);
 
-        for (CFMetaData schemaTable : ALL_TABLE_METADATA)
+        for (TableMetadata schemaTable : ALL_TABLE_METADATA)
             builder.update(schemaTable).delete();
 
         return builder;
     }
 
-    public static Mutation.SimpleBuilder makeCreateTypeMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
+    static Mutation.SimpleBuilder makeCreateTypeMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
@@ -469,7 +467,7 @@ public final class SchemaKeyspace
                 .add("field_types", type.fieldTypes().stream().map(AbstractType::asCQL3Type).map(CQL3Type::toString).collect(toList()));
     }
 
-    public static Mutation.SimpleBuilder dropTypeFromSchemaMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
+    static Mutation.SimpleBuilder dropTypeFromSchemaMutation(KeyspaceMetadata keyspace, UserType type, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
@@ -477,7 +475,7 @@ public final class SchemaKeyspace
         return builder;
     }
 
-    public static Mutation.SimpleBuilder makeCreateTableMutation(KeyspaceMetadata keyspace, CFMetaData table, long timestamp)
+    static Mutation.SimpleBuilder makeCreateTableMutation(KeyspaceMetadata keyspace, TableMetadata table, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
@@ -485,27 +483,27 @@ public final class SchemaKeyspace
         return builder;
     }
 
-    static void addTableToSchemaMutation(CFMetaData table, boolean withColumnsAndTriggers, Mutation.SimpleBuilder builder)
+    static void addTableToSchemaMutation(TableMetadata table, boolean withColumnsAndTriggers, Mutation.SimpleBuilder builder)
     {
         Row.SimpleBuilder rowBuilder = builder.update(Tables)
-                                              .row(table.cfName)
-                                              .add("id", table.cfId)
-                                              .add("flags", CFMetaData.flagsToStrings(table.flags()));
+                                              .row(table.name)
+                                              .add("id", table.id.asUUID())
+                                              .add("flags", TableMetadata.Flag.toStringSet(table.flags));
 
         addTableParamsToRowBuilder(table.params, rowBuilder);
 
         if (withColumnsAndTriggers)
         {
-            for (ColumnDefinition column : table.allColumns())
+            for (ColumnMetadata column : table.columns())
                 addColumnToSchemaMutation(table, column, builder);
 
-            for (CFMetaData.DroppedColumn column : table.getDroppedColumns().values())
+            for (DroppedColumn column : table.droppedColumns.values())
                 addDroppedColumnToSchemaMutation(table, column, builder);
 
-            for (TriggerMetadata trigger : table.getTriggers())
+            for (TriggerMetadata trigger : table.triggers)
                 addTriggerToSchemaMutation(table, trigger, builder);
 
-            for (IndexMetadata index : table.getIndexes())
+            for (IndexMetadata index : table.indexes)
                 addIndexToSchemaMutation(table, index, builder);
         }
     }
@@ -534,43 +532,42 @@ public final class SchemaKeyspace
             builder.add("cdc", params.cdc);
     }
 
-    public static Mutation.SimpleBuilder makeUpdateTableMutation(KeyspaceMetadata keyspace,
-                                                                 CFMetaData oldTable,
-                                                                 CFMetaData newTable,
+    static Mutation.SimpleBuilder makeUpdateTableMutation(KeyspaceMetadata keyspace,
+                                                                 TableMetadata oldTable,
+                                                                 TableMetadata newTable,
                                                                  long timestamp)
     {
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
         addTableToSchemaMutation(newTable, false, builder);
 
-        MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(oldTable.getColumnMetadata(),
-                                                                                 newTable.getColumnMetadata());
+        MapDifference<ByteBuffer, ColumnMetadata> columnDiff = Maps.difference(oldTable.columns, newTable.columns);
 
         // columns that are no longer needed
-        for (ColumnDefinition column : columnDiff.entriesOnlyOnLeft().values())
+        for (ColumnMetadata column : columnDiff.entriesOnlyOnLeft().values())
             dropColumnFromSchemaMutation(oldTable, column, builder);
 
         // newly added columns
-        for (ColumnDefinition column : columnDiff.entriesOnlyOnRight().values())
+        for (ColumnMetadata column : columnDiff.entriesOnlyOnRight().values())
             addColumnToSchemaMutation(newTable, column, builder);
 
         // old columns with updated attributes
         for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
-            addColumnToSchemaMutation(newTable, newTable.getColumnDefinition(name), builder);
+            addColumnToSchemaMutation(newTable, newTable.getColumn(name), builder);
 
         // dropped columns
-        MapDifference<ByteBuffer, CFMetaData.DroppedColumn> droppedColumnDiff =
-            Maps.difference(oldTable.getDroppedColumns(), newTable.getDroppedColumns());
+        MapDifference<ByteBuffer, DroppedColumn> droppedColumnDiff =
+            Maps.difference(oldTable.droppedColumns, newTable.droppedColumns);
 
         // newly dropped columns
-        for (CFMetaData.DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values())
+        for (DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values())
             addDroppedColumnToSchemaMutation(newTable, column, builder);
 
         // columns added then dropped again
         for (ByteBuffer name : droppedColumnDiff.entriesDiffering().keySet())
-            addDroppedColumnToSchemaMutation(newTable, newTable.getDroppedColumns().get(name), builder);
+            addDroppedColumnToSchemaMutation(newTable, newTable.droppedColumns.get(name), builder);
 
-        MapDifference<String, TriggerMetadata> triggerDiff = triggersDiff(oldTable.getTriggers(), newTable.getTriggers());
+        MapDifference<String, TriggerMetadata> triggerDiff = triggersDiff(oldTable.triggers, newTable.triggers);
 
         // dropped triggers
         for (TriggerMetadata trigger : triggerDiff.entriesOnlyOnLeft().values())
@@ -580,8 +577,7 @@ public final class SchemaKeyspace
         for (TriggerMetadata trigger : triggerDiff.entriesOnlyOnRight().values())
             addTriggerToSchemaMutation(newTable, trigger, builder);
 
-        MapDifference<String, IndexMetadata> indexesDiff = indexesDiff(oldTable.getIndexes(),
-                                                                       newTable.getIndexes());
+        MapDifference<String, IndexMetadata> indexesDiff = indexesDiff(oldTable.indexes, newTable.indexes);
 
         // dropped indexes
         for (IndexMetadata index : indexesDiff.entriesOnlyOnLeft().values())
@@ -620,33 +616,33 @@ public final class SchemaKeyspace
         return Maps.difference(beforeMap, afterMap);
     }
 
-    public static Mutation.SimpleBuilder makeDropTableMutation(KeyspaceMetadata keyspace, CFMetaData table, long timestamp)
+    static Mutation.SimpleBuilder makeDropTableMutation(KeyspaceMetadata keyspace, TableMetadata table, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
-        builder.update(Tables).row(table.cfName).delete();
+        builder.update(Tables).row(table.name).delete();
 
-        for (ColumnDefinition column : table.allColumns())
+        for (ColumnMetadata column : table.columns())
             dropColumnFromSchemaMutation(table, column, builder);
 
-        for (TriggerMetadata trigger : table.getTriggers())
+        for (TriggerMetadata trigger : table.triggers)
             dropTriggerFromSchemaMutation(table, trigger, builder);
 
-        for (IndexMetadata index : table.getIndexes())
+        for (IndexMetadata index : table.indexes)
             dropIndexFromSchemaMutation(table, index, builder);
 
         return builder;
     }
 
-    private static void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, Mutation.SimpleBuilder builder)
+    private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadata column, Mutation.SimpleBuilder builder)
     {
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
             type = ((ReversedType) type).baseType;
 
         builder.update(Columns)
-               .row(table.cfName, column.name.toString())
+               .row(table.name, column.name.toString())
                .add("column_name_bytes", column.name.bytes)
                .add("kind", column.kind.toString().toLowerCase())
                .add("position", column.position())
@@ -654,34 +650,34 @@ public final class SchemaKeyspace
                .add("type", type.asCQL3Type().toString());
     }
 
-    private static void dropColumnFromSchemaMutation(CFMetaData table, ColumnDefinition column, Mutation.SimpleBuilder builder)
+    private static void dropColumnFromSchemaMutation(TableMetadata table, ColumnMetadata column, Mutation.SimpleBuilder builder)
     {
         // Note: we do want to use name.toString(), not name.bytes directly for backward compatibility (For CQL3, this won't make a difference).
-        builder.update(Columns).row(table.cfName, column.name.toString()).delete();
+        builder.update(Columns).row(table.name, column.name.toString()).delete();
     }
 
-    private static void addDroppedColumnToSchemaMutation(CFMetaData table, CFMetaData.DroppedColumn column, Mutation.SimpleBuilder builder)
+    private static void addDroppedColumnToSchemaMutation(TableMetadata table, DroppedColumn column, Mutation.SimpleBuilder builder)
     {
         builder.update(DroppedColumns)
-               .row(table.cfName, column.name)
+               .row(table.name, column.column.name.toString())
                .add("dropped_time", new Date(TimeUnit.MICROSECONDS.toMillis(column.droppedTime)))
-               .add("type", expandUserTypes(column.type).asCQL3Type().toString())
-               .add("kind", column.kind.toString().toLowerCase());
+               .add("type", expandUserTypes(column.column.type).asCQL3Type().toString())
+               .add("kind", column.column.kind.toString().toLowerCase());
     }
 
-    private static void addTriggerToSchemaMutation(CFMetaData table, TriggerMetadata trigger, Mutation.SimpleBuilder builder)
+    private static void addTriggerToSchemaMutation(TableMetadata table, TriggerMetadata trigger, Mutation.SimpleBuilder builder)
     {
         builder.update(Triggers)
-               .row(table.cfName, trigger.name)
+               .row(table.name, trigger.name)
                .add("options", Collections.singletonMap("class", trigger.classOption));
     }
 
-    private static void dropTriggerFromSchemaMutation(CFMetaData table, TriggerMetadata trigger, Mutation.SimpleBuilder builder)
+    private static void dropTriggerFromSchemaMutation(TableMetadata table, TriggerMetadata trigger, Mutation.SimpleBuilder builder)
     {
-        builder.update(Triggers).row(table.cfName, trigger.name).delete();
+        builder.update(Triggers).row(table.name, trigger.name).delete();
     }
 
-    public static Mutation.SimpleBuilder makeCreateViewMutation(KeyspaceMetadata keyspace, ViewDefinition view, long timestamp)
+    static Mutation.SimpleBuilder makeCreateViewMutation(KeyspaceMetadata keyspace, ViewMetadata view, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
@@ -689,110 +685,106 @@ public final class SchemaKeyspace
         return builder;
     }
 
-    private static void addViewToSchemaMutation(ViewDefinition view, boolean includeColumns, Mutation.SimpleBuilder builder)
+    private static void addViewToSchemaMutation(ViewMetadata view, boolean includeColumns, Mutation.SimpleBuilder builder)
     {
-        CFMetaData table = view.metadata;
+        TableMetadata table = view.metadata;
         Row.SimpleBuilder rowBuilder = builder.update(Views)
-                                              .row(view.viewName)
+                                              .row(view.name)
                                               .add("include_all_columns", view.includeAllColumns)
-                                              .add("base_table_id", view.baseTableId)
-                                              .add("base_table_name", view.baseTableMetadata().cfName)
+                                              .add("base_table_id", view.baseTableId.asUUID())
+                                              .add("base_table_name", view.baseTableName)
                                               .add("where_clause", view.whereClause)
-                                              .add("id", table.cfId);
+                                              .add("id", table.id.asUUID());
 
         addTableParamsToRowBuilder(table.params, rowBuilder);
 
         if (includeColumns)
         {
-            for (ColumnDefinition column : table.allColumns())
+            for (ColumnMetadata column : table.columns())
                 addColumnToSchemaMutation(table, column, builder);
 
-            for (CFMetaData.DroppedColumn column : table.getDroppedColumns().values())
+            for (DroppedColumn column : table.droppedColumns.values())
                 addDroppedColumnToSchemaMutation(table, column, builder);
         }
     }
 
-    public static Mutation.SimpleBuilder makeDropViewMutation(KeyspaceMetadata keyspace, ViewDefinition view, long timestamp)
+    static Mutation.SimpleBuilder makeDropViewMutation(KeyspaceMetadata keyspace, ViewMetadata view, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
-        builder.update(Views).row(view.viewName).delete();
+        builder.update(Views).row(view.name).delete();
 
-        CFMetaData table = view.metadata;
-        for (ColumnDefinition column : table.allColumns())
+        TableMetadata table = view.metadata;
+        for (ColumnMetadata column : table.columns())
             dropColumnFromSchemaMutation(table, column, builder);
 
-        for (IndexMetadata index : table.getIndexes())
+        for (IndexMetadata index : table.indexes)
             dropIndexFromSchemaMutation(table, index, builder);
 
         return builder;
     }
 
-    public static Mutation.SimpleBuilder makeUpdateViewMutation(KeyspaceMetadata keyspace,
-                                                                ViewDefinition oldView,
-                                                                ViewDefinition newView,
+    static Mutation.SimpleBuilder makeUpdateViewMutation(KeyspaceMetadata keyspace,
+                                                                ViewMetadata oldView,
+                                                                ViewMetadata newView,
                                                                 long timestamp)
     {
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
         addViewToSchemaMutation(newView, false, builder);
 
-        MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(oldView.metadata.getColumnMetadata(),
-                                                                                 newView.metadata.getColumnMetadata());
+        MapDifference<ByteBuffer, ColumnMetadata> columnDiff = Maps.difference(oldView.metadata.columns,
+                                                                               newView.metadata.columns);
 
         // columns that are no longer needed
-        for (ColumnDefinition column : columnDiff.entriesOnlyOnLeft().values())
+        for (ColumnMetadata column : columnDiff.entriesOnlyOnLeft().values())
             dropColumnFromSchemaMutation(oldView.metadata, column, builder);
 
         // newly added columns
-        for (ColumnDefinition column : columnDiff.entriesOnlyOnRight().values())
+        for (ColumnMetadata column : columnDiff.entriesOnlyOnRight().values())
             addColumnToSchemaMutation(newView.metadata, column, builder);
 
         // old columns with updated attributes
         for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
-            addColumnToSchemaMutation(newView.metadata, newView.metadata.getColumnDefinition(name), builder);
+            addColumnToSchemaMutation(newView.metadata, newView.metadata.getColumn(name), builder);
 
         // dropped columns
-        MapDifference<ByteBuffer, CFMetaData.DroppedColumn> droppedColumnDiff =
-        Maps.difference(oldView.metadata.getDroppedColumns(), oldView.metadata.getDroppedColumns());
+        MapDifference<ByteBuffer, DroppedColumn> droppedColumnDiff =
+        Maps.difference(oldView.metadata.droppedColumns, oldView.metadata.droppedColumns);
 
         // newly dropped columns
-        for (CFMetaData.DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values())
+        for (DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values())
             addDroppedColumnToSchemaMutation(oldView.metadata, column, builder);
 
         // columns added then dropped again
         for (ByteBuffer name : droppedColumnDiff.entriesDiffering().keySet())
-            addDroppedColumnToSchemaMutation(newView.metadata, newView.metadata.getDroppedColumns().get(name), builder);
+            addDroppedColumnToSchemaMutation(newView.metadata, newView.metadata.droppedColumns.get(name), builder);
 
         return builder;
     }
 
-    private static void addIndexToSchemaMutation(CFMetaData table,
-                                                 IndexMetadata index,
-                                                 Mutation.SimpleBuilder builder)
+    private static void addIndexToSchemaMutation(TableMetadata table, IndexMetadata index, Mutation.SimpleBuilder builder)
     {
         builder.update(Indexes)
-               .row(table.cfName, index.name)
+               .row(table.name, index.name)
                .add("kind", index.kind.toString())
                .add("options", index.options);
     }
 
-    private static void dropIndexFromSchemaMutation(CFMetaData table,
-                                                    IndexMetadata index,
-                                                    Mutation.SimpleBuilder builder)
+    private static void dropIndexFromSchemaMutation(TableMetadata table, IndexMetadata index, Mutation.SimpleBuilder builder)
     {
-        builder.update(Indexes).row(table.cfName, index.name).delete();
+        builder.update(Indexes).row(table.name, index.name).delete();
     }
 
-    private static void addUpdatedIndexToSchemaMutation(CFMetaData table,
+    private static void addUpdatedIndexToSchemaMutation(TableMetadata table,
                                                         IndexMetadata index,
                                                         Mutation.SimpleBuilder builder)
     {
         addIndexToSchemaMutation(table, index, builder);
     }
 
-    public static Mutation.SimpleBuilder makeCreateFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
+    static Mutation.SimpleBuilder makeCreateFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
@@ -803,7 +795,7 @@ public final class SchemaKeyspace
     static void addFunctionToSchemaMutation(UDFunction function, Mutation.SimpleBuilder builder)
     {
         builder.update(Functions)
-               .row(function.name().name, functionArgumentsList(function))
+               .row(function.name().name, function.argumentsList())
                .add("body", function.body())
                .add("language", function.language())
                .add("return_type", function.returnType().asCQL3Type().toString())
@@ -823,24 +815,15 @@ public final class SchemaKeyspace
         }
     }
 
-    private static List<String> functionArgumentsList(AbstractFunction fun)
-    {
-        return fun.argTypes()
-                  .stream()
-                  .map(AbstractType::asCQL3Type)
-                  .map(CQL3Type::toString)
-                  .collect(toList());
-    }
-
-    public static Mutation.SimpleBuilder makeDropFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
+    static Mutation.SimpleBuilder makeDropFunctionMutation(KeyspaceMetadata keyspace, UDFunction function, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        builder.update(Functions).row(function.name().name, functionArgumentsList(function)).delete();
+        builder.update(Functions).row(function.name().name, function.argumentsList()).delete();
         return builder;
     }
 
-    public static Mutation.SimpleBuilder makeCreateAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
+    static Mutation.SimpleBuilder makeCreateAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
@@ -851,7 +834,7 @@ public final class SchemaKeyspace
     static void addAggregateToSchemaMutation(UDAggregate aggregate, Mutation.SimpleBuilder builder)
     {
         builder.update(Aggregates)
-               .row(aggregate.name().name, functionArgumentsList(aggregate))
+               .row(aggregate.name().name, aggregate.argumentsList())
                .add("return_type", aggregate.returnType().asCQL3Type().toString())
                .add("state_func", aggregate.stateFunction().name().name)
                .add("state_type", aggregate.stateType().asCQL3Type().toString())
@@ -862,11 +845,11 @@ public final class SchemaKeyspace
                                 : null);
     }
 
-    public static Mutation.SimpleBuilder makeDropAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
+    static Mutation.SimpleBuilder makeDropAggregateMutation(KeyspaceMetadata keyspace, UDAggregate aggregate, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
         Mutation.SimpleBuilder builder = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
-        builder.update(Aggregates).row(aggregate.name().name, functionArgumentsList(aggregate)).delete();
+        builder.update(Aggregates).row(aggregate.name().name, aggregate.argumentsList()).delete();
         return builder;
     }
 
@@ -874,7 +857,7 @@ public final class SchemaKeyspace
      * Fetching schema
      */
 
-    public static Keyspaces fetchNonSystemKeyspaces()
+    static Keyspaces fetchNonSystemKeyspaces()
     {
         return fetchKeyspacesWithout(SchemaConstants.SYSTEM_KEYSPACE_NAMES);
     }
@@ -893,20 +876,6 @@ public final class SchemaKeyspace
         return keyspaces.build();
     }
 
-    private static Keyspaces fetchKeyspacesOnly(Set<String> includedKeyspaceNames)
-    {
-        /*
-         * We know the keyspace names we are going to query, but we still want to run the SELECT IN
-         * query, to filter out the keyspaces that had been dropped by the applied mutation set.
-         */
-        String query = format("SELECT keyspace_name FROM %s.%s WHERE keyspace_name IN ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, KEYSPACES);
-
-        Keyspaces.Builder keyspaces = org.apache.cassandra.schema.Keyspaces.builder();
-        for (UntypedResultSet.Row row : query(query, new ArrayList<>(includedKeyspaceNames)))
-            keyspaces.add(fetchKeyspace(row.getString("keyspace_name")));
-        return keyspaces.build();
-    }
-
     private static KeyspaceMetadata fetchKeyspace(String keyspaceName)
     {
         KeyspaceParams params = fetchKeyspaceParams(keyspaceName);
@@ -952,7 +921,7 @@ public final class SchemaKeyspace
         return tables.build();
     }
 
-    private static CFMetaData fetchTable(String keyspaceName, String tableName, Types types)
+    private static TableMetadata fetchTable(String keyspaceName, String tableName, Types types)
     {
         String query = String.format("SELECT * FROM %s.%s WHERE keyspace_name = ? AND table_name = ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, TABLES);
         UntypedResultSet rows = query(query, keyspaceName, tableName);
@@ -960,43 +929,17 @@ public final class SchemaKeyspace
             throw new RuntimeException(String.format("%s:%s not found in the schema definitions keyspace.", keyspaceName, tableName));
         UntypedResultSet.Row row = rows.one();
 
-        UUID id = row.getUUID("id");
-
-        Set<CFMetaData.Flag> flags = CFMetaData.flagsFromStrings(row.getFrozenSet("flags", UTF8Type.instance));
-
-        boolean isSuper = flags.contains(CFMetaData.Flag.SUPER);
-        boolean isCounter = flags.contains(CFMetaData.Flag.COUNTER);
-        boolean isDense = flags.contains(CFMetaData.Flag.DENSE);
-        boolean isCompound = flags.contains(CFMetaData.Flag.COMPOUND);
-
-        List<ColumnDefinition> columns = fetchColumns(keyspaceName, tableName, types);
-        if (!columns.stream().anyMatch(ColumnDefinition::isPartitionKey))
-        {
-            String msg = String.format("Table %s.%s did not have any partition key columns in the schema tables", keyspaceName, tableName);
-            throw new AssertionError(msg);
-        }
-
-        Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns = fetchDroppedColumns(keyspaceName, tableName);
-        Indexes indexes = fetchIndexes(keyspaceName, tableName);
-        Triggers triggers = fetchTriggers(keyspaceName, tableName);
-
-        return CFMetaData.create(keyspaceName,
-                                 tableName,
-                                 id,
-                                 isDense,
-                                 isCompound,
-                                 isSuper,
-                                 isCounter,
-                                 false,
-                                 columns,
-                                 DatabaseDescriptor.getPartitioner())
-                         .params(createTableParamsFromRow(row))
-                         .droppedColumns(droppedColumns)
-                         .indexes(indexes)
-                         .triggers(triggers);
+        return TableMetadata.builder(keyspaceName, tableName, TableId.fromUUID(row.getUUID("id")))
+                            .flags(TableMetadata.Flag.fromStringSet(row.getFrozenSet("flags", UTF8Type.instance)))
+                            .params(createTableParamsFromRow(row))
+                            .addColumns(fetchColumns(keyspaceName, tableName, types))
+                            .droppedColumns(fetchDroppedColumns(keyspaceName, tableName))
+                            .indexes(fetchIndexes(keyspaceName, tableName))
+                            .triggers(fetchTriggers(keyspaceName, tableName))
+                            .build();
     }
 
-    public static TableParams createTableParamsFromRow(UntypedResultSet.Row row)
+    static TableParams createTableParamsFromRow(UntypedResultSet.Row row)
     {
         return TableParams.builder()
                           .bloomFilterFpChance(row.getDouble("bloom_filter_fp_chance"))
@@ -1014,29 +957,29 @@ public final class SchemaKeyspace
                           .readRepairChance(row.getDouble("read_repair_chance"))
                           .crcCheckChance(row.getDouble("crc_check_chance"))
                           .speculativeRetry(SpeculativeRetryParam.fromString(row.getString("speculative_retry")))
-                          .cdc(row.has("cdc") ? row.getBoolean("cdc") : false)
+                          .cdc(row.has("cdc") && row.getBoolean("cdc"))
                           .build();
     }
 
-    private static List<ColumnDefinition> fetchColumns(String keyspace, String table, Types types)
+    private static List<ColumnMetadata> fetchColumns(String keyspace, String table, Types types)
     {
         String query = format("SELECT * FROM %s.%s WHERE keyspace_name = ? AND table_name = ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, COLUMNS);
-        List<ColumnDefinition> columns = new ArrayList<>();
+        List<ColumnMetadata> columns = new ArrayList<>();
         query(query, keyspace, table).forEach(row -> columns.add(createColumnFromRow(row, types)));
         return columns;
     }
 
-    public static ColumnDefinition createColumnFromRow(UntypedResultSet.Row row, Types types)
+    static ColumnMetadata createColumnFromRow(UntypedResultSet.Row row, Types types)
     {
         String keyspace = row.getString("keyspace_name");
         String table = row.getString("table_name");
 
-        ColumnDefinition.Kind kind = ColumnDefinition.Kind.valueOf(row.getString("kind").toUpperCase());
+        ColumnMetadata.Kind kind = ColumnMetadata.Kind.valueOf(row.getString("kind").toUpperCase());
 
         int position = row.getInt("position");
         ClusteringOrder order = ClusteringOrder.valueOf(row.getString("clustering_order").toUpperCase());
 
-        AbstractType<?> type = parse(keyspace, row.getString("type"), types);
+        AbstractType<?> type = CQLTypeParser.parse(keyspace, row.getString("type"), types);
         if (order == ClusteringOrder.DESC)
             type = ReversedType.getInstance(type);
 
@@ -1044,38 +987,41 @@ public final class SchemaKeyspace
                                                              row.getBytes("column_name_bytes"),
                                                              row.getString("column_name"));
 
-        return new ColumnDefinition(keyspace, table, name, type, position, kind);
+        return new ColumnMetadata(keyspace, table, name, type, position, kind);
     }
 
-    private static Map<ByteBuffer, CFMetaData.DroppedColumn> fetchDroppedColumns(String keyspace, String table)
+    private static Map<ByteBuffer, DroppedColumn> fetchDroppedColumns(String keyspace, String table)
     {
         String query = format("SELECT * FROM %s.%s WHERE keyspace_name = ? AND table_name = ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, DROPPED_COLUMNS);
-        Map<ByteBuffer, CFMetaData.DroppedColumn> columns = new HashMap<>();
+        Map<ByteBuffer, DroppedColumn> columns = new HashMap<>();
         for (UntypedResultSet.Row row : query(query, keyspace, table))
         {
-            CFMetaData.DroppedColumn column = createDroppedColumnFromRow(row);
-            columns.put(UTF8Type.instance.decompose(column.name), column);
+            DroppedColumn column = createDroppedColumnFromRow(row);
+            columns.put(column.column.name.bytes, column);
         }
         return columns;
     }
 
-    private static CFMetaData.DroppedColumn createDroppedColumnFromRow(UntypedResultSet.Row row)
+    private static DroppedColumn createDroppedColumnFromRow(UntypedResultSet.Row row)
     {
         String keyspace = row.getString("keyspace_name");
+        String table = row.getString("table_name");
         String name = row.getString("column_name");
         /*
          * we never store actual UDT names in dropped column types (so that we can safely drop types if nothing refers to
          * them anymore), so before storing dropped columns in schema we expand UDTs to tuples. See expandUserTypes method.
          * Because of that, we can safely pass Types.none() to parse()
          */
-        AbstractType<?> type = parse(keyspace, row.getString("type"), org.apache.cassandra.schema.Types.none());
+        AbstractType<?> type = CQLTypeParser.parse(keyspace, row.getString("type"), org.apache.cassandra.schema.Types.none());
+        ColumnMetadata.Kind kind = row.has("kind")
+                                 ? ColumnMetadata.Kind.valueOf(row.getString("kind").toUpperCase())
+                                 : ColumnMetadata.Kind.REGULAR;
+        assert kind == ColumnMetadata.Kind.REGULAR || kind == ColumnMetadata.Kind.STATIC
+            : "Unexpected dropped column kind: " + kind.toString();
+
+        ColumnMetadata column = new ColumnMetadata(keyspace, table, ColumnIdentifier.getInterned(name, true), type, ColumnMetadata.NO_POSITION, kind);
         long droppedTime = TimeUnit.MILLISECONDS.toMicros(row.getLong("dropped_time"));
-        ColumnDefinition.Kind kind = row.has("kind")
-                                     ? ColumnDefinition.Kind.valueOf(row.getString("kind").toUpperCase())
-                                     : ColumnDefinition.Kind.REGULAR;
-        assert kind == ColumnDefinition.Kind.REGULAR || kind == ColumnDefinition.Kind.STATIC
-            : "Unexpected dropped column kind: " + kind;
-        return new CFMetaData.DroppedColumn(name, type, droppedTime, kind);
+        return new DroppedColumn(column, droppedTime);
     }
 
     private static Indexes fetchIndexes(String keyspace, String table)
@@ -1119,7 +1065,7 @@ public final class SchemaKeyspace
         return views.build();
     }
 
-    private static ViewDefinition fetchView(String keyspaceName, String viewName, Types types)
+    private static ViewMetadata fetchView(String keyspaceName, String viewName, Types types)
     {
         String query = String.format("SELECT * FROM %s.%s WHERE keyspace_name = ? AND view_name = ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, VIEWS);
         UntypedResultSet rows = query(query, keyspaceName, viewName);
@@ -1127,33 +1073,25 @@ public final class SchemaKeyspace
             throw new RuntimeException(String.format("%s:%s not found in the schema definitions keyspace.", keyspaceName, viewName));
         UntypedResultSet.Row row = rows.one();
 
-        UUID id = row.getUUID("id");
-        UUID baseTableId = row.getUUID("base_table_id");
+        TableId baseTableId = TableId.fromUUID(row.getUUID("base_table_id"));
         String baseTableName = row.getString("base_table_name");
         boolean includeAll = row.getBoolean("include_all_columns");
         String whereClause = row.getString("where_clause");
 
-        List<ColumnDefinition> columns = fetchColumns(keyspaceName, viewName, types);
+        List<ColumnMetadata> columns = fetchColumns(keyspaceName, viewName, types);
 
-        Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns = fetchDroppedColumns(keyspaceName, viewName);
-
-        CFMetaData cfm = CFMetaData.create(keyspaceName,
-                                           viewName,
-                                           id,
-                                           false,
-                                           true,
-                                           false,
-                                           false,
-                                           true,
-                                           columns,
-                                           DatabaseDescriptor.getPartitioner())
-                                   .params(createTableParamsFromRow(row))
-                                   .droppedColumns(droppedColumns);
+        TableMetadata metadata =
+            TableMetadata.builder(keyspaceName, viewName, TableId.fromUUID(row.getUUID("id")))
+                         .isView(true)
+                         .addColumns(columns)
+                         .droppedColumns(fetchDroppedColumns(keyspaceName, viewName))
+                         .params(createTableParamsFromRow(row))
+                         .build();
 
-            String rawSelect = View.buildSelectStatement(baseTableName, columns, whereClause);
-            SelectStatement.RawStatement rawStatement = (SelectStatement.RawStatement) QueryProcessor.parseStatement(rawSelect);
+        String rawSelect = View.buildSelectStatement(baseTableName, columns, whereClause);
+        SelectStatement.RawStatement rawStatement = (SelectStatement.RawStatement) QueryProcessor.parseStatement(rawSelect);
 
-            return new ViewDefinition(keyspaceName, viewName, baseTableId, baseTableName, includeAll, rawStatement, whereClause, cfm);
+        return new ViewMetadata(keyspaceName, viewName, baseTableId, baseTableName, includeAll, rawStatement, whereClause, metadata);
     }
 
     private static Functions fetchFunctions(String keyspaceName, Types types)
@@ -1189,14 +1127,18 @@ public final class SchemaKeyspace
 
         List<AbstractType<?>> argTypes = new ArrayList<>();
         for (String type : row.getFrozenList("argument_types", UTF8Type.instance))
-            argTypes.add(parse(ksName, type, types));
+            argTypes.add(CQLTypeParser.parse(ksName, type, types));
 
-        AbstractType<?> returnType = parse(ksName, row.getString("return_type"), types);
+        AbstractType<?> returnType = CQLTypeParser.parse(ksName, row.getString("return_type"), types);
 
         String language = row.getString("language");
         String body = row.getString("body");
         boolean calledOnNullInput = row.getBoolean("called_on_null_input");
 
+        /*
+         * TODO: find a way to get rid of Schema.instance dependency; evaluate if the opimisation below makes a difference
+         * in the first place. Remove if it isn't.
+         */
         org.apache.cassandra.cql3.functions.Function existing = Schema.instance.findFunction(name, argTypes).orElse(null);
         if (existing instanceof UDFunction)
         {
@@ -1205,7 +1147,8 @@ public final class SchemaKeyspace
             // statement, since CreateFunctionStatement needs to execute UDFunction.create but schema migration
             // also needs that (since it needs to handle its own change).
             UDFunction udf = (UDFunction) existing;
-            if (udf.argNames().equals(argNames) && // arg types checked in Functions.find call
+            if (udf.argNames().equals(argNames) &&
+                udf.argTypes().equals(argTypes) &&
                 udf.returnType().equals(returnType) &&
                 !udf.isAggregate() &&
                 udf.language().equals(language) &&
@@ -1247,14 +1190,14 @@ public final class SchemaKeyspace
         List<AbstractType<?>> argTypes =
             row.getFrozenList("argument_types", UTF8Type.instance)
                .stream()
-               .map(t -> parse(ksName, t, types))
+               .map(t -> CQLTypeParser.parse(ksName, t, types))
                .collect(toList());
 
-        AbstractType<?> returnType = parse(ksName, row.getString("return_type"), types);
+        AbstractType<?> returnType = CQLTypeParser.parse(ksName, row.getString("return_type"), types);
 
         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") ? parse(ksName, row.getString("state_type"), types) : null;
+        AbstractType<?> stateType = row.has("state_type") ? CQLTypeParser.parse(ksName, row.getString("state_type"), types) : null;
         ByteBuffer initcond = row.has("initcond") ? Terms.asBytes(ksName, row.getString("initcond"), stateType) : null;
 
         try
@@ -1277,116 +1220,35 @@ public final class SchemaKeyspace
      */
 
     /**
-     * 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
+     * Computes the set of names of keyspaces affected by the provided schema mutations.
      */
-    public static synchronized void mergeSchemaAndAnnounceVersion(Collection<Mutation> mutations) throws ConfigurationException
+    static Set<String> affectedKeyspaces(Collection<Mutation> mutations)
     {
-        mergeSchema(mutations);
-        Schema.instance.updateVersionAndAnnounce();
+        // only compare the keyspaces affected by this set of schema mutations
+        return mutations.stream()
+                        .map(m -> UTF8Type.instance.compose(m.key().getKey()))
+                        .collect(toSet());
     }
 
-    public static synchronized void mergeSchema(Collection<Mutation> mutations)
+    static void applyChanges(Collection<Mutation> mutations)
     {
-        // only compare the keyspaces affected by this set of schema mutations
-        Set<String> affectedKeyspaces =
-        mutations.stream()
-                 .map(m -> UTF8Type.instance.compose(m.key().getKey()))
-                 .collect(Collectors.toSet());
-
-        // fetch the current state of schema for the affected keyspaces only
-        Keyspaces before = Schema.instance.getKeyspaces(affectedKeyspaces);
-
-        // apply the schema mutations and flush
         mutations.forEach(Mutation::apply);
-        if (FLUSH_SCHEMA_TABLES)
-            flush();
-
-        // fetch the new state of schema from schema tables (not applied to Schema.instance yet)
-        Keyspaces after = fetchKeyspacesOnly(affectedKeyspaces);
-
-        // deal with the diff
-        MapDifference<String, KeyspaceMetadata> keyspacesDiff = before.diff(after);
-
-        // dropped keyspaces
-        for (KeyspaceMetadata keyspace : keyspacesDiff.entriesOnlyOnLeft().values())
-        {
-            keyspace.functions.udas().forEach(Schema.instance::dropAggregate);
-            keyspace.functions.udfs().forEach(Schema.instance::dropFunction);
-            keyspace.views.forEach(v -> Schema.instance.dropView(v.ksName, v.viewName));
-            keyspace.tables.forEach(t -> Schema.instance.dropTable(t.ksName, t.cfName));
-            keyspace.types.forEach(Schema.instance::dropType);
-            Schema.instance.dropKeyspace(keyspace.name);
-        }
+        if (SchemaKeyspace.FLUSH_SCHEMA_TABLES)
+            SchemaKeyspace.flush();
+    }
 
-        // new keyspaces
-        for (KeyspaceMetadata keyspace : keyspacesDiff.entriesOnlyOnRight().values())
-        {
-            Schema.instance.addKeyspace(KeyspaceMetadata.create(keyspace.name, keyspace.params));
-            keyspace.types.forEach(Schema.instance::addType);
-            keyspace.tables.forEach(Schema.instance::addTable);
-            keyspace.views.forEach(Schema.instance::addView);
-            keyspace.functions.udfs().forEach(Schema.instance::addFunction);
-            keyspace.functions.udas().forEach(Schema.instance::addAggregate);
-        }
+    static Keyspaces fetchKeyspaces(Set<String> toFetch)
+    {
+        /*
+         * We know the keyspace names we are going to query, but we still want to run the SELECT IN
+         * query, to filter out the keyspaces that had been dropped by the applied mutation set.
+         */
+        String query = format("SELECT keyspace_name FROM %s.%s WHERE keyspace_name IN ?", SchemaConstants.SCHEMA_KEYSPACE_NAME, KEYSPACES);
 
-        // updated keyspaces
-        for (Map.Entry<String, MapDifference.ValueDifference<KeyspaceMetadata>> diff : keyspacesDiff.entriesDiffering().entrySet())
-            updateKeyspace(diff.getKey(), diff.getValue().leftValue(), diff.getValue().rightValue());
-    }
-
-    private static void updateKeyspace(String keyspaceName, KeyspaceMetadata keyspaceBefore, KeyspaceMetadata keyspaceAfter)
-    {
-        // calculate the deltas
-        MapDifference<String, CFMetaData> tablesDiff = keyspaceBefore.tables.diff(keyspaceAfter.tables);
-        MapDifference<String, ViewDefinition> viewsDiff = keyspaceBefore.views.diff(keyspaceAfter.views);
-        MapDifference<ByteBuffer, UserType> typesDiff = keyspaceBefore.types.diff(keyspaceAfter.types);
-
-        Map<Pair<FunctionName, List<String>>, UDFunction> udfsBefore = new HashMap<>();
-        keyspaceBefore.functions.udfs().forEach(f -> udfsBefore.put(Pair.create(f.name(), functionArgumentsList(f)), f));
-        Map<Pair<FunctionName, List<String>>, UDFunction> udfsAfter = new HashMap<>();
-        keyspaceAfter.functions.udfs().forEach(f -> udfsAfter.put(Pair.create(f.name(), functionArgumentsList(f)), f));
-        MapDifference<Pair<FunctionName, List<String>>, UDFunction> udfsDiff = Maps.difference(udfsBefore, udfsAfter);
-
-        Map<Pair<FunctionName, List<String>>, UDAggregate> udasBefore = new HashMap<>();
-        keyspaceBefore.functions.udas().forEach(f -> udasBefore.put(Pair.create(f.name(), functionArgumentsList(f)), f));
-        Map<Pair<FunctionName, List<String>>, UDAggregate> udasAfter = new HashMap<>();
-        keyspaceAfter.functions.udas().forEach(f -> udasAfter.put(Pair.create(f.name(), functionArgumentsList(f)), f));
-        MapDifference<Pair<FunctionName, List<String>>, UDAggregate> udasDiff = Maps.difference(udasBefore, udasAfter);
-
-        // update keyspace params, if changed
-        if (!keyspaceBefore.params.equals(keyspaceAfter.params))
-            Schema.instance.updateKeyspace(keyspaceName, keyspaceAfter.params);
-
-        // drop everything removed
-        udasDiff.entriesOnlyOnLeft().values().forEach(Schema.instance::dropAggregate);
-        udfsDiff.entriesOnlyOnLeft().values().forEach(Schema.instance::dropFunction);
-        viewsDiff.entriesOnlyOnLeft().values().forEach(v -> Schema.instance.dropView(v.ksName, v.viewName));
-        tablesDiff.entriesOnlyOnLeft().values().forEach(t -> Schema.instance.dropTable(t.ksName, t.cfName));
-        typesDiff.entriesOnlyOnLeft().values().forEach(Schema.instance::dropType);
-
-        // add everything created
-        typesDiff.entriesOnlyOnRight().values().forEach(Schema.instance::addType);
-        tablesDiff.entriesOnlyOnRight().values().forEach(Schema.instance::addTable);
-        viewsDiff.entriesOnlyOnRight().values().forEach(Schema.instance::addView);
-        udfsDiff.entriesOnlyOnRight().values().forEach(Schema.instance::addFunction);
-        udasDiff.entriesOnlyOnRight().values().forEach(Schema.instance::addAggregate);
-
-        // update everything altered
-        for (MapDifference.ValueDifference<UserType> diff : typesDiff.entriesDiffering().values())
-            Schema.instance.updateType(diff.rightValue());
-        for (MapDifference.ValueDifference<CFMetaData> diff : tablesDiff.entriesDiffering().values())
-            Schema.instance.updateTable(diff.rightValue());
-        for (MapDifference.ValueDifference<ViewDefinition> diff : viewsDiff.entriesDiffering().values())
-            Schema.instance.updateView(diff.rightValue());
-        for (MapDifference.ValueDifference<UDFunction> diff : udfsDiff.entriesDiffering().values())
-            Schema.instance.updateFunction(diff.rightValue());
-        for (MapDifference.ValueDifference<UDAggregate> diff : udasDiff.entriesDiffering().values())
-            Schema.instance.updateAggregate(diff.rightValue());
+        Keyspaces.Builder keyspaces = org.apache.cassandra.schema.Keyspaces.builder();
+        for (UntypedResultSet.Row row : query(query, new ArrayList<>(toFetch)))
+            keyspaces.add(fetchKeyspace(row.getString("keyspace_name")));
+        return keyspaces.build();
     }
 
     /*

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java b/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java
new file mode 100644
index 0000000..45cf365
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/SchemaPullVerbHandler.java
@@ -0,0 +1,50 @@
+/*
+ * 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.util.Collection;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.net.IVerbHandler;
+import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.net.MessageOut;
+import org.apache.cassandra.net.MessagingService;
+
+/**
+ * Sends it's current schema state in form of mutations in reply to the remote node's request.
+ * Such a request is made when one of the nodes, by means of Gossip, detects schema disagreement in the ring.
+ */
+public final class SchemaPullVerbHandler implements IVerbHandler
+{
+    private static final Logger logger = LoggerFactory.getLogger(SchemaPullVerbHandler.class);
+
+    public void doVerb(MessageIn message, int id)
+    {
+        logger.trace("Received schema pull request from {}", message.from);
+
+        MessageOut<Collection<Mutation>> response =
+            new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE,
+                             SchemaKeyspace.convertSchemaToMutations(),
+                             MigrationManager.MigrationsSerializer.instance);
+
+        MessagingService.instance().sendReply(response, id, message.from);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/SchemaPushVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaPushVerbHandler.java b/src/java/org/apache/cassandra/schema/SchemaPushVerbHandler.java
new file mode 100644
index 0000000..f939cda
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/SchemaPushVerbHandler.java
@@ -0,0 +1,47 @@
+/*
+ * 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.util.Collection;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.net.IVerbHandler;
+import org.apache.cassandra.net.MessageIn;
+
+/**
+ * Called when node receives updated schema state from the schema migration coordinator node.
+ * Such happens when user makes local schema migration on one of the nodes in the ring
+ * (which is going to act as coordinator) and that node sends (pushes) it's updated schema state
+ * (in form of mutations) to all the alive nodes in the cluster.
+ */
+public final class SchemaPushVerbHandler implements IVerbHandler<Collection<Mutation>>
+{
+    private static final Logger logger = LoggerFactory.getLogger(SchemaPushVerbHandler.class);
+
+    public void doVerb(final MessageIn<Collection<Mutation>> message, int id)
+    {
+        logger.trace("Received schema push request from {}", message.from);
+
+        StageManager.getStage(Stage.MIGRATION).submit(() -> Schema.instance.mergeAndAnnounceVersion(message.payload));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/SchemaVersionVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaVersionVerbHandler.java b/src/java/org/apache/cassandra/schema/SchemaVersionVerbHandler.java
new file mode 100644
index 0000000..0a506e3
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/SchemaVersionVerbHandler.java
@@ -0,0 +1,46 @@
+/*
+ * 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.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.utils.UUIDSerializer;
+
+public final class SchemaVersionVerbHandler implements IVerbHandler
+{
+    private final Logger logger = LoggerFactory.getLogger(SchemaVersionVerbHandler.class);
+
+    public void doVerb(MessageIn message, int id)
+    {
+        logger.trace("Received schema version request from {}", message.from);
+
+        MessageOut<UUID> response =
+            new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE,
+                             Schema.instance.getVersion(),
+                             UUIDSerializer.serializer);
+
+        MessagingService.instance().sendReply(response, id, message.from);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/schema/TableId.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/TableId.java b/src/java/org/apache/cassandra/schema/TableId.java
new file mode 100644
index 0000000..4b2592e
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/TableId.java
@@ -0,0 +1,118 @@
+/*
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
+
+/**
+ * The unique identifier of a table.
+ * <p>
+ * This is essentially a UUID, but we wrap it as it's used quite a bit in the code and having a nicely name class make
+ * the code more readable.
+ */
+public class TableId
+{
+    private final UUID id;
+
+    private TableId(UUID id)
+    {
+        this.id = id;
+    }
+
+    public static TableId fromUUID(UUID id)
+    {
+        return new TableId(id);
+    }
+
+    public static TableId generate()
+    {
+        return new TableId(UUIDGen.getTimeUUID());
+    }
+
+    public static TableId fromString(String idString)
+    {
+        return new TableId(UUID.fromString(idString));
+    }
+
+    /**
+     * Creates the UUID of a system table.
+     *
+     * This is deterministically based on the table name as system tables are hardcoded and initialized independently
+     * on each node (they don't go through a CREATE), but we still want them to have the same ID everywhere.
+     *
+     * We shouldn't use this for any other table.
+     */
+    public static TableId forSystemTable(String keyspace, String table)
+    {
+        assert SchemaConstants.SYSTEM_KEYSPACE_NAMES.contains(keyspace)
+               || SchemaConstants.REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(keyspace);
+        return new TableId(UUID.nameUUIDFromBytes(ArrayUtils.addAll(keyspace.getBytes(), table.getBytes())));
+    }
+
+    public String toHexString()
+    {
+        return ByteBufferUtil.bytesToHex(ByteBufferUtil.bytes(id));
+    }
+
+    public UUID asUUID()
+    {
+        return id;
+    }
+
+    @Override
+    public final int hashCode()
+    {
+        return id.hashCode();
+    }
+
+    @Override
+    public final boolean equals(Object o)
+    {
+        return this == o || (o instanceof TableId && this.id.equals(((TableId) o).id));
+    }
+
+    @Override
+    public String toString()
+    {
+        return id.toString();
+    }
+
+    public void serialize(DataOutput out) throws IOException
+    {
+        out.writeLong(id.getMostSignificantBits());
+        out.writeLong(id.getLeastSignificantBits());
+    }
+
+    public int serializedSize()
+    {
+        return 16;
+    }
+
+    public static TableId deserialize(DataInput in) throws IOException
+    {
+        return new TableId(new UUID(in.readLong(), in.readLong()));
+    }
+}


[09/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/ColumnsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnsTest.java b/test/unit/org/apache/cassandra/db/ColumnsTest.java
index d64a5bd..ae0bbd8 100644
--- a/test/unit/org/apache/cassandra/db/ColumnsTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnsTest.java
@@ -30,14 +30,14 @@ import org.junit.AfterClass;
 import org.junit.Test;
 
 import junit.framework.Assert;
-import org.apache.cassandra.MockSchema;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.SetType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.MockSchema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.btree.BTreeSet;
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
@@ -49,7 +49,7 @@ public class ColumnsTest
         DatabaseDescriptor.daemonInitialization();
     }
 
-    private static final CFMetaData cfMetaData = MockSchema.newCFS().metadata;
+    private static final TableMetadata TABLE_METADATA = MockSchema.newCFS().metadata();
 
     // this tests most of our functionality, since each subset we perform
     // reasonably comprehensive tests of basic functionality against
@@ -64,8 +64,8 @@ public class ColumnsTest
     {
         // pick some arbitrary groupings of columns to remove at-once (to avoid factorial complexity)
         // whatever is left after each removal, we perform this logic on again, recursively
-        List<List<ColumnDefinition>> removeGroups = shuffleAndGroup(Lists.newArrayList(input.definitions));
-        for (List<ColumnDefinition> defs : removeGroups)
+        List<List<ColumnMetadata>> removeGroups = shuffleAndGroup(Lists.newArrayList(input.definitions));
+        for (List<ColumnMetadata> defs : removeGroups)
         {
             ColumnsCheck subset = input.remove(defs);
 
@@ -77,7 +77,7 @@ public class ColumnsTest
 
             // test .mergeTo
             Columns otherSubset = input.columns;
-            for (ColumnDefinition def : subset.definitions)
+            for (ColumnMetadata def : subset.definitions)
             {
                 otherSubset = otherSubset.without(def);
                 assertContents(otherSubset.mergeTo(subset.columns), input.definitions);
@@ -102,7 +102,7 @@ public class ColumnsTest
             testSerialize(randomColumns.columns, randomColumns.definitions);
     }
 
-    private void testSerialize(Columns columns, List<ColumnDefinition> definitions) throws IOException
+    private void testSerialize(Columns columns, List<ColumnMetadata> definitions) throws IOException
     {
         try (DataOutputBuffer out = new DataOutputBuffer())
         {
@@ -136,7 +136,7 @@ public class ColumnsTest
         for (int i = 0; i < 50; i++)
             names.add("clustering_" + i);
 
-        List<ColumnDefinition> defs = new ArrayList<>();
+        List<ColumnMetadata> defs = new ArrayList<>();
         addClustering(names, defs);
 
         Columns columns = Columns.from(new HashSet<>(defs));
@@ -153,8 +153,8 @@ public class ColumnsTest
     {
         testSerializeSubset(input.columns, input.columns, input.definitions);
         testSerializeSubset(input.columns, Columns.NONE, Collections.emptyList());
-        List<List<ColumnDefinition>> removeGroups = shuffleAndGroup(Lists.newArrayList(input.definitions));
-        for (List<ColumnDefinition> defs : removeGroups)
+        List<List<ColumnMetadata>> removeGroups = shuffleAndGroup(Lists.newArrayList(input.definitions));
+        for (List<ColumnMetadata> defs : removeGroups)
         {
             Collections.sort(defs);
             ColumnsCheck subset = input.remove(defs);
@@ -162,7 +162,7 @@ public class ColumnsTest
         }
     }
 
-    private void testSerializeSubset(Columns superset, Columns subset, List<ColumnDefinition> subsetDefinitions) throws IOException
+    private void testSerializeSubset(Columns superset, Columns subset, List<ColumnMetadata> subsetDefinitions) throws IOException
     {
         try (DataOutputBuffer out = new DataOutputBuffer())
         {
@@ -175,17 +175,17 @@ public class ColumnsTest
         }
     }
 
-    private static void assertContents(Columns columns, List<ColumnDefinition> defs)
+    private static void assertContents(Columns columns, List<ColumnMetadata> defs)
     {
         Assert.assertEquals(defs, Lists.newArrayList(columns));
         boolean hasSimple = false, hasComplex = false;
         int firstComplexIdx = 0;
         int i = 0;
-        Iterator<ColumnDefinition> simple = columns.simpleColumns();
-        Iterator<ColumnDefinition> complex = columns.complexColumns();
-        Iterator<ColumnDefinition> all = columns.iterator();
-        Predicate<ColumnDefinition> predicate = columns.inOrderInclusionTester();
-        for (ColumnDefinition def : defs)
+        Iterator<ColumnMetadata> simple = columns.simpleColumns();
+        Iterator<ColumnMetadata> complex = columns.complexColumns();
+        Iterator<ColumnMetadata> all = columns.iterator();
+        Predicate<ColumnMetadata> predicate = columns.inOrderInclusionTester();
+        for (ColumnMetadata def : defs)
         {
             Assert.assertEquals(def, all.next());
             Assert.assertTrue(columns.contains(def));
@@ -218,9 +218,9 @@ public class ColumnsTest
         // check select order
         if (!columns.hasSimple() || !columns.getSimple(0).kind.isPrimaryKeyKind())
         {
-            List<ColumnDefinition> selectOrderDefs = new ArrayList<>(defs);
+            List<ColumnMetadata> selectOrderDefs = new ArrayList<>(defs);
             Collections.sort(selectOrderDefs, (a, b) -> a.name.bytes.compareTo(b.name.bytes));
-            List<ColumnDefinition> selectOrderColumns = new ArrayList<>();
+            List<ColumnMetadata> selectOrderColumns = new ArrayList<>();
             Iterators.addAll(selectOrderColumns, columns.selectOrderIterator());
             Assert.assertEquals(selectOrderDefs, selectOrderColumns);
         }
@@ -262,27 +262,27 @@ public class ColumnsTest
     private static class ColumnsCheck
     {
         final Columns columns;
-        final List<ColumnDefinition> definitions;
+        final List<ColumnMetadata> definitions;
 
-        private ColumnsCheck(Columns columns, List<ColumnDefinition> definitions)
+        private ColumnsCheck(Columns columns, List<ColumnMetadata> definitions)
         {
             this.columns = columns;
             this.definitions = definitions;
         }
 
-        private ColumnsCheck(List<ColumnDefinition> definitions)
+        private ColumnsCheck(List<ColumnMetadata> definitions)
         {
             this.columns = Columns.from(BTreeSet.of(definitions));
             this.definitions = definitions;
         }
 
-        ColumnsCheck remove(List<ColumnDefinition> remove)
+        ColumnsCheck remove(List<ColumnMetadata> remove)
         {
             Columns subset = columns;
-            for (ColumnDefinition def : remove)
+            for (ColumnMetadata def : remove)
                 subset = subset.without(def);
             Assert.assertEquals(columns.size() - remove.size(), subset.size());
-            List<ColumnDefinition> remainingDefs = Lists.newArrayList(columns);
+            List<ColumnMetadata> remainingDefs = Lists.newArrayList(columns);
             remainingDefs.removeAll(remove);
             return new ColumnsCheck(subset, remainingDefs);
         }
@@ -332,7 +332,7 @@ public class ColumnsTest
         for (char c = 'a' ; c <= 'z' ; c++)
             names .add(Character.toString(c));
 
-        List<ColumnDefinition> result = new ArrayList<>();
+        List<ColumnMetadata> result = new ArrayList<>();
         addPartition(select(names, pkCount), result);
         addClustering(select(names, clCount), result);
         addRegular(select(names, regularCount), result);
@@ -356,7 +356,7 @@ public class ColumnsTest
 
     private static ColumnsCheck randomHuge(int pkCount, int clCount, int regularCount, int complexCount)
     {
-        List<ColumnDefinition> result = new ArrayList<>();
+        List<ColumnMetadata> result = new ArrayList<>();
         Set<String> usedNames = new HashSet<>();
         addPartition(names(pkCount, usedNames), result);
         addClustering(names(clCount, usedNames), result);
@@ -383,43 +383,44 @@ public class ColumnsTest
         return names;
     }
 
-    private static void addPartition(List<String> names, List<ColumnDefinition> results)
+    private static void addPartition(List<String> names, List<ColumnMetadata> results)
     {
         for (String name : names)
-            results.add(ColumnDefinition.partitionKeyDef(cfMetaData, bytes(name), UTF8Type.instance, 0));
+            results.add(ColumnMetadata.partitionKeyColumn(TABLE_METADATA, bytes(name), UTF8Type.instance, 0));
     }
 
-    private static void addClustering(List<String> names, List<ColumnDefinition> results)
+    private static void addClustering(List<String> names, List<ColumnMetadata> results)
     {
         int i = 0;
         for (String name : names)
-            results.add(ColumnDefinition.clusteringDef(cfMetaData, bytes(name), UTF8Type.instance, i++));
+            results.add(ColumnMetadata.clusteringColumn(TABLE_METADATA, bytes(name), UTF8Type.instance, i++));
     }
 
-    private static void addRegular(List<String> names, List<ColumnDefinition> results)
+    private static void addRegular(List<String> names, List<ColumnMetadata> results)
     {
         for (String name : names)
-            results.add(ColumnDefinition.regularDef(cfMetaData, bytes(name), UTF8Type.instance));
+            results.add(ColumnMetadata.regularColumn(TABLE_METADATA, bytes(name), UTF8Type.instance));
     }
 
-    private static <V> void addComplex(List<String> names, List<ColumnDefinition> results)
+    private static <V> void addComplex(List<String> names, List<ColumnMetadata> results)
     {
         for (String name : names)
-            results.add(ColumnDefinition.regularDef(cfMetaData, bytes(name), SetType.getInstance(UTF8Type.instance, true)));
+            results.add(ColumnMetadata.regularColumn(TABLE_METADATA, bytes(name), SetType.getInstance(UTF8Type.instance, true)));
     }
 
-    private static CFMetaData mock(Columns columns)
+    private static TableMetadata mock(Columns columns)
     {
         if (columns.isEmpty())
-            return cfMetaData;
-        CFMetaData.Builder builder = CFMetaData.Builder.create(cfMetaData.ksName, cfMetaData.cfName);
+            return TABLE_METADATA;
+
+        TableMetadata.Builder builder = TableMetadata.builder(TABLE_METADATA.keyspace, TABLE_METADATA.name);
         boolean hasPartitionKey = false;
-        for (ColumnDefinition def : columns)
+        for (ColumnMetadata def : columns)
         {
             switch (def.kind)
             {
                 case PARTITION_KEY:
-                    builder.addPartitionKey(def.name, def.type);
+                    builder.addPartitionKeyColumn(def.name, def.type);
                     hasPartitionKey = true;
                     break;
                 case CLUSTERING:
@@ -431,7 +432,7 @@ public class ColumnsTest
             }
         }
         if (!hasPartitionKey)
-            builder.addPartitionKey("219894021498309239rufejsfjdksfjheiwfhjes", UTF8Type.instance);
+            builder.addPartitionKeyColumn("219894021498309239rufejsfjdksfjheiwfhjes", UTF8Type.instance);
         return builder.build();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/CounterCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCacheTest.java b/test/unit/org/apache/cassandra/db/CounterCacheTest.java
index 4cfd848..a913133 100644
--- a/test/unit/org/apache/cassandra/db/CounterCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCacheTest.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.db;
 import java.util.Collections;
 import java.util.concurrent.ExecutionException;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -33,7 +33,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.service.CacheService;
 
@@ -52,15 +52,14 @@ public class CounterCacheTest
     {
         SchemaLoader.prepareServer();
 
-        CFMetaData counterTable = CFMetaData.Builder.create(KEYSPACE1, COUNTER1, false, true, true)
-                                  .addPartitionKey("key", Int32Type.instance)
-                                  .addClusteringColumn("name", Int32Type.instance)
-                                  .addRegularColumn("c", CounterColumnType.instance)
-                                  .build();
+        TableMetadata.Builder counterTable =
+            TableMetadata.builder(KEYSPACE1, COUNTER1)
+                         .isCounter(true)
+                         .addPartitionKeyColumn("key", Int32Type.instance)
+                         .addClusteringColumn("name", Int32Type.instance)
+                         .addRegularColumn("c", CounterColumnType.instance);
 
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    counterTable);
+        SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1), counterTable);
     }
 
     @AfterClass
@@ -76,9 +75,9 @@ public class CounterCacheTest
         cfs.truncateBlocking();
         CacheService.instance.invalidateCounterCache();
 
-        Clustering c1 = CBuilder.create(cfs.metadata.comparator).add(ByteBufferUtil.bytes(1)).build();
-        Clustering c2 = CBuilder.create(cfs.metadata.comparator).add(ByteBufferUtil.bytes(2)).build();
-        ColumnDefinition cd = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("c"));
+        Clustering c1 = CBuilder.create(cfs.metadata().comparator).add(ByteBufferUtil.bytes(1)).build();
+        Clustering c2 = CBuilder.create(cfs.metadata().comparator).add(ByteBufferUtil.bytes(2)).build();
+        ColumnMetadata cd = cfs.metadata().getColumn(ByteBufferUtil.bytes("c"));
 
         assertEquals(0, CacheService.instance.counterCache.size());
         assertNull(cfs.getCachedCounter(bytes(1), c1, cd, null));
@@ -104,9 +103,9 @@ public class CounterCacheTest
         cfs.truncateBlocking();
         CacheService.instance.invalidateCounterCache();
 
-        Clustering c1 = CBuilder.create(cfs.metadata.comparator).add(ByteBufferUtil.bytes(1)).build();
-        Clustering c2 = CBuilder.create(cfs.metadata.comparator).add(ByteBufferUtil.bytes(2)).build();
-        ColumnDefinition cd = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("c"));
+        Clustering c1 = CBuilder.create(cfs.metadata().comparator).add(ByteBufferUtil.bytes(1)).build();
+        Clustering c2 = CBuilder.create(cfs.metadata().comparator).add(ByteBufferUtil.bytes(2)).build();
+        ColumnMetadata cd = cfs.metadata().getColumn(ByteBufferUtil.bytes("c"));
 
         assertEquals(0, CacheService.instance.counterCache.size());
         assertNull(cfs.getCachedCounter(bytes(1), c1, cd, null));
@@ -149,10 +148,10 @@ public class CounterCacheTest
         cfs.truncateBlocking();
         CacheService.instance.invalidateCounterCache();
 
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(1)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(1)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(2)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(2)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(1)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(1)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(2)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(2)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
 
         assertEquals(4, CacheService.instance.counterCache.size());
 
@@ -165,9 +164,9 @@ public class CounterCacheTest
         CacheService.instance.counterCache.loadSaved();
         assertEquals(4, CacheService.instance.counterCache.size());
 
-        Clustering c1 = CBuilder.create(cfs.metadata.comparator).add(ByteBufferUtil.bytes(1)).build();
-        Clustering c2 = CBuilder.create(cfs.metadata.comparator).add(ByteBufferUtil.bytes(2)).build();
-        ColumnDefinition cd = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("c"));
+        Clustering c1 = CBuilder.create(cfs.metadata().comparator).add(ByteBufferUtil.bytes(1)).build();
+        Clustering c2 = CBuilder.create(cfs.metadata().comparator).add(ByteBufferUtil.bytes(2)).build();
+        ColumnMetadata cd = cfs.metadata().getColumn(ByteBufferUtil.bytes("c"));
 
         assertEquals(1L, cfs.getCachedCounter(bytes(1), c1, cd, null).count);
         assertEquals(2L, cfs.getCachedCounter(bytes(1), c2, cd, null).count);
@@ -182,10 +181,10 @@ public class CounterCacheTest
         cfs.truncateBlocking();
         CacheService.instance.invalidateCounterCache();
 
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(1)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(1)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(2)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(2)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(1)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(1)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(2)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(2)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
 
         // flush the counter cache and invalidate
         CacheService.instance.counterCache.submitWrite(Integer.MAX_VALUE).get();
@@ -213,10 +212,10 @@ public class CounterCacheTest
         cfs.truncateBlocking();
         CacheService.instance.invalidateCounterCache();
 
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(1)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(1)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(2)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new RowUpdateBuilder(cfs.metadata, 0, bytes(2)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(1)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(1)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(2)).clustering(1).add("c", 1L).build(), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new RowUpdateBuilder(cfs.metadata(), 0, bytes(2)).clustering(2).add("c", 2L).build(), ConsistencyLevel.ONE).apply();
 
         // flush the counter cache and invalidate
         CacheService.instance.counterCache.submitWrite(Integer.MAX_VALUE).get();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/CounterCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index a043b1b..8c1347d 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -28,7 +28,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.rows.BufferCell;
 import org.apache.cassandra.db.rows.Cell;
 import org.apache.cassandra.db.rows.Cells;
@@ -98,27 +98,27 @@ public class CounterCellTest
 
     private Cell createLegacyCounterCell(ColumnFamilyStore cfs, ByteBuffer colName, long count, long ts)
     {
-        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
+        ColumnMetadata cDef = cfs.metadata().getColumn(colName);
         ByteBuffer val = CounterContext.instance().createLocal(count);
         return BufferCell.live(cDef, ts, val);
     }
 
     private Cell createCounterCell(ColumnFamilyStore cfs, ByteBuffer colName, CounterId id, long count, long ts)
     {
-        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
+        ColumnMetadata cDef = cfs.metadata().getColumn(colName);
         ByteBuffer val = CounterContext.instance().createGlobal(id, ts, count);
         return BufferCell.live(cDef, ts, val);
     }
 
     private Cell createCounterCellFromContext(ColumnFamilyStore cfs, ByteBuffer colName, ContextState context, long ts)
     {
-        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
+        ColumnMetadata cDef = cfs.metadata().getColumn(colName);
         return BufferCell.live(cDef, ts, context.context);
     }
 
     private Cell createDeleted(ColumnFamilyStore cfs, ByteBuffer colName, long ts, int localDeletionTime)
     {
-        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
+        ColumnMetadata cDef = cfs.metadata().getColumn(colName);
         return BufferCell.tombstone(cDef, ts, localDeletionTime);
     }
 
@@ -272,7 +272,7 @@ public class CounterCellTest
 
         Cell original = createCounterCellFromContext(cfs, col, state, 5);
 
-        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(col);
+        ColumnMetadata cDef = cfs.metadata().getColumn(col);
         Cell cleared = BufferCell.live(cDef, 5, CounterContext.instance().clearAllLocal(state.context));
 
         CounterContext.instance().updateDigest(digest1, original.value());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/CounterMutationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterMutationTest.java b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
index c8d4703..f36deea 100644
--- a/test/unit/org/apache/cassandra/db/CounterMutationTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
@@ -21,7 +21,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.context.CounterContext;
@@ -53,7 +53,7 @@ public class CounterMutationTest
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
         cfs.truncateBlocking();
-        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
+        ColumnMetadata cDef = cfs.metadata().getColumn(ByteBufferUtil.bytes("val"));
 
         // Do the initial update (+1)
         addAndCheck(cfs, 1, 1);
@@ -67,8 +67,8 @@ public class CounterMutationTest
 
     private void addAndCheck(ColumnFamilyStore cfs, long toAdd, long expected)
     {
-        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
-        Mutation m = new RowUpdateBuilder(cfs.metadata, 5, "key1").clustering("cc").add("val", toAdd).build();
+        ColumnMetadata cDef = cfs.metadata().getColumn(ByteBufferUtil.bytes("val"));
+        Mutation m = new RowUpdateBuilder(cfs.metadata(), 5, "key1").clustering("cc").add("val", toAdd).build();
         new CounterMutation(m, ConsistencyLevel.ONE).apply();
 
         Row row = Util.getOnlyRow(Util.cmd(cfs).includeRow("cc").columns("val").build());
@@ -93,10 +93,10 @@ public class CounterMutationTest
 
     private void addTwoAndCheck(ColumnFamilyStore cfs, long addOne, long expectedOne, long addTwo, long expectedTwo)
     {
-        ColumnDefinition cDefOne = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
-        ColumnDefinition cDefTwo = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val2"));
+        ColumnMetadata cDefOne = cfs.metadata().getColumn(ByteBufferUtil.bytes("val"));
+        ColumnMetadata cDefTwo = cfs.metadata().getColumn(ByteBufferUtil.bytes("val2"));
 
-        Mutation m = new RowUpdateBuilder(cfs.metadata, 5, "key1")
+        Mutation m = new RowUpdateBuilder(cfs.metadata(), 5, "key1")
             .clustering("cc")
             .add("val", addOne)
             .add("val2", addTwo)
@@ -119,35 +119,35 @@ public class CounterMutationTest
 
         // Do the update (+1, -1), (+2, -2)
         Mutation batch = new Mutation(KEYSPACE1, Util.dk("key1"));
-        batch.add(new RowUpdateBuilder(cfsOne.metadata, 5, "key1")
+        batch.add(new RowUpdateBuilder(cfsOne.metadata(), 5, "key1")
             .clustering("cc")
             .add("val", 1L)
             .add("val2", -1L)
-            .build().get(cfsOne.metadata));
+            .build().get(cfsOne.metadata()));
 
-        batch.add(new RowUpdateBuilder(cfsTwo.metadata, 5, "key1")
+        batch.add(new RowUpdateBuilder(cfsTwo.metadata(), 5, "key1")
             .clustering("cc")
             .add("val", 2L)
             .add("val2", -2L)
-            .build().get(cfsTwo.metadata));
+            .build().get(cfsTwo.metadata()));
 
         new CounterMutation(batch, ConsistencyLevel.ONE).apply();
 
-        ColumnDefinition c1cfs1 = cfsOne.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
-        ColumnDefinition c2cfs1 = cfsOne.metadata.getColumnDefinition(ByteBufferUtil.bytes("val2"));
+        ColumnMetadata c1cfs1 = cfsOne.metadata().getColumn(ByteBufferUtil.bytes("val"));
+        ColumnMetadata c2cfs1 = cfsOne.metadata().getColumn(ByteBufferUtil.bytes("val2"));
 
         Row row = Util.getOnlyRow(Util.cmd(cfsOne).includeRow("cc").columns("val", "val2").build());
         assertEquals(1L, CounterContext.instance().total(row.getCell(c1cfs1).value()));
         assertEquals(-1L, CounterContext.instance().total(row.getCell(c2cfs1).value()));
 
-        ColumnDefinition c1cfs2 = cfsTwo.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
-        ColumnDefinition c2cfs2 = cfsTwo.metadata.getColumnDefinition(ByteBufferUtil.bytes("val2"));
+        ColumnMetadata c1cfs2 = cfsTwo.metadata().getColumn(ByteBufferUtil.bytes("val"));
+        ColumnMetadata c2cfs2 = cfsTwo.metadata().getColumn(ByteBufferUtil.bytes("val2"));
         row = Util.getOnlyRow(Util.cmd(cfsTwo).includeRow("cc").columns("val", "val2").build());
         assertEquals(2L, CounterContext.instance().total(row.getCell(c1cfs2).value()));
         assertEquals(-2L, CounterContext.instance().total(row.getCell(c2cfs2).value()));
 
         // Check the caches, separately
-        CBuilder cb = CBuilder.create(cfsOne.metadata.comparator);
+        CBuilder cb = CBuilder.create(cfsOne.metadata().comparator);
         cb.add("cc");
 
         assertEquals(1L, cfsOne.getCachedCounter(Util.dk("key1").getKey(), cb.build(), c1cfs1, null).count);
@@ -162,12 +162,12 @@ public class CounterMutationTest
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
         cfs.truncateBlocking();
-        ColumnDefinition cOne = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
-        ColumnDefinition cTwo = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val2"));
+        ColumnMetadata cOne = cfs.metadata().getColumn(ByteBufferUtil.bytes("val"));
+        ColumnMetadata cTwo = cfs.metadata().getColumn(ByteBufferUtil.bytes("val2"));
 
         // Do the initial update (+1, -1)
         new CounterMutation(
-            new RowUpdateBuilder(cfs.metadata, 5, "key1")
+            new RowUpdateBuilder(cfs.metadata(), 5, "key1")
                 .clustering("cc")
                 .add("val", 1L)
                 .add("val2", -1L)
@@ -180,7 +180,7 @@ public class CounterMutationTest
 
         // Remove the first counter, increment the second counter
         new CounterMutation(
-            new RowUpdateBuilder(cfs.metadata, 5, "key1")
+            new RowUpdateBuilder(cfs.metadata(), 5, "key1")
                 .clustering("cc")
                 .delete(cOne)
                 .add("val2", -5L)
@@ -193,7 +193,7 @@ public class CounterMutationTest
 
         // Increment the first counter, make sure it's still shadowed by the tombstone
         new CounterMutation(
-            new RowUpdateBuilder(cfs.metadata, 5, "key1")
+            new RowUpdateBuilder(cfs.metadata(), 5, "key1")
                 .clustering("cc")
                 .add("val", 1L)
                 .build(),
@@ -202,12 +202,12 @@ public class CounterMutationTest
         assertEquals(null, row.getCell(cOne));
 
         // Get rid of the complete partition
-        RowUpdateBuilder.deleteRow(cfs.metadata, 6, "key1", "cc").applyUnsafe();
+        RowUpdateBuilder.deleteRow(cfs.metadata(), 6, "key1", "cc").applyUnsafe();
         Util.assertEmpty(Util.cmd(cfs).includeRow("cc").columns("val", "val2").build());
 
         // Increment both counters, ensure that both stay dead
         new CounterMutation(
-            new RowUpdateBuilder(cfs.metadata, 6, "key1")
+            new RowUpdateBuilder(cfs.metadata(), 6, "key1")
                 .clustering("cc")
                 .add("val", 1L)
                 .add("val2", 1L)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/DeletePartitionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/DeletePartitionTest.java b/test/unit/org/apache/cassandra/db/DeletePartitionTest.java
index a65befd..95bb5a4 100644
--- a/test/unit/org/apache/cassandra/db/DeletePartitionTest.java
+++ b/test/unit/org/apache/cassandra/db/DeletePartitionTest.java
@@ -23,7 +23,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -59,10 +59,10 @@ public class DeletePartitionTest
     public void testDeletePartition(DecoratedKey key, boolean flushBeforeRemove, boolean flushAfterRemove)
     {
         ColumnFamilyStore store = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
-        ColumnDefinition column = store.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
+        ColumnMetadata column = store.metadata().getColumn(ByteBufferUtil.bytes("val"));
 
         // write
-        new RowUpdateBuilder(store.metadata, 0, key.getKey())
+        new RowUpdateBuilder(store.metadata(), 0, key.getKey())
                 .clustering("Column1")
                 .add("val", "asdf")
                 .build()
@@ -79,7 +79,7 @@ public class DeletePartitionTest
 
         // delete the partition
         new Mutation(KEYSPACE1, key)
-                .add(PartitionUpdate.fullPartitionDelete(store.metadata, key, 0, FBUtilities.nowInSeconds()))
+                .add(PartitionUpdate.fullPartitionDelete(store.metadata(), key, 0, FBUtilities.nowInSeconds()))
                 .applyUnsafe();
 
         if (flushAfterRemove)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
index 5841c36..904354a 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -30,8 +30,9 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.schema.Indexes;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.Config.DiskFailurePolicy;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.statements.IndexTarget;
@@ -45,7 +46,6 @@ import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.service.DefaultFSErrorHandler;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 
 import static org.junit.Assert.assertEquals;
@@ -60,7 +60,7 @@ public class DirectoriesTest
     private static final String KS = "ks";
     private static final String[] TABLES = new String[] { "cf1", "ks" };
 
-    private static final Set<CFMetaData> CFM = new HashSet<>(TABLES.length);
+    private static final Set<TableMetadata> CFM = new HashSet<>(TABLES.length);
 
     private static final Map<String, List<File>> files = new HashMap<>();
 
@@ -73,12 +73,10 @@ public class DirectoriesTest
 
         for (String table : TABLES)
         {
-            UUID tableID = CFMetaData.generateLegacyCfId(KS, table);
-            CFM.add(CFMetaData.Builder.create(KS, table)
-                                      .withId(tableID)
-                                      .addPartitionKey("thekey", UTF8Type.instance)
-                                      .addClusteringColumn("thecolumn", UTF8Type.instance)
-                                      .build());
+            CFM.add(TableMetadata.builder(KS, table)
+                                 .addPartitionKeyColumn("thekey", UTF8Type.instance)
+                                 .addClusteringColumn("thecolumn", UTF8Type.instance)
+                                 .build());
         }
 
         tempDataDir = File.createTempFile("cassandra", "unittest");
@@ -99,23 +97,23 @@ public class DirectoriesTest
 
     private static void createTestFiles() throws IOException
     {
-        for (CFMetaData cfm : CFM)
+        for (TableMetadata cfm : CFM)
         {
             List<File> fs = new ArrayList<>();
-            files.put(cfm.cfName, fs);
+            files.put(cfm.name, fs);
             File dir = cfDir(cfm);
             dir.mkdirs();
 
-            createFakeSSTable(dir, cfm.cfName, 1, fs);
-            createFakeSSTable(dir, cfm.cfName, 2, fs);
+            createFakeSSTable(dir, cfm.name, 1, fs);
+            createFakeSSTable(dir, cfm.name, 2, fs);
 
             File backupDir = new File(dir, Directories.BACKUPS_SUBDIR);
             backupDir.mkdir();
-            createFakeSSTable(backupDir, cfm.cfName, 1, fs);
+            createFakeSSTable(backupDir, cfm.name, 1, fs);
 
             File snapshotDir = new File(dir, Directories.SNAPSHOT_SUBDIR + File.separator + "42");
             snapshotDir.mkdirs();
-            createFakeSSTable(snapshotDir, cfm.cfName, 1, fs);
+            createFakeSSTable(snapshotDir, cfm.name, 1, fs);
         }
     }
 
@@ -130,33 +128,33 @@ public class DirectoriesTest
         }
     }
 
-    private static File cfDir(CFMetaData metadata)
+    private static File cfDir(TableMetadata metadata)
     {
-        String cfId = ByteBufferUtil.bytesToHex(ByteBufferUtil.bytes(metadata.cfId));
-        int idx = metadata.cfName.indexOf(Directories.SECONDARY_INDEX_NAME_SEPARATOR);
+        String tableId = metadata.id.toHexString();
+        int idx = metadata.name.indexOf(Directories.SECONDARY_INDEX_NAME_SEPARATOR);
         if (idx >= 0)
         {
             // secondary index
             return new File(tempDataDir,
-                            metadata.ksName + File.separator +
-                            metadata.cfName.substring(0, idx) + '-' + cfId + File.separator +
-                            metadata.cfName.substring(idx));
+                            metadata.keyspace + File.separator +
+                            metadata.name.substring(0, idx) + '-' + tableId + File.separator +
+                            metadata.name.substring(idx));
         }
         else
         {
-            return new File(tempDataDir, metadata.ksName + File.separator + metadata.cfName + '-' + cfId);
+            return new File(tempDataDir, metadata.keyspace + File.separator + metadata.name + '-' + tableId);
         }
     }
 
     @Test
     public void testStandardDirs()
     {
-        for (CFMetaData cfm : CFM)
+        for (TableMetadata cfm : CFM)
         {
             Directories directories = new Directories(cfm);
             assertEquals(cfDir(cfm), directories.getDirectoryForNewSSTables());
 
-            Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1, SSTableFormat.Type.BIG);
+            Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.name, 1, SSTableFormat.Type.BIG);
             File snapshotDir = new File(cfDir(cfm),  File.separator + Directories.SNAPSHOT_SUBDIR + File.separator + "42");
             assertEquals(snapshotDir, Directories.getSnapshotDirectory(desc, "42"));
 
@@ -168,21 +166,22 @@ public class DirectoriesTest
     @Test
     public void testSecondaryIndexDirectories()
     {
-        UUID tableID = CFMetaData.generateLegacyCfId(KS, "cf");
-        CFMetaData PARENT_CFM = CFMetaData.Builder.create(KS, "cf")
-                                  .withId(tableID)
-                                  .addPartitionKey("thekey", UTF8Type.instance)
-                                  .addClusteringColumn("col", UTF8Type.instance)
-                                  .build();
-        ColumnDefinition col = PARENT_CFM.getColumnDefinition(ByteBufferUtil.bytes("col"));
+        TableMetadata.Builder builder =
+            TableMetadata.builder(KS, "cf")
+                         .addPartitionKeyColumn("thekey", UTF8Type.instance)
+                         .addClusteringColumn("col", UTF8Type.instance);
+
+        ColumnIdentifier col = ColumnIdentifier.getInterned("col", true);
         IndexMetadata indexDef =
-            IndexMetadata.fromIndexTargets(PARENT_CFM,
-                                           Collections.singletonList(new IndexTarget(col.name, IndexTarget.Type.VALUES)),
+            IndexMetadata.fromIndexTargets(
+            Collections.singletonList(new IndexTarget(col, IndexTarget.Type.VALUES)),
                                            "idx",
                                            IndexMetadata.Kind.KEYS,
                                            Collections.emptyMap());
-        PARENT_CFM.indexes(PARENT_CFM.getIndexes().with(indexDef));
-        CFMetaData INDEX_CFM = CassandraIndex.indexCfsMetadata(PARENT_CFM, indexDef);
+        builder.indexes(Indexes.of(indexDef));
+
+        TableMetadata PARENT_CFM = builder.build();
+        TableMetadata INDEX_CFM = CassandraIndex.indexCfsMetadata(PARENT_CFM, indexDef);
         Directories parentDirectories = new Directories(PARENT_CFM);
         Directories indexDirectories = new Directories(INDEX_CFM);
         // secondary index has its own directory
@@ -190,8 +189,8 @@ public class DirectoriesTest
         {
             assertEquals(cfDir(INDEX_CFM), dir);
         }
-        Descriptor parentDesc = new Descriptor(parentDirectories.getDirectoryForNewSSTables(), KS, PARENT_CFM.cfName, 0, SSTableFormat.Type.BIG);
-        Descriptor indexDesc = new Descriptor(indexDirectories.getDirectoryForNewSSTables(), KS, INDEX_CFM.cfName, 0, SSTableFormat.Type.BIG);
+        Descriptor parentDesc = new Descriptor(parentDirectories.getDirectoryForNewSSTables(), KS, PARENT_CFM.name, 0, SSTableFormat.Type.BIG);
+        Descriptor indexDesc = new Descriptor(indexDirectories.getDirectoryForNewSSTables(), KS, INDEX_CFM.name, 0, SSTableFormat.Type.BIG);
 
         // snapshot dir should be created under its parent's
         File parentSnapshotDirectory = Directories.getSnapshotDirectory(parentDesc, "test");
@@ -208,9 +207,9 @@ public class DirectoriesTest
                      indexDirectories.snapshotCreationTime("test"));
 
         // check true snapshot size
-        Descriptor parentSnapshot = new Descriptor(parentSnapshotDirectory, KS, PARENT_CFM.cfName, 0, SSTableFormat.Type.BIG);
+        Descriptor parentSnapshot = new Descriptor(parentSnapshotDirectory, KS, PARENT_CFM.name, 0, SSTableFormat.Type.BIG);
         createFile(parentSnapshot.filenameFor(Component.DATA), 30);
-        Descriptor indexSnapshot = new Descriptor(indexSnapshotDirectory, KS, INDEX_CFM.cfName, 0, SSTableFormat.Type.BIG);
+        Descriptor indexSnapshot = new Descriptor(indexSnapshotDirectory, KS, INDEX_CFM.name, 0, SSTableFormat.Type.BIG);
         createFile(indexSnapshot.filenameFor(Component.DATA), 40);
 
         assertEquals(30, parentDirectories.trueSnapshotsSize());
@@ -246,20 +245,20 @@ public class DirectoriesTest
     @Test
     public void testSSTableLister()
     {
-        for (CFMetaData cfm : CFM)
+        for (TableMetadata cfm : CFM)
         {
             Directories directories = new Directories(cfm);
             checkFiles(cfm, directories);
         }
     }
 
-    private void checkFiles(CFMetaData cfm, Directories directories)
+    private void checkFiles(TableMetadata cfm, Directories directories)
     {
         Directories.SSTableLister lister;
         Set<File> listed;// List all but no snapshot, backup
         lister = directories.sstableLister(Directories.OnTxnErr.THROW);
         listed = new HashSet<>(lister.listFiles());
-        for (File f : files.get(cfm.cfName))
+        for (File f : files.get(cfm.name))
         {
             if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
                 assertFalse(f + " should not be listed", listed.contains(f));
@@ -270,7 +269,7 @@ public class DirectoriesTest
         // List all but including backup (but no snapshot)
         lister = directories.sstableLister(Directories.OnTxnErr.THROW).includeBackups(true);
         listed = new HashSet<>(lister.listFiles());
-        for (File f : files.get(cfm.cfName))
+        for (File f : files.get(cfm.name))
         {
             if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR))
                 assertFalse(f + " should not be listed", listed.contains(f));
@@ -281,7 +280,7 @@ public class DirectoriesTest
         // Skip temporary and compacted
         lister = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true);
         listed = new HashSet<>(lister.listFiles());
-        for (File f : files.get(cfm.cfName))
+        for (File f : files.get(cfm.name))
         {
             if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
                 assertFalse(f + " should not be listed", listed.contains(f));
@@ -295,7 +294,7 @@ public class DirectoriesTest
     @Test
     public void testTemporaryFile() throws IOException
     {
-        for (CFMetaData cfm : CFM)
+        for (TableMetadata cfm : CFM)
         {
             Directories directories = new Directories(cfm);
 
@@ -351,14 +350,14 @@ public class DirectoriesTest
     @Test
     public void testMTSnapshots() throws Exception
     {
-        for (final CFMetaData cfm : CFM)
+        for (final TableMetadata cfm : CFM)
         {
             final Directories directories = new Directories(cfm);
             assertEquals(cfDir(cfm), directories.getDirectoryForNewSSTables());
             final String n = Long.toString(System.nanoTime());
             Callable<File> directoryGetter = new Callable<File>() {
                 public File call() throws Exception {
-                    Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1, SSTableFormat.Type.BIG);
+                    Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.name, 1, SSTableFormat.Type.BIG);
                     return Directories.getSnapshotDirectory(desc, n);
                 }
             };

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/KeyCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index ada6b5b..d7a31d6 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -269,8 +269,8 @@ public class KeyCacheTest
         Mutation rm;
 
         // inserts
-        new RowUpdateBuilder(cfs.metadata, 0, "key1").clustering("1").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, "key2").clustering("2").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "key1").clustering("1").build().applyUnsafe();
+        new RowUpdateBuilder(cfs.metadata(), 0, "key2").clustering("2").build().applyUnsafe();
 
         // to make sure we have SSTable
         cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/KeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
index 5036749..d913f3e 100644
--- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
@@ -80,20 +80,20 @@ public class KeyspaceTest extends CQLTester
         {
             // slice with limit 1
             Row row = Util.getOnlyRow(Util.cmd(cfs, "0").columns("c").withLimit(1).build());
-            assertEquals(ByteBufferUtil.bytes(0), row.getCell(cfs.metadata.getColumnDefinition(new ColumnIdentifier("c", false))).value());
+            assertEquals(ByteBufferUtil.bytes(0), row.getCell(cfs.metadata().getColumn(new ColumnIdentifier("c", false))).value());
 
             // fetch each row by name
             for (int i = 0; i < 2; i++)
             {
                 row = Util.getOnlyRow(Util.cmd(cfs, "0").columns("c").includeRow(i).build());
-                assertEquals(ByteBufferUtil.bytes(i), row.getCell(cfs.metadata.getColumnDefinition(new ColumnIdentifier("c", false))).value());
+                assertEquals(ByteBufferUtil.bytes(i), row.getCell(cfs.metadata().getColumn(new ColumnIdentifier("c", false))).value());
             }
 
             // fetch each row by slice
             for (int i = 0; i < 2; i++)
             {
                 row = Util.getOnlyRow(Util.cmd(cfs, "0").columns("c").fromIncl(i).toIncl(i).build());
-                assertEquals(ByteBufferUtil.bytes(i), row.getCell(cfs.metadata.getColumnDefinition(new ColumnIdentifier("c", false))).value());
+                assertEquals(ByteBufferUtil.bytes(i), row.getCell(cfs.metadata().getColumn(new ColumnIdentifier("c", false))).value());
             }
 
             if (round == 0)
@@ -145,7 +145,7 @@ public class KeyspaceTest extends CQLTester
                     for (int i = sliceEnd; i >= sliceStart; i--)
                     {
                         Row row = rowIterator.next();
-                        Cell cell = row.getCell(cfs.metadata.getColumnDefinition(new ColumnIdentifier("c", false)));
+                        Cell cell = row.getCell(cfs.metadata().getColumn(new ColumnIdentifier("c", false)));
                         assertEquals(ByteBufferUtil.bytes(columnValuePrefix + i), cell.value());
                     }
                 }
@@ -154,7 +154,7 @@ public class KeyspaceTest extends CQLTester
                     for (int i = sliceStart; i <= sliceEnd; i++)
                     {
                         Row row = rowIterator.next();
-                        Cell cell = row.getCell(cfs.metadata.getColumnDefinition(new ColumnIdentifier("c", false)));
+                        Cell cell = row.getCell(cfs.metadata().getColumn(new ColumnIdentifier("c", false)));
                         assertEquals(ByteBufferUtil.bytes(columnValuePrefix + i), cell.value());
                     }
                 }
@@ -206,7 +206,7 @@ public class KeyspaceTest extends CQLTester
         {
             execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "0", i, i);
 
-            PartitionColumns columns = PartitionColumns.of(cfs.metadata.getColumnDefinition(new ColumnIdentifier("c", false)));
+            RegularAndStaticColumns columns = RegularAndStaticColumns.of(cfs.metadata().getColumn(new ColumnIdentifier("c", false)));
             ClusteringIndexSliceFilter filter = new ClusteringIndexSliceFilter(Slices.ALL, false);
             SinglePartitionReadCommand command = singlePartitionSlice(cfs, "0", filter, null);
             try (ReadExecutionController executionController = command.executionController();
@@ -215,7 +215,7 @@ public class KeyspaceTest extends CQLTester
                 try (RowIterator rowIterator = iterator.next())
                 {
                     Row row = rowIterator.next();
-                    Cell cell = row.getCell(cfs.metadata.getColumnDefinition(new ColumnIdentifier("c", false)));
+                    Cell cell = row.getCell(cfs.metadata().getColumn(new ColumnIdentifier("c", false)));
                     assertEquals(ByteBufferUtil.bytes(i), cell.value());
                 }
             }
@@ -230,7 +230,7 @@ public class KeyspaceTest extends CQLTester
             if (columnValues.length == 0)
             {
                 if (iterator.hasNext())
-                    fail("Didn't expect any results, but got rows starting with: " + iterator.next().next().toString(cfs.metadata));
+                    fail("Didn't expect any results, but got rows starting with: " + iterator.next().next().toString(cfs.metadata()));
                 return;
             }
 
@@ -239,7 +239,7 @@ public class KeyspaceTest extends CQLTester
                 for (int expected : columnValues)
                 {
                     Row row = rowIterator.next();
-                    Cell cell = row.getCell(cfs.metadata.getColumnDefinition(new ColumnIdentifier("c", false)));
+                    Cell cell = row.getCell(cfs.metadata().getColumn(new ColumnIdentifier("c", false)));
                     assertEquals(
                             String.format("Expected %s, but got %s", ByteBufferUtil.bytesToHex(ByteBufferUtil.bytes(expected)), ByteBufferUtil.bytesToHex(cell.value())),
                             ByteBufferUtil.bytes(expected), cell.value());
@@ -267,7 +267,7 @@ public class KeyspaceTest extends CQLTester
                          ? DataLimits.NONE
                          : DataLimits.cqlLimits(rowLimit);
         return SinglePartitionReadCommand.create(
-                cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, limit, Util.dk(key), filter);
+                cfs.metadata(), FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata()), RowFilter.NONE, limit, Util.dk(key), filter);
     }
 
     @Test
@@ -474,17 +474,17 @@ public class KeyspaceTest extends CQLTester
     {
         ClusteringIndexSliceFilter filter = slices(cfs, 1000, null, false);
         SinglePartitionReadCommand command = SinglePartitionReadCommand.create(
-                cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
+                cfs.metadata(), FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata()), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
         assertRowsInResult(cfs, command, 1000, 1001, 1002);
 
         filter = slices(cfs, 1195, null, false);
         command = SinglePartitionReadCommand.create(
-                cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
+                cfs.metadata(), FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata()), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
         assertRowsInResult(cfs, command, 1195, 1196, 1197);
 
         filter = slices(cfs, null, 1996, true);
         command = SinglePartitionReadCommand.create(
-                cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(1000), Util.dk("0"), filter);
+                cfs.metadata(), FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata()), RowFilter.NONE, DataLimits.cqlLimits(1000), Util.dk("0"), filter);
         int[] expectedValues = new int[997];
         for (int i = 0, v = 1996; v >= 1000; i++, v--)
             expectedValues[i] = v;
@@ -492,22 +492,22 @@ public class KeyspaceTest extends CQLTester
 
         filter = slices(cfs, 1990, null, false);
         command = SinglePartitionReadCommand.create(
-                cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
+                cfs.metadata(), FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata()), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
         assertRowsInResult(cfs, command, 1990, 1991, 1992);
 
         filter = slices(cfs, null, null, true);
         command = SinglePartitionReadCommand.create(
-                cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
+                cfs.metadata(), FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata()), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
         assertRowsInResult(cfs, command, 1999, 1998, 1997);
 
         filter = slices(cfs, null, 9000, true);
         command = SinglePartitionReadCommand.create(
-                cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
+                cfs.metadata(), FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata()), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
         assertRowsInResult(cfs, command, 1999, 1998, 1997);
 
         filter = slices(cfs, 9000, null, false);
         command = SinglePartitionReadCommand.create(
-                cfs.metadata, FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
+                cfs.metadata(), FBUtilities.nowInSeconds(), ColumnFilter.all(cfs.metadata()), RowFilter.NONE, DataLimits.cqlLimits(3), Util.dk("0"), filter);
         assertRowsInResult(cfs, command);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/NameSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NameSortTest.java b/test/unit/org/apache/cassandra/db/NameSortTest.java
index 1da6ea6..0b00f40 100644
--- a/test/unit/org/apache/cassandra/db/NameSortTest.java
+++ b/test/unit/org/apache/cassandra/db/NameSortTest.java
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.marshal.AsciiType;
@@ -77,7 +77,7 @@ public class NameSortTest
         for (int i = 0; i < N; i++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(Integer.toString(i));
-            RowUpdateBuilder rub = new RowUpdateBuilder(cfs.metadata, 0, key);
+            RowUpdateBuilder rub = new RowUpdateBuilder(cfs.metadata(), 0, key);
             rub.clustering("cc");
             for (int j = 0; j < 8; j++)
                 rub.add("val" + j, j % 2 == 0 ? "a" : "b");
@@ -94,7 +94,7 @@ public class NameSortTest
         {
             for (Row r : partition)
             {
-                for (ColumnDefinition cd : r.columns())
+                for (ColumnMetadata cd : r.columns())
                 {
                     if (r.getCell(cd) == null)
                         continue;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/NativeCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NativeCellTest.java b/test/unit/org/apache/cassandra/db/NativeCellTest.java
index 69e615b..cd7074f 100644
--- a/test/unit/org/apache/cassandra/db/NativeCellTest.java
+++ b/test/unit/org/apache/cassandra/db/NativeCellTest.java
@@ -29,7 +29,7 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.SetType;
@@ -92,7 +92,7 @@ public class NativeCellTest
 
     private static void rndcd(Row.Builder builder)
     {
-        ColumnDefinition col = rndcol();
+        ColumnMetadata col = rndcol();
         if (!col.isComplex())
         {
             builder.addCell(rndcell(col));
@@ -105,19 +105,19 @@ public class NativeCellTest
         }
     }
 
-    private static ColumnDefinition rndcol()
+    private static ColumnMetadata rndcol()
     {
         UUID uuid = new UUID(rand.nextLong(), rand.nextLong());
         boolean isComplex = rand.nextBoolean();
-        return new ColumnDefinition("",
-                                    "",
-                                    ColumnIdentifier.getInterned(uuid.toString(), false),
+        return new ColumnMetadata("",
+                                  "",
+                                  ColumnIdentifier.getInterned(uuid.toString(), false),
                                     isComplex ? new SetType<>(BytesType.instance, true) : BytesType.instance,
-                                    -1,
-                                    ColumnDefinition.Kind.REGULAR);
+                                  -1,
+                                  ColumnMetadata.Kind.REGULAR);
     }
 
-    private static Cell rndcell(ColumnDefinition col)
+    private static Cell rndcell(ColumnMetadata col)
     {
         long timestamp = rand.nextLong();
         int ttl = rand.nextInt();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java b/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java
index e409592..c433d8e 100644
--- a/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java
+++ b/test/unit/org/apache/cassandra/db/PartitionRangeReadTest.java
@@ -30,7 +30,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.cassandra.*;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.db.partitions.*;
@@ -62,10 +62,10 @@ public class PartitionRangeReadTest
     public void testInclusiveBounds()
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE2).getColumnFamilyStore(CF_STANDARD1);
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key1"))
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key1"))
                 .clustering("cc1")
                 .add("val", "asdf").build().applyUnsafe();
-        new RowUpdateBuilder(cfs.metadata, 0, ByteBufferUtil.bytes("key2"))
+        new RowUpdateBuilder(cfs.metadata(), 0, ByteBufferUtil.bytes("key2"))
                 .clustering("cc2")
                 .add("val", "asdf").build().applyUnsafe();
 
@@ -81,18 +81,18 @@ public class PartitionRangeReadTest
         cfs.truncateBlocking();
 
         ByteBuffer col = ByteBufferUtil.bytes("val");
-        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(col);
+        ColumnMetadata cDef = cfs.metadata().getColumn(col);
 
         // insert two columns that represent the same integer but have different binary forms (the
         // second one is padded with extra zeros)
-        new RowUpdateBuilder(cfs.metadata, 0, "k1")
+        new RowUpdateBuilder(cfs.metadata(), 0, "k1")
                 .clustering(new BigInteger(new byte[]{1}))
                 .add("val", "val1")
                 .build()
                 .applyUnsafe();
         cfs.forceBlockingFlush();
 
-        new RowUpdateBuilder(cfs.metadata, 1, "k1")
+        new RowUpdateBuilder(cfs.metadata(), 1, "k1")
                 .clustering(new BigInteger(new byte[]{0, 0, 1}))
                 .add("val", "val2")
                 .build()
@@ -119,7 +119,7 @@ public class PartitionRangeReadTest
 
         for (int i = 0; i < 10; ++i)
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 10, String.valueOf(i));
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 10, String.valueOf(i));
             builder.clustering("c");
             builder.add("val", String.valueOf(i));
             builder.build().applyUnsafe();
@@ -127,7 +127,7 @@ public class PartitionRangeReadTest
 
         cfs.forceBlockingFlush();
 
-        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val"));
+        ColumnMetadata cDef = cfs.metadata().getColumn(ByteBufferUtil.bytes("val"));
 
         List<FilteredPartition> partitions;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/PartitionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/PartitionTest.java b/test/unit/org/apache/cassandra/db/PartitionTest.java
index 7f44d51..c1796be 100644
--- a/test/unit/org/apache/cassandra/db/PartitionTest.java
+++ b/test/unit/org/apache/cassandra/db/PartitionTest.java
@@ -25,7 +25,7 @@ import java.util.Arrays;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.rows.EncodingStats;
 import org.apache.cassandra.db.rows.UnfilteredRowIterators;
 import org.apache.cassandra.db.marshal.AsciiType;
@@ -67,7 +67,7 @@ public class PartitionTest
     public void testSingleColumn() throws IOException
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
-        PartitionUpdate update = new RowUpdateBuilder(cfs.metadata, 5, "key1")
+        PartitionUpdate update = new RowUpdateBuilder(cfs.metadata(), 5, "key1")
                                  .clustering("c")
                                  .add("val", "val1")
                                  .buildUpdate();
@@ -80,7 +80,7 @@ public class PartitionTest
         CachedPartition deserialized = CachedPartition.cacheSerializer.deserialize(new DataInputBuffer(bufOut.getData()));
 
         assert deserialized != null;
-        assert deserialized.metadata().cfName.equals(CF_STANDARD1);
+        assert deserialized.metadata().name.equals(CF_STANDARD1);
         assert deserialized.partitionKey().equals(partition.partitionKey());
     }
 
@@ -88,7 +88,7 @@ public class PartitionTest
     public void testManyColumns() throws IOException
     {
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_TENCOL);
-        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 5, "key1")
+        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 5, "key1")
                                    .clustering("c")
                                    .add("val", "val1");
 
@@ -108,7 +108,7 @@ public class PartitionTest
         assertTrue(deserialized.columns().regulars.getSimple(1).equals(partition.columns().regulars.getSimple(1)));
         assertTrue(deserialized.columns().regulars.getSimple(5).equals(partition.columns().regulars.getSimple(5)));
 
-        ColumnDefinition cDef = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("val8"));
+        ColumnMetadata cDef = cfs.metadata().getColumn(ByteBufferUtil.bytes("val8"));
         assertTrue(partition.lastRow().getCell(cDef).value().equals(deserialized.lastRow().getCell(cDef).value()));
         assert deserialized.partitionKey().equals(partition.partitionKey());
     }
@@ -125,12 +125,12 @@ public class PartitionTest
 
         try
         {
-            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 5, "key1").clustering("c").add("val", "val1");
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 5, "key1").clustering("c").add("val", "val1");
             for (int i = 0; i < 10; i++)
                 builder.add("val" + i, "val" + i);
             builder.build().applyUnsafe();
 
-            new RowUpdateBuilder(cfs.metadata, 5, "key2").clustering("c").add("val", "val2").build().applyUnsafe();
+            new RowUpdateBuilder(cfs.metadata(), 5, "key2").clustering("c").add("val", "val2").build().applyUnsafe();
 
             ReadCommand cmd1 = Util.cmd(cfs, "key1").build();
             ReadCommand cmd2 = Util.cmd(cfs, "key2").build();
@@ -152,7 +152,7 @@ public class PartitionTest
             assertTrue(Arrays.equals(digest1.digest(), digest2.digest()));
 
             p1 = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, "key2").build());
-            RowUpdateBuilder.deleteRow(cfs.metadata, 6, "key2", "c").applyUnsafe();
+            RowUpdateBuilder.deleteRow(cfs.metadata(), 6, "key2", "c").applyUnsafe();
             p2 = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, "key2").build());
             digest1 = MessageDigest.getInstance("MD5");
             digest2 = MessageDigest.getInstance("MD5");
@@ -173,12 +173,12 @@ public class PartitionTest
         int localDeletionTime = (int) (timestamp / 1000);
 
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_TENCOL);
-        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata, 5, "key1").clustering("c").add("val", "val1");
+        RowUpdateBuilder builder = new RowUpdateBuilder(cfs.metadata(), 5, "key1").clustering("c").add("val", "val1");
         for (int i = 0; i < 10; i++)
             builder.add("val" + i, "val" + i);
         builder.build().applyUnsafe();
 
-        RowUpdateBuilder.deleteRowAt(cfs.metadata, 10L, localDeletionTime, "key1", "c").applyUnsafe();
+        RowUpdateBuilder.deleteRowAt(cfs.metadata(), 10L, localDeletionTime, "key1", "c").applyUnsafe();
         ImmutableBTreePartition partition = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, "key1").build());
         EncodingStats stats = partition.stats();
         assertEquals(localDeletionTime, stats.minLocalDeletionTime);


[30/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 43c22a3..228af33 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -25,10 +25,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.ColumnDefinition.Raw;
-import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.ColumnMetadata.Raw;
+import org.apache.cassandra.schema.ViewMetadata;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.conditions.ColumnCondition;
 import org.apache.cassandra.cql3.conditions.ColumnConditions;
@@ -71,24 +72,24 @@ public abstract class ModificationStatement implements CQLStatement
     protected final StatementType type;
 
     private final int boundTerms;
-    public final CFMetaData cfm;
+    public final TableMetadata metadata;
     private final Attributes attrs;
 
     private final StatementRestrictions restrictions;
 
     private final Operations operations;
 
-    private final PartitionColumns updatedColumns;
+    private final RegularAndStaticColumns updatedColumns;
 
     private final Conditions conditions;
 
-    private final PartitionColumns conditionColumns;
+    private final RegularAndStaticColumns conditionColumns;
 
-    private final PartitionColumns requiresRead;
+    private final RegularAndStaticColumns requiresRead;
 
     public ModificationStatement(StatementType type,
                                  int boundTerms,
-                                 CFMetaData cfm,
+                                 TableMetadata metadata,
                                  Operations operations,
                                  StatementRestrictions restrictions,
                                  Conditions conditions,
@@ -96,7 +97,7 @@ public abstract class ModificationStatement implements CQLStatement
     {
         this.type = type;
         this.boundTerms = boundTerms;
-        this.cfm = cfm;
+        this.metadata = metadata;
         this.restrictions = restrictions;
         this.operations = operations;
         this.conditions = conditions;
@@ -104,17 +105,17 @@ public abstract class ModificationStatement implements CQLStatement
 
         if (!conditions.isEmpty())
         {
-            checkFalse(cfm.isCounter(), "Conditional updates are not supported on counter tables");
+            checkFalse(metadata.isCounter(), "Conditional updates are not supported on counter tables");
             checkFalse(attrs.isTimestampSet(), "Cannot provide custom timestamp for conditional updates");
         }
 
-        PartitionColumns.Builder conditionColumnsBuilder = PartitionColumns.builder();
-        Iterable<ColumnDefinition> columns = conditions.getColumns();
+        RegularAndStaticColumns.Builder conditionColumnsBuilder = RegularAndStaticColumns.builder();
+        Iterable<ColumnMetadata> columns = conditions.getColumns();
         if (columns != null)
             conditionColumnsBuilder.addAll(columns);
 
-        PartitionColumns.Builder updatedColumnsBuilder = PartitionColumns.builder();
-        PartitionColumns.Builder requiresReadBuilder = PartitionColumns.builder();
+        RegularAndStaticColumns.Builder updatedColumnsBuilder = RegularAndStaticColumns.builder();
+        RegularAndStaticColumns.Builder requiresReadBuilder = RegularAndStaticColumns.builder();
         for (Operation operation : operations)
         {
             updatedColumnsBuilder.add(operation.column);
@@ -127,13 +128,13 @@ public abstract class ModificationStatement implements CQLStatement
             }
         }
 
-        PartitionColumns modifiedColumns = updatedColumnsBuilder.build();
+        RegularAndStaticColumns modifiedColumns = updatedColumnsBuilder.build();
         // Compact tables have not row marker. So if we don't actually update any particular column,
         // this means that we're only updating the PK, which we allow if only those were declared in
         // the definition. In that case however, we do went to write the compactValueColumn (since again
         // we can't use a "row marker") so add it automatically.
-        if (cfm.isCompactTable() && modifiedColumns.isEmpty() && updatesRegularRows())
-            modifiedColumns = cfm.partitionColumns();
+        if (metadata.isCompactTable() && modifiedColumns.isEmpty() && updatesRegularRows())
+            modifiedColumns = metadata.regularAndStaticColumns();
 
         this.updatedColumns = modifiedColumns;
         this.conditionColumns = conditionColumnsBuilder.build();
@@ -155,6 +156,11 @@ public abstract class ModificationStatement implements CQLStatement
         conditions.addFunctionsTo(functions);
     }
 
+    public TableMetadata metadata()
+    {
+        return metadata;
+    }
+
     /*
      * May be used by QueryHandler implementations
      */
@@ -174,22 +180,22 @@ public abstract class ModificationStatement implements CQLStatement
 
     public String keyspace()
     {
-        return cfm.ksName;
+        return metadata.keyspace;
     }
 
     public String columnFamily()
     {
-        return cfm.cfName;
+        return metadata.name;
     }
 
     public boolean isCounter()
     {
-        return cfm.isCounter();
+        return metadata().isCounter();
     }
 
     public boolean isView()
     {
-        return cfm.isView();
+        return metadata().isView();
     }
 
     public long getTimestamp(long now, QueryOptions options) throws InvalidRequestException
@@ -204,23 +210,23 @@ public abstract class ModificationStatement implements CQLStatement
 
     public int getTimeToLive(QueryOptions options) throws InvalidRequestException
     {
-        return attrs.getTimeToLive(options, cfm.params.defaultTimeToLive);
+        return attrs.getTimeToLive(options, metadata().params.defaultTimeToLive);
     }
 
     public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
     {
-        state.hasColumnFamilyAccess(cfm, Permission.MODIFY);
+        state.hasColumnFamilyAccess(metadata, Permission.MODIFY);
 
         // CAS updates can be used to simulate a SELECT query, so should require Permission.SELECT as well.
         if (hasConditions())
-            state.hasColumnFamilyAccess(cfm, Permission.SELECT);
+            state.hasColumnFamilyAccess(metadata, Permission.SELECT);
 
         // MV updates need to get the current state from the table, and might update the views
         // Require Permission.SELECT on the base table, and Permission.MODIFY on the views
-        Iterator<ViewDefinition> views = View.findAll(keyspace(), columnFamily()).iterator();
+        Iterator<ViewMetadata> views = View.findAll(keyspace(), columnFamily()).iterator();
         if (views.hasNext())
         {
-            state.hasColumnFamilyAccess(cfm, Permission.SELECT);
+            state.hasColumnFamilyAccess(metadata, Permission.SELECT);
             do
             {
                 state.hasColumnFamilyAccess(views.next().metadata, Permission.MODIFY);
@@ -239,12 +245,12 @@ public abstract class ModificationStatement implements CQLStatement
         checkFalse(isView(), "Cannot directly modify a materialized view");
     }
 
-    public PartitionColumns updatedColumns()
+    public RegularAndStaticColumns updatedColumns()
     {
         return updatedColumns;
     }
 
-    public PartitionColumns conditionColumns()
+    public RegularAndStaticColumns conditionColumns()
     {
         return conditionColumns;
     }
@@ -256,7 +262,7 @@ public abstract class ModificationStatement implements CQLStatement
         // columns is if we set some static columns, and in that case no clustering
         // columns should be given. So in practice, it's enough to check if we have
         // either the table has no clustering or if it has at least one of them set.
-        return cfm.clusteringColumns().isEmpty() || restrictions.hasClusteringColumnsRestrictions();
+        return metadata().clusteringColumns().isEmpty() || restrictions.hasClusteringColumnsRestrictions();
     }
 
     public boolean updatesStaticRow()
@@ -279,7 +285,7 @@ public abstract class ModificationStatement implements CQLStatement
         return operations;
     }
 
-    public Iterable<ColumnDefinition> getColumnsWithConditions()
+    public Iterable<ColumnMetadata> getColumnsWithConditions()
     {
          return conditions.getColumns();
     }
@@ -308,7 +314,7 @@ public abstract class ModificationStatement implements CQLStatement
     throws InvalidRequestException
     {
         if (appliesOnlyToStaticColumns() && !restrictions.hasClusteringColumnsRestrictions())
-            return FBUtilities.singleton(CBuilder.STATIC_BUILDER.build(), cfm.comparator);
+            return FBUtilities.singleton(CBuilder.STATIC_BUILDER.build(), metadata().comparator);
 
         return restrictions.getClusteringColumns(options);
     }
@@ -365,12 +371,12 @@ public abstract class ModificationStatement implements CQLStatement
         List<SinglePartitionReadCommand> commands = new ArrayList<>(partitionKeys.size());
         int nowInSec = FBUtilities.nowInSeconds();
         for (ByteBuffer key : partitionKeys)
-            commands.add(SinglePartitionReadCommand.create(cfm,
+            commands.add(SinglePartitionReadCommand.create(metadata(),
                                                            nowInSec,
                                                            ColumnFilter.selection(this.requiresRead),
                                                            RowFilter.NONE,
                                                            limits,
-                                                           cfm.decorateKey(key),
+                                                           metadata().partitioner.decorateKey(key),
                                                            filter));
 
         SinglePartitionReadCommand.Group group = new SinglePartitionReadCommand.Group(commands, DataLimits.NONE);
@@ -424,9 +430,9 @@ public abstract class ModificationStatement implements CQLStatement
     {
         ConsistencyLevel cl = options.getConsistency();
         if (isCounter())
-            cl.validateCounterForWrite(cfm);
+            cl.validateCounterForWrite(metadata());
         else
-            cl.validateForWrite(cfm.ksName);
+            cl.validateForWrite(metadata.keyspace);
 
         Collection<? extends IMutation> mutations = getMutations(options, false, options.getTimestamp(queryState), queryStartNanoTime);
         if (!mutations.isEmpty())
@@ -461,7 +467,7 @@ public abstract class ModificationStatement implements CQLStatement
                    "IN on the partition key is not supported with conditional %s",
                    type.isUpdate()? "updates" : "deletions");
 
-        DecoratedKey key = cfm.decorateKey(keys.get(0));
+        DecoratedKey key = metadata().partitioner.decorateKey(keys.get(0));
         long now = options.getTimestamp(queryState);
 
         checkFalse(restrictions.clusteringKeyRestrictionsHasIN(),
@@ -469,7 +475,7 @@ public abstract class ModificationStatement implements CQLStatement
                     type.isUpdate()? "updates" : "deletions");
 
         Clustering clustering = Iterables.getOnlyElement(createClustering(options));
-        CQL3CasRequest request = new CQL3CasRequest(cfm, key, false, conditionColumns(), updatesRegularRows(), updatesStaticRow());
+        CQL3CasRequest request = new CQL3CasRequest(metadata(), key, false, conditionColumns(), updatesRegularRows(), updatesStaticRow());
 
         addConditions(clustering, request, options);
         request.addRowUpdate(clustering, this, options, now);
@@ -487,7 +493,7 @@ public abstract class ModificationStatement implements CQLStatement
         return buildCasResultSet(keyspace(), columnFamily(), partition, getColumnsWithConditions(), false, options);
     }
 
-    public static ResultSet buildCasResultSet(String ksName, String tableName, RowIterator partition, Iterable<ColumnDefinition> columnsWithConditions, boolean isBatch, QueryOptions options)
+    public static ResultSet buildCasResultSet(String ksName, String tableName, RowIterator partition, Iterable<ColumnMetadata> columnsWithConditions, boolean isBatch, QueryOptions options)
     throws InvalidRequestException
     {
         boolean success = partition == null;
@@ -523,35 +529,31 @@ public abstract class ModificationStatement implements CQLStatement
         return new ResultSet(new ResultSet.ResultMetadata(specs), rows);
     }
 
-    private static ResultSet buildCasFailureResultSet(RowIterator partition, Iterable<ColumnDefinition> columnsWithConditions, boolean isBatch, QueryOptions options)
+    private static ResultSet buildCasFailureResultSet(RowIterator partition, Iterable<ColumnMetadata> columnsWithConditions, boolean isBatch, QueryOptions options)
     throws InvalidRequestException
     {
-        CFMetaData cfm = partition.metadata();
+        TableMetadata metadata = partition.metadata();
         Selection selection;
         if (columnsWithConditions == null)
         {
-            selection = Selection.wildcard(cfm);
+            selection = Selection.wildcard(metadata);
         }
         else
         {
             // We can have multiple conditions on the same columns (for collections) so use a set
             // to avoid duplicate, but preserve the order just to it follows the order of IF in the query in general
-            Set<ColumnDefinition> defs = new LinkedHashSet<>();
+            Set<ColumnMetadata> defs = new LinkedHashSet<>();
             // Adding the partition key for batches to disambiguate if the conditions span multipe rows (we don't add them outside
             // of batches for compatibility sakes).
             if (isBatch)
-            {
-                defs.addAll(cfm.partitionKeyColumns());
-                defs.addAll(cfm.clusteringColumns());
-            }
-            for (ColumnDefinition def : columnsWithConditions)
-                defs.add(def);
-            selection = Selection.forColumns(cfm, new ArrayList<>(defs));
+                Iterables.addAll(defs, metadata.primaryKeyColumns());
+            Iterables.addAll(defs, columnsWithConditions);
+            selection = Selection.forColumns(metadata, new ArrayList<>(defs));
 
         }
 
         Selection.ResultSetBuilder builder = selection.resultSetBuilder(options, false);
-        SelectStatement.forSelection(cfm, selection).processPartition(partition,
+        SelectStatement.forSelection(metadata, selection).processPartition(partition,
                                                                       options,
                                                                       builder,
                                                                       FBUtilities.nowInSeconds());
@@ -616,7 +618,7 @@ public abstract class ModificationStatement implements CQLStatement
      */
     private Collection<? extends IMutation> getMutations(QueryOptions options, boolean local, long now, long queryStartNanoTime)
     {
-        UpdatesCollector collector = new UpdatesCollector(Collections.singletonMap(cfm.cfId, updatedColumns), 1);
+        UpdatesCollector collector = new UpdatesCollector(Collections.singletonMap(metadata.id, updatedColumns), 1);
         addUpdates(collector, options, local, now, queryStartNanoTime);
         collector.validateIndexedColumns();
 
@@ -650,10 +652,10 @@ public abstract class ModificationStatement implements CQLStatement
                                                            queryStartNanoTime);
             for (ByteBuffer key : keys)
             {
-                Validation.validateKey(cfm, key);
-                DecoratedKey dk = cfm.decorateKey(key);
+                Validation.validateKey(metadata(), key);
+                DecoratedKey dk = metadata().partitioner.decorateKey(key);
 
-                PartitionUpdate upd = collector.getPartitionUpdate(cfm, dk, options.getConsistency());
+                PartitionUpdate upd = collector.getPartitionUpdate(metadata(), dk, options.getConsistency());
 
                 for (Slice slice : slices)
                     addUpdateForKey(upd, slice, params);
@@ -667,10 +669,10 @@ public abstract class ModificationStatement implements CQLStatement
 
             for (ByteBuffer key : keys)
             {
-                Validation.validateKey(cfm, key);
-                DecoratedKey dk = cfm.decorateKey(key);
+                Validation.validateKey(metadata(), key);
+                DecoratedKey dk = metadata().partitioner.decorateKey(key);
 
-                PartitionUpdate upd = collector.getPartitionUpdate(cfm, dk, options.getConsistency());
+                PartitionUpdate upd = collector.getPartitionUpdate(metadata, dk, options.getConsistency());
 
                 if (!restrictions.hasClusteringColumnsRestrictions())
                 {
@@ -738,14 +740,14 @@ public abstract class ModificationStatement implements CQLStatement
     {
         // Some lists operation requires reading
         Map<DecoratedKey, Partition> lists = readRequiredLists(keys, filter, limits, local, options.getConsistency(), queryStartNanoTime);
-        return new UpdateParameters(cfm, updatedColumns(), options, getTimestamp(now, options), getTimeToLive(options), lists);
+        return new UpdateParameters(metadata(), updatedColumns(), options, getTimestamp(now, options), getTimeToLive(options), lists);
     }
 
     private Slices toSlices(SortedSet<ClusteringBound> startBounds, SortedSet<ClusteringBound> endBounds)
     {
         assert startBounds.size() == endBounds.size();
 
-        Slices.Builder builder = new Slices.Builder(cfm.comparator);
+        Slices.Builder builder = new Slices.Builder(metadata().comparator);
 
         Iterator<ClusteringBound> starts = startBounds.iterator();
         Iterator<ClusteringBound> ends = endBounds.iterator();
@@ -753,7 +755,7 @@ public abstract class ModificationStatement implements CQLStatement
         while (starts.hasNext())
         {
             Slice slice = Slice.make(starts.next(), ends.next());
-            if (!slice.isEmpty(cfm.comparator))
+            if (!slice.isEmpty(metadata().comparator))
             {
                 builder.add(slice);
             }
@@ -766,21 +768,21 @@ public abstract class ModificationStatement implements CQLStatement
     {
         protected final StatementType type;
         private final Attributes.Raw attrs;
-        private final List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions;
+        private final List<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>> conditions;
         private final boolean ifNotExists;
         private final boolean ifExists;
 
         protected Parsed(CFName name,
                          StatementType type,
                          Attributes.Raw attrs,
-                         List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions,
+                         List<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>> conditions,
                          boolean ifNotExists,
                          boolean ifExists)
         {
             super(name);
             this.type = type;
             this.attrs = attrs;
-            this.conditions = conditions == null ? Collections.<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>>emptyList() : conditions;
+            this.conditions = conditions == null ? Collections.<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>>emptyList() : conditions;
             this.ifNotExists = ifNotExists;
             this.ifExists = ifExists;
         }
@@ -789,23 +791,20 @@ public abstract class ModificationStatement implements CQLStatement
         {
             VariableSpecifications boundNames = getBoundVariables();
             ModificationStatement statement = prepare(boundNames);
-            CFMetaData cfm = Validation.validateColumnFamily(keyspace(), columnFamily());
-            return new ParsedStatement.Prepared(statement, boundNames, boundNames.getPartitionKeyBindIndexes(cfm));
+            TableMetadata metadata = Schema.instance.validateTable(keyspace(), columnFamily());
+            return new ParsedStatement.Prepared(statement, boundNames, boundNames.getPartitionKeyBindIndexes(metadata));
         }
 
         public ModificationStatement prepare(VariableSpecifications boundNames)
         {
-            CFMetaData metadata = Validation.validateColumnFamily(keyspace(), columnFamily());
+            TableMetadata metadata = Schema.instance.validateTable(keyspace(), columnFamily());
 
             Attributes preparedAttributes = attrs.prepare(keyspace(), columnFamily());
             preparedAttributes.collectMarkerSpecification(boundNames);
 
             Conditions preparedConditions = prepareConditions(metadata, boundNames);
 
-            return prepareInternal(metadata,
-                                   boundNames,
-                                   preparedConditions,
-                                   preparedAttributes);
+            return prepareInternal(metadata, boundNames, preparedConditions, preparedAttributes);
         }
 
         /**
@@ -815,7 +814,7 @@ public abstract class ModificationStatement implements CQLStatement
          * @param boundNames the bound names
          * @return the column conditions.
          */
-        private Conditions prepareConditions(CFMetaData metadata, VariableSpecifications boundNames)
+        private Conditions prepareConditions(TableMetadata metadata, VariableSpecifications boundNames)
         {
             // To have both 'IF EXISTS'/'IF NOT EXISTS' and some other conditions doesn't make sense.
             // So far this is enforced by the parser, but let's assert it for sanity if ever the parse changes.
@@ -846,15 +845,15 @@ public abstract class ModificationStatement implements CQLStatement
          * @param boundNames the bound names
          * @return the column conditions.
          */
-        private ColumnConditions prepareColumnConditions(CFMetaData metadata, VariableSpecifications boundNames)
+        private ColumnConditions prepareColumnConditions(TableMetadata metadata, VariableSpecifications boundNames)
         {
             checkNull(attrs.timestamp, "Cannot provide custom timestamp for conditional updates");
 
             ColumnConditions.Builder builder = ColumnConditions.newBuilder();
 
-            for (Pair<ColumnDefinition.Raw, ColumnCondition.Raw> entry : conditions)
+            for (Pair<ColumnMetadata.Raw, ColumnCondition.Raw> entry : conditions)
             {
-                ColumnDefinition def = entry.left.prepare(metadata);
+                ColumnMetadata def = entry.left.prepare(metadata);
                 ColumnCondition condition = entry.right.prepare(keyspace(), def, metadata);
                 condition.collectMarkerSpecification(boundNames);
 
@@ -864,7 +863,7 @@ public abstract class ModificationStatement implements CQLStatement
             return builder.build();
         }
 
-        protected abstract ModificationStatement prepareInternal(CFMetaData cfm,
+        protected abstract ModificationStatement prepareInternal(TableMetadata metadata,
                                                                  VariableSpecifications boundNames,
                                                                  Conditions conditions,
                                                                  Attributes attrs);
@@ -872,14 +871,14 @@ public abstract class ModificationStatement implements CQLStatement
         /**
          * Creates the restrictions.
          *
-         * @param cfm the column family meta data
+         * @param metadata the column family meta data
          * @param boundNames the bound names
          * @param operations the column operations
          * @param where the where clause
          * @param conditions the conditions
          * @return the restrictions
          */
-        protected StatementRestrictions newRestrictions(CFMetaData cfm,
+        protected StatementRestrictions newRestrictions(TableMetadata metadata,
                                                         VariableSpecifications boundNames,
                                                         Operations operations,
                                                         WhereClause where,
@@ -889,19 +888,19 @@ public abstract class ModificationStatement implements CQLStatement
                 throw new InvalidRequestException(CUSTOM_EXPRESSIONS_NOT_ALLOWED);
 
             boolean applyOnlyToStaticColumns = appliesOnlyToStaticColumns(operations, conditions);
-            return new StatementRestrictions(type, cfm, where, boundNames, applyOnlyToStaticColumns, false, false);
+            return new StatementRestrictions(type, metadata, where, boundNames, applyOnlyToStaticColumns, false, false);
         }
 
         /**
-         * Retrieves the <code>ColumnDefinition</code> corresponding to the specified raw <code>ColumnIdentifier</code>.
+         * Retrieves the <code>ColumnMetadata</code> corresponding to the specified raw <code>ColumnIdentifier</code>.
          *
-         * @param cfm the column family meta data
+         * @param metadata the column family meta data
          * @param rawId the raw <code>ColumnIdentifier</code>
-         * @return the <code>ColumnDefinition</code> corresponding to the specified raw <code>ColumnIdentifier</code>
+         * @return the <code>ColumnMetadata</code> corresponding to the specified raw <code>ColumnIdentifier</code>
          */
-        protected static ColumnDefinition getColumnDefinition(CFMetaData cfm, Raw rawId)
+        protected static ColumnMetadata getColumnDefinition(TableMetadata metadata, Raw rawId)
         {
-            return rawId.prepare(cfm);
+            return rawId.prepare(metadata);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java b/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java
index 3ae6bd8..8d508fc 100644
--- a/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java
@@ -21,7 +21,7 @@ import java.util.Set;
 
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.cql3.RoleName;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 7e66dc4..652b549 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -26,8 +26,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
@@ -82,7 +84,7 @@ public class SelectStatement implements CQLStatement
     public static final int DEFAULT_PAGE_SIZE = 10000;
 
     private final int boundTerms;
-    public final CFMetaData cfm;
+    public final TableMetadata table;
     public final Parameters parameters;
     private final Selection selection;
     private final Term limit;
@@ -111,7 +113,7 @@ public class SelectStatement implements CQLStatement
                                                                        false,
                                                                        false);
 
-    public SelectStatement(CFMetaData cfm,
+    public SelectStatement(TableMetadata table,
                            int boundTerms,
                            Parameters parameters,
                            Selection selection,
@@ -122,7 +124,7 @@ public class SelectStatement implements CQLStatement
                            Term limit,
                            Term perPartitionLimit)
     {
-        this.cfm = cfm;
+        this.table = table;
         this.boundTerms = boundTerms;
         this.selection = selection;
         this.restrictions = restrictions;
@@ -159,11 +161,11 @@ public class SelectStatement implements CQLStatement
     private ColumnFilter gatherQueriedColumns()
     {
         if (selection.isWildcard())
-            return ColumnFilter.all(cfm);
+            return ColumnFilter.all(table);
 
-        ColumnFilter.Builder builder = ColumnFilter.allRegularColumnsBuilder(cfm);
+        ColumnFilter.Builder builder = ColumnFilter.allRegularColumnsBuilder(table);
         // Adds all selected columns
-        for (ColumnDefinition def : selection.getColumns())
+        for (ColumnMetadata def : selection.getColumns())
             if (!def.isPrimaryKeyColumn())
                 builder.add(def);
         // as well as any restricted column (so we can actually apply the restriction)
@@ -171,8 +173,8 @@ public class SelectStatement implements CQLStatement
 
         // In a number of cases, we want to distinguish between a partition truly empty and one with only static content
         // (but no rows). In those cases, we should force querying all static columns (to make the distinction).
-        if (cfm.hasStaticColumns() && returnStaticContentOnPartitionWithNoRows())
-            builder.addAll(cfm.partitionColumns().statics);
+        if (table.hasStaticColumns() && returnStaticContentOnPartitionWithNoRows())
+            builder.addAll(table.staticColumns());
 
         return builder.build();
     }
@@ -189,13 +191,13 @@ public class SelectStatement implements CQLStatement
     // Creates a simple select based on the given selection.
     // Note that the results select statement should not be used for actual queries, but only for processing already
     // queried data through processColumnFamily.
-    static SelectStatement forSelection(CFMetaData cfm, Selection selection)
+    static SelectStatement forSelection(TableMetadata table, Selection selection)
     {
-        return new SelectStatement(cfm,
+        return new SelectStatement(table,
                                    0,
                                    defaultParameters,
                                    selection,
-                                   StatementRestrictions.empty(StatementType.SELECT, cfm),
+                                   StatementRestrictions.empty(StatementType.SELECT, table),
                                    false,
                                    null,
                                    null,
@@ -215,15 +217,15 @@ public class SelectStatement implements CQLStatement
 
     public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
     {
-        if (cfm.isView())
+        if (table.isView())
         {
-            CFMetaData baseTable = View.findBaseTable(keyspace(), columnFamily());
+            TableMetadataRef baseTable = View.findBaseTable(keyspace(), columnFamily());
             if (baseTable != null)
                 state.hasColumnFamilyAccess(baseTable, Permission.SELECT);
         }
         else
         {
-            state.hasColumnFamilyAccess(cfm, Permission.SELECT);
+            state.hasColumnFamilyAccess(table, Permission.SELECT);
         }
 
         for (Function function : getFunctions())
@@ -453,12 +455,12 @@ public class SelectStatement implements CQLStatement
 
     public String keyspace()
     {
-        return cfm.ksName;
+        return table.keyspace;
     }
 
     public String columnFamily()
     {
-        return cfm.cfName;
+        return table.name;
     }
 
     /**
@@ -495,8 +497,8 @@ public class SelectStatement implements CQLStatement
         for (ByteBuffer key : keys)
         {
             QueryProcessor.validateKey(key);
-            DecoratedKey dk = cfm.decorateKey(ByteBufferUtil.clone(key));
-            commands.add(SinglePartitionReadCommand.create(cfm, nowInSec, queriedColumns, rowFilter, limit, dk, filter));
+            DecoratedKey dk = table.partitioner.decorateKey(ByteBufferUtil.clone(key));
+            commands.add(SinglePartitionReadCommand.create(table, nowInSec, queriedColumns, rowFilter, limit, dk, filter));
         }
 
         return new SinglePartitionReadCommand.Group(commands, limit);
@@ -517,7 +519,7 @@ public class SelectStatement implements CQLStatement
         if (filter instanceof ClusteringIndexSliceFilter)
             return ((ClusteringIndexSliceFilter)filter).requestedSlices();
 
-        Slices.Builder builder = new Slices.Builder(cfm.comparator);
+        Slices.Builder builder = new Slices.Builder(table.comparator);
         for (Clustering clustering: ((ClusteringIndexNamesFilter)filter).requestedRows())
             builder.add(Slice.make(clustering));
         return builder.build();
@@ -532,7 +534,7 @@ public class SelectStatement implements CQLStatement
         QueryOptions options = QueryOptions.forInternalCalls(Collections.emptyList());
         ClusteringIndexFilter filter = makeClusteringIndexFilter(options);
         RowFilter rowFilter = getRowFilter(options);
-        return SinglePartitionReadCommand.create(cfm, nowInSec, queriedColumns, rowFilter, DataLimits.NONE, key, filter);
+        return SinglePartitionReadCommand.create(table, nowInSec, queriedColumns, rowFilter, DataLimits.NONE, key, filter);
     }
 
     /**
@@ -557,7 +559,7 @@ public class SelectStatement implements CQLStatement
         if (keyBounds == null)
             return ReadQuery.EMPTY;
 
-        PartitionRangeReadCommand command = new PartitionRangeReadCommand(cfm,
+        PartitionRangeReadCommand command = new PartitionRangeReadCommand(table,
                                                                           nowInSec,
                                                                           queriedColumns,
                                                                           rowFilter,
@@ -622,12 +624,12 @@ public class SelectStatement implements CQLStatement
         {
             ClusteringBound start = startBounds.first();
             ClusteringBound end = endBounds.first();
-            return cfm.comparator.compare(start, end) > 0
+            return table.comparator.compare(start, end) > 0
                  ? Slices.NONE
-                 : Slices.with(cfm.comparator, Slice.make(start, end));
+                 : Slices.with(table.comparator, Slice.make(start, end));
         }
 
-        Slices.Builder builder = new Slices.Builder(cfm.comparator, startBounds.size());
+        Slices.Builder builder = new Slices.Builder(table.comparator, startBounds.size());
         Iterator<ClusteringBound> startIter = startBounds.iterator();
         Iterator<ClusteringBound> endIter = endBounds.iterator();
         while (startIter.hasNext() && endIter.hasNext())
@@ -636,7 +638,7 @@ public class SelectStatement implements CQLStatement
             ClusteringBound end = endIter.next();
 
             // Ignore slices that are nonsensical
-            if (cfm.comparator.compare(start, end) > 0)
+            if (table.comparator.compare(start, end) > 0)
                 continue;
 
             builder.add(start, end);
@@ -774,12 +776,12 @@ public class SelectStatement implements CQLStatement
         return cqlRows;
     }
 
-    public static ByteBuffer[] getComponents(CFMetaData cfm, DecoratedKey dk)
+    public static ByteBuffer[] getComponents(TableMetadata metadata, DecoratedKey dk)
     {
         ByteBuffer key = dk.getKey();
-        if (cfm.getKeyValidator() instanceof CompositeType)
+        if (metadata.partitionKeyType instanceof CompositeType)
         {
-            return ((CompositeType)cfm.getKeyValidator()).split(key);
+            return ((CompositeType)metadata.partitionKeyType).split(key);
         }
         else
         {
@@ -796,7 +798,7 @@ public class SelectStatement implements CQLStatement
         // query, then we include that content.
         // We make an exception for "static compact" table are from a CQL standpoint we always want to show their static
         // content for backward compatiblity.
-        return queriesFullPartitions() || cfm.isStaticCompactTable();
+        return queriesFullPartitions() || table.isStaticCompactTable();
     }
 
     // Used by ModificationStatement for CAS operations
@@ -805,7 +807,7 @@ public class SelectStatement implements CQLStatement
     {
         ProtocolVersion protocolVersion = options.getProtocolVersion();
 
-        ByteBuffer[] keyComponents = getComponents(cfm, partition.partitionKey());
+        ByteBuffer[] keyComponents = getComponents(table, partition.partitionKey());
 
         Row staticRow = partition.staticRow();
         // If there is no rows, we include the static content if we should and we're done.
@@ -814,7 +816,7 @@ public class SelectStatement implements CQLStatement
             if (!staticRow.isEmpty() && returnStaticContentOnPartitionWithNoRows())
             {
                 result.newRow(partition.partitionKey(), staticRow.clustering());
-                for (ColumnDefinition def : selection.getColumns())
+                for (ColumnMetadata def : selection.getColumns())
                 {
                     switch (def.kind)
                     {
@@ -837,7 +839,7 @@ public class SelectStatement implements CQLStatement
             Row row = partition.next();
             result.newRow( partition.partitionKey(), row.clustering());
             // Respect selection order
-            for (ColumnDefinition def : selection.getColumns())
+            for (ColumnMetadata def : selection.getColumns())
             {
                 switch (def.kind)
                 {
@@ -867,7 +869,7 @@ public class SelectStatement implements CQLStatement
         return !restrictions.hasClusteringColumnsRestrictions() && !restrictions.hasRegularColumnsRestrictions();
     }
 
-    private static void addValue(Selection.ResultSetBuilder result, ColumnDefinition def, Row row, int nowInSec, ProtocolVersion protocolVersion)
+    private static void addValue(Selection.ResultSetBuilder result, ColumnMetadata def, Row row, int nowInSec, ProtocolVersion protocolVersion)
     {
         if (def.isComplex())
         {
@@ -932,22 +934,22 @@ public class SelectStatement implements CQLStatement
 
         public ParsedStatement.Prepared prepare(boolean forView) throws InvalidRequestException
         {
-            CFMetaData cfm = Validation.validateColumnFamily(keyspace(), columnFamily());
+            TableMetadata table = Schema.instance.validateTable(keyspace(), columnFamily());
             VariableSpecifications boundNames = getBoundVariables();
 
             Selection selection = selectClause.isEmpty()
-                                  ? Selection.wildcard(cfm)
-                                  : Selection.fromSelectors(cfm, selectClause, boundNames, !parameters.groups.isEmpty());
+                                  ? Selection.wildcard(table)
+                                  : Selection.fromSelectors(table, selectClause, boundNames, !parameters.groups.isEmpty());
 
-            StatementRestrictions restrictions = prepareRestrictions(cfm, boundNames, selection, forView);
+            StatementRestrictions restrictions = prepareRestrictions(table, boundNames, selection, forView);
 
             if (parameters.isDistinct)
             {
                 checkNull(perPartitionLimit, "PER PARTITION LIMIT is not allowed with SELECT DISTINCT queries");
-                validateDistinctSelection(cfm, selection, restrictions);
+                validateDistinctSelection(table, selection, restrictions);
             }
 
-            AggregationSpecification aggregationSpec = getAggregationSpecification(cfm,
+            AggregationSpecification aggregationSpec = getAggregationSpecification(table,
                                                                                    selection,
                                                                                    restrictions,
                                                                                    parameters.isDistinct);
@@ -962,15 +964,15 @@ public class SelectStatement implements CQLStatement
             {
                 assert !forView;
                 verifyOrderingIsAllowed(restrictions);
-                orderingComparator = getOrderingComparator(cfm, selection, restrictions);
-                isReversed = isReversed(cfm);
+                orderingComparator = getOrderingComparator(table, selection, restrictions);
+                isReversed = isReversed(table);
                 if (isReversed)
                     orderingComparator = Collections.reverseOrder(orderingComparator);
             }
 
             checkNeedsFiltering(restrictions);
 
-            SelectStatement stmt = new SelectStatement(cfm,
+            SelectStatement stmt = new SelectStatement(table,
                                                        boundNames.size(),
                                                        parameters,
                                                        selection,
@@ -981,25 +983,25 @@ public class SelectStatement implements CQLStatement
                                                        prepareLimit(boundNames, limit, keyspace(), limitReceiver()),
                                                        prepareLimit(boundNames, perPartitionLimit, keyspace(), perPartitionLimitReceiver()));
 
-            return new ParsedStatement.Prepared(stmt, boundNames, boundNames.getPartitionKeyBindIndexes(cfm));
+            return new ParsedStatement.Prepared(stmt, boundNames, boundNames.getPartitionKeyBindIndexes(table));
         }
 
         /**
          * Prepares the restrictions.
          *
-         * @param cfm the column family meta data
+         * @param metadata the column family meta data
          * @param boundNames the variable specifications
          * @param selection the selection
          * @return the restrictions
          * @throws InvalidRequestException if a problem occurs while building the restrictions
          */
-        private StatementRestrictions prepareRestrictions(CFMetaData cfm,
+        private StatementRestrictions prepareRestrictions(TableMetadata metadata,
                                                           VariableSpecifications boundNames,
                                                           Selection selection,
                                                           boolean forView) throws InvalidRequestException
         {
             return new StatementRestrictions(StatementType.SELECT,
-                                             cfm,
+                                             metadata,
                                              whereClause,
                                              boundNames,
                                              selection.containsOnlyStaticColumns(),
@@ -1025,17 +1027,17 @@ public class SelectStatement implements CQLStatement
             checkFalse(restrictions.isKeyRange(), "ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
         }
 
-        private static void validateDistinctSelection(CFMetaData cfm,
+        private static void validateDistinctSelection(TableMetadata metadata,
                                                       Selection selection,
                                                       StatementRestrictions restrictions)
                                                       throws InvalidRequestException
         {
             checkFalse(restrictions.hasClusteringColumnsRestrictions() ||
-                       (restrictions.hasNonPrimaryKeyRestrictions() && !restrictions.nonPKRestrictedColumns(true).stream().allMatch(ColumnDefinition::isStatic)),
+                       (restrictions.hasNonPrimaryKeyRestrictions() && !restrictions.nonPKRestrictedColumns(true).stream().allMatch(ColumnMetadata::isStatic)),
                        "SELECT DISTINCT with WHERE clause only supports restriction by partition key and/or static columns.");
 
-            Collection<ColumnDefinition> requestedColumns = selection.getColumns();
-            for (ColumnDefinition def : requestedColumns)
+            Collection<ColumnMetadata> requestedColumns = selection.getColumns();
+            for (ColumnMetadata def : requestedColumns)
                 checkFalse(!def.isPartitionKey() && !def.isStatic(),
                            "SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)",
                            def.name);
@@ -1045,7 +1047,7 @@ public class SelectStatement implements CQLStatement
             if (!restrictions.isKeyRange())
                 return;
 
-            for (ColumnDefinition def : cfm.partitionKeyColumns())
+            for (ColumnMetadata def : metadata.partitionKeyColumns())
                 checkTrue(requestedColumns.contains(def),
                           "SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name);
         }
@@ -1053,13 +1055,13 @@ public class SelectStatement implements CQLStatement
         /**
          * Creates the <code>AggregationSpecification</code>s used to make the aggregates.
          *
-         * @param cfm the column family metadata
+         * @param metadata the table metadata
          * @param selection the selection
          * @param restrictions the restrictions
          * @param isDistinct <code>true</code> if the query is a DISTINCT one. 
          * @return the <code>AggregationSpecification</code>s used to make the aggregates
          */
-        private AggregationSpecification getAggregationSpecification(CFMetaData cfm,
+        private AggregationSpecification getAggregationSpecification(TableMetadata metadata,
                                                                      Selection selection,
                                                                      StatementRestrictions restrictions,
                                                                      boolean isDistinct)
@@ -1070,10 +1072,10 @@ public class SelectStatement implements CQLStatement
 
             int clusteringPrefixSize = 0;
 
-            Iterator<ColumnDefinition> pkColumns = cfm.primaryKeyColumns().iterator();
-            for (ColumnDefinition.Raw raw : parameters.groups)
+            Iterator<ColumnMetadata> pkColumns = metadata.primaryKeyColumns().iterator();
+            for (ColumnMetadata.Raw raw : parameters.groups)
             {
-                ColumnDefinition def = raw.prepare(cfm);
+                ColumnMetadata def = raw.prepare(metadata);
 
                 checkTrue(def.isPartitionKey() || def.isClusteringColumn(),
                           "Group by is currently only supported on the columns of the PRIMARY KEY, got %s", def.name);
@@ -1083,7 +1085,7 @@ public class SelectStatement implements CQLStatement
                     checkTrue(pkColumns.hasNext(),
                               "Group by currently only support groups of columns following their declared order in the PRIMARY KEY");
 
-                    ColumnDefinition pkColumn = pkColumns.next();
+                    ColumnMetadata pkColumn = pkColumns.next();
 
                     if (pkColumn.isClusteringColumn())
                         clusteringPrefixSize++;
@@ -1104,10 +1106,10 @@ public class SelectStatement implements CQLStatement
             checkFalse(clusteringPrefixSize > 0 && isDistinct,
                        "Grouping on clustering columns is not allowed for SELECT DISTINCT queries");
 
-            return AggregationSpecification.aggregatePkPrefix(cfm.comparator, clusteringPrefixSize);
+            return AggregationSpecification.aggregatePkPrefix(metadata.comparator, clusteringPrefixSize);
         }
 
-        private Comparator<List<ByteBuffer>> getOrderingComparator(CFMetaData cfm,
+        private Comparator<List<ByteBuffer>> getOrderingComparator(TableMetadata metadata,
                                                                    Selection selection,
                                                                    StatementRestrictions restrictions)
                                                                    throws InvalidRequestException
@@ -1115,14 +1117,14 @@ public class SelectStatement implements CQLStatement
             if (!restrictions.keyIsInRelation())
                 return null;
 
-            Map<ColumnIdentifier, Integer> orderingIndexes = getOrderingIndex(cfm, selection);
+            Map<ColumnIdentifier, Integer> orderingIndexes = getOrderingIndex(metadata, selection);
 
             List<Integer> idToSort = new ArrayList<Integer>();
             List<Comparator<ByteBuffer>> sorters = new ArrayList<Comparator<ByteBuffer>>();
 
-            for (ColumnDefinition.Raw raw : parameters.orderings.keySet())
+            for (ColumnMetadata.Raw raw : parameters.orderings.keySet())
             {
-                ColumnDefinition orderingColumn = raw.prepare(cfm);
+                ColumnMetadata orderingColumn = raw.prepare(metadata);
                 idToSort.add(orderingIndexes.get(orderingColumn.name));
                 sorters.add(orderingColumn.type);
             }
@@ -1130,16 +1132,16 @@ public class SelectStatement implements CQLStatement
                     : new CompositeComparator(sorters, idToSort);
         }
 
-        private Map<ColumnIdentifier, Integer> getOrderingIndex(CFMetaData cfm, Selection selection)
+        private Map<ColumnIdentifier, Integer> getOrderingIndex(TableMetadata table, Selection selection)
                 throws InvalidRequestException
         {
             // If we order post-query (see orderResults), the sorted column needs to be in the ResultSet for sorting,
             // even if we don't
             // ultimately ship them to the client (CASSANDRA-4911).
             Map<ColumnIdentifier, Integer> orderingIndexes = new HashMap<>();
-            for (ColumnDefinition.Raw raw : parameters.orderings.keySet())
+            for (ColumnMetadata.Raw raw : parameters.orderings.keySet())
             {
-                final ColumnDefinition def = raw.prepare(cfm);
+                final ColumnMetadata def = raw.prepare(table);
                 int index = selection.getResultSetIndex(def);
                 if (index < 0)
                     index = selection.addColumnForOrdering(def);
@@ -1148,13 +1150,13 @@ public class SelectStatement implements CQLStatement
             return orderingIndexes;
         }
 
-        private boolean isReversed(CFMetaData cfm) throws InvalidRequestException
+        private boolean isReversed(TableMetadata table) throws InvalidRequestException
         {
-            Boolean[] reversedMap = new Boolean[cfm.clusteringColumns().size()];
+            Boolean[] reversedMap = new Boolean[table.clusteringColumns().size()];
             int i = 0;
-            for (Map.Entry<ColumnDefinition.Raw, Boolean> entry : parameters.orderings.entrySet())
+            for (Map.Entry<ColumnMetadata.Raw, Boolean> entry : parameters.orderings.entrySet())
             {
-                ColumnDefinition def = entry.getKey().prepare(cfm);
+                ColumnMetadata def = entry.getKey().prepare(table);
                 boolean reversed = entry.getValue();
 
                 checkTrue(def.isClusteringColumn(),
@@ -1223,14 +1225,14 @@ public class SelectStatement implements CQLStatement
     public static class Parameters
     {
         // Public because CASSANDRA-9858
-        public final Map<ColumnDefinition.Raw, Boolean> orderings;
-        public final List<ColumnDefinition.Raw> groups;
+        public final Map<ColumnMetadata.Raw, Boolean> orderings;
+        public final List<ColumnMetadata.Raw> groups;
         public final boolean isDistinct;
         public final boolean allowFiltering;
         public final boolean isJson;
 
-        public Parameters(Map<ColumnDefinition.Raw, Boolean> orderings,
-                          List<ColumnDefinition.Raw> groups,
+        public Parameters(Map<ColumnMetadata.Raw, Boolean> orderings,
+                          List<ColumnMetadata.Raw> groups,
                           boolean isDistinct,
                           boolean allowFiltering,
                           boolean isJson)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/TableAttributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/TableAttributes.java b/src/java/org/apache/cassandra/cql3/statements/TableAttributes.java
index 63e1eec..779b6c2 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TableAttributes.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TableAttributes.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.cql3.statements;
 
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 
 import com.google.common.collect.ImmutableSet;
 
@@ -61,15 +60,15 @@ public final class TableAttributes extends PropertyDefinitions
     {
         if (getId() != null)
             throw new ConfigurationException("Cannot alter table id.");
-        return build(TableParams.builder(previous));
+        return build(previous.unbuild());
     }
 
-    public UUID getId() throws ConfigurationException
+    public TableId getId() throws ConfigurationException
     {
         String id = getSimple(KW_ID);
         try
         {
-            return id != null ? UUID.fromString(id) : null;
+            return id != null ? TableId.fromString(id) : null;
         }
         catch (IllegalArgumentException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
index 927cdda..300d8f4 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -20,16 +20,16 @@ package org.apache.cassandra.cql3.statements;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.transport.messages.ResultMessage;
 
 public class TruncateStatement extends CFStatement implements CQLStatement
 {
@@ -55,14 +55,14 @@ public class TruncateStatement extends CFStatement implements CQLStatement
 
     public void validate(ClientState state) throws InvalidRequestException
     {
-        Validation.validateColumnFamily(keyspace(), columnFamily());
+        Schema.instance.validateTable(keyspace(), columnFamily());
     }
 
     public ResultMessage execute(QueryState state, QueryOptions options, long queryStartNanoTime) throws InvalidRequestException, TruncateException
     {
         try
         {
-            CFMetaData metaData = Schema.instance.getCFMetaData(keyspace(), columnFamily());
+            TableMetadata metaData = Schema.instance.getTableMetadata(keyspace(), columnFamily());
             if (metaData.isView())
                 throw new InvalidRequestException("Cannot TRUNCATE materialized view directly; must truncate base table instead");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index 01d47bd..7a2a1ba 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -21,8 +21,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.conditions.ColumnCondition;
 import org.apache.cassandra.cql3.conditions.Conditions;
@@ -31,6 +29,8 @@ import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.CompactTables;
 import org.apache.cassandra.db.Slice;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 
@@ -48,13 +48,13 @@ public class UpdateStatement extends ModificationStatement
 
     private UpdateStatement(StatementType type,
                             int boundTerms,
-                            CFMetaData cfm,
+                            TableMetadata metadata,
                             Operations operations,
                             StatementRestrictions restrictions,
                             Conditions conditions,
                             Attributes attrs)
     {
-        super(type, boundTerms, cfm, operations, restrictions, conditions, attrs);
+        super(type, boundTerms, metadata, operations, restrictions, conditions, attrs);
     }
 
     public boolean requireFullClusteringKey()
@@ -72,7 +72,7 @@ public class UpdateStatement extends ModificationStatement
             // We update the row timestamp (ex-row marker) only on INSERT (#6782)
             // Further, COMPACT tables semantic differs from "CQL3" ones in that a row exists only if it has
             // a non-null column, so we don't want to set the row timestamp for them.
-            if (type.isInsert() && cfm.isCQLTable())
+            if (type.isInsert() && metadata().isCQLTable())
                 params.addPrimaryKeyLivenessInfo();
 
             List<Operation> updates = getRegularOperations();
@@ -80,13 +80,13 @@ public class UpdateStatement extends ModificationStatement
             // For compact table, we don't accept an insert/update that only sets the PK unless the is no
             // declared non-PK columns (which we recognize because in that case
             // the compact value is of type "EmptyType").
-            if (cfm.isCompactTable() && updates.isEmpty())
+            if (metadata().isCompactTable() && updates.isEmpty())
             {
-                checkTrue(CompactTables.hasEmptyCompactValue(cfm),
+                checkTrue(CompactTables.hasEmptyCompactValue(metadata),
                           "Column %s is mandatory for this COMPACT STORAGE table",
-                          cfm.compactValueColumn().name);
+                          metadata().compactValueColumn.name);
 
-                updates = Collections.<Operation>singletonList(new Constants.Setter(cfm.compactValueColumn(), EMPTY));
+                updates = Collections.<Operation>singletonList(new Constants.Setter(metadata().compactValueColumn, EMPTY));
             }
 
             for (Operation op : updates)
@@ -112,7 +112,7 @@ public class UpdateStatement extends ModificationStatement
 
     public static class ParsedInsert extends ModificationStatement.Parsed
     {
-        private final List<ColumnDefinition.Raw> columnNames;
+        private final List<ColumnMetadata.Raw> columnNames;
         private final List<Term.Raw> columnValues;
 
         /**
@@ -126,7 +126,7 @@ public class UpdateStatement extends ModificationStatement
          */
         public ParsedInsert(CFName name,
                             Attributes.Raw attrs,
-                            List<ColumnDefinition.Raw> columnNames,
+                            List<ColumnMetadata.Raw> columnNames,
                             List<Term.Raw> columnValues,
                             boolean ifNotExists)
         {
@@ -136,14 +136,14 @@ public class UpdateStatement extends ModificationStatement
         }
 
         @Override
-        protected ModificationStatement prepareInternal(CFMetaData cfm,
+        protected ModificationStatement prepareInternal(TableMetadata metadata,
                                                         VariableSpecifications boundNames,
                                                         Conditions conditions,
                                                         Attributes attrs)
         {
 
             // Created from an INSERT
-            checkFalse(cfm.isCounter(), "INSERT statements are not allowed on counter tables, use UPDATE instead");
+            checkFalse(metadata.isCounter(), "INSERT statements are not allowed on counter tables, use UPDATE instead");
 
             checkFalse(columnNames == null, "Column names for INSERT must be provided when using VALUES");
             checkFalse(columnNames.isEmpty(), "No columns provided to INSERT");
@@ -156,7 +156,7 @@ public class UpdateStatement extends ModificationStatement
 
             for (int i = 0; i < columnNames.size(); i++)
             {
-                ColumnDefinition def = getColumnDefinition(cfm, columnNames.get(i));
+                ColumnMetadata def = getColumnDefinition(metadata, columnNames.get(i));
 
                 if (def.isClusteringColumn())
                     hasClusteringColumnsSet = true;
@@ -169,7 +169,7 @@ public class UpdateStatement extends ModificationStatement
                 }
                 else
                 {
-                    Operation operation = new Operation.SetValue(value).prepare(cfm, def);
+                    Operation operation = new Operation.SetValue(value).prepare(metadata, def);
                     operation.collectMarkerSpecification(boundNames);
                     operations.add(operation);
                 }
@@ -178,7 +178,7 @@ public class UpdateStatement extends ModificationStatement
             boolean applyOnlyToStaticColumns = !hasClusteringColumnsSet && appliesOnlyToStaticColumns(operations, conditions);
 
             StatementRestrictions restrictions = new StatementRestrictions(type,
-                                                                           cfm,
+                                                                           metadata,
                                                                            whereClause.build(),
                                                                            boundNames,
                                                                            applyOnlyToStaticColumns,
@@ -187,7 +187,7 @@ public class UpdateStatement extends ModificationStatement
 
             return new UpdateStatement(type,
                                        boundNames.size(),
-                                       cfm,
+                                       metadata,
                                        operations,
                                        restrictions,
                                        conditions,
@@ -211,21 +211,21 @@ public class UpdateStatement extends ModificationStatement
         }
 
         @Override
-        protected ModificationStatement prepareInternal(CFMetaData cfm,
+        protected ModificationStatement prepareInternal(TableMetadata metadata,
                                                         VariableSpecifications boundNames,
                                                         Conditions conditions,
                                                         Attributes attrs)
         {
-            checkFalse(cfm.isCounter(), "INSERT statements are not allowed on counter tables, use UPDATE instead");
+            checkFalse(metadata.isCounter(), "INSERT statements are not allowed on counter tables, use UPDATE instead");
 
-            Collection<ColumnDefinition> defs = cfm.allColumns();
-            Json.Prepared prepared = jsonValue.prepareAndCollectMarkers(cfm, defs, boundNames);
+            Collection<ColumnMetadata> defs = metadata.columns();
+            Json.Prepared prepared = jsonValue.prepareAndCollectMarkers(metadata, defs, boundNames);
 
             WhereClause.Builder whereClause = new WhereClause.Builder();
             Operations operations = new Operations(type);
             boolean hasClusteringColumnsSet = false;
 
-            for (ColumnDefinition def : defs)
+            for (ColumnMetadata def : defs)
             {
                 if (def.isClusteringColumn())
                     hasClusteringColumnsSet = true;
@@ -233,11 +233,11 @@ public class UpdateStatement extends ModificationStatement
                 Term.Raw raw = prepared.getRawTermForColumn(def, defaultUnset);
                 if (def.isPrimaryKeyColumn())
                 {
-                    whereClause.add(new SingleColumnRelation(ColumnDefinition.Raw.forColumn(def), Operator.EQ, raw));
+                    whereClause.add(new SingleColumnRelation(ColumnMetadata.Raw.forColumn(def), Operator.EQ, raw));
                 }
                 else
                 {
-                    Operation operation = new Operation.SetValue(raw).prepare(cfm, def);
+                    Operation operation = new Operation.SetValue(raw).prepare(metadata, def);
                     operation.collectMarkerSpecification(boundNames);
                     operations.add(operation);
                 }
@@ -246,7 +246,7 @@ public class UpdateStatement extends ModificationStatement
             boolean applyOnlyToStaticColumns = !hasClusteringColumnsSet && appliesOnlyToStaticColumns(operations, conditions);
 
             StatementRestrictions restrictions = new StatementRestrictions(type,
-                                                                           cfm,
+                                                                           metadata,
                                                                            whereClause.build(),
                                                                            boundNames,
                                                                            applyOnlyToStaticColumns,
@@ -255,7 +255,7 @@ public class UpdateStatement extends ModificationStatement
 
             return new UpdateStatement(type,
                                        boundNames.size(),
-                                       cfm,
+                                       metadata,
                                        operations,
                                        restrictions,
                                        conditions,
@@ -266,7 +266,7 @@ public class UpdateStatement extends ModificationStatement
     public static class ParsedUpdate extends ModificationStatement.Parsed
     {
         // Provided for an UPDATE
-        private final List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> updates;
+        private final List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> updates;
         private final WhereClause whereClause;
 
         /**
@@ -281,9 +281,9 @@ public class UpdateStatement extends ModificationStatement
          * */
         public ParsedUpdate(CFName name,
                             Attributes.Raw attrs,
-                            List<Pair<ColumnDefinition.Raw, Operation.RawUpdate>> updates,
+                            List<Pair<ColumnMetadata.Raw, Operation.RawUpdate>> updates,
                             WhereClause whereClause,
-                            List<Pair<ColumnDefinition.Raw, ColumnCondition.Raw>> conditions,
+                            List<Pair<ColumnMetadata.Raw, ColumnCondition.Raw>> conditions,
                             boolean ifExists)
         {
             super(name, StatementType.UPDATE, attrs, conditions, false, ifExists);
@@ -292,25 +292,25 @@ public class UpdateStatement extends ModificationStatement
         }
 
         @Override
-        protected ModificationStatement prepareInternal(CFMetaData cfm,
+        protected ModificationStatement prepareInternal(TableMetadata metadata,
                                                         VariableSpecifications boundNames,
                                                         Conditions conditions,
                                                         Attributes attrs)
         {
             Operations operations = new Operations(type);
 
-            for (Pair<ColumnDefinition.Raw, Operation.RawUpdate> entry : updates)
+            for (Pair<ColumnMetadata.Raw, Operation.RawUpdate> entry : updates)
             {
-                ColumnDefinition def = getColumnDefinition(cfm, entry.left);
+                ColumnMetadata def = getColumnDefinition(metadata, entry.left);
 
                 checkFalse(def.isPrimaryKeyColumn(), "PRIMARY KEY part %s found in SET part", def.name);
 
-                Operation operation = entry.right.prepare(cfm, def);
+                Operation operation = entry.right.prepare(metadata, def);
                 operation.collectMarkerSpecification(boundNames);
                 operations.add(operation);
             }
 
-            StatementRestrictions restrictions = newRestrictions(cfm,
+            StatementRestrictions restrictions = newRestrictions(metadata,
                                                                  boundNames,
                                                                  operations,
                                                                  whereClause,
@@ -318,7 +318,7 @@ public class UpdateStatement extends ModificationStatement
 
             return new UpdateStatement(type,
                                        boundNames.size(),
-                                       cfm,
+                                       metadata,
                                        operations,
                                        restrictions,
                                        conditions,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/cql3/statements/UpdatesCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdatesCollector.java b/src/java/org/apache/cassandra/cql3/statements/UpdatesCollector.java
index 1d65a78..f16c619 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdatesCollector.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdatesCollector.java
@@ -20,7 +20,8 @@ package org.apache.cassandra.cql3.statements;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 
@@ -36,7 +37,7 @@ final class UpdatesCollector
     /**
      * The columns that will be updated for each table (keyed by the table ID).
      */
-    private final Map<UUID, PartitionColumns> updatedColumns;
+    private final Map<TableId, RegularAndStaticColumns> updatedColumns;
 
     /**
      * The estimated number of updated row.
@@ -48,7 +49,7 @@ final class UpdatesCollector
      */
     private final Map<String, Map<ByteBuffer, IMutation>> mutations = new HashMap<>();
 
-    public UpdatesCollector(Map<UUID, PartitionColumns> updatedColumns, int updatedRows)
+    public UpdatesCollector(Map<TableId, RegularAndStaticColumns> updatedColumns, int updatedRows)
     {
         super();
         this.updatedColumns = updatedColumns;
@@ -59,20 +60,20 @@ final class UpdatesCollector
      * Gets the <code>PartitionUpdate</code> for the specified column family and key. If the update does not
      * exist it will be created.
      *
-     * @param cfm the column family meta data
+     * @param metadata the column family meta data
      * @param dk the partition key
      * @param consistency the consistency level
      * @return the <code>PartitionUpdate</code> for the specified column family and key
      */
-    public PartitionUpdate getPartitionUpdate(CFMetaData cfm, DecoratedKey dk, ConsistencyLevel consistency)
+    public PartitionUpdate getPartitionUpdate(TableMetadata metadata, DecoratedKey dk, ConsistencyLevel consistency)
     {
-        Mutation mut = getMutation(cfm, dk, consistency);
-        PartitionUpdate upd = mut.get(cfm);
+        Mutation mut = getMutation(metadata, dk, consistency);
+        PartitionUpdate upd = mut.get(metadata);
         if (upd == null)
         {
-            PartitionColumns columns = updatedColumns.get(cfm.cfId);
+            RegularAndStaticColumns columns = updatedColumns.get(metadata.id);
             assert columns != null;
-            upd = new PartitionUpdate(cfm, dk, columns, updatedRows);
+            upd = new PartitionUpdate(metadata, dk, columns, updatedRows);
             mut.add(upd);
         }
         return upd;
@@ -90,18 +91,18 @@ final class UpdatesCollector
                     Keyspace.openAndGetStore(update.metadata()).indexManager.validate(update);
     }
 
-    private Mutation getMutation(CFMetaData cfm, DecoratedKey dk, ConsistencyLevel consistency)
+    private Mutation getMutation(TableMetadata metadata, DecoratedKey dk, ConsistencyLevel consistency)
     {
-        String ksName = cfm.ksName;
+        String ksName = metadata.keyspace;
         IMutation mutation = keyspaceMap(ksName).get(dk.getKey());
         if (mutation == null)
         {
             Mutation mut = new Mutation(ksName, dk);
-            mutation = cfm.isCounter() ? new CounterMutation(mut, consistency) : mut;
+            mutation = metadata.isCounter() ? new CounterMutation(mut, consistency) : mut;
             keyspaceMap(ksName).put(dk.getKey(), mutation);
             return mut;
         }
-        return cfm.isCounter() ? ((CounterMutation) mutation).getMutation() : (Mutation) mutation;
+        return metadata.isCounter() ? ((CounterMutation) mutation).getMutation() : (Mutation) mutation;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
index 849e684..5a049d3 100644
--- a/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
+++ b/src/java/org/apache/cassandra/db/AbstractReadCommandBuilder.java
@@ -21,8 +21,8 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.filter.*;
@@ -64,28 +64,28 @@ public abstract class AbstractReadCommandBuilder
     public AbstractReadCommandBuilder fromIncl(Object... values)
     {
         assert lowerClusteringBound == null && clusterings == null;
-        this.lowerClusteringBound = ClusteringBound.create(cfs.metadata.comparator, true, true, values);
+        this.lowerClusteringBound = ClusteringBound.create(cfs.metadata().comparator, true, true, values);
         return this;
     }
 
     public AbstractReadCommandBuilder fromExcl(Object... values)
     {
         assert lowerClusteringBound == null && clusterings == null;
-        this.lowerClusteringBound = ClusteringBound.create(cfs.metadata.comparator, true, false, values);
+        this.lowerClusteringBound = ClusteringBound.create(cfs.metadata().comparator, true, false, values);
         return this;
     }
 
     public AbstractReadCommandBuilder toIncl(Object... values)
     {
         assert upperClusteringBound == null && clusterings == null;
-        this.upperClusteringBound = ClusteringBound.create(cfs.metadata.comparator, false, true, values);
+        this.upperClusteringBound = ClusteringBound.create(cfs.metadata().comparator, false, true, values);
         return this;
     }
 
     public AbstractReadCommandBuilder toExcl(Object... values)
     {
         assert upperClusteringBound == null && clusterings == null;
-        this.upperClusteringBound = ClusteringBound.create(cfs.metadata.comparator, false, false, values);
+        this.upperClusteringBound = ClusteringBound.create(cfs.metadata().comparator, false, false, values);
         return this;
     }
 
@@ -94,9 +94,9 @@ public abstract class AbstractReadCommandBuilder
         assert lowerClusteringBound == null && upperClusteringBound == null;
 
         if (this.clusterings == null)
-            this.clusterings = new TreeSet<>(cfs.metadata.comparator);
+            this.clusterings = new TreeSet<>(cfs.metadata().comparator);
 
-        this.clusterings.add(cfs.metadata.comparator.make(values));
+        this.clusterings.add(cfs.metadata().comparator.make(values));
         return this;
     }
 
@@ -163,7 +163,7 @@ public abstract class AbstractReadCommandBuilder
 
     public AbstractReadCommandBuilder filterOn(String column, Operator op, Object value)
     {
-        ColumnDefinition def = cfs.metadata.getColumnDefinition(ColumnIdentifier.getInterned(column, true));
+        ColumnMetadata def = cfs.metadata().getColumn(ColumnIdentifier.getInterned(column, true));
         assert def != null;
 
         AbstractType<?> type = def.type;
@@ -179,11 +179,11 @@ public abstract class AbstractReadCommandBuilder
     protected ColumnFilter makeColumnFilter()
     {
         if (columns == null || columns.isEmpty())
-            return ColumnFilter.all(cfs.metadata);
+            return ColumnFilter.all(cfs.metadata());
 
         ColumnFilter.Builder filter = ColumnFilter.selectionBuilder();
         for (ColumnIdentifier column : columns)
-            filter.add(cfs.metadata.getColumnDefinition(column));
+            filter.add(cfs.metadata().getColumn(column));
         return filter.build();
     }
 
@@ -197,7 +197,7 @@ public abstract class AbstractReadCommandBuilder
         {
             Slice slice = Slice.make(lowerClusteringBound == null ? ClusteringBound.BOTTOM : lowerClusteringBound,
                                      upperClusteringBound == null ? ClusteringBound.TOP : upperClusteringBound);
-            return new ClusteringIndexSliceFilter(Slices.with(cfs.metadata.comparator, slice), reversed);
+            return new ClusteringIndexSliceFilter(Slices.with(cfs.metadata().comparator, slice), reversed);
         }
     }
 
@@ -224,7 +224,7 @@ public abstract class AbstractReadCommandBuilder
         @Override
         public ReadCommand build()
         {
-            return SinglePartitionReadCommand.create(cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), partitionKey, makeFilter());
+            return SinglePartitionReadCommand.create(cfs.metadata(), nowInSeconds, makeColumnFilter(), filter, makeLimits(), partitionKey, makeFilter());
         }
     }
 
@@ -255,7 +255,7 @@ public abstract class AbstractReadCommandBuilder
         @Override
         public ReadCommand build()
         {
-            return SinglePartitionReadCommand.create(cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), partitionKey, makeFilter());
+            return SinglePartitionReadCommand.create(cfs.metadata(), nowInSeconds, makeColumnFilter(), filter, makeLimits(), partitionKey, makeFilter());
         }
     }
 
@@ -275,7 +275,7 @@ public abstract class AbstractReadCommandBuilder
         {
             assert startKey == null;
             this.startInclusive = true;
-            this.startKey = makeKey(cfs.metadata, values);
+            this.startKey = makeKey(cfs.metadata(), values);
             return this;
         }
 
@@ -283,7 +283,7 @@ public abstract class AbstractReadCommandBuilder
         {
             assert startKey == null;
             this.startInclusive = false;
-            this.startKey = makeKey(cfs.metadata, values);
+            this.startKey = makeKey(cfs.metadata(), values);
             return this;
         }
 
@@ -291,7 +291,7 @@ public abstract class AbstractReadCommandBuilder
         {
             assert endKey == null;
             this.endInclusive = true;
-            this.endKey = makeKey(cfs.metadata, values);
+            this.endKey = makeKey(cfs.metadata(), values);
             return this;
         }
 
@@ -299,7 +299,7 @@ public abstract class AbstractReadCommandBuilder
         {
             assert endKey == null;
             this.endInclusive = false;
-            this.endKey = makeKey(cfs.metadata, values);
+            this.endKey = makeKey(cfs.metadata(), values);
             return this;
         }
 
@@ -329,16 +329,16 @@ public abstract class AbstractReadCommandBuilder
             else
                 bounds = new ExcludingBounds<>(start, end);
 
-            return new PartitionRangeReadCommand(cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()), Optional.empty());
+            return new PartitionRangeReadCommand(cfs.metadata(), nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()), Optional.empty());
         }
 
-        static DecoratedKey makeKey(CFMetaData metadata, Object... partitionKey)
+        static DecoratedKey makeKey(TableMetadata metadata, Object... partitionKey)
         {
             if (partitionKey.length == 1 && partitionKey[0] instanceof DecoratedKey)
                 return (DecoratedKey)partitionKey[0];
 
-            ByteBuffer key = CFMetaData.serializePartitionKey(metadata.getKeyValidatorAsClusteringComparator().make(partitionKey));
-            return metadata.decorateKey(key);
+            ByteBuffer key = metadata.partitionKeyAsClusteringComparator().make(partitionKey).serializeAsPartitionKey();
+            return metadata.partitioner.decorateKey(key);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/Clustering.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Clustering.java b/src/java/org/apache/cassandra/db/Clustering.java
index fa38ce1..772961f 100644
--- a/src/java/org/apache/cassandra/db/Clustering.java
+++ b/src/java/org/apache/cassandra/db/Clustering.java
@@ -22,8 +22,8 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -54,23 +54,23 @@ public interface Clustering extends ClusteringPrefix
         return new BufferClustering(newValues);
     }
 
-    public default String toString(CFMetaData metadata)
+    public default String toString(TableMetadata metadata)
     {
         StringBuilder sb = new StringBuilder();
         for (int i = 0; i < size(); i++)
         {
-            ColumnDefinition c = metadata.clusteringColumns().get(i);
+            ColumnMetadata c = metadata.clusteringColumns().get(i);
             sb.append(i == 0 ? "" : ", ").append(c.name).append('=').append(get(i) == null ? "null" : c.type.getString(get(i)));
         }
         return sb.toString();
     }
 
-    public default String toCQLString(CFMetaData metadata)
+    public default String toCQLString(TableMetadata metadata)
     {
         StringBuilder sb = new StringBuilder();
         for (int i = 0; i < size(); i++)
         {
-            ColumnDefinition c = metadata.clusteringColumns().get(i);
+            ColumnMetadata c = metadata.clusteringColumns().get(i);
             sb.append(i == 0 ? "" : ", ").append(c.type.getString(get(i)));
         }
         return sb.toString();
@@ -100,7 +100,7 @@ public interface Clustering extends ClusteringPrefix
         }
 
         @Override
-        public String toString(CFMetaData metadata)
+        public String toString(TableMetadata metadata)
         {
             return toString();
         }
@@ -110,7 +110,7 @@ public interface Clustering extends ClusteringPrefix
     public static final Clustering EMPTY = new BufferClustering(EMPTY_VALUES_ARRAY)
     {
         @Override
-        public String toString(CFMetaData metadata)
+        public String toString(TableMetadata metadata)
         {
             return "EMPTY";
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ClusteringBoundOrBoundary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ClusteringBoundOrBoundary.java b/src/java/org/apache/cassandra/db/ClusteringBoundOrBoundary.java
index 7a2cce1..84a9e30 100644
--- a/src/java/org/apache/cassandra/db/ClusteringBoundOrBoundary.java
+++ b/src/java/org/apache/cassandra/db/ClusteringBoundOrBoundary.java
@@ -24,7 +24,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -111,7 +111,7 @@ public abstract class ClusteringBoundOrBoundary extends AbstractBufferClustering
         return create(kind(), newValues);
     }
 
-    public String toString(CFMetaData metadata)
+    public String toString(TableMetadata metadata)
     {
         return toString(metadata.comparator);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/ClusteringComparator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ClusteringComparator.java b/src/java/org/apache/cassandra/db/ClusteringComparator.java
index 4374a46..50cf5bf 100644
--- a/src/java/org/apache/cassandra/db/ClusteringComparator.java
+++ b/src/java/org/apache/cassandra/db/ClusteringComparator.java
@@ -53,7 +53,7 @@ public class ClusteringComparator implements Comparator<Clusterable>
         this(ImmutableList.copyOf(clusteringTypes));
     }
 
-    public ClusteringComparator(List<AbstractType<?>> clusteringTypes)
+    public ClusteringComparator(Iterable<AbstractType<?>> clusteringTypes)
     {
         // copy the list to ensure despatch is monomorphic
         this.clusteringTypes = ImmutableList.copyOf(clusteringTypes);
@@ -62,7 +62,7 @@ public class ClusteringComparator implements Comparator<Clusterable>
         this.indexReverseComparator = (o1, o2) -> ClusteringComparator.this.compare(o1.firstName, o2.firstName);
         this.reverseComparator = (c1, c2) -> ClusteringComparator.this.compare(c2, c1);
         for (AbstractType<?> type : clusteringTypes)
-            type.checkComparable(); // this should already be enforced by CFMetaData.rebuild, but we check again for other constructors
+            type.checkComparable(); // this should already be enforced by TableMetadata.Builder.addColumn, but we check again for other constructors
     }
 
     /**


[05/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java b/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
index bfa9796..fdd34f1 100644
--- a/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
+++ b/test/unit/org/apache/cassandra/db/partition/PartitionUpdateTest.java
@@ -18,7 +18,7 @@
 package org.apache.cassandra.db.partition;
 
 import org.apache.cassandra.UpdateBuilder;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -33,7 +33,7 @@ public class PartitionUpdateTest extends CQLTester
     public void testOperationCount()
     {
         createTable("CREATE TABLE %s (key text, clustering int, a int, s int static, PRIMARY KEY(key, clustering))");
-        CFMetaData cfm = currentTableMetadata();
+        TableMetadata cfm = currentTableMetadata();
 
         UpdateBuilder builder = UpdateBuilder.create(cfm, "key0");
         Assert.assertEquals(0, builder.build().operationCount());
@@ -52,7 +52,7 @@ public class PartitionUpdateTest extends CQLTester
     public void testOperationCountWithCompactTable()
     {
         createTable("CREATE TABLE %s (key text PRIMARY KEY, a int) WITH COMPACT STORAGE");
-        CFMetaData cfm = currentTableMetadata();
+        TableMetadata cfm = currentTableMetadata();
 
         PartitionUpdate update = new RowUpdateBuilder(cfm, FBUtilities.timestampMicros(), "key0").add("a", 1)
                                                                                                  .buildUpdate();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
index e4c04fb..fb35ead 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
@@ -29,18 +29,18 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.ClusteringPrefix;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -55,23 +55,22 @@ public class RowAndDeletionMergeIteratorTest
     private int nowInSeconds;
     private DecoratedKey dk;
     private ColumnFamilyStore cfs;
-    private CFMetaData cfm;
-    private ColumnDefinition defA;
+    private TableMetadata cfm;
+    private ColumnMetadata defA;
 
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
         DatabaseDescriptor.daemonInitialization();
-        CFMetaData cfMetadata = CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD1)
-                                                  .addPartitionKey("key", AsciiType.instance)
-                                                  .addClusteringColumn("col1", Int32Type.instance)
-                                                  .addRegularColumn("a", Int32Type.instance)
-                                                  .build();
-        SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    KeyspaceParams.simple(1),
-                                    cfMetadata);
 
+        TableMetadata.Builder builder =
+            TableMetadata.builder(KEYSPACE1, CF_STANDARD1)
+                         .addPartitionKeyColumn("key", AsciiType.instance)
+                         .addClusteringColumn("col1", Int32Type.instance)
+                         .addRegularColumn("a", Int32Type.instance);
+
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1, KeyspaceParams.simple(1), builder);
     }
 
     @Before
@@ -80,8 +79,8 @@ public class RowAndDeletionMergeIteratorTest
         nowInSeconds = FBUtilities.nowInSeconds();
         dk = Util.dk("key0");
         cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
-        cfm = cfs.metadata;
-        defA = cfm.getColumnDefinition(new ColumnIdentifier("a", true));
+        cfm = cfs.metadata();
+        defA = cfm.getColumn(new ColumnIdentifier("a", true));
     }
 
     @Test
@@ -367,7 +366,7 @@ public class RowAndDeletionMergeIteratorTest
 
     private Iterator<Row> createRowIterator()
     {
-        PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.partitionColumns(), 1);
+        PartitionUpdate update = new PartitionUpdate(cfm, dk, cfm.regularAndStaticColumns(), 1);
         for (int i = 0; i < 5; i++)
             addRow(update, i, i);
 
@@ -401,9 +400,9 @@ public class RowAndDeletionMergeIteratorTest
         update.add(BTreeRow.singleCellRow(update.metadata().comparator.make(col1), makeCell(defA, a, 0)));
     }
 
-    private Cell makeCell(ColumnDefinition columnDefinition, int value, long timestamp)
+    private Cell makeCell(ColumnMetadata columnMetadata, int value, long timestamp)
     {
-        return BufferCell.live(columnDefinition, timestamp, ((AbstractType)columnDefinition.cellValueType()).decompose(value));
+        return BufferCell.live(columnMetadata, timestamp, ((AbstractType) columnMetadata.cellValueType()).decompose(value));
     }
 
     private static RangeTombstone atLeast(int start, long tstamp, int delTime)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/rows/RowBuilder.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowBuilder.java b/test/unit/org/apache/cassandra/db/rows/RowBuilder.java
index b1223f1..5eed774 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowBuilder.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowBuilder.java
@@ -21,7 +21,7 @@ package org.apache.cassandra.db.rows;
 import java.util.LinkedList;
 import java.util.List;
 
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.db.Clustering;
 import org.apache.cassandra.db.DeletionTime;
 import org.apache.cassandra.db.LivenessInfo;
@@ -37,7 +37,7 @@ public class RowBuilder implements Row.Builder
     public Clustering clustering = null;
     public LivenessInfo livenessInfo = null;
     public Row.Deletion deletionTime = null;
-    public List<Pair<ColumnDefinition, DeletionTime>> complexDeletions = new LinkedList<>();
+    public List<Pair<ColumnMetadata, DeletionTime>> complexDeletions = new LinkedList<>();
 
     public void addCell(Cell cell)
     {
@@ -72,7 +72,7 @@ public class RowBuilder implements Row.Builder
         deletionTime = deletion;
     }
 
-    public void addComplexDeletion(ColumnDefinition column, DeletionTime complexDeletion)
+    public void addComplexDeletion(ColumnMetadata column, DeletionTime complexDeletion)
     {
         complexDeletions.add(Pair.create(column, complexDeletion));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/rows/RowsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowsTest.java b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
index 5629f3f..3f59003 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowsTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
@@ -33,8 +33,8 @@ import com.google.common.collect.Sets;
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.Clustering;
@@ -50,23 +50,24 @@ public class RowsTest
 {
     private static final String KEYSPACE = "rows_test";
     private static final String KCVM_TABLE = "kcvm";
-    private static final CFMetaData kcvm;
-    private static final ColumnDefinition v;
-    private static final ColumnDefinition m;
+    private static final TableMetadata kcvm;
+    private static final ColumnMetadata v;
+    private static final ColumnMetadata m;
     private static final Clustering c1;
 
     static
     {
         DatabaseDescriptor.daemonInitialization();
-        kcvm = CFMetaData.Builder.create(KEYSPACE, KCVM_TABLE)
-                                 .addPartitionKey("k", IntegerType.instance)
-                                 .addClusteringColumn("c", IntegerType.instance)
-                                 .addRegularColumn("v", IntegerType.instance)
-                                 .addRegularColumn("m", MapType.getInstance(IntegerType.instance, IntegerType.instance, true))
-                                 .build();
-
-        v = kcvm.getColumnDefinition(new ColumnIdentifier("v", false));
-        m = kcvm.getColumnDefinition(new ColumnIdentifier("m", false));
+        kcvm =
+            TableMetadata.builder(KEYSPACE, KCVM_TABLE)
+                         .addPartitionKeyColumn("k", IntegerType.instance)
+                         .addClusteringColumn("c", IntegerType.instance)
+                         .addRegularColumn("v", IntegerType.instance)
+                         .addRegularColumn("m", MapType.getInstance(IntegerType.instance, IntegerType.instance, true))
+                         .build();
+
+        v = kcvm.getColumn(new ColumnIdentifier("v", false));
+        m = kcvm.getColumn(new ColumnIdentifier("m", false));
         c1 = kcvm.comparator.make(BigInteger.valueOf(1));
     }
 
@@ -158,8 +159,8 @@ public class RowsTest
             updates++;
         }
 
-        Map<ColumnDefinition, List<MergedPair<DeletionTime>>> complexDeletions = new HashMap<>();
-        public void onComplexDeletion(int i, Clustering clustering, ColumnDefinition column, DeletionTime merged, DeletionTime original)
+        Map<ColumnMetadata, List<MergedPair<DeletionTime>>> complexDeletions = new HashMap<>();
+        public void onComplexDeletion(int i, Clustering clustering, ColumnMetadata column, DeletionTime merged, DeletionTime original)
         {
             updateClustering(clustering);
             if (!complexDeletions.containsKey(column)) complexDeletions.put(column, new LinkedList<>());
@@ -550,14 +551,14 @@ public class RowsTest
     }
 
     // Creates a dummy cell for a (regular) column for the provided name and without a cellPath.
-    private static Cell liveCell(ColumnDefinition name)
+    private static Cell liveCell(ColumnMetadata name)
     {
         return liveCell(name, -1);
     }
 
     // Creates a dummy cell for a (regular) column for the provided name.
     // If path >= 0, the cell will have a CellPath containing path as an Int32Type.
-    private static Cell liveCell(ColumnDefinition name, int path)
+    private static Cell liveCell(ColumnMetadata name, int path)
     {
         CellPath cp = path < 0 ? null : CellPath.create(ByteBufferUtil.bytes(path));
         return new BufferCell(name, 0L, Cell.NO_TTL, Cell.NO_DELETION_TIME, ByteBuffer.allocate(1), cp);
@@ -593,20 +594,21 @@ public class RowsTest
         // Creates a table with
         //   - 3 Simple columns: a, c and e
         //   - 2 Complex columns: b and d
-        CFMetaData metadata = CFMetaData.Builder.create("dummy_ks", "dummy_tbl")
-                                        .addPartitionKey("k", BytesType.instance)
-                                        .addRegularColumn("a", BytesType.instance)
-                                        .addRegularColumn("b", MapType.getInstance(Int32Type.instance, BytesType.instance, true))
-                                        .addRegularColumn("c", BytesType.instance)
-                                        .addRegularColumn("d", MapType.getInstance(Int32Type.instance, BytesType.instance, true))
-                                        .addRegularColumn("e", BytesType.instance)
-                                        .build();
-
-        ColumnDefinition a = metadata.getColumnDefinition(new ColumnIdentifier("a", false));
-        ColumnDefinition b = metadata.getColumnDefinition(new ColumnIdentifier("b", false));
-        ColumnDefinition c = metadata.getColumnDefinition(new ColumnIdentifier("c", false));
-        ColumnDefinition d = metadata.getColumnDefinition(new ColumnIdentifier("d", false));
-        ColumnDefinition e = metadata.getColumnDefinition(new ColumnIdentifier("e", false));
+        TableMetadata metadata =
+            TableMetadata.builder("dummy_ks", "dummy_tbl")
+                         .addPartitionKeyColumn("k", BytesType.instance)
+                         .addRegularColumn("a", BytesType.instance)
+                         .addRegularColumn("b", MapType.getInstance(Int32Type.instance, BytesType.instance, true))
+                         .addRegularColumn("c", BytesType.instance)
+                         .addRegularColumn("d", MapType.getInstance(Int32Type.instance, BytesType.instance, true))
+                         .addRegularColumn("e", BytesType.instance)
+                         .build();
+
+        ColumnMetadata a = metadata.getColumn(new ColumnIdentifier("a", false));
+        ColumnMetadata b = metadata.getColumn(new ColumnIdentifier("b", false));
+        ColumnMetadata c = metadata.getColumn(new ColumnIdentifier("c", false));
+        ColumnMetadata d = metadata.getColumn(new ColumnIdentifier("d", false));
+        ColumnMetadata e = metadata.getColumn(new ColumnIdentifier("e", false));
 
         Row row;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java b/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
index 4578ad1..8c99057 100644
--- a/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
@@ -31,7 +31,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.AsciiType;
@@ -47,11 +47,13 @@ public class UnfilteredRowIteratorsMergeTest
     }
     static DecoratedKey partitionKey = Util.dk("key");
     static DeletionTime partitionLevelDeletion = DeletionTime.LIVE;
-    static CFMetaData metadata = CFMetaData.Builder.create("UnfilteredRowIteratorsMergeTest", "Test").
-            addPartitionKey("key", AsciiType.instance).
-            addClusteringColumn("clustering", Int32Type.instance).
-            addRegularColumn("data", Int32Type.instance).
-            build();
+    static TableMetadata metadata =
+        TableMetadata.builder("UnfilteredRowIteratorsMergeTest", "Test")
+                     .addPartitionKeyColumn("key", AsciiType.instance)
+                     .addClusteringColumn("clustering", Int32Type.instance)
+                     .addRegularColumn("data", Int32Type.instance)
+                     .build();
+
     static Comparator<Clusterable> comparator = new ClusteringComparator(Int32Type.instance);
     static int nowInSec = FBUtilities.nowInSeconds();
 
@@ -424,7 +426,7 @@ public class UnfilteredRowIteratorsMergeTest
             super(UnfilteredRowIteratorsMergeTest.metadata,
                   UnfilteredRowIteratorsMergeTest.partitionKey,
                   UnfilteredRowIteratorsMergeTest.partitionLevelDeletion,
-                  UnfilteredRowIteratorsMergeTest.metadata.partitionColumns(),
+                  UnfilteredRowIteratorsMergeTest.metadata.regularAndStaticColumns(),
                   null,
                   reversed,
                   EncodingStats.NO_STATS);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/rows/UnfilteredRowsGenerator.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/UnfilteredRowsGenerator.java b/test/unit/org/apache/cassandra/db/rows/UnfilteredRowsGenerator.java
index 7cdccdb..1f8793a 100644
--- a/test/unit/org/apache/cassandra/db/rows/UnfilteredRowsGenerator.java
+++ b/test/unit/org/apache/cassandra/db/rows/UnfilteredRowsGenerator.java
@@ -25,10 +25,10 @@ import java.util.regex.Pattern;
 
 import org.junit.Assert;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.rows.Unfiltered.Kind;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.btree.BTree;
 
 public class UnfilteredRowsGenerator
@@ -289,12 +289,12 @@ public class UnfilteredRowsGenerator
                                              new DeletionTime(delTime, delTime));
     }
 
-    public static UnfilteredRowIterator source(Iterable<Unfiltered> content, CFMetaData metadata, DecoratedKey partitionKey)
+    public static UnfilteredRowIterator source(Iterable<Unfiltered> content, TableMetadata metadata, DecoratedKey partitionKey)
     {
         return source(content, metadata, partitionKey, DeletionTime.LIVE);
     }
 
-    public static UnfilteredRowIterator source(Iterable<Unfiltered> content, CFMetaData metadata, DecoratedKey partitionKey, DeletionTime delTime)
+    public static UnfilteredRowIterator source(Iterable<Unfiltered> content, TableMetadata metadata, DecoratedKey partitionKey, DeletionTime delTime)
     {
         return new Source(content.iterator(), metadata, partitionKey, delTime, false);
     }
@@ -303,12 +303,12 @@ public class UnfilteredRowsGenerator
     {
         Iterator<Unfiltered> content;
 
-        protected Source(Iterator<Unfiltered> content, CFMetaData metadata, DecoratedKey partitionKey, DeletionTime partitionLevelDeletion, boolean reversed)
+        protected Source(Iterator<Unfiltered> content, TableMetadata metadata, DecoratedKey partitionKey, DeletionTime partitionLevelDeletion, boolean reversed)
         {
             super(metadata,
                   partitionKey,
                   partitionLevelDeletion,
-                  metadata.partitionColumns(),
+                  metadata.regularAndStaticColumns(),
                   Rows.EMPTY_STATIC_ROW,
                   reversed,
                   EncodingStats.NO_STATS);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java b/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
index 89bb44a..658b87a 100644
--- a/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
+++ b/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
@@ -28,7 +28,7 @@ import org.junit.Test;
 
 import junit.framework.Assert;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -74,7 +74,7 @@ public class ViewUtilsTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         Optional<InetAddress> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
                                                                        new StringToken("CA"),
@@ -107,7 +107,7 @@ public class ViewUtilsTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         Optional<InetAddress> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
                                                                        new StringToken("CA"),
@@ -139,7 +139,7 @@ public class ViewUtilsTest
 
         Keyspace.clear("Keyspace1");
         KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
-        Schema.instance.setKeyspaceMetadata(meta);
+        Schema.instance.load(meta);
 
         Optional<InetAddress> naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
                                                                        new StringToken("AB"),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
index ba0352d..9481201 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -41,7 +41,7 @@ import org.junit.runner.RunWith;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.tokenallocator.TokenAllocation;
 import org.apache.cassandra.exceptions.ConfigurationException;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java b/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
index c0ff9ca..5b5365d 100644
--- a/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
@@ -27,7 +27,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.RowUpdateBuilder;
@@ -95,7 +95,7 @@ public class KeyCollisionTest
 
     private void insert(String key)
     {
-        RowUpdateBuilder builder = new RowUpdateBuilder(Schema.instance.getCFMetaData(KEYSPACE1, CF), FBUtilities.timestampMicros(), key);
+        RowUpdateBuilder builder = new RowUpdateBuilder(Schema.instance.getTableMetadata(KEYSPACE1, CF), FBUtilities.timestampMicros(), key);
         builder.clustering("c").add("val", "asdf").build().applyUnsafe();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/LengthPartitioner.java b/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
index 97f2dcc..bd6f3d4 100644
--- a/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
+++ b/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
@@ -22,8 +22,8 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ThreadLocalRandom;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -143,12 +143,12 @@ public class LengthPartitioner implements IPartitioner
 
         for (String ks : Schema.instance.getKeyspaces())
         {
-            for (CFMetaData cfmd : Schema.instance.getTablesAndViews(ks))
+            for (TableMetadata cfmd : Schema.instance.getTablesAndViews(ks))
             {
                 for (Range<Token> r : sortedRanges)
                 {
                     // Looping over every KS:CF:Range, get the splits size and add it to the count
-                    allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.cfName, r, 1).size());
+                    allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.name, r, 1).size());
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/hints/AlteredHints.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/AlteredHints.java b/test/unit/org/apache/cassandra/hints/AlteredHints.java
index 7efe08f..6cc507b 100644
--- a/test/unit/org/apache/cassandra/hints/AlteredHints.java
+++ b/test/unit/org/apache/cassandra/hints/AlteredHints.java
@@ -32,8 +32,9 @@ import org.junit.Assert;
 import org.junit.BeforeClass;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -51,7 +52,7 @@ public abstract class AlteredHints
 
     private static Mutation createMutation(int index, long timestamp)
     {
-        CFMetaData table = Schema.instance.getCFMetaData(KEYSPACE, TABLE);
+        TableMetadata table = Schema.instance.getTableMetadata(KEYSPACE, TABLE);
         return new RowUpdateBuilder(table, timestamp, bytes(index))
                .clustering(bytes(index))
                .add("val", bytes(index))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/hints/HintMessageTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintMessageTest.java b/test/unit/org/apache/cassandra/hints/HintMessageTest.java
index 7ffaa54..bb015a8 100644
--- a/test/unit/org/apache/cassandra/hints/HintMessageTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintMessageTest.java
@@ -23,8 +23,8 @@ import java.util.UUID;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.io.util.DataInputBuffer;
@@ -53,7 +53,7 @@ public class HintMessageTest
         UUID hostId = UUID.randomUUID();
         long now = FBUtilities.timestampMicros();
 
-        CFMetaData table = Schema.instance.getCFMetaData(KEYSPACE, TABLE);
+        TableMetadata table = Schema.instance.getTableMetadata(KEYSPACE, TABLE);
         Mutation mutation =
             new RowUpdateBuilder(table, now, bytes("key"))
                 .clustering("column")

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/hints/HintTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintTest.java b/test/unit/org/apache/cassandra/hints/HintTest.java
index e4a33fd..23189e1 100644
--- a/test/unit/org/apache/cassandra/hints/HintTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintTest.java
@@ -43,8 +43,12 @@ import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.TableParams;
+import org.apache.cassandra.schema.MigrationManager;
 import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
@@ -54,7 +58,6 @@ import static junit.framework.Assert.*;
 import static org.apache.cassandra.Util.dk;
 import static org.apache.cassandra.hints.HintsTestUtil.assertHintsEqual;
 import static org.apache.cassandra.hints.HintsTestUtil.assertPartitionsEqual;
-import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
 public class HintTest
 {
@@ -83,8 +86,8 @@ public class HintTest
         tokenMeta.updateHostId(UUID.randomUUID(), local);
         tokenMeta.updateNormalTokens(BootStrapper.getRandomTokens(tokenMeta, 1), local);
 
-        for (CFMetaData table : Schema.instance.getTablesAndViews(KEYSPACE))
-            table.gcGraceSeconds(TableParams.DEFAULT_GC_GRACE_SECONDS);
+        for (TableMetadata table : Schema.instance.getTablesAndViews(KEYSPACE))
+            MigrationManager.announceTableUpdate(table.unbuild().gcGraceSeconds(TableParams.DEFAULT_GC_GRACE_SECONDS).build(), true);
     }
 
     @Test
@@ -125,7 +128,7 @@ public class HintTest
 
         // assert that we can read the inserted partitions
         for (PartitionUpdate partition : mutation.getPartitionUpdates())
-            assertPartitionsEqual(partition, readPartition(key, partition.metadata().cfName, partition.columns()));
+            assertPartitionsEqual(partition, readPartition(key, partition.metadata().name, partition.columns()));
     }
 
     @Test
@@ -150,9 +153,9 @@ public class HintTest
         assertNoPartitions(key, TABLE1);
 
         // TABLE0 and TABLE2 updates should have been applied successfully
-        PartitionUpdate upd0 = mutation.getPartitionUpdate(Schema.instance.getId(KEYSPACE, TABLE0));
+        PartitionUpdate upd0 = mutation.getPartitionUpdate(Schema.instance.getTableMetadata(KEYSPACE, TABLE0));
         assertPartitionsEqual(upd0, readPartition(key, TABLE0, upd0.columns()));
-        PartitionUpdate upd2 = mutation.getPartitionUpdate(Schema.instance.getId(KEYSPACE, TABLE2));
+        PartitionUpdate upd2 = mutation.getPartitionUpdate(Schema.instance.getTableMetadata(KEYSPACE, TABLE2));
         assertPartitionsEqual(upd2, readPartition(key, TABLE2, upd2.columns()));
     }
 
@@ -161,7 +164,6 @@ public class HintTest
     {
         long now = FBUtilities.timestampMicros();
         String key = "testApplyWithRegularExpiration";
-        Mutation mutation = createMutation(key, now);
 
         // sanity check that there is no data inside yet
         assertNoPartitions(key, TABLE0);
@@ -169,8 +171,15 @@ public class HintTest
         assertNoPartitions(key, TABLE2);
 
         // lower the GC GS on TABLE0 to 0 BEFORE the hint is created
-        Schema.instance.getCFMetaData(KEYSPACE, TABLE0).gcGraceSeconds(0);
+        TableMetadata updated =
+            Schema.instance
+                  .getTableMetadata(KEYSPACE, TABLE0)
+                  .unbuild()
+                  .gcGraceSeconds(0)
+                  .build();
+        MigrationManager.announceTableUpdate(updated, true);
 
+        Mutation mutation = createMutation(key, now);
         Hint.create(mutation, now / 1000).apply();
 
         // all updates should have been skipped and not applied, as expired
@@ -184,8 +193,6 @@ public class HintTest
     {
         long now = FBUtilities.timestampMicros();
         String key = "testApplyWithGCGSReducedLater";
-        Mutation mutation = createMutation(key, now);
-        Hint hint = Hint.create(mutation, now / 1000);
 
         // sanity check that there is no data inside yet
         assertNoPartitions(key, TABLE0);
@@ -193,8 +200,16 @@ public class HintTest
         assertNoPartitions(key, TABLE2);
 
         // lower the GC GS on TABLE0 AFTER the hint is already created
-        Schema.instance.getCFMetaData(KEYSPACE, TABLE0).gcGraceSeconds(0);
+        TableMetadata updated =
+            Schema.instance
+                  .getTableMetadata(KEYSPACE, TABLE0)
+                  .unbuild()
+                  .gcGraceSeconds(0)
+                  .build();
+        MigrationManager.announceTableUpdate(updated, true);
 
+        Mutation mutation = createMutation(key, now);
+        Hint hint = Hint.create(mutation, now / 1000);
         hint.apply();
 
         // all updates should have been skipped and not applied, as expired
@@ -298,17 +313,17 @@ public class HintTest
     {
         Mutation.SimpleBuilder builder = Mutation.simpleBuilder(KEYSPACE, dk(key));
 
-        builder.update(Schema.instance.getCFMetaData(KEYSPACE, TABLE0))
+        builder.update(Schema.instance.getTableMetadata(KEYSPACE, TABLE0))
                .timestamp(now)
                .row("column0")
                .add("val", "value0");
 
-        builder.update(Schema.instance.getCFMetaData(KEYSPACE, TABLE1))
+        builder.update(Schema.instance.getTableMetadata(KEYSPACE, TABLE1))
                .timestamp(now + 1)
                .row("column1")
                .add("val", "value1");
 
-        builder.update(Schema.instance.getCFMetaData(KEYSPACE, TABLE2))
+        builder.update(Schema.instance.getTableMetadata(KEYSPACE, TABLE2))
                .timestamp(now + 2)
                .row("column2")
                .add("val", "value2");
@@ -318,14 +333,14 @@ public class HintTest
 
     private static ColumnFamilyStore cfs(String table)
     {
-        return Schema.instance.getColumnFamilyStoreInstance(Schema.instance.getCFMetaData(KEYSPACE, table).cfId);
+        return Schema.instance.getColumnFamilyStoreInstance(Schema.instance.getTableMetadata(KEYSPACE, table).id);
     }
 
-    private static FilteredPartition readPartition(String key, String table, PartitionColumns columns)
+    private static FilteredPartition readPartition(String key, String table, RegularAndStaticColumns columns)
     {
         String[] columnNames = new String[columns.size()];
         int i = 0;
-        for (ColumnDefinition column : columns)
+        for (ColumnMetadata column : columns)
             columnNames[i++] = column.name.toString();
 
         return Util.getOnlyPartition(Util.cmd(cfs(table), key).columns(columnNames).build());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/hints/HintsBufferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintsBufferTest.java b/test/unit/org/apache/cassandra/hints/HintsBufferTest.java
index 08f7ec0..a4cb651 100644
--- a/test/unit/org/apache/cassandra/hints/HintsBufferTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintsBufferTest.java
@@ -29,8 +29,6 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.RowUpdateBuilder;
 import org.apache.cassandra.db.rows.Cell;
@@ -39,6 +37,8 @@ import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
 
 import static junit.framework.Assert.*;
 
@@ -197,7 +197,7 @@ public class HintsBufferTest
 
     private static Mutation createMutation(int index, long timestamp)
     {
-        CFMetaData table = Schema.instance.getCFMetaData(KEYSPACE, TABLE);
+        TableMetadata table = Schema.instance.getTableMetadata(KEYSPACE, TABLE);
         return new RowUpdateBuilder(table, timestamp, bytes(index))
                    .clustering(bytes(index))
                    .add("val", bytes(index))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/hints/HintsServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintsServiceTest.java b/test/unit/org/apache/cassandra/hints/HintsServiceTest.java
index 077a9d1..2da8993 100644
--- a/test/unit/org/apache/cassandra/hints/HintsServiceTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintsServiceTest.java
@@ -34,8 +34,8 @@ import org.junit.Test;
 
 import com.datastax.driver.core.utils.MoreFutures;
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.gms.IFailureDetectionEventListener;
@@ -78,7 +78,7 @@ public class HintsServiceTest
     }
 
     @Before
-    public void reinstanciateService() throws ExecutionException, InterruptedException
+    public void reinstanciateService() throws Throwable
     {
         MessagingService.instance().clearMessageSinks();
 
@@ -89,7 +89,9 @@ public class HintsServiceTest
         }
 
         failureDetector.isAlive = true;
+
         HintsService.instance = new HintsService(failureDetector);
+
         HintsService.instance.startDispatch();
     }
 
@@ -202,8 +204,8 @@ public class HintsServiceTest
         {
             long now = System.currentTimeMillis();
             DecoratedKey dkey = dk(String.valueOf(i));
-            CFMetaData cfMetaData = Schema.instance.getCFMetaData(KEYSPACE, TABLE);
-            PartitionUpdate.SimpleBuilder builder = PartitionUpdate.simpleBuilder(cfMetaData, dkey).timestamp(now);
+            TableMetadata metadata = Schema.instance.getTableMetadata(KEYSPACE, TABLE);
+            PartitionUpdate.SimpleBuilder builder = PartitionUpdate.simpleBuilder(metadata, dkey).timestamp(now);
             builder.row("column0").add("val", "value0");
             Hint hint = Hint.create(builder.buildAsMutation(), now);
             HintsService.instance.write(hostId, hint);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/hints/HintsTestUtil.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintsTestUtil.java b/test/unit/org/apache/cassandra/hints/HintsTestUtil.java
index 89b532f..c1c6192 100644
--- a/test/unit/org/apache/cassandra/hints/HintsTestUtil.java
+++ b/test/unit/org/apache/cassandra/hints/HintsTestUtil.java
@@ -17,11 +17,8 @@
  */
 package org.apache.cassandra.hints;
 
-import java.util.UUID;
-
 import com.google.common.collect.Iterators;
 
-import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.partitions.AbstractBTreePartition;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 
@@ -30,15 +27,6 @@ import static junit.framework.Assert.assertTrue;
 
 final class HintsTestUtil
 {
-    static void assertMutationsEqual(Mutation expected, Mutation actual)
-    {
-        assertEquals(expected.key(), actual.key());
-        assertEquals(expected.getPartitionUpdates().size(), actual.getPartitionUpdates().size());
-
-        for (UUID id : expected.getColumnFamilyIds())
-            assertPartitionsEqual(expected.getPartitionUpdate(id), actual.getPartitionUpdate(id));
-    }
-
     static void assertPartitionsEqual(AbstractBTreePartition expected, AbstractBTreePartition actual)
     {
         assertEquals(expected.partitionKey(), actual.partitionKey());
@@ -51,9 +39,9 @@ final class HintsTestUtil
     {
         assertEquals(expected.mutation.getKeyspaceName(), actual.mutation.getKeyspaceName());
         assertEquals(expected.mutation.key(), actual.mutation.key());
-        assertEquals(expected.mutation.getColumnFamilyIds(), actual.mutation.getColumnFamilyIds());
+        assertEquals(expected.mutation.getTableIds(), actual.mutation.getTableIds());
         for (PartitionUpdate partitionUpdate : expected.mutation.getPartitionUpdates())
-            assertPartitionsEqual(partitionUpdate, actual.mutation.getPartitionUpdate(partitionUpdate.metadata().cfId));
+            assertPartitionsEqual(partitionUpdate, actual.mutation.getPartitionUpdate(partitionUpdate.metadata()));
         assertEquals(expected.creationTime, actual.creationTime);
         assertEquals(expected.gcgs, actual.gcgs);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/index/CustomIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/CustomIndexTest.java b/test/unit/org/apache/cassandra/index/CustomIndexTest.java
index a095ef4..b6b401d 100644
--- a/test/unit/org/apache/cassandra/index/CustomIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/CustomIndexTest.java
@@ -27,12 +27,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import org.junit.Test;
 
 import com.datastax.driver.core.exceptions.QueryValidationException;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.restrictions.IndexRestrictions;
@@ -54,7 +53,6 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
-import static org.apache.cassandra.Util.throwAssert;
 import static org.apache.cassandra.cql3.statements.IndexTarget.CUSTOM_INDEX_OPTION_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -233,21 +231,21 @@ public class CustomIndexTest extends CQLTester
         createTable("CREATE TABLE %s(k int, c int, v1 int, v2 int, PRIMARY KEY(k,c))");
 
         createIndex(String.format("CREATE CUSTOM INDEX ON %%s(v1, v2) USING '%s'", StubIndex.class.getName()));
-        assertEquals(1, getCurrentColumnFamilyStore().metadata.getIndexes().size());
+        assertEquals(1, getCurrentColumnFamilyStore().metadata().indexes.size());
         assertIndexCreated(currentTable() + "_idx", "v1", "v2");
 
         createIndex(String.format("CREATE CUSTOM INDEX ON %%s(c, v1, v2) USING '%s'", StubIndex.class.getName()));
-        assertEquals(2, getCurrentColumnFamilyStore().metadata.getIndexes().size());
+        assertEquals(2, getCurrentColumnFamilyStore().metadata().indexes.size());
         assertIndexCreated(currentTable() + "_idx_1", "c", "v1", "v2");
 
         createIndex(String.format("CREATE CUSTOM INDEX ON %%s(c, v2) USING '%s'", StubIndex.class.getName()));
-        assertEquals(3, getCurrentColumnFamilyStore().metadata.getIndexes().size());
+        assertEquals(3, getCurrentColumnFamilyStore().metadata().indexes.size());
         assertIndexCreated(currentTable() + "_idx_2", "c", "v2");
 
         // duplicate the previous index with some additional options and check the name is generated as expected
         createIndex(String.format("CREATE CUSTOM INDEX ON %%s(c, v2) USING '%s' WITH OPTIONS = {'foo':'bar'}",
                                   StubIndex.class.getName()));
-        assertEquals(4, getCurrentColumnFamilyStore().metadata.getIndexes().size());
+        assertEquals(4, getCurrentColumnFamilyStore().metadata().indexes.size());
         Map<String, String> options = new HashMap<>();
         options.put("foo", "bar");
         assertIndexCreated(currentTable() + "_idx_3", options, "c", "v2");
@@ -279,7 +277,19 @@ public class CustomIndexTest extends CQLTester
         testCreateIndex("idx_5", "c2", "v1");
         testCreateIndex("idx_6", "v1", "v2");
         testCreateIndex("idx_7", "pk2", "c2", "v2");
-        testCreateIndex("idx_8", "pk1", "c1", "v1", "mval", "sval", "lval");
+
+        createIndex(String.format("CREATE CUSTOM INDEX idx_8 ON %%s(" +
+                                  "  pk1, c1, v1, values(mval), values(sval), values(lval)" +
+                                  ") USING '%s'",
+                                  StubIndex.class.getName()));
+        assertIndexCreated("idx_8",
+                           new HashMap<>(),
+                           ImmutableList.of(indexTarget("pk1", IndexTarget.Type.SIMPLE),
+                                            indexTarget("c1", IndexTarget.Type.SIMPLE),
+                                            indexTarget("v1", IndexTarget.Type.SIMPLE),
+                                            indexTarget("mval", IndexTarget.Type.VALUES),
+                                            indexTarget("sval", IndexTarget.Type.VALUES),
+                                            indexTarget("lval", IndexTarget.Type.VALUES)));
 
         createIndex(String.format("CREATE CUSTOM INDEX inc_frozen ON %%s(" +
                                   "  pk2, c2, v2, full(fmap), full(fset), full(flist)" +
@@ -287,9 +297,9 @@ public class CustomIndexTest extends CQLTester
                                   StubIndex.class.getName()));
         assertIndexCreated("inc_frozen",
                            new HashMap<>(),
-                           ImmutableList.of(indexTarget("pk2", IndexTarget.Type.VALUES),
-                                            indexTarget("c2", IndexTarget.Type.VALUES),
-                                            indexTarget("v2", IndexTarget.Type.VALUES),
+                           ImmutableList.of(indexTarget("pk2", IndexTarget.Type.SIMPLE),
+                                            indexTarget("c2", IndexTarget.Type.SIMPLE),
+                                            indexTarget("v2", IndexTarget.Type.SIMPLE),
                                             indexTarget("fmap", IndexTarget.Type.FULL),
                                             indexTarget("fset", IndexTarget.Type.FULL),
                                             indexTarget("flist", IndexTarget.Type.FULL)));
@@ -300,12 +310,12 @@ public class CustomIndexTest extends CQLTester
                                   StubIndex.class.getName()));
         assertIndexCreated("all_teh_things",
                            new HashMap<>(),
-                           ImmutableList.of(indexTarget("pk1", IndexTarget.Type.VALUES),
-                                            indexTarget("pk2", IndexTarget.Type.VALUES),
-                                            indexTarget("c1", IndexTarget.Type.VALUES),
-                                            indexTarget("c2", IndexTarget.Type.VALUES),
-                                            indexTarget("v1", IndexTarget.Type.VALUES),
-                                            indexTarget("v2", IndexTarget.Type.VALUES),
+                           ImmutableList.of(indexTarget("pk1", IndexTarget.Type.SIMPLE),
+                                            indexTarget("pk2", IndexTarget.Type.SIMPLE),
+                                            indexTarget("c1", IndexTarget.Type.SIMPLE),
+                                            indexTarget("c2", IndexTarget.Type.SIMPLE),
+                                            indexTarget("v1", IndexTarget.Type.SIMPLE),
+                                            indexTarget("v2", IndexTarget.Type.SIMPLE),
                                             indexTarget("mval", IndexTarget.Type.KEYS),
                                             indexTarget("lval", IndexTarget.Type.VALUES),
                                             indexTarget("sval", IndexTarget.Type.VALUES),
@@ -320,16 +330,6 @@ public class CustomIndexTest extends CQLTester
         String myType = KEYSPACE + '.' + createType("CREATE TYPE %s (a int, b int)");
         createTable("CREATE TABLE %s (k int PRIMARY KEY, v1 int, v2 frozen<" + myType + ">)");
         testCreateIndex("udt_idx", "v1", "v2");
-        Indexes indexes = getCurrentColumnFamilyStore().metadata.getIndexes();
-        IndexMetadata expected = IndexMetadata.fromIndexTargets(getCurrentColumnFamilyStore().metadata,
-                                                                ImmutableList.of(indexTarget("v1", IndexTarget.Type.VALUES),
-                                                                                 indexTarget("v2", IndexTarget.Type.VALUES)),
-                                                                "udt_idx",
-                                                                IndexMetadata.Kind.CUSTOM,
-                                                                ImmutableMap.of(CUSTOM_INDEX_OPTION_NAME,
-                                                                                StubIndex.class.getName()));
-        IndexMetadata actual = indexes.get("udt_idx").orElseThrow(throwAssert("Index udt_idx not found"));
-        assertEquals(expected, actual);
     }
 
     @Test
@@ -355,13 +355,13 @@ public class CustomIndexTest extends CQLTester
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", row);
 
 
-        assertInvalidMessage(String.format(IndexRestrictions.INDEX_NOT_FOUND, indexName, keyspace(), currentTable()),
+        assertInvalidMessage(String.format(IndexRestrictions.INDEX_NOT_FOUND, indexName, currentTableMetadata().toString()),
                              String.format("SELECT * FROM %%s WHERE expr(%s, 'foo bar baz')", indexName));
 
         createIndex(String.format("CREATE CUSTOM INDEX %s ON %%s(c) USING '%s'", indexName, StubIndex.class.getName()));
 
         assertInvalidThrowMessage(Optional.of(ProtocolVersion.CURRENT),
-                                  String.format(IndexRestrictions.INDEX_NOT_FOUND, "no_such_index", keyspace(), currentTable()),
+                                  String.format(IndexRestrictions.INDEX_NOT_FOUND, "no_such_index", currentTableMetadata().toString()),
                                   QueryValidationException.class,
                                   "SELECT * FROM %s WHERE expr(no_such_index, 'foo bar baz ')");
 
@@ -536,7 +536,7 @@ public class CustomIndexTest extends CQLTester
     @Test
     public void reloadIndexMetadataOnBaseCfsReload() throws Throwable
     {
-        // verify that whenever the base table CFMetadata is reloaded, a reload of the index
+        // verify that whenever the base table TableMetadata is reloaded, a reload of the index
         // metadata is performed
         createTable("CREATE TABLE %s (k int, v1 int, PRIMARY KEY(k))");
         createIndex(String.format("CREATE CUSTOM INDEX reload_counter ON %%s() USING '%s'",
@@ -616,13 +616,13 @@ public class CustomIndexTest extends CQLTester
     }
 
     @Test
-    public void validateOptionsWithCFMetaData() throws Throwable
+    public void validateOptionsWithTableMetadata() throws Throwable
     {
         createTable("CREATE TABLE %s(k int, c int, v1 int, v2 int, PRIMARY KEY(k,c))");
         createIndex(String.format("CREATE CUSTOM INDEX ON %%s(c, v2) USING '%s' WITH OPTIONS = {'foo':'bar'}",
                                   IndexWithOverloadedValidateOptions.class.getName()));
-        CFMetaData cfm = getCurrentColumnFamilyStore().metadata;
-        assertEquals(cfm, IndexWithOverloadedValidateOptions.cfm);
+        TableMetadata table = getCurrentColumnFamilyStore().metadata();
+        assertEquals(table, IndexWithOverloadedValidateOptions.table);
         assertNotNull(IndexWithOverloadedValidateOptions.options);
         assertEquals("bar", IndexWithOverloadedValidateOptions.options.get("foo"));
     }
@@ -823,8 +823,7 @@ public class CustomIndexTest extends CQLTester
     private void assertIndexCreated(String name, Map<String, String> options, String... targetColumnNames)
     {
         List<IndexTarget> targets = Arrays.stream(targetColumnNames)
-                                          .map(s -> new IndexTarget(ColumnIdentifier.getInterned(s, true),
-                                                                    IndexTarget.Type.VALUES))
+                                          .map(s -> new IndexTarget(ColumnIdentifier.getInterned(s, true), IndexTarget.Type.SIMPLE))
                                           .collect(Collectors.toList());
         assertIndexCreated(name, options, targets);
     }
@@ -834,14 +833,13 @@ public class CustomIndexTest extends CQLTester
         // all tests here use StubIndex as the custom index class,
         // so add that to the map of options
         options.put(CUSTOM_INDEX_OPTION_NAME, StubIndex.class.getName());
-        CFMetaData cfm = getCurrentColumnFamilyStore().metadata;
-        IndexMetadata expected = IndexMetadata.fromIndexTargets(cfm, targets, name, IndexMetadata.Kind.CUSTOM, options);
-        Indexes indexes = getCurrentColumnFamilyStore().metadata.getIndexes();
+        IndexMetadata expected = IndexMetadata.fromIndexTargets(targets, name, IndexMetadata.Kind.CUSTOM, options);
+        Indexes indexes = getCurrentColumnFamilyStore().metadata().indexes;
         for (IndexMetadata actual : indexes)
             if (actual.equals(expected))
                 return;
 
-        fail(String.format("Index %s not found in CFMetaData", expected));
+        fail(String.format("Index %s not found", expected));
     }
 
     private static IndexTarget indexTarget(String name, IndexTarget.Type type)
@@ -998,7 +996,7 @@ public class CustomIndexTest extends CQLTester
 
     public static final class IndexWithOverloadedValidateOptions extends StubIndex
     {
-        public static CFMetaData cfm;
+        public static TableMetadata table;
         public static Map<String, String> options;
 
         public IndexWithOverloadedValidateOptions(ColumnFamilyStore baseCfs, IndexMetadata metadata)
@@ -1006,10 +1004,10 @@ public class CustomIndexTest extends CQLTester
             super(baseCfs, metadata);
         }
 
-        public static Map<String, String> validateOptions(Map<String, String> options, CFMetaData cfm)
+        public static Map<String, String> validateOptions(Map<String, String> options, TableMetadata table)
         {
             IndexWithOverloadedValidateOptions.options = options;
-            IndexWithOverloadedValidateOptions.cfm = cfm;
+            IndexWithOverloadedValidateOptions.table = table;
             return new HashMap<>();
         }
     }
@@ -1047,7 +1045,7 @@ public class CustomIndexTest extends CQLTester
         // various OpOrder.Groups, which it can obtain from this index.
 
         public Indexer indexerFor(final DecoratedKey key,
-                                  PartitionColumns columns,
+                                  RegularAndStaticColumns columns,
                                   int nowInSec,
                                   OpOrder.Group opGroup,
                                   IndexTransaction.Type transactionType)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/index/StubIndex.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/StubIndex.java b/test/unit/org/apache/cassandra/index/StubIndex.java
index c80f0d9..00c47d1 100644
--- a/test/unit/org/apache/cassandra/index/StubIndex.java
+++ b/test/unit/org/apache/cassandra/index/StubIndex.java
@@ -23,7 +23,7 @@ import java.util.concurrent.Callable;
 import java.util.function.BiFunction;
 
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.RowFilter;
@@ -77,12 +77,12 @@ public class StubIndex implements Index
         return false;
     }
 
-    public boolean dependsOn(ColumnDefinition column)
+    public boolean dependsOn(ColumnMetadata column)
     {
         return false;
     }
 
-    public boolean supportsExpression(ColumnDefinition column, Operator operator)
+    public boolean supportsExpression(ColumnMetadata column, Operator operator)
     {
         return operator == Operator.EQ;
     }
@@ -98,7 +98,7 @@ public class StubIndex implements Index
     }
 
     public Indexer indexerFor(final DecoratedKey key,
-                              PartitionColumns columns,
+                              RegularAndStaticColumns columns,
                               int nowInSec,
                               OpOrder.Group opGroup,
                               IndexTransaction.Type transactionType)
@@ -161,7 +161,7 @@ public class StubIndex implements Index
         return Optional.empty();
     }
 
-    public Collection<ColumnDefinition> getIndexedColumns()
+    public Collection<ColumnMetadata> getIndexedColumns()
     {
         return Collections.emptySet();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java b/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java
index 5c7d840..9bedb68 100644
--- a/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/internal/CassandraIndexTest.java
@@ -26,9 +26,6 @@ import com.google.common.base.Joiner;
 import com.google.common.collect.*;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.SchemaConstants;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
@@ -40,6 +37,9 @@ import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.Unfiltered;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -488,20 +488,20 @@ public class CassandraIndexTest extends CQLTester
     }
 
     // this is slightly annoying, but we cannot read rows from the methods in Util as
-    // ReadCommand#executeInternal uses metadata retrieved via the cfId, which the index
+    // ReadCommand#executeInternal uses metadata retrieved via the tableId, which the index
     // CFS inherits from the base CFS. This has the 'wrong' partitioner (the index table
     // uses LocalPartition, the base table a real one, so we cannot read from the index
     // table with executeInternal
     private void assertIndexRowTtl(ColumnFamilyStore indexCfs, int indexedValue, int ttl) throws Throwable
     {
         DecoratedKey indexKey = indexCfs.decorateKey(ByteBufferUtil.bytes(indexedValue));
-        ClusteringIndexFilter filter = new ClusteringIndexSliceFilter(Slices.with(indexCfs.metadata.comparator,
+        ClusteringIndexFilter filter = new ClusteringIndexSliceFilter(Slices.with(indexCfs.metadata().comparator,
                                                                                   Slice.ALL),
                                                                       false);
-        SinglePartitionReadCommand command = SinglePartitionReadCommand.create(indexCfs.metadata,
+        SinglePartitionReadCommand command = SinglePartitionReadCommand.create(indexCfs.metadata(),
                                                                                FBUtilities.nowInSeconds(),
                                                                                indexKey,
-                                                                               ColumnFilter.all(indexCfs.metadata),
+                                                                               ColumnFilter.all(indexCfs.metadata()),
                                                                                filter);
         try (ReadExecutionController executionController = command.executionController();
              UnfilteredRowIterator iter = command.queryMemtableAndDisk(indexCfs, executionController))
@@ -603,7 +603,7 @@ public class CassandraIndexTest extends CQLTester
             if (updateExpression != null)
                 assertNotNull(postUpdateQueryExpression);
 
-            // first, create the table as we need the CFMetaData to build the other cql statements
+            // first, create the table as we need the Tablemetadata to build the other cql statements
             createTable(tableDefinition);
 
             // now setup the cql statements the test will run through. Some are dependent on
@@ -704,7 +704,7 @@ public class CassandraIndexTest extends CQLTester
         private void assertPrimaryKeyColumnsOnly(UntypedResultSet resultSet, Object[] row)
         {
             assertFalse(resultSet.isEmpty());
-            CFMetaData cfm = getCurrentColumnFamilyStore().metadata;
+            TableMetadata cfm = getCurrentColumnFamilyStore().metadata();
             int columnCount = cfm.partitionKeyColumns().size();
             if (cfm.isCompound())
                 columnCount += cfm.clusteringColumns().size();
@@ -714,14 +714,12 @@ public class CassandraIndexTest extends CQLTester
 
         private String getInsertCql()
         {
-            CFMetaData cfm = getCurrentColumnFamilyStore().metadata;
+            TableMetadata metadata = getCurrentColumnFamilyStore().metadata();
             String columns = Joiner.on(", ")
-                                   .join(Iterators.transform(cfm.allColumnsInSelectOrder(),
+                                   .join(Iterators.transform(metadata.allColumnsInSelectOrder(),
                                                              (column) -> column.name.toString()));
-            String markers = Joiner.on(", ").join(Iterators.transform(cfm.allColumnsInSelectOrder(),
-                                                                      (column) -> {
-                                                                          return "?";
-                                                                      }));
+            String markers = Joiner.on(", ").join(Iterators.transform(metadata.allColumnsInSelectOrder(),
+                                                                      (column) -> "?"));
             return String.format("INSERT INTO %%s (%s) VALUES (%s)", columns, markers);
         }
 
@@ -740,15 +738,15 @@ public class CassandraIndexTest extends CQLTester
 
         private String getDeletePartitionCql()
         {
-            CFMetaData cfm = getCurrentColumnFamilyStore().metadata;
+            TableMetadata cfm = getCurrentColumnFamilyStore().metadata();
             return StreamSupport.stream(cfm.partitionKeyColumns().spliterator(), false)
                                 .map(column -> column.name.toString() + "=?")
                                 .collect(Collectors.joining(" AND ", "DELETE FROM %s WHERE ", ""));
         }
 
-        private Stream<ColumnDefinition> getPrimaryKeyColumns()
+        private Stream<ColumnMetadata> getPrimaryKeyColumns()
         {
-            CFMetaData cfm = getCurrentColumnFamilyStore().metadata;
+            TableMetadata cfm = getCurrentColumnFamilyStore().metadata();
             if (cfm.isCompactTable())
                 return cfm.partitionKeyColumns().stream();
             else
@@ -757,7 +755,7 @@ public class CassandraIndexTest extends CQLTester
 
         private Object[] getPrimaryKeyValues(Object[] row)
         {
-            CFMetaData cfm = getCurrentColumnFamilyStore().metadata;
+            TableMetadata cfm = getCurrentColumnFamilyStore().metadata();
             if (cfm.isCompactTable())
                 return getPartitionKeyValues(row);
 
@@ -766,7 +764,7 @@ public class CassandraIndexTest extends CQLTester
 
         private Object[] getPartitionKeyValues(Object[] row)
         {
-            CFMetaData cfm = getCurrentColumnFamilyStore().metadata;
+            TableMetadata cfm = getCurrentColumnFamilyStore().metadata();
             return copyValuesFromRow(row, cfm.partitionKeyColumns().size());
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
index 1173801..1f1d8b9 100644
--- a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
+++ b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
@@ -32,8 +32,10 @@ import org.apache.cassandra.index.TargetParser;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.cql3.statements.IndexTarget;
 import org.apache.cassandra.db.*;
@@ -73,7 +75,7 @@ public class CustomCassandraIndex implements Index
     public final ColumnFamilyStore baseCfs;
     protected IndexMetadata metadata;
     protected ColumnFamilyStore indexCfs;
-    protected ColumnDefinition indexedColumn;
+    protected ColumnMetadata indexedColumn;
     protected CassandraIndexFunctions functions;
 
     public CustomCassandraIndex(ColumnFamilyStore baseCfs, IndexMetadata indexDef)
@@ -88,19 +90,19 @@ public class CustomCassandraIndex implements Index
      * @param operator
      * @return
      */
-    protected boolean supportsOperator(ColumnDefinition indexedColumn, Operator operator)
+    protected boolean supportsOperator(ColumnMetadata indexedColumn, Operator operator)
     {
         return operator.equals(Operator.EQ);
     }
 
-    public ColumnDefinition getIndexedColumn()
+    public ColumnMetadata getIndexedColumn()
     {
         return indexedColumn;
     }
 
     public ClusteringComparator getIndexComparator()
     {
-        return indexCfs.metadata.comparator;
+        return indexCfs.metadata().comparator;
     }
 
     public ColumnFamilyStore getIndexCfs()
@@ -150,7 +152,6 @@ public class CustomCassandraIndex implements Index
     {
         setMetadata(indexDef);
         return () -> {
-            indexCfs.metadata.reloadIndexMetadataProperties(baseCfs.metadata);
             indexCfs.reload();
             return null;
         };
@@ -159,12 +160,12 @@ public class CustomCassandraIndex implements Index
     private void setMetadata(IndexMetadata indexDef)
     {
         metadata = indexDef;
-        Pair<ColumnDefinition, IndexTarget.Type> target = TargetParser.parse(baseCfs.metadata, indexDef);
+        Pair<ColumnMetadata, IndexTarget.Type> target = TargetParser.parse(baseCfs.metadata(), indexDef);
         functions = getFunctions(indexDef, target);
-        CFMetaData cfm = indexCfsMetadata(baseCfs.metadata, indexDef);
+        TableMetadata cfm = indexCfsMetadata(baseCfs.metadata(), indexDef);
         indexCfs = ColumnFamilyStore.createColumnFamilyStore(baseCfs.keyspace,
-                                                             cfm.cfName,
-                                                             cfm,
+                                                             cfm.name,
+                                                             TableMetadataRef.forOfflineTools(cfm),
                                                              baseCfs.getTracker().loadsstables);
         indexedColumn = target.left;
     }
@@ -182,12 +183,12 @@ public class CustomCassandraIndex implements Index
         return true;
     }
 
-    public boolean dependsOn(ColumnDefinition column)
+    public boolean dependsOn(ColumnMetadata column)
     {
         return column.equals(indexedColumn);
     }
 
-    public boolean supportsExpression(ColumnDefinition column, Operator operator)
+    public boolean supportsExpression(ColumnMetadata column, Operator operator)
     {
         return indexedColumn.name.equals(column.name)
                && supportsOperator(indexedColumn, operator);
@@ -285,7 +286,7 @@ public class CustomCassandraIndex implements Index
     }
 
     public Indexer indexerFor(final DecoratedKey key,
-                              final PartitionColumns columns,
+                              final RegularAndStaticColumns columns,
                               final int nowInSec,
                               final OpOrder.Group opGroup,
                               final IndexTransaction.Type transactionType)
@@ -553,8 +554,8 @@ public class CustomCassandraIndex implements Index
                                                            "Cannot index value of size %d for index %s on %s.%s(%s) (maximum allowed size=%d)",
                                                            value.remaining(),
                                                            metadata.name,
-                                                           baseCfs.metadata.ksName,
-                                                           baseCfs.metadata.cfName,
+                                                           baseCfs.metadata.keyspace,
+                                                           baseCfs.metadata.name,
                                                            indexedColumn.name.toString(),
                                                            FBUtilities.MAX_UNSIGNED_SHORT));
     }
@@ -586,7 +587,7 @@ public class CustomCassandraIndex implements Index
 
     private PartitionUpdate partitionUpdate(DecoratedKey valueKey, Row row)
     {
-        return PartitionUpdate.singleRowUpdate(indexCfs.metadata, valueKey, row);
+        return PartitionUpdate.singleRowUpdate(indexCfs.metadata(), valueKey, row);
     }
 
     private void invalidate()
@@ -629,8 +630,8 @@ public class CustomCassandraIndex implements Index
             if (sstables.isEmpty())
             {
                 logger.info("No SSTable data for {}.{} to build index {} from, marking empty index as built",
-                            baseCfs.metadata.ksName,
-                            baseCfs.metadata.cfName,
+                            baseCfs.metadata.keyspace,
+                            baseCfs.metadata.name,
                             metadata.name);
                 baseCfs.indexManager.markIndexBuilt(metadata.name);
                 return;


[26/37] cassandra git commit: Make TableMetadata immutable, optimize Schema

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/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 5b63ba6..e826dd8 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -38,34 +38,34 @@ import com.google.common.io.ByteStreams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SchemaConstants;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.db.commitlog.CommitLogPosition;
 import org.apache.cassandra.db.compaction.CompactionHistoryTabularData;
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.db.rows.Rows;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.Rows;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.*;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.*;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.service.paxos.PaxosState;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.*;
 
+import static java.lang.String.format;
 import static java.util.Collections.emptyMap;
 import static java.util.Collections.singletonMap;
+
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
 
@@ -102,187 +102,203 @@ public final class SystemKeyspace
     public static final String BUILT_VIEWS = "built_views";
     public static final String PREPARED_STATEMENTS = "prepared_statements";
 
-    public static final CFMetaData Batches =
-        compile(BATCHES,
-                "batches awaiting replay",
-                "CREATE TABLE %s ("
-                + "id timeuuid,"
-                + "mutations list<blob>,"
-                + "version int,"
-                + "PRIMARY KEY ((id)))")
-                .copy(new LocalPartitioner(TimeUUIDType.instance))
-                .compaction(CompactionParams.scts(singletonMap("min_threshold", "2")))
-                .gcGraceSeconds(0);
-
-    private static final CFMetaData Paxos =
-        compile(PAXOS,
-                "in-progress paxos proposals",
-                "CREATE TABLE %s ("
-                + "row_key blob,"
-                + "cf_id UUID,"
-                + "in_progress_ballot timeuuid,"
-                + "most_recent_commit blob,"
-                + "most_recent_commit_at timeuuid,"
-                + "most_recent_commit_version int,"
-                + "proposal blob,"
-                + "proposal_ballot timeuuid,"
-                + "proposal_version int,"
-                + "PRIMARY KEY ((row_key), cf_id))")
-                .compaction(CompactionParams.lcs(emptyMap()));
-
-    private static final CFMetaData BuiltIndexes =
-        compile(BUILT_INDEXES,
-                "built column indexes",
-                "CREATE TABLE \"%s\" ("
-                + "table_name text," // table_name here is the name of the keyspace - don't be fooled
-                + "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,"
-                + "broadcast_address inet,"
-                + "cluster_name text,"
-                + "cql_version text,"
-                + "data_center text,"
-                + "gossip_generation int,"
-                + "host_id uuid,"
-                + "listen_address inet,"
-                + "native_protocol_version text,"
-                + "partitioner text,"
-                + "rack text,"
-                + "release_version text,"
-                + "rpc_address inet,"
-                + "schema_version uuid,"
-                + "tokens set<varchar>,"
-                + "truncated_at map<uuid, blob>,"
-                + "PRIMARY KEY ((key)))"
-                ).recordDeprecatedSystemColumn("thrift_version", UTF8Type.instance);
-
-    private static final CFMetaData Peers =
-        compile(PEERS,
-                "information about known peers in the cluster",
-                "CREATE TABLE %s ("
-                + "peer inet,"
-                + "data_center text,"
-                + "host_id uuid,"
-                + "preferred_ip inet,"
-                + "rack text,"
-                + "release_version text,"
-                + "rpc_address inet,"
-                + "schema_version uuid,"
-                + "tokens set<varchar>,"
-                + "PRIMARY KEY ((peer)))");
-
-    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 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 CompactionHistory =
-        compile(COMPACTION_HISTORY,
-                "week-long compaction history",
-                "CREATE TABLE %s ("
-                + "id uuid,"
-                + "bytes_in bigint,"
-                + "bytes_out bigint,"
-                + "columnfamily_name text,"
-                + "compacted_at timestamp,"
-                + "keyspace_name text,"
-                + "rows_merged map<int, bigint>,"
-                + "PRIMARY KEY ((id)))")
-                .defaultTimeToLive((int) TimeUnit.DAYS.toSeconds(7));
-
-    private static final CFMetaData SSTableActivity =
-        compile(SSTABLE_ACTIVITY,
-                "historic sstable read rates",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "columnfamily_name text,"
-                + "generation int,"
-                + "rate_120m double,"
-                + "rate_15m double,"
-                + "PRIMARY KEY ((keyspace_name, columnfamily_name, generation)))");
-
-    private static final CFMetaData SizeEstimates =
-        compile(SIZE_ESTIMATES,
-                "per-table primary range size estimates",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "table_name text,"
-                + "range_start text,"
-                + "range_end text,"
-                + "mean_partition_size bigint,"
-                + "partitions_count bigint,"
-                + "PRIMARY KEY ((keyspace_name), table_name, range_start, range_end))")
-                .gcGraceSeconds(0);
-
-    private static final CFMetaData AvailableRanges =
-        compile(AVAILABLE_RANGES,
-                "available keyspace/ranges during bootstrap/replace that are ready to be served",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "ranges set<blob>,"
-                + "PRIMARY KEY ((keyspace_name)))");
-
-    private static final CFMetaData TransferredRanges =
-        compile(TRANSFERRED_RANGES,
-                "record of transferred ranges for streaming operation",
-                "CREATE TABLE %s ("
-                + "operation text,"
-                + "peer inet,"
-                + "keyspace_name text,"
-                + "ranges set<blob>,"
-                + "PRIMARY KEY ((operation, keyspace_name), peer))");
-
-    private static final CFMetaData ViewsBuildsInProgress =
-        compile(VIEWS_BUILDS_IN_PROGRESS,
-                "views builds current progress",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "view_name text,"
-                + "last_token varchar,"
-                + "generation_number int,"
-                + "PRIMARY KEY ((keyspace_name), view_name))");
-
-    private static final CFMetaData BuiltViews =
-        compile(BUILT_VIEWS,
-                "built views",
-                "CREATE TABLE %s ("
-                + "keyspace_name text,"
-                + "view_name text,"
-                + "status_replicated boolean,"
-                + "PRIMARY KEY ((keyspace_name), view_name))");
-
-    private static final CFMetaData PreparedStatements =
-        compile(PREPARED_STATEMENTS,
-                "prepared statements",
-                "CREATE TABLE %s ("
-                + "prepared_id blob,"
-                + "logged_keyspace text,"
-                + "query_string text,"
-                + "PRIMARY KEY ((prepared_id)))");
-
-
-    private static CFMetaData compile(String name, String description, String schema)
-    {
-        return CFMetaData.compile(String.format(schema, name), SchemaConstants.SYSTEM_KEYSPACE_NAME)
-                         .comment(description);
+    public static final TableMetadata Batches =
+        parse(BATCHES,
+              "batches awaiting replay",
+              "CREATE TABLE %s ("
+              + "id timeuuid,"
+              + "mutations list<blob>,"
+              + "version int,"
+              + "PRIMARY KEY ((id)))")
+              .partitioner(new LocalPartitioner(TimeUUIDType.instance))
+              .compaction(CompactionParams.scts(singletonMap("min_threshold", "2")))
+              .build();
+
+    private static final TableMetadata Paxos =
+        parse(PAXOS,
+              "in-progress paxos proposals",
+              "CREATE TABLE %s ("
+              + "row_key blob,"
+              + "cf_id UUID,"
+              + "in_progress_ballot timeuuid,"
+              + "most_recent_commit blob,"
+              + "most_recent_commit_at timeuuid,"
+              + "most_recent_commit_version int,"
+              + "proposal blob,"
+              + "proposal_ballot timeuuid,"
+              + "proposal_version int,"
+              + "PRIMARY KEY ((row_key), cf_id))")
+              .compaction(CompactionParams.lcs(emptyMap()))
+              .build();
+
+    private static final TableMetadata BuiltIndexes =
+        parse(BUILT_INDEXES,
+              "built column indexes",
+              "CREATE TABLE \"%s\" ("
+              + "table_name text," // table_name here is the name of the keyspace - don't be fooled
+              + "index_name text,"
+              + "PRIMARY KEY ((table_name), index_name)) "
+              + "WITH COMPACT STORAGE")
+              .build();
+
+    private static final TableMetadata Local =
+        parse(LOCAL,
+              "information about the local node",
+              "CREATE TABLE %s ("
+              + "key text,"
+              + "bootstrapped text,"
+              + "broadcast_address inet,"
+              + "cluster_name text,"
+              + "cql_version text,"
+              + "data_center text,"
+              + "gossip_generation int,"
+              + "host_id uuid,"
+              + "listen_address inet,"
+              + "native_protocol_version text,"
+              + "partitioner text,"
+              + "rack text,"
+              + "release_version text,"
+              + "rpc_address inet,"
+              + "schema_version uuid,"
+              + "tokens set<varchar>,"
+              + "truncated_at map<uuid, blob>,"
+              + "PRIMARY KEY ((key)))")
+              .recordDeprecatedSystemColumn("thrift_version", UTF8Type.instance)
+              .build();
+
+    private static final TableMetadata Peers =
+        parse(PEERS,
+              "information about known peers in the cluster",
+              "CREATE TABLE %s ("
+              + "peer inet,"
+              + "data_center text,"
+              + "host_id uuid,"
+              + "preferred_ip inet,"
+              + "rack text,"
+              + "release_version text,"
+              + "rpc_address inet,"
+              + "schema_version uuid,"
+              + "tokens set<varchar>,"
+              + "PRIMARY KEY ((peer)))")
+              .build();
+
+    private static final TableMetadata PeerEvents =
+        parse(PEER_EVENTS,
+              "events related to peers",
+              "CREATE TABLE %s ("
+              + "peer inet,"
+              + "hints_dropped map<uuid, int>,"
+              + "PRIMARY KEY ((peer)))")
+              .build();
+
+    private static final TableMetadata RangeXfers =
+        parse(RANGE_XFERS,
+              "ranges requested for transfer",
+              "CREATE TABLE %s ("
+              + "token_bytes blob,"
+              + "requested_at timestamp,"
+              + "PRIMARY KEY ((token_bytes)))")
+              .build();
+
+    private static final TableMetadata CompactionHistory =
+        parse(COMPACTION_HISTORY,
+              "week-long compaction history",
+              "CREATE TABLE %s ("
+              + "id uuid,"
+              + "bytes_in bigint,"
+              + "bytes_out bigint,"
+              + "columnfamily_name text,"
+              + "compacted_at timestamp,"
+              + "keyspace_name text,"
+              + "rows_merged map<int, bigint>,"
+              + "PRIMARY KEY ((id)))")
+              .defaultTimeToLive((int) TimeUnit.DAYS.toSeconds(7))
+              .build();
+
+    private static final TableMetadata SSTableActivity =
+        parse(SSTABLE_ACTIVITY,
+              "historic sstable read rates",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "columnfamily_name text,"
+              + "generation int,"
+              + "rate_120m double,"
+              + "rate_15m double,"
+              + "PRIMARY KEY ((keyspace_name, columnfamily_name, generation)))")
+              .build();
+
+    private static final TableMetadata SizeEstimates =
+        parse(SIZE_ESTIMATES,
+              "per-table primary range size estimates",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "table_name text,"
+              + "range_start text,"
+              + "range_end text,"
+              + "mean_partition_size bigint,"
+              + "partitions_count bigint,"
+              + "PRIMARY KEY ((keyspace_name), table_name, range_start, range_end))")
+              .build();
+
+    private static final TableMetadata AvailableRanges =
+        parse(AVAILABLE_RANGES,
+              "available keyspace/ranges during bootstrap/replace that are ready to be served",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "ranges set<blob>,"
+              + "PRIMARY KEY ((keyspace_name)))")
+              .build();
+
+    private static final TableMetadata TransferredRanges =
+        parse(TRANSFERRED_RANGES,
+              "record of transferred ranges for streaming operation",
+              "CREATE TABLE %s ("
+              + "operation text,"
+              + "peer inet,"
+              + "keyspace_name text,"
+              + "ranges set<blob>,"
+              + "PRIMARY KEY ((operation, keyspace_name), peer))")
+              .build();
+
+    private static final TableMetadata ViewsBuildsInProgress =
+        parse(VIEWS_BUILDS_IN_PROGRESS,
+              "views builds current progress",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "view_name text,"
+              + "last_token varchar,"
+              + "generation_number int,"
+              + "PRIMARY KEY ((keyspace_name), view_name))")
+              .build();
+
+    private static final TableMetadata BuiltViews =
+        parse(BUILT_VIEWS,
+              "built views",
+              "CREATE TABLE %s ("
+              + "keyspace_name text,"
+              + "view_name text,"
+              + "status_replicated boolean,"
+              + "PRIMARY KEY ((keyspace_name), view_name))")
+              .build();
+
+    private static final TableMetadata PreparedStatements =
+        parse(PREPARED_STATEMENTS,
+              "prepared statements",
+              "CREATE TABLE %s ("
+              + "prepared_id blob,"
+              + "logged_keyspace text,"
+              + "query_string text,"
+              + "PRIMARY KEY ((prepared_id)))")
+              .build();
+
+    private static TableMetadata.Builder parse(String table, String description, String cql)
+    {
+        return CreateTableStatement.parse(format(cql, table), SchemaConstants.SYSTEM_KEYSPACE_NAME)
+                                   .id(TableId.forSystemTable(SchemaConstants.SYSTEM_KEYSPACE_NAME, table))
+                                   .dcLocalReadRepairChance(0.0)
+                                   .gcGraceSeconds(0)
+                                   .memtableFlushPeriod((int) TimeUnit.HOURS.toMillis(1))
+                                   .comment(description);
     }
 
     public static KeyspaceMetadata metadata()
@@ -321,7 +337,7 @@ public final class SystemKeyspace
                         .build();
     }
 
-    private static volatile Map<UUID, Pair<CommitLogPosition, Long>> truncationRecords;
+    private static volatile Map<TableId, Pair<CommitLogPosition, Long>> truncationRecords;
 
     public enum BootstrapState
     {
@@ -352,7 +368,7 @@ public final class SystemKeyspace
                      "listen_address" +
                      ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        executeOnceInternal(String.format(req, LOCAL),
+        executeOnceInternal(format(req, LOCAL),
                             LOCAL,
                             DatabaseDescriptor.getClusterName(),
                             FBUtilities.getReleaseVersionString(),
@@ -377,7 +393,7 @@ public final class SystemKeyspace
         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),
+        executeInternal(format(req, COMPACTION_HISTORY),
                         UUIDGen.getTimeUUID(),
                         ksname,
                         cfname,
@@ -389,21 +405,21 @@ public final class SystemKeyspace
 
     public static TabularData getCompactionHistory() throws OpenDataException
     {
-        UntypedResultSet queryResultSet = executeInternal(String.format("SELECT * from system.%s", COMPACTION_HISTORY));
+        UntypedResultSet queryResultSet = executeInternal(format("SELECT * from system.%s", COMPACTION_HISTORY));
         return CompactionHistoryTabularData.from(queryResultSet);
     }
 
     public static boolean isViewBuilt(String keyspaceName, String viewName)
     {
         String req = "SELECT view_name FROM %s.\"%s\" WHERE keyspace_name=? AND view_name=?";
-        UntypedResultSet result = executeInternal(String.format(req, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_VIEWS), keyspaceName, viewName);
+        UntypedResultSet result = executeInternal(format(req, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_VIEWS), keyspaceName, viewName);
         return !result.isEmpty();
     }
 
     public static boolean isViewStatusReplicated(String keyspaceName, String viewName)
     {
         String req = "SELECT status_replicated FROM %s.\"%s\" WHERE keyspace_name=? AND view_name=?";
-        UntypedResultSet result = executeInternal(String.format(req, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_VIEWS), keyspaceName, viewName);
+        UntypedResultSet result = executeInternal(format(req, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_VIEWS), keyspaceName, viewName);
 
         if (result.isEmpty())
             return false;
@@ -417,7 +433,7 @@ public final class SystemKeyspace
             return;
 
         String req = "INSERT INTO %s.\"%s\" (keyspace_name, view_name, status_replicated) VALUES (?, ?, ?)";
-        executeInternal(String.format(req, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_VIEWS), keyspaceName, viewName, replicated);
+        executeInternal(format(req, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_VIEWS), keyspaceName, viewName, replicated);
         forceBlockingFlush(BUILT_VIEWS);
     }
 
@@ -434,7 +450,7 @@ public final class SystemKeyspace
 
     public static void beginViewBuild(String ksname, String viewName, int generationNumber)
     {
-        executeInternal(String.format("INSERT INTO system.%s (keyspace_name, view_name, generation_number) VALUES (?, ?, ?)", VIEWS_BUILDS_IN_PROGRESS),
+        executeInternal(format("INSERT INTO system.%s (keyspace_name, view_name, generation_number) VALUES (?, ?, ?)", VIEWS_BUILDS_IN_PROGRESS),
                         ksname,
                         viewName,
                         generationNumber);
@@ -461,13 +477,13 @@ public final class SystemKeyspace
     {
         String req = "INSERT INTO system.%s (keyspace_name, view_name, last_token) VALUES (?, ?, ?)";
         Token.TokenFactory factory = ViewsBuildsInProgress.partitioner.getTokenFactory();
-        executeInternal(String.format(req, VIEWS_BUILDS_IN_PROGRESS), ksname, viewName, factory.toString(token));
+        executeInternal(format(req, VIEWS_BUILDS_IN_PROGRESS), ksname, viewName, factory.toString(token));
     }
 
     public static Pair<Integer, Token> getViewBuildStatus(String ksname, String viewName)
     {
         String req = "SELECT generation_number, last_token FROM system.%s WHERE keyspace_name = ? AND view_name = ?";
-        UntypedResultSet queryResultSet = executeInternal(String.format(req, VIEWS_BUILDS_IN_PROGRESS), ksname, viewName);
+        UntypedResultSet queryResultSet = executeInternal(format(req, VIEWS_BUILDS_IN_PROGRESS), ksname, viewName);
         if (queryResultSet == null || queryResultSet.isEmpty())
             return null;
 
@@ -489,7 +505,7 @@ public final class SystemKeyspace
     public static synchronized void saveTruncationRecord(ColumnFamilyStore cfs, long truncatedAt, CommitLogPosition position)
     {
         String req = "UPDATE system.%s SET truncated_at = truncated_at + ? WHERE key = '%s'";
-        executeInternal(String.format(req, LOCAL, LOCAL), truncationAsMapEntry(cfs, truncatedAt, position));
+        executeInternal(format(req, LOCAL, LOCAL), truncationAsMapEntry(cfs, truncatedAt, position));
         truncationRecords = null;
         forceBlockingFlush(LOCAL);
     }
@@ -497,14 +513,14 @@ public final class SystemKeyspace
     /**
      * This method is used to remove information about truncation time for specified column family
      */
-    public static synchronized void removeTruncationRecord(UUID cfId)
+    public static synchronized void removeTruncationRecord(TableId id)
     {
-        Pair<CommitLogPosition, Long> truncationRecord = getTruncationRecord(cfId);
+        Pair<CommitLogPosition, Long> truncationRecord = getTruncationRecord(id);
         if (truncationRecord == null)
             return;
 
         String req = "DELETE truncated_at[?] from system.%s WHERE key = '%s'";
-        executeInternal(String.format(req, LOCAL, LOCAL), cfId);
+        executeInternal(format(req, LOCAL, LOCAL), id.asUUID());
         truncationRecords = null;
         forceBlockingFlush(LOCAL);
     }
@@ -515,7 +531,7 @@ public final class SystemKeyspace
         {
             CommitLogPosition.serializer.serialize(position, out);
             out.writeLong(truncatedAt);
-            return singletonMap(cfs.metadata.cfId, out.asNewBuffer());
+            return singletonMap(cfs.metadata.id.asUUID(), out.asNewBuffer());
         }
         catch (IOException e)
         {
@@ -523,36 +539,36 @@ public final class SystemKeyspace
         }
     }
 
-    public static CommitLogPosition getTruncatedPosition(UUID cfId)
+    public static CommitLogPosition getTruncatedPosition(TableId id)
     {
-        Pair<CommitLogPosition, Long> record = getTruncationRecord(cfId);
+        Pair<CommitLogPosition, Long> record = getTruncationRecord(id);
         return record == null ? null : record.left;
     }
 
-    public static long getTruncatedAt(UUID cfId)
+    public static long getTruncatedAt(TableId id)
     {
-        Pair<CommitLogPosition, Long> record = getTruncationRecord(cfId);
+        Pair<CommitLogPosition, Long> record = getTruncationRecord(id);
         return record == null ? Long.MIN_VALUE : record.right;
     }
 
-    private static synchronized Pair<CommitLogPosition, Long> getTruncationRecord(UUID cfId)
+    private static synchronized Pair<CommitLogPosition, Long> getTruncationRecord(TableId id)
     {
         if (truncationRecords == null)
             truncationRecords = readTruncationRecords();
-        return truncationRecords.get(cfId);
+        return truncationRecords.get(id);
     }
 
-    private static Map<UUID, Pair<CommitLogPosition, Long>> readTruncationRecords()
+    private static Map<TableId, Pair<CommitLogPosition, Long>> readTruncationRecords()
     {
-        UntypedResultSet rows = executeInternal(String.format("SELECT truncated_at FROM system.%s WHERE key = '%s'", LOCAL, LOCAL));
+        UntypedResultSet rows = executeInternal(format("SELECT truncated_at FROM system.%s WHERE key = '%s'", LOCAL, LOCAL));
 
-        Map<UUID, Pair<CommitLogPosition, Long>> records = new HashMap<>();
+        Map<TableId, Pair<CommitLogPosition, Long>> records = new HashMap<>();
 
         if (!rows.isEmpty() && rows.one().has("truncated_at"))
         {
             Map<UUID, ByteBuffer> map = rows.one().getMap("truncated_at", UUIDType.instance, BytesType.instance);
             for (Map.Entry<UUID, ByteBuffer> entry : map.entrySet())
-                records.put(entry.getKey(), truncationRecordFromBlob(entry.getValue()));
+                records.put(TableId.fromUUID(entry.getKey()), truncationRecordFromBlob(entry.getValue()));
         }
 
         return records;
@@ -579,7 +595,7 @@ public final class SystemKeyspace
             return;
 
         String req = "INSERT INTO system.%s (peer, tokens) VALUES (?, ?)";
-        executeInternal(String.format(req, PEERS), ep, tokensAsSet(tokens));
+        executeInternal(format(req, PEERS), ep, tokensAsSet(tokens));
     }
 
     public static synchronized void updatePreferredIP(InetAddress ep, InetAddress preferred_ip)
@@ -588,7 +604,7 @@ public final class SystemKeyspace
             return;
 
         String req = "INSERT INTO system.%s (peer, preferred_ip) VALUES (?, ?)";
-        executeInternal(String.format(req, PEERS), ep, preferred_ip);
+        executeInternal(format(req, PEERS), ep, preferred_ip);
         forceBlockingFlush(PEERS);
     }
 
@@ -598,20 +614,20 @@ public final class SystemKeyspace
             return;
 
         String req = "INSERT INTO system.%s (peer, %s) VALUES (?, ?)";
-        executeInternal(String.format(req, PEERS, columnName), ep, value);
+        executeInternal(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), timePeriod, value, ep);
+        executeInternal(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, LOCAL), version);
+        executeInternal(format(req, LOCAL, LOCAL), version);
     }
 
     private static Set<String> tokensAsSet(Collection<Token> tokens)
@@ -640,7 +656,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), ep);
+        executeInternal(format(req, PEERS), ep);
         forceBlockingFlush(PEERS);
     }
 
@@ -656,7 +672,7 @@ public final class SystemKeyspace
             return;
 
         String req = "INSERT INTO system.%s (key, tokens) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL, LOCAL), tokensAsSet(tokens));
+        executeInternal(format(req, LOCAL, LOCAL), tokensAsSet(tokens));
         forceBlockingFlush(LOCAL);
     }
 
@@ -710,7 +726,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), ep);
+        UntypedResultSet result = executeInternal(format(req, PEERS), ep);
         if (!result.isEmpty() && result.one().has("preferred_ip"))
             return result.one().getInetAddress("preferred_ip");
         return ep;
@@ -752,7 +768,7 @@ public final class SystemKeyspace
                 return new CassandraVersion(FBUtilities.getReleaseVersionString());
             }
             String req = "SELECT release_version FROM system.%s WHERE peer=?";
-            UntypedResultSet result = executeInternal(String.format(req, PEERS), ep);
+            UntypedResultSet result = executeInternal(format(req, PEERS), ep);
             if (result != null && result.one().has("release_version"))
             {
                 return new CassandraVersion(result.one().getString("release_version"));
@@ -791,7 +807,7 @@ public final class SystemKeyspace
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(LOCAL);
 
         String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
+        UntypedResultSet result = executeInternal(format(req, LOCAL, LOCAL));
 
         if (result.isEmpty() || !result.one().has("cluster_name"))
         {
@@ -811,7 +827,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, LOCAL));
+        UntypedResultSet result = executeInternal(format(req, LOCAL, LOCAL));
         return result.isEmpty() || !result.one().has("tokens")
              ? Collections.<Token>emptyList()
              : deserializeTokens(result.one().getSet("tokens", UTF8Type.instance));
@@ -820,7 +836,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, LOCAL));
+        UntypedResultSet result = executeInternal(format(req, LOCAL, LOCAL));
 
         int generation;
         if (result.isEmpty() || !result.one().has("gossip_generation"))
@@ -848,7 +864,7 @@ public final class SystemKeyspace
         }
 
         req = "INSERT INTO system.%s (key, gossip_generation) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL, LOCAL), generation);
+        executeInternal(format(req, LOCAL, LOCAL), generation);
         forceBlockingFlush(LOCAL);
 
         return generation;
@@ -857,7 +873,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, LOCAL));
+        UntypedResultSet result = executeInternal(format(req, LOCAL, LOCAL));
 
         if (result.isEmpty() || !result.one().has("bootstrapped"))
             return BootstrapState.NEEDS_BOOTSTRAP;
@@ -886,14 +902,14 @@ public final class SystemKeyspace
             return;
 
         String req = "INSERT INTO system.%s (key, bootstrapped) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL, LOCAL), state.name());
+        executeInternal(format(req, LOCAL, LOCAL), state.name());
         forceBlockingFlush(LOCAL);
     }
 
     public static boolean isIndexBuilt(String keyspaceName, String indexName)
     {
         String req = "SELECT index_name FROM %s.\"%s\" WHERE table_name=? AND index_name=?";
-        UntypedResultSet result = executeInternal(String.format(req, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_INDEXES), keyspaceName, indexName);
+        UntypedResultSet result = executeInternal(format(req, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_INDEXES), keyspaceName, indexName);
         return !result.isEmpty();
     }
 
@@ -915,7 +931,7 @@ public final class SystemKeyspace
     {
         List<String> names = new ArrayList<>(indexNames);
         String req = "SELECT index_name from %s.\"%s\" WHERE table_name=? AND index_name IN ?";
-        UntypedResultSet results = executeInternal(String.format(req, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_INDEXES), keyspaceName, names);
+        UntypedResultSet results = executeInternal(format(req, SchemaConstants.SYSTEM_KEYSPACE_NAME, BUILT_INDEXES), keyspaceName, names);
         return StreamSupport.stream(results.spliterator(), false)
                             .map(r -> r.getString("index_name"))
                             .collect(Collectors.toList());
@@ -928,7 +944,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, LOCAL));
+        UntypedResultSet result = executeInternal(format(req, LOCAL, LOCAL));
 
         // Look up the Host UUID (return it if found)
         if (!result.isEmpty() && result.one().has("host_id"))
@@ -946,7 +962,7 @@ 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, LOCAL), hostId);
+        executeInternal(format(req, LOCAL, LOCAL), hostId);
         return hostId;
     }
 
@@ -956,7 +972,7 @@ public final class SystemKeyspace
     public static String getRack()
     {
         String req = "SELECT rack FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
+        UntypedResultSet result = executeInternal(format(req, LOCAL, LOCAL));
 
         // Look up the Rack (return it if found)
         if (!result.isEmpty() && result.one().has("rack"))
@@ -971,7 +987,7 @@ public final class SystemKeyspace
     public static String getDatacenter()
     {
         String req = "SELECT data_center FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
+        UntypedResultSet result = executeInternal(format(req, LOCAL, LOCAL));
 
         // Look up the Data center (return it if found)
         if (!result.isEmpty() && result.one().has("data_center"))
@@ -980,16 +996,16 @@ public final class SystemKeyspace
         return null;
     }
 
-    public static PaxosState loadPaxosState(DecoratedKey key, CFMetaData metadata, int nowInSec)
+    public static PaxosState loadPaxosState(DecoratedKey key, TableMetadata metadata, int nowInSec)
     {
         String req = "SELECT * FROM system.%s WHERE row_key = ? AND cf_id = ?";
-        UntypedResultSet results = QueryProcessor.executeInternalWithNow(nowInSec, System.nanoTime(), String.format(req, PAXOS), key.getKey(), metadata.cfId);
+        UntypedResultSet results = QueryProcessor.executeInternalWithNow(nowInSec, System.nanoTime(), format(req, PAXOS), key.getKey(), metadata.id.asUUID());
         if (results.isEmpty())
             return new PaxosState(key, metadata);
         UntypedResultSet.Row row = results.one();
 
         Commit promised = row.has("in_progress_ballot")
-                        ? new Commit(row.getUUID("in_progress_ballot"), new PartitionUpdate(metadata, key, metadata.partitionColumns(), 1))
+                        ? new Commit(row.getUUID("in_progress_ballot"), new PartitionUpdate(metadata, key, metadata.regularAndStaticColumns(), 1))
                         : Commit.emptyCommit(key, metadata);
         // either we have both a recently accepted ballot and update or we have neither
         Commit accepted = row.has("proposal_version") && row.has("proposal")
@@ -1007,27 +1023,27 @@ 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),
+        executeInternal(format(req, PAXOS),
                         UUIDGen.microsTimestamp(promise.ballot),
                         paxosTtlSec(promise.update.metadata()),
                         promise.ballot,
                         promise.update.partitionKey().getKey(),
-                        promise.update.metadata().cfId);
+                        promise.update.metadata().id.asUUID());
     }
 
     public static void savePaxosProposal(Commit proposal)
     {
-        executeInternal(String.format("UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = ?, proposal = ?, proposal_version = ? WHERE row_key = ? AND cf_id = ?", PAXOS),
+        executeInternal(format("UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = ?, proposal = ?, proposal_version = ? WHERE row_key = ? AND cf_id = ?", PAXOS),
                         UUIDGen.microsTimestamp(proposal.ballot),
                         paxosTtlSec(proposal.update.metadata()),
                         proposal.ballot,
                         PartitionUpdate.toBytes(proposal.update, MessagingService.current_version),
                         MessagingService.current_version,
                         proposal.update.partitionKey().getKey(),
-                        proposal.update.metadata().cfId);
+                        proposal.update.metadata().id.asUUID());
     }
 
-    public static int paxosTtlSec(CFMetaData metadata)
+    public static int paxosTtlSec(TableMetadata metadata)
     {
         // keep paxos state around for at least 3h
         return Math.max(3 * 3600, metadata.params.gcGraceSeconds);
@@ -1038,14 +1054,14 @@ 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 = ?, most_recent_commit_version = ? WHERE row_key = ? AND cf_id = ?";
-        executeInternal(String.format(cql, PAXOS),
+        executeInternal(format(cql, PAXOS),
                         UUIDGen.microsTimestamp(commit.ballot),
                         paxosTtlSec(commit.update.metadata()),
                         commit.ballot,
                         PartitionUpdate.toBytes(commit.update, MessagingService.current_version),
                         MessagingService.current_version,
                         commit.update.partitionKey().getKey(),
-                        commit.update.metadata().cfId);
+                        commit.update.metadata().id.asUUID());
     }
 
     /**
@@ -1058,7 +1074,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), keyspace, table, generation);
+        UntypedResultSet results = executeInternal(format(cql, SSTABLE_ACTIVITY), keyspace, table, generation);
 
         if (results.isEmpty())
             return new RestorableMeter();
@@ -1076,7 +1092,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),
+        executeInternal(format(cql, SSTABLE_ACTIVITY),
                         keyspace,
                         table,
                         generation,
@@ -1090,7 +1106,7 @@ 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), keyspace, table, generation);
+        executeInternal(format(cql, SSTABLE_ACTIVITY), keyspace, table, generation);
     }
 
     /**
@@ -1099,7 +1115,7 @@ public final class SystemKeyspace
     public static void updateSizeEstimates(String keyspace, String table, Map<Range<Token>, Pair<Long, Long>> estimates)
     {
         long timestamp = FBUtilities.timestampMicros();
-        PartitionUpdate update = new PartitionUpdate(SizeEstimates, UTF8Type.instance.decompose(keyspace), SizeEstimates.partitionColumns(), estimates.size());
+        PartitionUpdate update = new PartitionUpdate(SizeEstimates, UTF8Type.instance.decompose(keyspace), SizeEstimates.regularAndStaticColumns(), estimates.size());
         Mutation mutation = new Mutation(update);
 
         // delete all previous values with a single range tombstone.
@@ -1126,7 +1142,7 @@ public final class SystemKeyspace
      */
     public static void clearSizeEstimates(String keyspace, String table)
     {
-        String cql = String.format("DELETE FROM %s.%s WHERE keyspace_name = ? AND table_name = ?", SchemaConstants.SYSTEM_KEYSPACE_NAME, SIZE_ESTIMATES);
+        String cql = format("DELETE FROM %s WHERE keyspace_name = ? AND table_name = ?", SizeEstimates.toString());
         executeInternal(cql, keyspace, table);
     }
 
@@ -1138,14 +1154,14 @@ public final class SystemKeyspace
         {
             rangesToUpdate.add(rangeToBytes(range));
         }
-        executeInternal(String.format(cql, AVAILABLE_RANGES), rangesToUpdate, keyspace);
+        executeInternal(format(cql, AVAILABLE_RANGES), rangesToUpdate, keyspace);
     }
 
     public static synchronized Set<Range<Token>> getAvailableRanges(String keyspace, IPartitioner partitioner)
     {
         Set<Range<Token>> result = new HashSet<>();
         String query = "SELECT * FROM system.%s WHERE keyspace_name=?";
-        UntypedResultSet rs = executeInternal(String.format(query, AVAILABLE_RANGES), keyspace);
+        UntypedResultSet rs = executeInternal(format(query, AVAILABLE_RANGES), keyspace);
         for (UntypedResultSet.Row row : rs)
         {
             Set<ByteBuffer> rawRanges = row.getSet("ranges", BytesType.instance);
@@ -1174,14 +1190,14 @@ public final class SystemKeyspace
         {
             rangesToUpdate.add(rangeToBytes(range));
         }
-        executeInternal(String.format(cql, TRANSFERRED_RANGES), rangesToUpdate, description, peer, keyspace);
+        executeInternal(format(cql, TRANSFERRED_RANGES), rangesToUpdate, description, peer, keyspace);
     }
 
     public static synchronized Map<InetAddress, Set<Range<Token>>> getTransferredRanges(String description, String keyspace, IPartitioner partitioner)
     {
         Map<InetAddress, Set<Range<Token>>> result = new HashMap<>();
         String query = "SELECT * FROM system.%s WHERE operation = ? AND keyspace_name = ?";
-        UntypedResultSet rs = executeInternal(String.format(query, TRANSFERRED_RANGES), description, keyspace);
+        UntypedResultSet rs = executeInternal(format(query, TRANSFERRED_RANGES), description, keyspace);
         for (UntypedResultSet.Row row : rs)
         {
             InetAddress peer = row.getInetAddress("peer");
@@ -1213,9 +1229,9 @@ public final class SystemKeyspace
 
         {
             logger.info("Detected version upgrade from {} to {}, snapshotting system keyspace", previous, next);
-            String snapshotName = Keyspace.getTimestampedSnapshotName(String.format("upgrade-%s-%s",
-                                                                                    previous,
-                                                                                    next));
+            String snapshotName = Keyspace.getTimestampedSnapshotName(format("upgrade-%s-%s",
+                                                                             previous,
+                                                                             next));
             Keyspace systemKs = Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME);
             systemKs.snapshot(snapshotName, null);
             return true;
@@ -1238,7 +1254,7 @@ public final class SystemKeyspace
     private static String getPreviousVersionString()
     {
         String req = "SELECT release_version FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, SystemKeyspace.LOCAL, SystemKeyspace.LOCAL));
+        UntypedResultSet result = executeInternal(format(req, SystemKeyspace.LOCAL, SystemKeyspace.LOCAL));
         if (result.isEmpty() || !result.one().has("release_version"))
         {
             // it isn't inconceivable that one might try to upgrade a node straight from <= 1.1 to whatever
@@ -1298,24 +1314,21 @@ public final class SystemKeyspace
 
     public static void writePreparedStatement(String loggedKeyspace, MD5Digest key, String cql)
     {
-        executeInternal(String.format("INSERT INTO %s.%s"
-                                      + " (logged_keyspace, prepared_id, query_string) VALUES (?, ?, ?)",
-                                      SchemaConstants.SYSTEM_KEYSPACE_NAME, PREPARED_STATEMENTS),
+        executeInternal(format("INSERT INTO %s (logged_keyspace, prepared_id, query_string) VALUES (?, ?, ?)",
+                               PreparedStatements.toString()),
                         loggedKeyspace, key.byteBuffer(), cql);
         logger.debug("stored prepared statement for logged keyspace '{}': '{}'", loggedKeyspace, cql);
     }
 
     public static void removePreparedStatement(MD5Digest key)
     {
-        executeInternal(String.format("DELETE FROM %s.%s"
-                                      + " WHERE prepared_id = ?",
-                                      SchemaConstants.SYSTEM_KEYSPACE_NAME, PREPARED_STATEMENTS),
+        executeInternal(format("DELETE FROM %s WHERE prepared_id = ?", PreparedStatements.toString()),
                         key.byteBuffer());
     }
 
     public static List<Pair<String, String>> loadPreparedStatements()
     {
-        String query = String.format("SELECT logged_keyspace, query_string FROM %s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, PREPARED_STATEMENTS);
+        String query = format("SELECT logged_keyspace, query_string FROM %s", PreparedStatements.toString());
         UntypedResultSet resultSet = executeOnceInternal(query);
         List<Pair<String, String>> r = new ArrayList<>();
         for (UntypedResultSet.Row row : resultSet)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
index c32a642..d361bac 100644
--- a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
+++ b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
@@ -18,19 +18,13 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
-import java.io.IOError;
-import java.util.*;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.PeekingIterator;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.cassandra.config.CFMetaData;
+
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.util.DataInputPlus;
-import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.net.MessagingService;
 
 /**
  * Helper class to deserialize Unfiltered object from disk efficiently.
@@ -43,7 +37,7 @@ public class UnfilteredDeserializer
 {
     private static final Logger logger = LoggerFactory.getLogger(UnfilteredDeserializer.class);
 
-    protected final CFMetaData metadata;
+    protected final TableMetadata metadata;
     protected final DataInputPlus in;
     protected final SerializationHelper helper;
 
@@ -57,7 +51,7 @@ public class UnfilteredDeserializer
 
     private final Row.Builder builder;
 
-    private UnfilteredDeserializer(CFMetaData metadata,
+    private UnfilteredDeserializer(TableMetadata metadata,
                                    DataInputPlus in,
                                    SerializationHeader header,
                                    SerializationHelper helper)
@@ -70,7 +64,7 @@ public class UnfilteredDeserializer
         this.builder = BTreeRow.sortedBuilder();
     }
 
-    public static UnfilteredDeserializer create(CFMetaData metadata,
+    public static UnfilteredDeserializer create(TableMetadata metadata,
                                                 DataInputPlus in,
                                                 SerializationHeader header,
                                                 SerializationHelper helper)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/UnknownColumnException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/UnknownColumnException.java b/src/java/org/apache/cassandra/db/UnknownColumnException.java
deleted file mode 100644
index 55dc453..0000000
--- a/src/java/org/apache/cassandra/db/UnknownColumnException.java
+++ /dev/null
@@ -1,51 +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.nio.ByteBuffer;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-/**
- * Exception thrown when we read a column internally that is unknown. Note that
- * this is an internal exception and is not meant to be user facing.
- */
-public class UnknownColumnException extends Exception
-{
-    public final ByteBuffer columnName;
-
-    public UnknownColumnException(CFMetaData metadata, ByteBuffer columnName)
-    {
-        super(String.format("Unknown column %s in table %s.%s", stringify(columnName), metadata.ksName, metadata.cfName));
-        this.columnName = columnName;
-    }
-
-    private static String stringify(ByteBuffer name)
-    {
-        try
-        {
-            return UTF8Type.instance.getString(name);
-        }
-        catch (Exception e)
-        {
-            return ByteBufferUtil.bytesToHex(name);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/UnknownColumnFamilyException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/UnknownColumnFamilyException.java b/src/java/org/apache/cassandra/db/UnknownColumnFamilyException.java
deleted file mode 100644
index c43b50a..0000000
--- a/src/java/org/apache/cassandra/db/UnknownColumnFamilyException.java
+++ /dev/null
@@ -1,33 +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.util.UUID;
-
-
-public class UnknownColumnFamilyException extends IOException
-{
-    public final UUID cfId;
-
-    public UnknownColumnFamilyException(String msg, UUID cfId)
-    {
-        super(msg);
-        this.cfId = cfId;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
index 71f8c43..3936ce4 100644
--- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
@@ -22,7 +22,7 @@ import java.util.Comparator;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.*;
@@ -37,6 +37,9 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 public abstract class AbstractSSTableIterator implements UnfilteredRowIterator
 {
     protected final SSTableReader sstable;
+    // We could use sstable.metadata(), but that can change during execution so it's good hygiene to grab an immutable instance
+    protected final TableMetadata metadata;
+
     protected final DecoratedKey key;
     protected final DeletionTime partitionLevelDeletion;
     protected final ColumnFilter columns;
@@ -62,11 +65,12 @@ public abstract class AbstractSSTableIterator implements UnfilteredRowIterator
                                       FileHandle ifile)
     {
         this.sstable = sstable;
+        this.metadata = sstable.metadata();
         this.ifile = ifile;
         this.key = key;
         this.columns = columnFilter;
         this.slices = slices;
-        this.helper = new SerializationHelper(sstable.metadata, sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL, columnFilter);
+        this.helper = new SerializationHelper(metadata, sstable.descriptor.version.correspondingMessagingVersion(), SerializationHelper.Flag.LOCAL, columnFilter);
 
         if (indexEntry == null)
         {
@@ -178,12 +182,12 @@ public abstract class AbstractSSTableIterator implements UnfilteredRowIterator
                                 : createReaderInternal(indexEntry, file, shouldCloseFile);
     };
 
-    public CFMetaData metadata()
+    public TableMetadata metadata()
     {
-        return sstable.metadata;
+        return metadata;
     }
 
-    public PartitionColumns columns()
+    public RegularAndStaticColumns columns()
     {
         return columns.fetchedColumns();
     }
@@ -306,7 +310,7 @@ public abstract class AbstractSSTableIterator implements UnfilteredRowIterator
         private void createDeserializer()
         {
             assert file != null && deserializer == null;
-            deserializer = UnfilteredDeserializer.create(sstable.metadata, file, sstable.header, helper);
+            deserializer = UnfilteredDeserializer.create(metadata, file, sstable.header, helper);
         }
 
         protected void seekToPosition(long position) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
index 5d9ca37..3e6cc27 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
@@ -199,7 +199,7 @@ public class SSTableIterator extends AbstractSSTableIterator
         private ForwardIndexedReader(RowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
         {
             super(file, shouldCloseFile);
-            this.indexState = new IndexState(this, sstable.metadata.comparator, indexEntry, false, ifile);
+            this.indexState = new IndexState(this, metadata.comparator, indexEntry, false, ifile);
             this.lastBlockIdx = indexState.blocksCount(); // if we never call setForSlice, that's where we want to stop
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index 4de234c..6443a01 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.db.columniterator;
 import java.io.IOException;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.partitions.ImmutableBTreePartition;
@@ -28,6 +27,7 @@ import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.btree.BTree;
 
 /**
@@ -88,7 +88,7 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         protected ReusablePartitionData createBuffer(int blocksCount)
         {
             int estimatedRowCount = 16;
-            int columnCount = metadata().partitionColumns().regulars.size();
+            int columnCount = metadata().regularColumns().size();
             if (columnCount == 0 || metadata().clusteringColumns().isEmpty())
             {
                 estimatedRowCount = 1;
@@ -222,7 +222,7 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
         private ReverseIndexedReader(RowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
         {
             super(file, shouldCloseFile);
-            this.indexState = new IndexState(this, sstable.metadata.comparator, indexEntry, true, ifile);
+            this.indexState = new IndexState(this, metadata.comparator, indexEntry, true, ifile);
         }
 
         @Override
@@ -321,18 +321,18 @@ public class SSTableReversedIterator extends AbstractSSTableIterator
 
     private class ReusablePartitionData
     {
-        private final CFMetaData metadata;
+        private final TableMetadata metadata;
         private final DecoratedKey partitionKey;
-        private final PartitionColumns columns;
+        private final RegularAndStaticColumns columns;
 
         private MutableDeletionInfo.Builder deletionBuilder;
         private MutableDeletionInfo deletionInfo;
         private BTree.Builder<Row> rowBuilder;
         private ImmutableBTreePartition built;
 
-        private ReusablePartitionData(CFMetaData metadata,
+        private ReusablePartitionData(TableMetadata metadata,
                                       DecoratedKey partitionKey,
-                                      PartitionColumns columns,
+                                      RegularAndStaticColumns columns,
                                       int initialRowCapacity)
         {
             this.metadata = metadata;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
index 0ab941b..7e94911 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
@@ -32,8 +32,10 @@ import org.slf4j.LoggerFactory;
 import net.nicoulaj.compilecommand.annotations.DontInline;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.schema.Schema;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
@@ -272,7 +274,7 @@ public abstract class AbstractCommitLogSegmentManager
      *
      * Flushes any dirty CFs for this segment and any older segments, and then discards the segments
      */
-    void forceRecycleAll(Iterable<UUID> droppedCfs)
+    void forceRecycleAll(Iterable<TableId> droppedTables)
     {
         List<CommitLogSegment> segmentsToRecycle = new ArrayList<>(activeSegments);
         CommitLogSegment last = segmentsToRecycle.get(segmentsToRecycle.size() - 1);
@@ -292,8 +294,8 @@ public abstract class AbstractCommitLogSegmentManager
             future.get();
 
             for (CommitLogSegment segment : activeSegments)
-                for (UUID cfId : droppedCfs)
-                    segment.markClean(cfId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition());
+                for (TableId tableId : droppedTables)
+                    segment.markClean(tableId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition());
 
             // now recycle segments that are unused, as we may not have triggered a discardCompletedSegments()
             // if the previous active segment was the only one to recycle (since an active segment isn't
@@ -367,27 +369,26 @@ public abstract class AbstractCommitLogSegmentManager
         final CommitLogPosition maxCommitLogPosition = segments.get(segments.size() - 1).getCurrentCommitLogPosition();
 
         // a map of CfId -> forceFlush() to ensure we only queue one flush per cf
-        final Map<UUID, ListenableFuture<?>> flushes = new LinkedHashMap<>();
+        final Map<TableId, ListenableFuture<?>> flushes = new LinkedHashMap<>();
 
         for (CommitLogSegment segment : segments)
         {
-            for (UUID dirtyCFId : segment.getDirtyCFIDs())
+            for (TableId dirtyTableId : segment.getDirtyTableIds())
             {
-                Pair<String,String> pair = Schema.instance.getCF(dirtyCFId);
-                if (pair == null)
+                TableMetadata metadata = Schema.instance.getTableMetadata(dirtyTableId);
+                if (metadata == null)
                 {
                     // even though we remove the schema entry before a final flush when dropping a CF,
                     // it's still possible for a writer to race and finish his append after the flush.
-                    logger.trace("Marking clean CF {} that doesn't exist anymore", dirtyCFId);
-                    segment.markClean(dirtyCFId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition());
+                    logger.trace("Marking clean CF {} that doesn't exist anymore", dirtyTableId);
+                    segment.markClean(dirtyTableId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition());
                 }
-                else if (!flushes.containsKey(dirtyCFId))
+                else if (!flushes.containsKey(dirtyTableId))
                 {
-                    String keyspace = pair.left;
-                    final ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(dirtyCFId);
+                    final ColumnFamilyStore cfs = Keyspace.open(metadata.keyspace).getColumnFamilyStore(dirtyTableId);
                     // can safely call forceFlush here as we will only ever block (briefly) for other attempts to flush,
                     // no deadlock possibility since switchLock removal
-                    flushes.put(dirtyCFId, force ? cfs.forceFlush() : cfs.forceFlush(maxCommitLogPosition));
+                    flushes.put(dirtyTableId, force ? cfs.forceFlush() : cfs.forceFlush(maxCommitLogPosition));
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index 750fabc..e93a131 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -44,6 +44,7 @@ import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.metrics.CommitLogMetrics;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.CompressionParams;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.security.EncryptionContext;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
@@ -205,9 +206,9 @@ public class CommitLog implements CommitLogMBean
     /**
      * Flushes all dirty CFs, waiting for them to free and recycle any segments they were retaining
      */
-    public void forceRecycleAllSegments(Iterable<UUID> droppedCfs)
+    public void forceRecycleAllSegments(Iterable<TableId> droppedTables)
     {
-        segmentManager.forceRecycleAll(droppedCfs);
+        segmentManager.forceRecycleAll(droppedTables);
     }
 
     /**
@@ -215,7 +216,7 @@ public class CommitLog implements CommitLogMBean
      */
     public void forceRecycleAllSegments()
     {
-        segmentManager.forceRecycleAll(Collections.<UUID>emptyList());
+        segmentManager.forceRecycleAll(Collections.emptyList());
     }
 
     /**
@@ -295,13 +296,13 @@ public class CommitLog implements CommitLogMBean
      * Modifies the per-CF dirty cursors of any commit log segments for the column family according to the position
      * given. Discards any commit log segments that are no longer used.
      *
-     * @param cfId    the column family ID that was flushed
+     * @param id         the table that was flushed
      * @param lowerBound the lowest covered replay position of the flush
      * @param lowerBound the highest covered replay position of the flush
      */
-    public void discardCompletedSegments(final UUID cfId, final CommitLogPosition lowerBound, final CommitLogPosition upperBound)
+    public void discardCompletedSegments(final TableId id, final CommitLogPosition lowerBound, final CommitLogPosition upperBound)
     {
-        logger.trace("discard completed log segments for {}-{}, table {}", lowerBound, upperBound, cfId);
+        logger.trace("discard completed log segments for {}-{}, table {}", lowerBound, upperBound, id);
 
         // Go thru the active segment files, which are ordered oldest to newest, marking the
         // flushed CF as clean, until we reach the segment file containing the CommitLogPosition passed
@@ -310,7 +311,7 @@ public class CommitLog implements CommitLogMBean
         for (Iterator<CommitLogSegment> iter = segmentManager.getActiveSegments().iterator(); iter.hasNext();)
         {
             CommitLogSegment segment = iter.next();
-            segment.markClean(cfId, lowerBound, upperBound);
+            segment.markClean(id, lowerBound, upperBound);
 
             if (segment.isUnused())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
index d25609a..1da0cee 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReader.java
@@ -29,15 +29,16 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.UnknownColumnFamilyException;
 import org.apache.cassandra.db.commitlog.CommitLogReadHandler.CommitLogReadErrorReason;
 import org.apache.cassandra.db.commitlog.CommitLogReadHandler.CommitLogReadException;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.SerializationHelper;
+import org.apache.cassandra.exceptions.UnknownTableException;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.io.util.RebufferingInputStream;
+import org.apache.cassandra.schema.TableId;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt;
@@ -51,7 +52,7 @@ public class CommitLogReader
     @VisibleForTesting
     public static final int ALL_MUTATIONS = -1;
     private final CRC32 checksum;
-    private final Map<UUID, AtomicInteger> invalidMutations;
+    private final Map<TableId, AtomicInteger> invalidMutations;
 
     private byte[] buffer;
 
@@ -62,7 +63,7 @@ public class CommitLogReader
         buffer = new byte[4096];
     }
 
-    public Set<Map.Entry<UUID, AtomicInteger>> getInvalidMutations()
+    public Set<Map.Entry<TableId, AtomicInteger>> getInvalidMutations()
     {
         return invalidMutations.entrySet();
     }
@@ -367,15 +368,15 @@ public class CommitLogReader
             for (PartitionUpdate upd : mutation.getPartitionUpdates())
                 upd.validate();
         }
-        catch (UnknownColumnFamilyException ex)
+        catch (UnknownTableException ex)
         {
-            if (ex.cfId == null)
+            if (ex.id == null)
                 return;
-            AtomicInteger i = invalidMutations.get(ex.cfId);
+            AtomicInteger i = invalidMutations.get(ex.id);
             if (i == null)
             {
                 i = new AtomicInteger(1);
-                invalidMutations.put(ex.cfId, i);
+                invalidMutations.put(ex.id, i);
             }
             else
                 i.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/af3fe39d/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index 4d2971f..961107c 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -21,15 +21,12 @@ package org.apache.cassandra.db.commitlog;
 import java.io.File;
 import java.io.IOException;
 import java.util.*;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
-import com.google.common.base.Throwables;
 import com.google.common.collect.*;
-import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.commons.lang3.StringUtils;
 import org.cliffc.high_scale_lib.NonBlockingHashSet;
 import org.slf4j.Logger;
@@ -37,10 +34,11 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadataRef;
 import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
@@ -64,7 +62,7 @@ public class CommitLogReplayer implements CommitLogReadHandler
     private final Queue<Future<Integer>> futures;
 
     private final AtomicInteger replayedCount;
-    private final Map<UUID, IntervalSet<CommitLogPosition>> cfPersisted;
+    private final Map<TableId, IntervalSet<CommitLogPosition>> cfPersisted;
     private final CommitLogPosition globalPosition;
 
     // Used to throttle speed of replay of mutations if we pass the max outstanding count
@@ -78,11 +76,11 @@ public class CommitLogReplayer implements CommitLogReadHandler
 
     CommitLogReplayer(CommitLog commitLog,
                       CommitLogPosition globalPosition,
-                      Map<UUID, IntervalSet<CommitLogPosition>> cfPersisted,
+                      Map<TableId, IntervalSet<CommitLogPosition>> cfPersisted,
                       ReplayFilter replayFilter)
     {
-        this.keyspacesReplayed = new NonBlockingHashSet<Keyspace>();
-        this.futures = new ArrayDeque<Future<Integer>>();
+        this.keyspacesReplayed = new NonBlockingHashSet<>();
+        this.futures = new ArrayDeque<>();
         // count the number of replayed mutation. We don't really care about atomicity, but we need it to be a reference.
         this.replayedCount = new AtomicInteger();
         this.cfPersisted = cfPersisted;
@@ -95,35 +93,35 @@ public class CommitLogReplayer implements CommitLogReadHandler
     public static CommitLogReplayer construct(CommitLog commitLog)
     {
         // compute per-CF and global replay intervals
-        Map<UUID, IntervalSet<CommitLogPosition>> cfPersisted = new HashMap<>();
+        Map<TableId, IntervalSet<CommitLogPosition>> cfPersisted = new HashMap<>();
         ReplayFilter replayFilter = ReplayFilter.create();
 
         for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
         {
             // but, if we've truncated the cf in question, then we need to need to start replay after the truncation
-            CommitLogPosition truncatedAt = SystemKeyspace.getTruncatedPosition(cfs.metadata.cfId);
+            CommitLogPosition truncatedAt = SystemKeyspace.getTruncatedPosition(cfs.metadata.id);
             if (truncatedAt != null)
             {
                 // Point in time restore is taken to mean that the tables need to be replayed even if they were
                 // deleted at a later point in time. Any truncation record after that point must thus be cleared prior
                 // to replay (CASSANDRA-9195).
                 long restoreTime = commitLog.archiver.restorePointInTime;
-                long truncatedTime = SystemKeyspace.getTruncatedAt(cfs.metadata.cfId);
+                long truncatedTime = SystemKeyspace.getTruncatedAt(cfs.metadata.id);
                 if (truncatedTime > restoreTime)
                 {
                     if (replayFilter.includes(cfs.metadata))
                     {
                         logger.info("Restore point in time is before latest truncation of table {}.{}. Clearing truncation record.",
-                                    cfs.metadata.ksName,
-                                    cfs.metadata.cfName);
-                        SystemKeyspace.removeTruncationRecord(cfs.metadata.cfId);
+                                    cfs.metadata.keyspace,
+                                    cfs.metadata.name);
+                        SystemKeyspace.removeTruncationRecord(cfs.metadata.id);
                         truncatedAt = null;
                     }
                 }
             }
 
             IntervalSet<CommitLogPosition> filter = persistedIntervals(cfs.getLiveSSTables(), truncatedAt);
-            cfPersisted.put(cfs.metadata.cfId, filter);
+            cfPersisted.put(cfs.metadata.id, filter);
         }
         CommitLogPosition globalPosition = firstNotCovered(cfPersisted.values());
         logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPersisted));
@@ -146,7 +144,7 @@ public class CommitLogReplayer implements CommitLogReadHandler
      */
     public int blockForWrites()
     {
-        for (Map.Entry<UUID, AtomicInteger> entry : commitLogReader.getInvalidMutations())
+        for (Map.Entry<TableId, AtomicInteger> entry : commitLogReader.getInvalidMutations())
             logger.warn("Skipped {} mutations from unknown (probably removed) CF with id {}", entry.getValue(), entry.getKey());
 
         // wait for all the writes to finish on the mutation stage
@@ -192,7 +190,7 @@ public class CommitLogReplayer implements CommitLogReadHandler
             {
                 public void runMayThrow()
                 {
-                    if (Schema.instance.getKSMetaData(mutation.getKeyspaceName()) == null)
+                    if (Schema.instance.getKeyspaceMetadata(mutation.getKeyspaceName()) == null)
                         return;
                     if (commitLogReplayer.pointInTimeExceeded(mutation))
                         return;
@@ -207,12 +205,12 @@ public class CommitLogReplayer implements CommitLogReadHandler
                     Mutation newMutation = null;
                     for (PartitionUpdate update : commitLogReplayer.replayFilter.filter(mutation))
                     {
-                        if (Schema.instance.getCF(update.metadata().cfId) == null)
+                        if (Schema.instance.getTableMetadata(update.metadata().id) == null)
                             continue; // dropped
 
                         // replay if current segment is newer than last flushed one or,
                         // if it is the last known segment, if we are after the commit log segment position
-                        if (commitLogReplayer.shouldReplay(update.metadata().cfId, new CommitLogPosition(segmentId, entryLocation)))
+                        if (commitLogReplayer.shouldReplay(update.metadata().id, new CommitLogPosition(segmentId, entryLocation)))
                         {
                             if (newMutation == null)
                                 newMutation = new Mutation(mutation.getKeyspaceName(), mutation.key());
@@ -272,7 +270,7 @@ public class CommitLogReplayer implements CommitLogReadHandler
     {
         public abstract Iterable<PartitionUpdate> filter(Mutation mutation);
 
-        public abstract boolean includes(CFMetaData metadata);
+        public abstract boolean includes(TableMetadataRef metadata);
 
         public static ReplayFilter create()
         {
@@ -307,7 +305,7 @@ public class CommitLogReplayer implements CommitLogReadHandler
             return mutation.getPartitionUpdates();
         }
 
-        public boolean includes(CFMetaData metadata)
+        public boolean includes(TableMetadataRef metadata)
         {
             return true;
         }
@@ -332,14 +330,14 @@ public class CommitLogReplayer implements CommitLogReadHandler
             {
                 public boolean apply(PartitionUpdate upd)
                 {
-                    return cfNames.contains(upd.metadata().cfName);
+                    return cfNames.contains(upd.metadata().name);
                 }
             });
         }
 
-        public boolean includes(CFMetaData metadata)
+        public boolean includes(TableMetadataRef metadata)
         {
-            return toReplay.containsEntry(metadata.ksName, metadata.cfName);
+            return toReplay.containsEntry(metadata.keyspace, metadata.name);
         }
     }
 
@@ -349,9 +347,9 @@ public class CommitLogReplayer implements CommitLogReadHandler
      *
      * @return true iff replay is necessary
      */
-    private boolean shouldReplay(UUID cfId, CommitLogPosition position)
+    private boolean shouldReplay(TableId tableId, CommitLogPosition position)
     {
-        return !cfPersisted.get(cfId).contains(position);
+        return !cfPersisted.get(tableId).contains(position);
     }
 
     protected boolean pointInTimeExceeded(Mutation fm)