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:09 UTC

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

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));
+        }
+    }
+}