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

[2/2] cassandra git commit: Cleanup and move hardcoded system tables away from CFMetaData

Cleanup and move hardcoded system tables away from CFMetaData

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


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

Branch: refs/heads/trunk
Commit: 611d1baecc448cde9f4e359e639287fed6f8e357
Parents: f096eb6
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Nov 12 15:16:15 2014 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Nov 12 15:16:15 2014 +0300

----------------------------------------------------------------------
 .../apache/cassandra/cache/AutoSavingCache.java |   6 +-
 .../org/apache/cassandra/config/CFMetaData.java | 240 +--------
 .../cassandra/config/ColumnDefinition.java      |  10 +-
 .../cassandra/config/DatabaseDescriptor.java    |  14 +-
 .../org/apache/cassandra/config/KSMetaData.java |  54 +--
 .../org/apache/cassandra/config/Schema.java     |  25 +-
 .../cassandra/config/TriggerDefinition.java     |  10 +-
 .../org/apache/cassandra/config/UTMetaData.java |  14 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |   3 +-
 .../cassandra/cql3/functions/Functions.java     |   3 +-
 .../cassandra/cql3/functions/UDFunction.java    |  15 +-
 .../cql3/statements/AlterKeyspaceStatement.java |   4 +-
 .../apache/cassandra/db/AtomicBTreeColumns.java |   2 +-
 .../apache/cassandra/db/BatchlogManager.java    |  23 +-
 .../org/apache/cassandra/db/DefsTables.java     |  31 +-
 .../cassandra/db/HintedHandOffManager.java      |  25 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |  11 +-
 src/java/org/apache/cassandra/db/Memtable.java  |   2 +-
 .../org/apache/cassandra/db/SystemKeyspace.java | 482 ++++++++++++++-----
 .../io/sstable/format/SSTableReader.java        |   2 +-
 .../cassandra/service/CassandraDaemon.java      |   4 +-
 .../apache/cassandra/service/ClientState.java   |  11 +-
 .../cassandra/service/MigrationManager.java     |   4 +-
 .../apache/cassandra/service/StorageProxy.java  |  10 +-
 .../cassandra/service/StorageService.java       |  25 +-
 .../cassandra/thrift/ThriftValidation.java      |   2 +-
 .../org/apache/cassandra/tools/BulkLoader.java  |  11 +-
 .../org/apache/cassandra/tools/NodeTool.java    |   6 +-
 .../apache/cassandra/tracing/TraceKeyspace.java |  73 +++
 .../apache/cassandra/tracing/TraceState.java    |   4 +-
 .../org/apache/cassandra/tracing/Tracing.java   |  26 +-
 .../apache/cassandra/config/CFMetaDataTest.java |   4 +-
 .../cassandra/db/BatchlogManagerTest.java       |   4 +-
 .../apache/cassandra/db/HintedHandOffTest.java  |   8 +-
 .../cassandra/db/RecoveryManager2Test.java      |   2 +-
 .../service/StorageServiceServerTest.java       |   3 +-
 36 files changed, 599 insertions(+), 574 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/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 4aeb822..9a79e38 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -30,12 +30,12 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.*;
 import org.apache.cassandra.service.CacheService;
@@ -198,7 +198,7 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
             else
                 type = OperationType.UNKNOWN;
 
-            info = new CompactionInfo(CFMetaData.denseCFMetaData(Keyspace.SYSTEM_KS, cacheType.toString(), BytesType.instance),
+            info = new CompactionInfo(CFMetaData.denseCFMetaData(SystemKeyspace.NAME, cacheType.toString(), BytesType.instance),
                                       type,
                                       0,
                                       keys.size(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index a8d528e..14271c0 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -50,7 +50,6 @@ import org.apache.cassandra.io.compress.CompressionParameters;
 import org.apache.cassandra.io.compress.LZ4Compressor;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
@@ -59,9 +58,6 @@ import static org.apache.cassandra.utils.FBUtilities.fromJsonList;
 import static org.apache.cassandra.utils.FBUtilities.fromJsonMap;
 import static org.apache.cassandra.utils.FBUtilities.json;
 
-/**
- * This class can be tricky to modify. Please read http://wiki.apache.org/cassandra/ConfigurationNotes for how to do so safely.
- */
 public final class CFMetaData
 {
     private static final Logger logger = LoggerFactory.getLogger(CFMetaData.class);
@@ -90,214 +86,6 @@ public final class CFMetaData
         }
     };
 
-    public static final CFMetaData IndexCf = compile("CREATE TABLE \"" + SystemKeyspace.INDEX_CF + "\" ("
-                                                     + "table_name text,"
-                                                     + "index_name text,"
-                                                     + "PRIMARY KEY (table_name, index_name)"
-                                                     + ") WITH COMPACT STORAGE AND COMMENT='indexes that have been completed'");
-
-    public static final CFMetaData SchemaKeyspacesCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_KEYSPACES_CF + " ("
-                                                               + "keyspace_name text PRIMARY KEY,"
-                                                               + "durable_writes boolean,"
-                                                               + "strategy_class text,"
-                                                               + "strategy_options text"
-                                                               + ") WITH COMPACT STORAGE AND COMMENT='keyspace definitions' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData SchemaColumnFamiliesCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF + " ("
-                                                                    + "keyspace_name text,"
-                                                                    + "columnfamily_name text,"
-                                                                    + "cf_id uuid," // post-2.1 UUID cfid
-                                                                    + "type text,"
-                                                                    + "is_dense boolean,"
-                                                                    + "comparator text,"
-                                                                    + "subcomparator text,"
-                                                                    + "comment text,"
-                                                                    + "read_repair_chance double,"
-                                                                    + "local_read_repair_chance double,"
-                                                                    + "gc_grace_seconds int,"
-                                                                    + "default_validator text,"
-                                                                    + "key_validator text,"
-                                                                    + "min_compaction_threshold int,"
-                                                                    + "max_compaction_threshold int,"
-                                                                    + "memtable_flush_period_in_ms int,"
-                                                                    + "key_aliases text,"
-                                                                    + "bloom_filter_fp_chance double,"
-                                                                    + "caching text,"
-                                                                    + "default_time_to_live int,"
-                                                                    + "compaction_strategy_class text,"
-                                                                    + "compression_parameters text,"
-                                                                    + "value_alias text,"
-                                                                    + "column_aliases text,"
-                                                                    + "compaction_strategy_options text,"
-                                                                    + "speculative_retry text,"
-                                                                    + "index_interval int,"
-                                                                    + "min_index_interval int,"
-                                                                    + "max_index_interval int,"
-                                                                    + "dropped_columns map<text, bigint>,"
-                                                                    + "PRIMARY KEY (keyspace_name, columnfamily_name)"
-                                                                    + ") WITH COMMENT='ColumnFamily definitions' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData SchemaColumnsCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_COLUMNS_CF + " ("
-                                                             + "keyspace_name text,"
-                                                             + "columnfamily_name text,"
-                                                             + "column_name text,"
-                                                             + "validator text,"
-                                                             + "index_type text,"
-                                                             + "index_options text,"
-                                                             + "index_name text,"
-                                                             + "component_index int,"
-                                                             + "type text,"
-                                                             + "PRIMARY KEY(keyspace_name, columnfamily_name, column_name)"
-                                                             + ") WITH COMMENT='ColumnFamily column attributes' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData SchemaTriggersCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_TRIGGERS_CF + " ("
-                                                              + "keyspace_name text,"
-                                                              + "columnfamily_name text,"
-                                                              + "trigger_name text,"
-                                                              + "trigger_options map<text, text>,"
-                                                              + "PRIMARY KEY (keyspace_name, columnfamily_name, trigger_name)"
-                                                              + ") WITH COMMENT='triggers metadata table' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData SchemaUserTypesCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_USER_TYPES_CF + " ("
-                                                               + "keyspace_name text,"
-                                                               + "type_name text,"
-                                                               + "field_names list<text>,"
-                                                               + "field_types list<text>,"
-                                                               + "PRIMARY KEY (keyspace_name, type_name)"
-                                                               + ") WITH COMMENT='Defined user types' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData SchemaFunctionsCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_FUNCTIONS_CF + " ("
-                                                               + "namespace text,"
-                                                               + "name text,"
-                                                               + "signature blob,"
-                                                               + "argument_names list<text>,"
-                                                               + "argument_types list<text>,"
-                                                               + "return_type text,"
-                                                               + "deterministic boolean,"
-                                                               + "language text,"
-                                                               + "body text,"
-                                                               + "primary key ((namespace, name), signature)"
-                                                               + ") WITH COMMENT='user defined functions' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData HintsCf = compile("CREATE TABLE " + SystemKeyspace.HINTS_CF + " ("
-                                                     + "target_id uuid,"
-                                                     + "hint_id timeuuid,"
-                                                     + "message_version int,"
-                                                     + "mutation blob,"
-                                                     + "PRIMARY KEY (target_id, hint_id, message_version)"
-                                                     + ") WITH COMPACT STORAGE "
-                                                     + "AND COMPACTION={'class' : 'SizeTieredCompactionStrategy', 'enabled' : false} "
-                                                     + "AND COMMENT='hints awaiting delivery'"
-                                                     + "AND gc_grace_seconds=0");
-
-    public static final CFMetaData PeersCf = compile("CREATE TABLE " + SystemKeyspace.PEERS_CF + " ("
-                                                     + "peer inet PRIMARY KEY,"
-                                                     + "host_id uuid,"
-                                                     + "tokens set<varchar>,"
-                                                     + "schema_version uuid,"
-                                                     + "release_version text,"
-                                                     + "rpc_address inet,"
-                                                     + "preferred_ip inet,"
-                                                     + "data_center text,"
-                                                     + "rack text"
-                                                     + ") WITH COMMENT='known peers in the cluster'");
-
-    public static final CFMetaData PeerEventsCf = compile("CREATE TABLE " + SystemKeyspace.PEER_EVENTS_CF + " ("
-                                                          + "peer inet PRIMARY KEY,"
-                                                          + "hints_dropped map<uuid, int>"
-                                                          + ") WITH COMMENT='cf contains events related to peers'");
-
-    public static final CFMetaData LocalCf = compile("CREATE TABLE " + SystemKeyspace.LOCAL_CF + " ("
-                                                     + "key text PRIMARY KEY,"
-                                                     + "tokens set<varchar>,"
-                                                     + "cluster_name text,"
-                                                     + "gossip_generation int,"
-                                                     + "bootstrapped text,"
-                                                     + "host_id uuid,"
-                                                     + "release_version text,"
-                                                     + "thrift_version text,"
-                                                     + "cql_version text,"
-                                                     + "native_protocol_version text,"
-                                                     + "data_center text,"
-                                                     + "rack text,"
-                                                     + "partitioner text,"
-                                                     + "schema_version uuid,"
-                                                     + "truncated_at map<uuid, blob>"
-                                                     + ") WITH COMMENT='information about the local node'");
-
-    public static final CFMetaData TraceSessionsCf = compile("CREATE TABLE " + Tracing.SESSIONS_CF + " ("
-                                                             + "session_id uuid PRIMARY KEY,"
-                                                             + "coordinator inet,"
-                                                             + "request text,"
-                                                             + "started_at timestamp,"
-                                                             + "parameters map<text, text>,"
-                                                             + "duration int"
-                                                             + ") WITH COMMENT='traced sessions'",
-                                                             Tracing.TRACE_KS);
-
-    public static final CFMetaData TraceEventsCf = compile("CREATE TABLE " + Tracing.EVENTS_CF + " ("
-                                                           + "session_id uuid,"
-                                                           + "event_id timeuuid,"
-                                                           + "source inet,"
-                                                           + "thread text,"
-                                                           + "activity text,"
-                                                           + "source_elapsed int,"
-                                                           + "PRIMARY KEY (session_id, event_id)"
-                                                           + ")",
-                                                           Tracing.TRACE_KS);
-
-    public static final CFMetaData BatchlogCf = compile("CREATE TABLE " + SystemKeyspace.BATCHLOG_CF + " ("
-                                                        + "id uuid PRIMARY KEY,"
-                                                        + "written_at timestamp,"
-                                                        + "data blob,"
-                                                        + "version int,"
-                                                        + ") WITH COMMENT='uncommited batches' AND gc_grace_seconds=0 "
-                                                        + "AND COMPACTION={'class' : 'SizeTieredCompactionStrategy', 'min_threshold' : 2}");
-
-    public static final CFMetaData RangeXfersCf = compile("CREATE TABLE " + SystemKeyspace.RANGE_XFERS_CF + " ("
-                                                          + "token_bytes blob PRIMARY KEY,"
-                                                          + "requested_at timestamp"
-                                                          + ") WITH COMMENT='ranges requested for transfer here'");
-
-    public static final CFMetaData CompactionLogCf = compile("CREATE TABLE " + SystemKeyspace.COMPACTION_LOG + " ("
-                                                             + "id uuid PRIMARY KEY,"
-                                                             + "keyspace_name text,"
-                                                             + "columnfamily_name text,"
-                                                             + "inputs set<int>"
-                                                             + ") WITH COMMENT='unfinished compactions'");
-
-    public static final CFMetaData PaxosCf = compile("CREATE TABLE " + SystemKeyspace.PAXOS_CF + " ("
-                                                     + "row_key blob,"
-                                                     + "cf_id UUID,"
-                                                     + "in_progress_ballot timeuuid,"
-                                                     + "proposal_ballot timeuuid,"
-                                                     + "proposal blob,"
-                                                     + "most_recent_commit_at timeuuid,"
-                                                     + "most_recent_commit blob,"
-                                                     + "PRIMARY KEY (row_key, cf_id)"
-                                                     + ") WITH COMMENT='in-progress paxos proposals' "
-                                                     + "AND COMPACTION={'class' : 'LeveledCompactionStrategy'}");
-
-    public static final CFMetaData SSTableActivityCF = compile("CREATE TABLE " + SystemKeyspace.SSTABLE_ACTIVITY_CF + " ("
-                                                               + "keyspace_name text,"
-                                                               + "columnfamily_name text,"
-                                                               + "generation int,"
-                                                               + "rate_15m double,"
-                                                               + "rate_120m double,"
-                                                               + "PRIMARY KEY ((keyspace_name, columnfamily_name, generation))"
-                                                               + ") WITH COMMENT='historic sstable read rates'");
-
-    public static final CFMetaData CompactionHistoryCf = compile("CREATE TABLE " + SystemKeyspace.COMPACTION_HISTORY_CF + " ("
-                                                                 + "id uuid,"
-                                                                 + "keyspace_name text,"
-                                                                 + "columnfamily_name text,"
-                                                                 + "compacted_at timestamp,"
-                                                                 + "bytes_in bigint,"
-                                                                 + "bytes_out bigint,"
-                                                                 + "rows_merged map<int, bigint>,"
-                                                                 + "PRIMARY KEY (id)"
-                                                                 + ") WITH COMMENT='show all compaction history' AND DEFAULT_TIME_TO_LIVE=604800");
-
     public static class SpeculativeRetry
     {
         public enum RetryType
@@ -500,12 +288,6 @@ public final class CFMetaData
         return triggers;
     }
 
-    private static CFMetaData compile(String cql)
-    {
-        return compile(cql, Keyspace.SYSTEM_KS);
-    }
-
-    @VisibleForTesting
     public static CFMetaData compile(String cql, String keyspace)
     {
         try
@@ -937,7 +719,7 @@ public final class CFMetaData
 
     public void reload()
     {
-        Row cfDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, ksName, cfName);
+        Row cfDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, ksName, cfName);
 
         if (cfDefRow.cf == null || !cfDefRow.cf.hasColumns())
             throw new RuntimeException(String.format("%s not found in the schema definitions keyspace.", ksName + ":" + cfName));
@@ -1343,7 +1125,7 @@ public final class CFMetaData
      */
     public Mutation toSchemaUpdate(CFMetaData newState, long modificationTimestamp, boolean fromThrift)
     {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(ksName));
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(ksName));
 
         newState.toSchemaNoColumnsNoTriggers(mutation, modificationTimestamp);
 
@@ -1393,11 +1175,11 @@ public final class CFMetaData
      */
     public Mutation dropFromSchema(long timestamp)
     {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(ksName));
-        ColumnFamily cf = mutation.addOrGet(SchemaColumnFamiliesCf);
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(ksName));
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnFamiliesTable);
         int ldt = (int) (System.currentTimeMillis() / 1000);
 
-        Composite prefix = SchemaColumnFamiliesCf.comparator.make(cfName);
+        Composite prefix = SystemKeyspace.SchemaColumnFamiliesTable.comparator.make(cfName);
         cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
 
         for (ColumnDefinition cd : allColumns())
@@ -1408,7 +1190,7 @@ public final class CFMetaData
 
         for (String indexName : Keyspace.open(this.ksName).getColumnFamilyStore(this.cfName).getBuiltIndexes())
         {
-            ColumnFamily indexCf = mutation.addOrGet(IndexCf);
+            ColumnFamily indexCf = mutation.addOrGet(SystemKeyspace.BuiltIndexesTable);
             indexCf.addTombstone(indexCf.getComparator().makeCellName(indexName), ldt, timestamp);
         }
 
@@ -1440,8 +1222,8 @@ public final class CFMetaData
     {
         // For property that can be null (and can be changed), we insert tombstones, to make sure
         // we don't keep a property the user has removed
-        ColumnFamily cf = mutation.addOrGet(SchemaColumnFamiliesCf);
-        Composite prefix = SchemaColumnFamiliesCf.comparator.make(cfName);
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnFamiliesTable);
+        Composite prefix = SystemKeyspace.SchemaColumnFamiliesTable.comparator.make(cfName);
         CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
 
         adder.add("cf_id", cfId);
@@ -1618,10 +1400,10 @@ public final class CFMetaData
         String ksName = result.getString("keyspace_name");
         String cfName = result.getString("columnfamily_name");
 
-        Row serializedColumns = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_COLUMNS_CF, ksName, cfName);
+        Row serializedColumns = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_COLUMNS_TABLE, ksName, cfName);
         CFMetaData cfm = fromSchemaNoTriggers(result, ColumnDefinition.resultify(serializedColumns));
 
-        Row serializedTriggers = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_TRIGGERS_CF, ksName, cfName);
+        Row serializedTriggers = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_TRIGGERS_TABLE, ksName, cfName);
         addTriggerDefinitionsFromSchema(cfm, serializedTriggers);
 
         return cfm;
@@ -1671,7 +1453,7 @@ public final class CFMetaData
      */
     public Mutation toSchema(long timestamp) throws ConfigurationException
     {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(ksName));
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(ksName));
         toSchema(mutation, timestamp);
         return mutation;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index 3f3ecd5..10a5a8b 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -252,18 +252,18 @@ public class ColumnDefinition extends ColumnSpecification
      */
     public void deleteFromSchema(Mutation mutation, long timestamp)
     {
-        ColumnFamily cf = mutation.addOrGet(CFMetaData.SchemaColumnsCf);
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnsTable);
         int ldt = (int) (System.currentTimeMillis() / 1000);
 
         // Note: we do want to use name.toString(), not name.bytes directly for backward compatibility (For CQL3, this won't make a difference).
-        Composite prefix = CFMetaData.SchemaColumnsCf.comparator.make(cfName, name.toString());
+        Composite prefix = SystemKeyspace.SchemaColumnsTable.comparator.make(cfName, name.toString());
         cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
     }
 
     public void toSchema(Mutation mutation, long timestamp)
     {
-        ColumnFamily cf = mutation.addOrGet(CFMetaData.SchemaColumnsCf);
-        Composite prefix = CFMetaData.SchemaColumnsCf.comparator.make(cfName, name.toString());
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnsTable);
+        Composite prefix = SystemKeyspace.SchemaColumnsTable.comparator.make(cfName, name.toString());
         CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
 
         adder.add(TYPE, type.toString());
@@ -303,7 +303,7 @@ public class ColumnDefinition extends ColumnSpecification
 
     public static UntypedResultSet resultify(Row serializedColumns)
     {
-        String query = String.format("SELECT * FROM %s.%s", Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNS_CF);
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.SCHEMA_COLUMNS_TABLE);
         return QueryProcessor.resultify(query, serializedColumns);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 7c8068e..a359cce 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -25,7 +25,6 @@ import java.net.NetworkInterface;
 import java.net.SocketException;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -586,13 +585,10 @@ public class DatabaseDescriptor
             conf.server_encryption_options = conf.encryption_options;
         }
 
-        // Hardcoded system keyspaces
-        List<KSMetaData> systemKeyspaces = Arrays.asList(KSMetaData.systemKeyspace());
-        assert systemKeyspaces.size() == Schema.systemKeyspaceNames.size();
-        for (KSMetaData ksmd : systemKeyspaces)
-            Schema.instance.load(ksmd);
+        // hardcoded system keyspace
+        Schema.instance.load(SystemKeyspace.definition());
 
-        /* Load the seeds for node contact points */
+        // load the seeds for node contact points
         if (conf.seed_provider == null)
         {
             throw new ConfigurationException("seeds configuration is missing; a minimum of one seed is required.");
@@ -627,7 +623,7 @@ public class DatabaseDescriptor
     /** load keyspace (keyspace) definitions, but do not initialize the keyspace instances. */
     public static void loadSchemas()
     {
-        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_KEYSPACES_CF);
+        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_KEYSPACES_TABLE);
 
         // if keyspace with definitions is empty try loading the old way
         if (schemaCFS.estimateKeys() == 0)
@@ -659,7 +655,7 @@ public class DatabaseDescriptor
                 {
                     public boolean accept(File pathname)
                     {
-                        return (pathname.isDirectory() && !Schema.systemKeyspaceNames.contains(pathname.getName()));
+                        return pathname.isDirectory() && !pathname.getName().equals(SystemKeyspace.NAME);
                     }
                 }).length;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/config/KSMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/KSMetaData.java b/src/java/org/apache/cassandra/config/KSMetaData.java
index f30fe74..d3ff62c 100644
--- a/src/java/org/apache/cassandra/config/KSMetaData.java
+++ b/src/java/org/apache/cassandra/config/KSMetaData.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.config;
 import java.util.*;
 
 import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableMap;
 
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -28,7 +27,6 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.*;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.tracing.Tracing;
 
 import static org.apache.cassandra.utils.FBUtilities.*;
 
@@ -89,34 +87,6 @@ public final class KSMetaData
         return new KSMetaData(ksm.name, ksm.strategyClass, ksm.strategyOptions, ksm.durableWrites, cfDefs, ksm.userTypes);
     }
 
-    public static KSMetaData systemKeyspace()
-    {
-        List<CFMetaData> cfDefs = Arrays.asList(CFMetaData.BatchlogCf,
-                                                CFMetaData.RangeXfersCf,
-                                                CFMetaData.LocalCf,
-                                                CFMetaData.PeersCf,
-                                                CFMetaData.PeerEventsCf,
-                                                CFMetaData.HintsCf,
-                                                CFMetaData.IndexCf,
-                                                CFMetaData.SchemaKeyspacesCf,
-                                                CFMetaData.SchemaColumnFamiliesCf,
-                                                CFMetaData.SchemaColumnsCf,
-                                                CFMetaData.SchemaTriggersCf,
-                                                CFMetaData.SchemaUserTypesCf,
-                                                CFMetaData.SchemaFunctionsCf,
-                                                CFMetaData.CompactionLogCf,
-                                                CFMetaData.CompactionHistoryCf,
-                                                CFMetaData.PaxosCf,
-                                                CFMetaData.SSTableActivityCF);
-        return new KSMetaData(Keyspace.SYSTEM_KS, LocalStrategy.class, Collections.<String, String>emptyMap(), true, cfDefs);
-    }
-
-    public static KSMetaData traceKeyspace()
-    {
-        List<CFMetaData> cfDefs = Arrays.asList(CFMetaData.TraceSessionsCf, CFMetaData.TraceEventsCf);
-        return new KSMetaData(Tracing.TRACE_KS, SimpleStrategy.class, ImmutableMap.of("replication_factor", "2"), true, cfDefs);
-    }
-
     public static KSMetaData testMetadata(String name, Class<? extends AbstractReplicationStrategy> strategyClass, Map<String, String> strategyOptions, CFMetaData... cfDefs)
     {
         return new KSMetaData(name, strategyClass, strategyOptions, true, Arrays.asList(cfDefs));
@@ -198,33 +168,33 @@ public final class KSMetaData
 
     public KSMetaData reloadAttributes()
     {
-        Row ksDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_KEYSPACES_CF, name);
+        Row ksDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, name);
 
         if (ksDefRow.cf == null)
-            throw new RuntimeException(String.format("%s not found in the schema definitions keyspaceName (%s).", name, SystemKeyspace.SCHEMA_KEYSPACES_CF));
+            throw new RuntimeException(String.format("%s not found in the schema definitions keyspaceName (%s).", name, SystemKeyspace.SCHEMA_KEYSPACES_TABLE));
 
         return fromSchema(ksDefRow, Collections.<CFMetaData>emptyList(), userTypes);
     }
 
     public Mutation dropFromSchema(long timestamp)
     {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(name));
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(name));
 
-        mutation.delete(SystemKeyspace.SCHEMA_KEYSPACES_CF, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_COLUMNS_CF, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_TRIGGERS_CF, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_USER_TYPES_CF, timestamp);
-        mutation.delete(SystemKeyspace.INDEX_CF, timestamp);
+        mutation.delete(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, timestamp);
+        mutation.delete(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, timestamp);
+        mutation.delete(SystemKeyspace.SCHEMA_COLUMNS_TABLE, timestamp);
+        mutation.delete(SystemKeyspace.SCHEMA_TRIGGERS_TABLE, timestamp);
+        mutation.delete(SystemKeyspace.SCHEMA_USER_TYPES_TABLE, timestamp);
+        mutation.delete(SystemKeyspace.BUILT_INDEXES_TABLE, timestamp);
 
         return mutation;
     }
 
     public Mutation toSchema(long timestamp)
     {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(name));
-        ColumnFamily cf = mutation.addOrGet(CFMetaData.SchemaKeyspacesCf);
-        CFRowAdder adder = new CFRowAdder(cf, CFMetaData.SchemaKeyspacesCf.comparator.builder().build(), timestamp);
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(name));
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaKeyspacesTable);
+        CFRowAdder adder = new CFRowAdder(cf, SystemKeyspace.SchemaKeyspacesTable.comparator.builder().build(), timestamp);
 
         adder.add("durable_writes", durableWrites);
         adder.add("strategy_class", strategyClass.getName());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index fed38d8..43cc6b5 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -23,7 +23,6 @@ import java.security.NoSuchAlgorithmException;
 import java.util.*;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,10 +52,10 @@ public class Schema
     public static final int NAME_LENGTH = 48;
 
     /* metadata map for faster keyspace lookup */
-    private final Map<String, KSMetaData> keyspaces = new NonBlockingHashMap<String, KSMetaData>();
+    private final Map<String, KSMetaData> 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<String, 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<>();
@@ -65,7 +64,6 @@ public class Schema
 
     // 59adb24e-f3cd-3e02-97f0-5b395827453f
     public static final UUID emptyVersion;
-    public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Keyspace.SYSTEM_KS);
 
     static
     {
@@ -213,21 +211,6 @@ public class Schema
     }
 
     /**
-     * Get type of the ColumnFamily but it's keyspace/name
-     *
-     * @param ksName The keyspace name
-     * @param cfName The ColumnFamily name
-     *
-     * @return The type of the ColumnFamily
-     */
-    public ColumnFamilyType getColumnFamilyType(String ksName, String cfName)
-    {
-        assert ksName != null && cfName != null;
-        CFMetaData cfMetaData = getCFMetaData(ksName, cfName);
-        return (cfMetaData == null) ? null : cfMetaData.cfType;
-    }
-
-    /**
      * Get metadata about keyspace by its name
      *
      * @param keyspaceName The name of the keyspace
@@ -245,7 +228,7 @@ public class Schema
      */
     public List<String> getNonSystemKeyspaces()
     {
-        return ImmutableList.copyOf(Sets.difference(keyspaces.keySet(), systemKeyspaceNames));
+        return ImmutableList.copyOf(Sets.difference(keyspaces.keySet(), Collections.singleton(SystemKeyspace.NAME)));
     }
 
     /**
@@ -425,7 +408,7 @@ public class Schema
     {
         try
         {
-            return systemKeyspaceNames.contains(ByteBufferUtil.string(row.key.getKey()));
+            return ByteBufferUtil.string(row.key.getKey()).equals(SystemKeyspace.NAME);
         }
         catch (CharacterCodingException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/config/TriggerDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/TriggerDefinition.java b/src/java/org/apache/cassandra/config/TriggerDefinition.java
index df37cbc..a395549 100644
--- a/src/java/org/apache/cassandra/config/TriggerDefinition.java
+++ b/src/java/org/apache/cassandra/config/TriggerDefinition.java
@@ -60,7 +60,7 @@ public class TriggerDefinition
     public static List<TriggerDefinition> fromSchema(Row serializedTriggers)
     {
         List<TriggerDefinition> triggers = new ArrayList<>();
-        String query = String.format("SELECT * FROM %s.%s", Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_TRIGGERS_CF);
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.SCHEMA_TRIGGERS_TABLE);
         for (UntypedResultSet.Row row : QueryProcessor.resultify(query, serializedTriggers))
         {
             String name = row.getString(TRIGGER_NAME);
@@ -79,9 +79,9 @@ public class TriggerDefinition
      */
     public void toSchema(Mutation mutation, String cfName, long timestamp)
     {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_CF);
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_TABLE);
 
-        CFMetaData cfm = CFMetaData.SchemaTriggersCf;
+        CFMetaData cfm = SystemKeyspace.SchemaTriggersTable;
         Composite prefix = cfm.comparator.make(cfName, name);
         CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
 
@@ -97,10 +97,10 @@ public class TriggerDefinition
      */
     public void deleteFromSchema(Mutation mutation, String cfName, long timestamp)
     {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_CF);
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_TABLE);
         int ldt = (int) (System.currentTimeMillis() / 1000);
 
-        Composite prefix = CFMetaData.SchemaTriggersCf.comparator.make(cfName, name);
+        Composite prefix = SystemKeyspace.SchemaTriggersTable.comparator.make(cfName, name);
         cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/config/UTMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/UTMetaData.java b/src/java/org/apache/cassandra/config/UTMetaData.java
index ee653a8..46a7a4f 100644
--- a/src/java/org/apache/cassandra/config/UTMetaData.java
+++ b/src/java/org/apache/cassandra/config/UTMetaData.java
@@ -75,7 +75,7 @@ public final class UTMetaData
 
     public static Map<ByteBuffer, UserType> fromSchema(Row row)
     {
-        UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_USER_TYPES_CF, row);
+        UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_USER_TYPES_TABLE, row);
         Map<ByteBuffer, UserType> types = new HashMap<>(results.size());
         for (UntypedResultSet.Row result : results)
         {
@@ -87,14 +87,14 @@ public final class UTMetaData
 
     public static Mutation toSchema(UserType newType, long timestamp)
     {
-        return toSchema(new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(newType.keyspace)), newType, timestamp);
+        return toSchema(new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(newType.keyspace)), newType, timestamp);
     }
 
     public static Mutation toSchema(Mutation mutation, UserType newType, long timestamp)
     {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_USER_TYPES_CF);
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_USER_TYPES_TABLE);
 
-        Composite prefix = CFMetaData.SchemaUserTypesCf.comparator.make(newType.name);
+        Composite prefix = SystemKeyspace.SchemaUserTypesTable.comparator.make(newType.name);
         CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
 
         adder.resetCollection("field_names");
@@ -117,11 +117,11 @@ public final class UTMetaData
 
     public static Mutation dropFromSchema(UserType droppedType, long timestamp)
     {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(droppedType.keyspace));
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_USER_TYPES_CF);
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(droppedType.keyspace));
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_USER_TYPES_TABLE);
         int ldt = (int) (System.currentTimeMillis() / 1000);
 
-        Composite prefix = CFMetaData.SchemaUserTypesCf.comparator.make(droppedType.name);
+        Composite prefix = SystemKeyspace.SchemaUserTypesTable.comparator.make(droppedType.name);
         cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
 
         return mutation;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/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 b20a771..680f9f2 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -22,7 +22,6 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.primitives.Ints;
 import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap;
 import com.googlecode.concurrentlinkedhashmap.EntryWeigher;
@@ -147,7 +146,7 @@ public class QueryProcessor implements QueryHandler
             ClientState state = ClientState.forInternalCalls();
             try
             {
-                state.setKeyspace(Keyspace.SYSTEM_KS);
+                state.setKeyspace(SystemKeyspace.NAME);
             }
             catch (InvalidRequestException e)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/cql3/functions/Functions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/Functions.java b/src/java/org/apache/cassandra/cql3/functions/Functions.java
index e8d6181..62de2bc 100644
--- a/src/java/org/apache/cassandra/cql3/functions/Functions.java
+++ b/src/java/org/apache/cassandra/cql3/functions/Functions.java
@@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -40,7 +39,7 @@ public abstract class Functions
     // to handle it as a special case.
     private static final FunctionName TOKEN_FUNCTION_NAME = new FunctionName("token");
 
-    private static final String SELECT_UDFS = "SELECT * FROM " + Keyspace.SYSTEM_KS + '.' + SystemKeyspace.SCHEMA_FUNCTIONS_CF;
+    private static final String SELECT_UDFS = "SELECT * FROM " + SystemKeyspace.NAME + '.' + SystemKeyspace.SCHEMA_FUNCTIONS_TABLE;
 
     private Functions() {}
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/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 3741008..bf011a7 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -26,7 +26,6 @@ import com.google.common.base.Objects;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.Composite;
@@ -153,16 +152,16 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
 
     private static Mutation makeSchemaMutation(FunctionName name)
     {
-        CompositeType kv = (CompositeType)CFMetaData.SchemaFunctionsCf.getKeyValidator();
-        return new Mutation(Keyspace.SYSTEM_KS, kv.decompose(name.namespace, name.name));
+        CompositeType kv = (CompositeType)SystemKeyspace.SchemaFunctionsTable.getKeyValidator();
+        return new Mutation(SystemKeyspace.NAME, kv.decompose(name.namespace, name.name));
     }
 
     public Mutation toSchemaDrop(long timestamp)
     {
         Mutation mutation = makeSchemaMutation(name);
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_FUNCTIONS_CF);
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE);
 
-        Composite prefix = CFMetaData.SchemaFunctionsCf.comparator.make(computeSignature(argTypes));
+        Composite prefix = SystemKeyspace.SchemaFunctionsTable.comparator.make(computeSignature(argTypes));
         int ldt = (int) (System.currentTimeMillis() / 1000);
         cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
 
@@ -172,9 +171,9 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
     public Mutation toSchemaUpdate(long timestamp)
     {
         Mutation mutation = makeSchemaMutation(name);
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_FUNCTIONS_CF);
+        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE);
 
-        Composite prefix = CFMetaData.SchemaFunctionsCf.comparator.make(computeSignature(argTypes));
+        Composite prefix = SystemKeyspace.SchemaFunctionsTable.comparator.make(computeSignature(argTypes));
         CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
 
         adder.resetCollection("argument_names");
@@ -254,7 +253,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
 
     public static Map<ByteBuffer, UDFunction> fromSchema(Row row)
     {
-        UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_FUNCTIONS_CF, row);
+        UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_FUNCTIONS_TABLE, row);
         Map<ByteBuffer, UDFunction> udfs = new HashMap<>(results.size());
         for (UntypedResultSet.Row result : results)
             udfs.put(result.getBlob("signature"), fromSchema(result));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/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 e65a51e..50c3f00 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
@@ -21,7 +21,7 @@ import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.service.ClientState;
@@ -57,7 +57,7 @@ public class AlterKeyspaceStatement extends SchemaAlteringStatement
         KSMetaData ksm = Schema.instance.getKSMetaData(name);
         if (ksm == null)
             throw new InvalidRequestException("Unknown keyspace " + name);
-        if (ksm.name.equalsIgnoreCase(Keyspace.SYSTEM_KS))
+        if (ksm.name.equalsIgnoreCase(SystemKeyspace.NAME))
             throw new InvalidRequestException("Cannot alter system keyspace");
 
         attrs.validate();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
index ec9bb94..7e48287 100644
--- a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
+++ b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
@@ -59,7 +59,7 @@ import static org.apache.cassandra.db.index.SecondaryIndexManager.Updater;
  */
 public class AtomicBTreeColumns extends ColumnFamily
 {
-    static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreeColumns(CFMetaData.IndexCf, null))
+    static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreeColumns(SystemKeyspace.BuiltIndexesTable, null))
             + ObjectSizes.measure(new Holder(null, null));
 
     // Reserved values for wasteTracker field. These values must not be consecutive (see avoidReservedValues)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/db/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BatchlogManager.java b/src/java/org/apache/cassandra/db/BatchlogManager.java
index cf36a64..b37de8b 100644
--- a/src/java/org/apache/cassandra/db/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/db/BatchlogManager.java
@@ -36,7 +36,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.compaction.CompactionManager;
@@ -97,7 +96,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
     public int countAllBatches()
     {
-        String query = String.format("SELECT count(*) FROM %s.%s", Keyspace.SYSTEM_KS, SystemKeyspace.BATCHLOG_CF);
+        String query = String.format("SELECT count(*) FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.BATCHLOG_TABLE);
         return (int) executeInternal(query).one().getLong("count");
     }
 
@@ -132,12 +131,12 @@ public class BatchlogManager implements BatchlogManagerMBean
     @VisibleForTesting
     static Mutation getBatchlogMutationFor(Collection<Mutation> mutations, UUID uuid, int version, long now)
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(CFMetaData.BatchlogCf);
-        CFRowAdder adder = new CFRowAdder(cf, CFMetaData.BatchlogCf.comparator.builder().build(), now);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(SystemKeyspace.BatchlogTable);
+        CFRowAdder adder = new CFRowAdder(cf, SystemKeyspace.BatchlogTable.comparator.builder().build(), now);
         adder.add("data", serializeMutations(mutations, version))
              .add("written_at", new Date(now / 1000))
              .add("version", version);
-        return new Mutation(Keyspace.SYSTEM_KS, UUIDType.instance.decompose(uuid), cf);
+        return new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(uuid), cf);
     }
 
     private static ByteBuffer serializeMutations(Collection<Mutation> mutations, int version)
@@ -168,8 +167,8 @@ public class BatchlogManager implements BatchlogManagerMBean
         RateLimiter rateLimiter = RateLimiter.create(throttleInKB == 0 ? Double.MAX_VALUE : throttleInKB * 1024);
 
         UntypedResultSet page = executeInternal(String.format("SELECT id, data, written_at, version FROM %s.%s LIMIT %d",
-                                                              Keyspace.SYSTEM_KS,
-                                                              SystemKeyspace.BATCHLOG_CF,
+                                                              SystemKeyspace.NAME,
+                                                              SystemKeyspace.BATCHLOG_TABLE,
                                                               PAGE_SIZE));
 
         while (!page.isEmpty())
@@ -180,8 +179,8 @@ public class BatchlogManager implements BatchlogManagerMBean
                 break; // we've exhausted the batchlog, next query would be empty.
 
             page = executeInternal(String.format("SELECT id, data, written_at, version FROM %s.%s WHERE token(id) > token(?) LIMIT %d",
-                                                 Keyspace.SYSTEM_KS,
-                                                 SystemKeyspace.BATCHLOG_CF,
+                                                 SystemKeyspace.NAME,
+                                                 SystemKeyspace.BATCHLOG_TABLE,
                                                  PAGE_SIZE),
                                    id);
         }
@@ -193,8 +192,8 @@ public class BatchlogManager implements BatchlogManagerMBean
 
     private void deleteBatch(UUID id)
     {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, UUIDType.instance.decompose(id));
-        mutation.delete(SystemKeyspace.BATCHLOG_CF, FBUtilities.timestampMicros());
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(id));
+        mutation.delete(SystemKeyspace.BATCHLOG_TABLE, FBUtilities.timestampMicros());
         mutation.apply();
     }
 
@@ -442,7 +441,7 @@ public class BatchlogManager implements BatchlogManagerMBean
     // force flush + compaction to reclaim space from the replayed batches
     private void cleanup() throws ExecutionException, InterruptedException
     {
-        ColumnFamilyStore cfs = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(SystemKeyspace.BATCHLOG_CF);
+        ColumnFamilyStore cfs = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG_TABLE);
         cfs.forceBlockingFlush();
         Collection<Descriptor> descriptors = new ArrayList<>();
         for (SSTableReader sstr : cfs.getSSTables())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/src/java/org/apache/cassandra/db/DefsTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefsTables.java b/src/java/org/apache/cassandra/db/DefsTables.java
index 748d3f4..a02f65e 100644
--- a/src/java/org/apache/cassandra/db/DefsTables.java
+++ b/src/java/org/apache/cassandra/db/DefsTables.java
@@ -42,7 +42,6 @@ import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
@@ -55,13 +54,13 @@ public class DefsTables
     private static final Logger logger = LoggerFactory.getLogger(DefsTables.class);
 
     /**
-     * Load keyspace definitions for the system keyspace (system.SCHEMA_KEYSPACES_CF)
+     * Load keyspace definitions for the system keyspace (system.SCHEMA_KEYSPACES_TABLE)
      *
      * @return Collection of found keyspace definitions
      */
     public static Collection<KSMetaData> loadFromKeyspace()
     {
-        List<Row> serializedSchema = SystemKeyspace.serializedSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF);
+        List<Row> serializedSchema = SystemKeyspace.serializedSchema(SystemKeyspace.SCHEMA_KEYSPACES_TABLE);
 
         List<KSMetaData> keyspaces = new ArrayList<>(serializedSchema.size());
 
@@ -78,17 +77,17 @@ public class DefsTables
 
     private static Row serializedColumnFamilies(DecoratedKey ksNameKey)
     {
-        ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF);
+        ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE);
         return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey,
-                                                                                         SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF,
+                                                                                         SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE,
                                                                                          System.currentTimeMillis())));
     }
 
     private static Row serializedUserTypes(DecoratedKey ksNameKey)
     {
-        ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_USER_TYPES_CF);
+        ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_USER_TYPES_TABLE);
         return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey,
-                                                                                         SystemKeyspace.SCHEMA_USER_TYPES_CF,
+                                                                                         SystemKeyspace.SCHEMA_USER_TYPES_TABLE,
                                                                                          System.currentTimeMillis())));
     }
 
@@ -115,10 +114,10 @@ public class DefsTables
             keyspaces.add(ByteBufferUtil.string(mutation.key()));
 
         // current state of the schema
-        Map<DecoratedKey, ColumnFamily> oldKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF, keyspaces);
-        Map<DecoratedKey, ColumnFamily> oldColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, keyspaces);
-        Map<DecoratedKey, ColumnFamily> oldTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_CF, keyspaces);
-        Map<DecoratedKey, ColumnFamily> oldFunctions = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_FUNCTIONS_CF);
+        Map<DecoratedKey, ColumnFamily> oldKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_TABLE, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldFunctions = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE);
 
         for (Mutation mutation : mutations)
             mutation.apply();
@@ -127,10 +126,10 @@ public class DefsTables
             flushSchemaCFs();
 
         // with new data applied
-        Map<DecoratedKey, ColumnFamily> newKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF, keyspaces);
-        Map<DecoratedKey, ColumnFamily> newColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, keyspaces);
-        Map<DecoratedKey, ColumnFamily> newTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_CF, keyspaces);
-        Map<DecoratedKey, ColumnFamily> newFunctions = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_FUNCTIONS_CF);
+        Map<DecoratedKey, ColumnFamily> newKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_TABLE, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newFunctions = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE);
 
         Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
         mergeColumnFamilies(oldColumnFamilies, newColumnFamilies);
@@ -531,7 +530,7 @@ public class DefsTables
 
     private static void flushSchemaCFs()
     {
-        for (String cf : SystemKeyspace.allSchemaCfs)
+        for (String cf : SystemKeyspace.ALL_SCHEMA_TABLES)
             SystemKeyspace.forceBlockingFlush(cf);
     }
 }

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/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 8986154..09fc338 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -57,7 +57,6 @@ import org.apache.cassandra.metrics.KeyspaceMetrics;
  */
 public class Keyspace
 {
-    public static final String SYSTEM_KS = "system";
     private static final int DEFAULT_PAGE_SIZE = 10000;
 
     private static final Logger logger = LoggerFactory.getLogger(Keyspace.class);
@@ -75,7 +74,7 @@ public class Keyspace
     public final OpOrder writeOrder = new OpOrder();
 
     /* ColumnFamilyStore per column family */
-    private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<UUID, ColumnFamilyStore>();
+    private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<>();
     private volatile AbstractReplicationStrategy replicationStrategy;
 
     public static final Function<String,Keyspace> keyspaceTransformer = new Function<String, Keyspace>()
@@ -94,7 +93,7 @@ public class Keyspace
 
     public static Keyspace open(String keyspaceName)
     {
-        assert initialized || keyspaceName.equals(SYSTEM_KS);
+        assert initialized || keyspaceName.equals(SystemKeyspace.NAME);
         return open(keyspaceName, Schema.instance, true);
     }
 
@@ -273,7 +272,7 @@ public class Keyspace
         createReplicationStrategy(metadata);
 
         this.metric = new KeyspaceMetrics(this);
-        for (CFMetaData cfm : new ArrayList<CFMetaData>(metadata.cfMetaData().values()))
+        for (CFMetaData cfm : new ArrayList<>(metadata.cfMetaData().values()))
         {
             logger.debug("Initializing {}.{}", getName(), cfm.cfName);
             initCf(cfm.cfId, cfm.cfName, loadSSTables);
@@ -425,7 +424,7 @@ public class Keyspace
 
     public List<Future<?>> flush()
     {
-        List<Future<?>> futures = new ArrayList<Future<?>>(columnFamilyStores.size());
+        List<Future<?>> futures = new ArrayList<>(columnFamilyStores.size());
         for (UUID cfId : columnFamilyStores.keySet())
             futures.add(columnFamilyStores.get(cfId).forceFlush());
         return futures;
@@ -443,7 +442,7 @@ public class Keyspace
 
     public static Iterable<Keyspace> system()
     {
-        return Iterables.transform(Schema.systemKeyspaceNames, keyspaceTransformer);
+        return Iterables.transform(Collections.singleton(SystemKeyspace.NAME), keyspaceTransformer);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/611d1bae/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 21a36b8..565bf04 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -343,7 +343,7 @@ public class Memtable
                         // and BL data is strictly local, so we don't need to preserve tombstones for repair.
                         // If we have a data row + row level tombstone, then writing it is effectively an expensive no-op so we skip it.
                         // See CASSANDRA-4667.
-                        if (cfs.name.equals(SystemKeyspace.BATCHLOG_CF) && cfs.keyspace.getName().equals(Keyspace.SYSTEM_KS))
+                        if (cfs.name.equals(SystemKeyspace.BATCHLOG_TABLE) && cfs.keyspace.getName().equals(SystemKeyspace.NAME))
                             continue;
                     }