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 2013/06/27 20:36:37 UTC

[01/11] Rename Table to Keyspace

Updated Branches:
  refs/heads/trunk e22de4316 -> 0e96e5858


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
index 6652bd2..6bd5f56 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
@@ -33,7 +33,7 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.db.marshal.IntegerType;
@@ -47,8 +47,8 @@ public class SSTableMetadataTest extends SchemaLoader
     @Test
     public void testTrackMaxDeletionTime() throws ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         long timestamp = System.currentTimeMillis();
         for(int i = 0; i < 10; i++)
         {
@@ -120,8 +120,8 @@ public class SSTableMetadataTest extends SchemaLoader
     @Test
     public void testWithDeletes() throws ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard2");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
         long timestamp = System.currentTimeMillis();
         DecoratedKey key = Util.dk("deletetest");
         RowMutation rm = new RowMutation("Keyspace1", key.key);
@@ -170,8 +170,8 @@ public class SSTableMetadataTest extends SchemaLoader
     @Test
     public void trackMaxMinColNames() throws CharacterCodingException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard3");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard3");
         store.getCompactionStrategy();
         for (int j = 0; j < 8; j++)
         {
@@ -225,9 +225,9 @@ public class SSTableMetadataTest extends SchemaLoader
         ---------------------
         meaning max columns are b9 and 9, min is a0 and 0
          */
-        Table table = Table.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
 
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("StandardComposite2");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardComposite2");
 
         CompositeType ct = CompositeType.getInstance(BytesType.instance, IntegerType.instance);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index eeec8d9..eeb22e6 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -67,8 +67,8 @@ public class SSTableReaderTest extends SchemaLoader
     @Test
     public void testGetPositionsForRanges() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard2");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
 
         // insert data and compact to a single sstable
         CompactionManager.instance.disableAutoCompaction();
@@ -108,8 +108,8 @@ public class SSTableReaderTest extends SchemaLoader
     {
         MmappedSegmentedFile.MAX_SEGMENT_SIZE = 40; // each index entry is ~11 bytes, so this will generate lots of segments
 
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
 
         // insert a bunch of data and compact to a single sstable
         CompactionManager.instance.disableAutoCompaction();
@@ -145,8 +145,8 @@ public class SSTableReaderTest extends SchemaLoader
     public void testPersistentStatistics() throws IOException, ExecutionException, InterruptedException
     {
 
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
 
         for (int j = 0; j < 100; j += 2)
         {
@@ -170,8 +170,8 @@ public class SSTableReaderTest extends SchemaLoader
     @Test
     public void testGetPositionsForRangesWithKeyCache() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard2");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
         CacheService.instance.keyCache.setCapacity(100);
 
         // insert data and compact to a single sstable
@@ -205,8 +205,8 @@ public class SSTableReaderTest extends SchemaLoader
     public void testPersistentStatisticsWithSecondaryIndex() throws IOException, ExecutionException, InterruptedException
     {
         // Create secondary index and flush to disk
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Indexed1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
         ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
         RowMutation rm = new RowMutation("Keyspace1", key);
         rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), System.currentTimeMillis());
@@ -224,8 +224,8 @@ public class SSTableReaderTest extends SchemaLoader
         String cf = "Standard1";
 
         // clear and create just one sstable for this test
-        Table table = Table.open(ks);
-        ColumnFamilyStore store = table.getColumnFamilyStore(cf);
+        Keyspace keyspace = Keyspace.open(ks);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(cf);
         store.clearUnsafe();
         store.disableAutoCompaction();
 
@@ -261,8 +261,8 @@ public class SSTableReaderTest extends SchemaLoader
     @Test
     public void testLoadingSummaryUsesCorrectPartitioner() throws Exception
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Indexed1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
         ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
         RowMutation rm = new RowMutation("Keyspace1", key);
         rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), System.currentTimeMillis());
@@ -288,8 +288,8 @@ public class SSTableReaderTest extends SchemaLoader
     @Test
     public void testGetScannerForNoIntersectingRanges()
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
         RowMutation rm = new RowMutation("Keyspace1", key);
         rm.add("Standard1", ByteBufferUtil.bytes("xyz"), ByteBufferUtil.bytes("abc"), 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
index 1b14613..e80d2bb 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
@@ -39,15 +39,15 @@ public class SSTableSimpleWriterTest extends SchemaLoader
         final int INC = 5;
         final int NBCOL = 10;
 
-        String tablename = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfname = "StandardInteger1";
 
-        Table t = Table.open(tablename); // make sure we create the directory
-        File dir = Directories.create(tablename, cfname).getDirectoryForNewSSTables(0);
+        Keyspace t = Keyspace.open(keyspaceName); // make sure we create the directory
+        File dir = Directories.create(keyspaceName, cfname).getDirectoryForNewSSTables(0);
         assert dir.exists();
 
         IPartitioner partitioner = StorageService.getPartitioner();
-        SSTableSimpleUnsortedWriter writer = new SSTableSimpleUnsortedWriter(dir, partitioner, tablename, cfname, IntegerType.instance, null, 16);
+        SSTableSimpleUnsortedWriter writer = new SSTableSimpleUnsortedWriter(dir, partitioner, keyspaceName, cfname, IntegerType.instance, null, 16);
 
         int k = 0;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 01b174b..feeac7c 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -32,34 +32,34 @@ import static org.junit.Assert.assertEquals;
 
 public class SSTableUtils
 {
-    // first configured table and cf
-    public static String TABLENAME = "Keyspace1";
+    // first configured keyspace and cf
+    public static String KEYSPACENAME = "Keyspace1";
     public static String CFNAME = "Standard1";
 
     public static ColumnFamily createCF(long mfda, int ldt, Column... cols)
     {
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(TABLENAME, CFNAME);
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(KEYSPACENAME, CFNAME);
         cf.delete(new DeletionInfo(mfda, ldt));
         for (Column col : cols)
             cf.addColumn(col);
         return cf;
     }
 
-    public static File tempSSTableFile(String tablename, String cfname) throws IOException
+    public static File tempSSTableFile(String keyspaceName, String cfname) throws IOException
     {
-        return tempSSTableFile(tablename, cfname, 0);
+        return tempSSTableFile(keyspaceName, cfname, 0);
     }
 
-    public static File tempSSTableFile(String tablename, String cfname, int generation) throws IOException
+    public static File tempSSTableFile(String keyspaceName, String cfname, int generation) throws IOException
     {
-        File tempdir = File.createTempFile(tablename, cfname);
+        File tempdir = File.createTempFile(keyspaceName, cfname);
         if(!tempdir.delete() || !tempdir.mkdir())
             throw new IOException("Temporary directory creation failed.");
         tempdir.deleteOnExit();
-        File tabledir = new File(tempdir, tablename);
-        tabledir.mkdir();
-        tabledir.deleteOnExit();
-        File datafile = new File(new Descriptor(tabledir, tablename, cfname, generation, false).filenameFor("Data.db"));
+        File keyspaceDir = new File(tempdir, keyspaceName);
+        keyspaceDir.mkdir();
+        keyspaceDir.deleteOnExit();
+        File datafile = new File(new Descriptor(keyspaceDir, keyspaceName, cfname, generation, false).filenameFor("Data.db"));
         if (!datafile.createNewFile())
             throw new IOException("unable to create file " + datafile);
         datafile.deleteOnExit();
@@ -117,7 +117,7 @@ public class SSTableUtils
 
     public static class Context
     {
-        private String ksname = TABLENAME;
+        private String ksname = KEYSPACENAME;
         private String cfname = CFNAME;
         private Descriptor dest = null;
         private boolean cleanup = true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java b/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
index 6de0b09..82bd42c 100644
--- a/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
@@ -45,7 +45,7 @@ import com.google.common.collect.Multimap;
 
 public class NetworkTopologyStrategyTest
 {
-    private String table = "Keyspace1";
+    private String keyspaceName = "Keyspace1";
     private static final Logger logger = LoggerFactory.getLogger(NetworkTopologyStrategyTest.class);
 
     @Test
@@ -62,7 +62,7 @@ public class NetworkTopologyStrategyTest
         configOptions.put("DC3", "1");
 
         // Set the localhost to the tokenmetadata. Embedded cassandra way?
-        NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(table, metadata, snitch, configOptions);
+        NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(keyspaceName, metadata, snitch, configOptions);
         assert strategy.getReplicationFactor("DC1") == 3;
         assert strategy.getReplicationFactor("DC2") == 2;
         assert strategy.getReplicationFactor("DC3") == 1;
@@ -86,7 +86,7 @@ public class NetworkTopologyStrategyTest
         configOptions.put("DC3", "0");
 
         // Set the localhost to the tokenmetadata. Embedded cassandra way?
-        NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(table, metadata, snitch, configOptions);
+        NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(keyspaceName, metadata, snitch, configOptions);
         assert strategy.getReplicationFactor("DC1") == 3;
         assert strategy.getReplicationFactor("DC2") == 3;
         assert strategy.getReplicationFactor("DC3") == 0;
@@ -128,7 +128,7 @@ public class NetworkTopologyStrategyTest
         }
         metadata.updateNormalTokens(tokens);
 
-        NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(table, metadata, snitch, configOptions);
+        NetworkTopologyStrategy strategy = new NetworkTopologyStrategy(keyspaceName, metadata, snitch, configOptions);
 
         for (String testToken : new String[]{"123456", "200000", "000402", "ffffff", "400200"})
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 277677f..e59019b 100644
--- a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
+++ b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
@@ -22,7 +22,7 @@ package org.apache.cassandra.locator;
 import java.net.InetAddress;
 import java.util.*;
 
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 
 import org.apache.commons.lang.StringUtils;
 import org.junit.Test;
@@ -43,7 +43,7 @@ public class ReplicationStrategyEndpointCacheTest extends SchemaLoader
         tmd = new TokenMetadata();
         searchToken = new BigIntegerToken(String.valueOf(15));
 
-        strategy = getStrategyWithNewTokenMetadata(Table.open("Keyspace3").getReplicationStrategy(), tmd);
+        strategy = getStrategyWithNewTokenMetadata(Keyspace.open("Keyspace3").getReplicationStrategy(), tmd);
 
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(10)), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(20)), InetAddress.getByName("127.0.0.2"));
@@ -114,9 +114,9 @@ public class ReplicationStrategyEndpointCacheTest extends SchemaLoader
     {
         private boolean called = false;
 
-        public FakeSimpleStrategy(String table, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
+        public FakeSimpleStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
         {
-            super(table, tokenMetadata, snitch, configOptions);
+            super(keyspaceName, tokenMetadata, snitch, configOptions);
         }
 
         public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
@@ -131,9 +131,9 @@ public class ReplicationStrategyEndpointCacheTest extends SchemaLoader
     {
         private boolean called = false;
 
-        public FakeOldNetworkTopologyStrategy(String table, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
+        public FakeOldNetworkTopologyStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
         {
-            super(table, tokenMetadata, snitch, configOptions);
+            super(keyspaceName, tokenMetadata, snitch, configOptions);
         }
 
         public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
@@ -148,9 +148,9 @@ public class ReplicationStrategyEndpointCacheTest extends SchemaLoader
     {
         private boolean called = false;
 
-        public FakeNetworkTopologyStrategy(String table, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions) throws ConfigurationException
+        public FakeNetworkTopologyStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions) throws ConfigurationException
         {
-            super(table, tokenMetadata, snitch, configOptions);
+            super(keyspaceName, tokenMetadata, snitch, configOptions);
         }
 
         public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)
@@ -164,7 +164,7 @@ public class ReplicationStrategyEndpointCacheTest extends SchemaLoader
     private AbstractReplicationStrategy getStrategyWithNewTokenMetadata(AbstractReplicationStrategy strategy, TokenMetadata newTmd) throws ConfigurationException
     {
         return AbstractReplicationStrategy.createReplicationStrategy(
-                strategy.tableName,
+                strategy.keyspaceName,
                 AbstractReplicationStrategy.getClass(strategy.getClass().getName()),
                 newTmd,
                 strategy.snitch,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 a457df8..2656642 100644
--- a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
@@ -32,7 +32,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.StorageServiceAccessor;
@@ -43,9 +43,9 @@ import static org.junit.Assert.*;
 public class SimpleStrategyTest extends SchemaLoader
 {
     @Test
-    public void tryValidTable()
+    public void tryValidKeyspace()
     {
-        assert Table.open("Keyspace1").getReplicationStrategy() != null;
+        assert Keyspace.open("Keyspace1").getReplicationStrategy() != null;
     }
 
     @Test
@@ -80,10 +80,10 @@ public class SimpleStrategyTest extends SchemaLoader
     {
         TokenMetadata tmd;
         AbstractReplicationStrategy strategy;
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
             tmd = new TokenMetadata();
-            strategy = getStrategy(table, tmd);
+            strategy = getStrategy(keyspaceName, tmd);
             List<InetAddress> hosts = new ArrayList<InetAddress>();
             for (int i = 0; i < endpointTokens.length; i++)
             {
@@ -135,17 +135,17 @@ public class SimpleStrategyTest extends SchemaLoader
         tmd.addBootstrapToken(bsToken, bootstrapEndpoint);
 
         AbstractReplicationStrategy strategy = null;
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            strategy = getStrategy(table, tmd);
+            strategy = getStrategy(keyspaceName, tmd);
 
-            StorageService.calculatePendingRanges(strategy, table);
+            StorageService.calculatePendingRanges(strategy, keyspaceName);
 
             int replicationFactor = strategy.getReplicationFactor();
 
             for (int i = 0; i < keyTokens.length; i++)
             {
-                Collection<InetAddress> endpoints = tmd.getWriteEndpoints(keyTokens[i], table, strategy.getNaturalEndpoints(keyTokens[i]));
+                Collection<InetAddress> endpoints = tmd.getWriteEndpoints(keyTokens[i], keyspaceName, strategy.getNaturalEndpoints(keyTokens[i]));
                 assertTrue(endpoints.size() >= replicationFactor);
 
                 for (int j = 0; j < replicationFactor; j++)
@@ -165,11 +165,11 @@ public class SimpleStrategyTest extends SchemaLoader
         StorageServiceAccessor.setTokenMetadata(oldTmd);
     }
 
-    private AbstractReplicationStrategy getStrategy(String table, TokenMetadata tmd) throws ConfigurationException
+    private AbstractReplicationStrategy getStrategy(String keyspaceName, TokenMetadata tmd) throws ConfigurationException
     {
-        KSMetaData ksmd = Schema.instance.getKSMetaData(table);
+        KSMetaData ksmd = Schema.instance.getKSMetaData(keyspaceName);
         return AbstractReplicationStrategy.createReplicationStrategy(
-                table,
+                keyspaceName,
                 ksmd.strategyClass,
                 tmd,
                 new SimpleSnitch(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 12abd24..622f67d 100644
--- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java
+++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
@@ -26,7 +26,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.TreeMapBackedSortedColumns;
 import org.apache.cassandra.db.compaction.PrecompactedRow;
 import org.apache.cassandra.dht.IPartitioner;
@@ -100,7 +100,7 @@ public class ValidatorTest extends SchemaLoader
 
         InetAddress remote = InetAddress.getByName("127.0.0.2");
 
-        ColumnFamilyStore cfs = Table.open(keyspace).getColumnFamilyStore(columnFamily);
+        ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily);
 
         Validator validator = new Validator(desc, remote, 0);
         validator.prepare(cfs);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java b/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
index 8689703..0950f1d 100644
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
+++ b/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
@@ -31,14 +31,14 @@ public class AntiEntropyServiceCounterTest extends AntiEntropyServiceTestAbstrac
 {
     public void init()
     {
-        tablename = "Keyspace5";
+        keyspaceName = "Keyspace5";
         cfname    = "Counter1";
     }
 
     public List<IMutation> getWriteData()
     {
         List<IMutation> rms = new LinkedList<IMutation>();
-        RowMutation rm = new RowMutation(tablename, ByteBufferUtil.bytes("key1"));
+        RowMutation rm = new RowMutation(keyspaceName, ByteBufferUtil.bytes("key1"));
         rm.addCounter(cfname, ByteBufferUtil.bytes("Column1"), 42);
         rms.add(new CounterMutation(rm, ConsistencyLevel.ONE));
         return rms;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java b/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
index 76b0af2..169be2d 100644
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
+++ b/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
@@ -30,7 +30,7 @@ public class AntiEntropyServiceStandardTest extends AntiEntropyServiceTestAbstra
 {
     public void init()
     {
-        tablename = "Keyspace5";
+        keyspaceName = "Keyspace5";
         cfname    = "Standard1";
     }
 
@@ -38,7 +38,7 @@ public class AntiEntropyServiceStandardTest extends AntiEntropyServiceTestAbstra
     {
         List<IMutation> rms = new LinkedList<IMutation>();
         RowMutation rm;
-        rm = new RowMutation(tablename, ByteBufferUtil.bytes("key1"));
+        rm = new RowMutation(keyspaceName, ByteBufferUtil.bytes("key1"));
         rm.add(cfname, ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdfasdf"), 0);
         rms.add(rm);
         return rms;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java b/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
index 8905830..4023910 100644
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
+++ b/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
@@ -35,7 +35,7 @@ import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.Gossiper;
@@ -50,10 +50,10 @@ import static org.junit.Assert.assertEquals;
 
 public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
 {
-    // table and column family to test against
+    // keyspace and column family to test against
     public ActiveRepairService aes;
 
-    public String tablename;
+    public String keyspaceName;
     public String cfname;
     public RepairJobDesc desc;
     public ColumnFamilyStore store;
@@ -80,7 +80,7 @@ public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
             // generate a fake endpoint for which we can spoof receiving/sending trees
             REMOTE = InetAddress.getByName("127.0.0.2");
             store = null;
-            for (ColumnFamilyStore cfs : Table.open(tablename).getColumnFamilyStores())
+            for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
             {
                 if (cfs.name.equals(cfname))
                 {
@@ -101,9 +101,9 @@ public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
         MessagingService.instance().setVersion(REMOTE, MessagingService.current_version);
         Gossiper.instance.initializeNodeUnsafe(REMOTE, UUID.randomUUID(), 1);
 
-        local_range = StorageService.instance.getPrimaryRangesForEndpoint(tablename, LOCAL).iterator().next();
+        local_range = StorageService.instance.getPrimaryRangesForEndpoint(keyspaceName, LOCAL).iterator().next();
 
-        desc = new RepairJobDesc(UUID.randomUUID(), tablename, cfname, local_range);
+        desc = new RepairJobDesc(UUID.randomUUID(), keyspaceName, cfname, local_range);
         // Set a fake session corresponding to this fake request
         ActiveRepairService.instance.submitArtificialRepairSession(desc);
     }
@@ -118,13 +118,13 @@ public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
     public void testGetNeighborsPlusOne() throws Throwable
     {
         // generate rf+1 nodes, and ensure that all nodes are returned
-        Set<InetAddress> expected = addTokens(1 + Table.open(tablename).getReplicationStrategy().getReplicationFactor());
+        Set<InetAddress> expected = addTokens(1 + Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor());
         expected.remove(FBUtilities.getBroadcastAddress());
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(tablename);
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspaceName);
         Set<InetAddress> neighbors = new HashSet<InetAddress>();
         for (Range<Token> range : ranges)
         {
-            neighbors.addAll(ActiveRepairService.getNeighbors(tablename, range, false));
+            neighbors.addAll(ActiveRepairService.getNeighbors(keyspaceName, range, false));
         }
         assertEquals(expected, neighbors);
     }
@@ -135,19 +135,19 @@ public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
 
         // generate rf*2 nodes, and ensure that only neighbors specified by the ARS are returned
-        addTokens(2 * Table.open(tablename).getReplicationStrategy().getReplicationFactor());
-        AbstractReplicationStrategy ars = Table.open(tablename).getReplicationStrategy();
+        addTokens(2 * Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor());
+        AbstractReplicationStrategy ars = Keyspace.open(keyspaceName).getReplicationStrategy();
         Set<InetAddress> expected = new HashSet<InetAddress>();
         for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
         {
             expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
         }
         expected.remove(FBUtilities.getBroadcastAddress());
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(tablename);
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspaceName);
         Set<InetAddress> neighbors = new HashSet<InetAddress>();
         for (Range<Token> range : ranges)
         {
-            neighbors.addAll(ActiveRepairService.getNeighbors(tablename, range, false));
+            neighbors.addAll(ActiveRepairService.getNeighbors(keyspaceName, range, false));
         }
         assertEquals(expected, neighbors);
     }
@@ -158,18 +158,18 @@ public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
         
         // generate rf+1 nodes, and ensure that all nodes are returned
-        Set<InetAddress> expected = addTokens(1 + Table.open(tablename).getReplicationStrategy().getReplicationFactor());
+        Set<InetAddress> expected = addTokens(1 + Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor());
         expected.remove(FBUtilities.getBroadcastAddress());
         // remove remote endpoints
         TokenMetadata.Topology topology = tmd.cloneOnlyTokenMap().getTopology();
         HashSet<InetAddress> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
         expected = Sets.intersection(expected, localEndpoints);
 
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(tablename);
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspaceName);
         Set<InetAddress> neighbors = new HashSet<InetAddress>();
         for (Range<Token> range : ranges)
         {
-            neighbors.addAll(ActiveRepairService.getNeighbors(tablename, range, true));
+            neighbors.addAll(ActiveRepairService.getNeighbors(keyspaceName, range, true));
         }
         assertEquals(expected, neighbors);
     }
@@ -180,8 +180,8 @@ public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
 
         // generate rf*2 nodes, and ensure that only neighbors specified by the ARS are returned
-        addTokens(2 * Table.open(tablename).getReplicationStrategy().getReplicationFactor());
-        AbstractReplicationStrategy ars = Table.open(tablename).getReplicationStrategy();
+        addTokens(2 * Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor());
+        AbstractReplicationStrategy ars = Keyspace.open(keyspaceName).getReplicationStrategy();
         Set<InetAddress> expected = new HashSet<InetAddress>();
         for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
         {
@@ -193,11 +193,11 @@ public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
         HashSet<InetAddress> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
         expected = Sets.intersection(expected, localEndpoints);
         
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(tablename);
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspaceName);
         Set<InetAddress> neighbors = new HashSet<InetAddress>();
         for (Range<Token> range : ranges)
         {
-            neighbors.addAll(ActiveRepairService.getNeighbors(tablename, range, true));
+            neighbors.addAll(ActiveRepairService.getNeighbors(keyspaceName, range, true));
         }
         assertEquals(expected, neighbors);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
index c3e83cb..ef7b800 100644
--- a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
+++ b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
@@ -87,7 +87,7 @@ public class LeaveAndBootstrapTest
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, RING_SIZE);
 
         Map<Token, List<InetAddress>> expectedEndpoints = new HashMap<Token, List<InetAddress>>();
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
             for (Token token : keyTokens)
             {
@@ -108,14 +108,14 @@ public class LeaveAndBootstrapTest
         assertTrue(tmd.isLeaving(hosts.get(LEAVING_NODE)));
 
         AbstractReplicationStrategy strategy;
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            strategy = getStrategy(table, tmd);
+            strategy = getStrategy(keyspaceName, tmd);
             for (Token token : keyTokens)
             {
                 int replicationFactor = strategy.getReplicationFactor();
 
-                HashSet<InetAddress> actual = new HashSet<InetAddress>(tmd.getWriteEndpoints(token, table, strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
+                HashSet<InetAddress> actual = new HashSet<InetAddress>(tmd.getWriteEndpoints(token, keyspaceName, strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
                 HashSet<InetAddress> expected = new HashSet<InetAddress>();
 
                 for (int i = 0; i < replicationFactor; i++)
@@ -179,10 +179,10 @@ public class LeaveAndBootstrapTest
         Collection<InetAddress> endpoints = null;
 
         /* don't require test update every time a new keyspace is added to test/conf/cassandra.yaml */
-        Map<String, AbstractReplicationStrategy> tableStrategyMap = new HashMap<String, AbstractReplicationStrategy>();
+        Map<String, AbstractReplicationStrategy> keyspaceStrategyMap = new HashMap<String, AbstractReplicationStrategy>();
         for (int i=1; i<=4; i++)
         {
-            tableStrategyMap.put("Keyspace" + i, getStrategy("Keyspace" + i, tmd));
+            keyspaceStrategyMap.put("Keyspace" + i, getStrategy("Keyspace" + i, tmd));
         }
 
         // pre-calculate the results.
@@ -232,16 +232,16 @@ public class LeaveAndBootstrapTest
         expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
         expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1", "127.0.0.2", "127.0.0.3"));
 
-        for (Map.Entry<String, AbstractReplicationStrategy> tableStrategy : tableStrategyMap.entrySet())
+        for (Map.Entry<String, AbstractReplicationStrategy> keyspaceStrategy : keyspaceStrategyMap.entrySet())
         {
-            String table = tableStrategy.getKey();
-            AbstractReplicationStrategy strategy = tableStrategy.getValue();
+            String keyspaceName = keyspaceStrategy.getKey();
+            AbstractReplicationStrategy strategy = keyspaceStrategy.getValue();
 
             for (int i = 0; i < keyTokens.size(); i++)
             {
-                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), table, strategy.getNaturalEndpoints(keyTokens.get(i)));
-                assertTrue(expectedEndpoints.get(table).get(keyTokens.get(i)).size() == endpoints.size());
-                assertTrue(expectedEndpoints.get(table).get(keyTokens.get(i)).containsAll(endpoints));
+                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(i)));
+                assertTrue(expectedEndpoints.get(keyspaceName).get(keyTokens.get(i)).size() == endpoints.size());
+                assertTrue(expectedEndpoints.get(keyspaceName).get(keyTokens.get(i)).containsAll(endpoints));
             }
 
             // just to be sure that things still work according to the old tests, run them:
@@ -250,7 +250,7 @@ public class LeaveAndBootstrapTest
             // tokens 5, 15 and 25 should go three nodes
             for (int i=0; i<3; ++i)
             {
-                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), table, strategy.getNaturalEndpoints(keyTokens.get(i)));
+                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(i)));
                 assertTrue(endpoints.size() == 3);
                 assertTrue(endpoints.contains(hosts.get(i+1)));
                 assertTrue(endpoints.contains(hosts.get(i+2)));
@@ -258,7 +258,7 @@ public class LeaveAndBootstrapTest
             }
 
             // token 35 should go to nodes 4, 5, 6, 7 and boot1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(3), table, strategy.getNaturalEndpoints(keyTokens.get(3)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(3), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(3)));
             assertTrue(endpoints.size() == 5);
             assertTrue(endpoints.contains(hosts.get(4)));
             assertTrue(endpoints.contains(hosts.get(5)));
@@ -267,7 +267,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(boot1));
 
             // token 45 should go to nodes 5, 6, 7, 0, boot1 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), table, strategy.getNaturalEndpoints(keyTokens.get(4)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(4)));
             assertTrue(endpoints.size() == 6);
             assertTrue(endpoints.contains(hosts.get(5)));
             assertTrue(endpoints.contains(hosts.get(6)));
@@ -277,7 +277,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(boot2));
 
             // token 55 should go to nodes 6, 7, 8, 0, 1, boot1 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), table, strategy.getNaturalEndpoints(keyTokens.get(5)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(5)));
             assertTrue(endpoints.size() == 7);
             assertTrue(endpoints.contains(hosts.get(6)));
             assertTrue(endpoints.contains(hosts.get(7)));
@@ -288,7 +288,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(boot2));
 
             // token 65 should go to nodes 7, 8, 9, 0, 1 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), table, strategy.getNaturalEndpoints(keyTokens.get(6)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(6)));
             assertTrue(endpoints.size() == 6);
             assertTrue(endpoints.contains(hosts.get(7)));
             assertTrue(endpoints.contains(hosts.get(8)));
@@ -298,7 +298,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(boot2));
 
             // token 75 should to go nodes 8, 9, 0, 1, 2 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), table, strategy.getNaturalEndpoints(keyTokens.get(7)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(7)));
             assertTrue(endpoints.size() == 6);
             assertTrue(endpoints.contains(hosts.get(8)));
             assertTrue(endpoints.contains(hosts.get(9)));
@@ -308,7 +308,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(boot2));
 
             // token 85 should go to nodes 9, 0, 1 and 2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), table, strategy.getNaturalEndpoints(keyTokens.get(8)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(8)));
             assertTrue(endpoints.size() == 4);
             assertTrue(endpoints.contains(hosts.get(9)));
             assertTrue(endpoints.contains(hosts.get(0)));
@@ -316,7 +316,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(hosts.get(2)));
 
             // token 95 should go to nodes 0, 1 and 2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), table, strategy.getNaturalEndpoints(keyTokens.get(9)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(9)));
             assertTrue(endpoints.size() == 3);
             assertTrue(endpoints.contains(hosts.get(0)));
             assertTrue(endpoints.contains(hosts.get(1)));
@@ -352,16 +352,16 @@ public class LeaveAndBootstrapTest
         expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("75")).removeAll(makeAddrs("127.0.0.10"));
         expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
 
-        for (Map.Entry<String, AbstractReplicationStrategy> tableStrategy : tableStrategyMap.entrySet())
+        for (Map.Entry<String, AbstractReplicationStrategy> keyspaceStrategy : keyspaceStrategyMap.entrySet())
         {
-            String table = tableStrategy.getKey();
-            AbstractReplicationStrategy strategy = tableStrategy.getValue();
+            String keyspaceName = keyspaceStrategy.getKey();
+            AbstractReplicationStrategy strategy = keyspaceStrategy.getValue();
 
             for (int i = 0; i < keyTokens.size(); i++)
             {
-                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), table, strategy.getNaturalEndpoints(keyTokens.get(i)));
-                assertTrue(expectedEndpoints.get(table).get(keyTokens.get(i)).size() == endpoints.size());
-                assertTrue(expectedEndpoints.get(table).get(keyTokens.get(i)).containsAll(endpoints));
+                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(i)));
+                assertTrue(expectedEndpoints.get(keyspaceName).get(keyTokens.get(i)).size() == endpoints.size());
+                assertTrue(expectedEndpoints.get(keyspaceName).get(keyTokens.get(i)).containsAll(endpoints));
             }
 
             if (strategy.getReplicationFactor() != 3)
@@ -370,7 +370,7 @@ public class LeaveAndBootstrapTest
             // tokens 5, 15 and 25 should go three nodes
             for (int i=0; i<3; ++i)
             {
-                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), table, strategy.getNaturalEndpoints(keyTokens.get(i)));
+                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(i)));
                 assertTrue(endpoints.size() == 3);
                 assertTrue(endpoints.contains(hosts.get(i+1)));
                 assertTrue(endpoints.contains(hosts.get(i+2)));
@@ -378,21 +378,21 @@ public class LeaveAndBootstrapTest
             }
 
             // token 35 goes to nodes 4, 5 and boot1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(3), table, strategy.getNaturalEndpoints(keyTokens.get(3)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(3), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(3)));
             assertTrue(endpoints.size() == 3);
             assertTrue(endpoints.contains(hosts.get(4)));
             assertTrue(endpoints.contains(hosts.get(5)));
             assertTrue(endpoints.contains(boot1));
 
             // token 45 goes to nodes 5, boot1 and node7
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), table, strategy.getNaturalEndpoints(keyTokens.get(4)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(4)));
             assertTrue(endpoints.size() == 3);
             assertTrue(endpoints.contains(hosts.get(5)));
             assertTrue(endpoints.contains(boot1));
             assertTrue(endpoints.contains(hosts.get(7)));
 
             // token 55 goes to boot1, 7, boot2, 8 and 0
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), table, strategy.getNaturalEndpoints(keyTokens.get(5)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(5)));
             assertTrue(endpoints.size() == 5);
             assertTrue(endpoints.contains(boot1));
             assertTrue(endpoints.contains(hosts.get(7)));
@@ -401,7 +401,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(hosts.get(0)));
 
             // token 65 goes to nodes 7, boot2, 8, 0 and 1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), table, strategy.getNaturalEndpoints(keyTokens.get(6)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(6)));
             assertTrue(endpoints.size() == 5);
             assertTrue(endpoints.contains(hosts.get(7)));
             assertTrue(endpoints.contains(boot2));
@@ -410,7 +410,7 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(hosts.get(1)));
 
             // token 75 goes to nodes boot2, 8, 0, 1 and 2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), table, strategy.getNaturalEndpoints(keyTokens.get(7)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(7)));
             assertTrue(endpoints.size() == 5);
             assertTrue(endpoints.contains(boot2));
             assertTrue(endpoints.contains(hosts.get(8)));
@@ -419,14 +419,14 @@ public class LeaveAndBootstrapTest
             assertTrue(endpoints.contains(hosts.get(2)));
 
             // token 85 goes to nodes 0, 1 and 2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), table, strategy.getNaturalEndpoints(keyTokens.get(8)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(8)));
             assertTrue(endpoints.size() == 3);
             assertTrue(endpoints.contains(hosts.get(0)));
             assertTrue(endpoints.contains(hosts.get(1)));
             assertTrue(endpoints.contains(hosts.get(2)));
 
             // token 95 goes to nodes 0, 1 and 2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), table, strategy.getNaturalEndpoints(keyTokens.get(9)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(9)));
             assertTrue(endpoints.size() == 3);
             assertTrue(endpoints.contains(hosts.get(0)));
             assertTrue(endpoints.contains(hosts.get(1)));
@@ -652,11 +652,11 @@ public class LeaveAndBootstrapTest
         return addrs;
     }
 
-    private AbstractReplicationStrategy getStrategy(String table, TokenMetadata tmd) throws ConfigurationException
+    private AbstractReplicationStrategy getStrategy(String keyspaceName, TokenMetadata tmd) throws ConfigurationException
     {
-        KSMetaData ksmd = Schema.instance.getKSMetaData(table);
+        KSMetaData ksmd = Schema.instance.getKSMetaData(keyspaceName);
         return AbstractReplicationStrategy.createReplicationStrategy(
-                table,
+                keyspaceName,
                 ksmd.strategyClass,
                 tmd,
                 new SimpleSnitch(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/service/MoveTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java b/test/unit/org/apache/cassandra/service/MoveTest.java
index 5454127..f239671 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -113,15 +113,15 @@ public class MoveTest
         assertTrue(tmd.isMoving(hosts.get(MOVING_NODE)));
 
         AbstractReplicationStrategy strategy;
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            strategy = getStrategy(table, tmd);
+            strategy = getStrategy(keyspaceName, tmd);
             int numMoved = 0;
             for (Token token : keyTokens)
             {
                 int replicationFactor = strategy.getReplicationFactor();
 
-                HashSet<InetAddress> actual = new HashSet<InetAddress>(tmd.getWriteEndpoints(token, table, strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
+                HashSet<InetAddress> actual = new HashSet<InetAddress>(tmd.getWriteEndpoints(token, keyspaceName, strategy.calculateNaturalEndpoints(token, tmd.cloneOnlyTokenMap())));
                 HashSet<InetAddress> expected = new HashSet<InetAddress>();
 
                 for (int i = 0; i < replicationFactor; i++)
@@ -199,10 +199,10 @@ public class MoveTest
                     valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(7))));
 
         // don't require test update every time a new keyspace is added to test/conf/cassandra.yaml
-        Map<String, AbstractReplicationStrategy> tableStrategyMap = new HashMap<String, AbstractReplicationStrategy>();
+        Map<String, AbstractReplicationStrategy> keyspaceStrategyMap = new HashMap<String, AbstractReplicationStrategy>();
         for (int i = 1; i <= 4; i++)
         {
-            tableStrategyMap.put("Keyspace" + i, getStrategy("Keyspace" + i, tmd));
+            keyspaceStrategyMap.put("Keyspace" + i, getStrategy("Keyspace" + i, tmd));
         }
 
        /**
@@ -221,7 +221,7 @@ public class MoveTest
         *  }
         */
 
-        Multimap<InetAddress, Range<Token>> keyspace1ranges = tableStrategyMap.get("Keyspace1").getAddressRanges();
+        Multimap<InetAddress, Range<Token>> keyspace1ranges = keyspaceStrategyMap.get("Keyspace1").getAddressRanges();
         Collection<Range<Token>> ranges1 = keyspace1ranges.get(InetAddress.getByName("127.0.0.1"));
         assertEquals(collectionSize(ranges1), 1);
         assertTrue(ranges1.iterator().next().equals(generateRange(97, 0)));
@@ -270,7 +270,7 @@ public class MoveTest
         * }
         */
 
-        Multimap<InetAddress, Range<Token>> keyspace3ranges = tableStrategyMap.get("Keyspace3").getAddressRanges();
+        Multimap<InetAddress, Range<Token>> keyspace3ranges = keyspaceStrategyMap.get("Keyspace3").getAddressRanges();
         ranges1 = keyspace3ranges.get(InetAddress.getByName("127.0.0.1"));
         assertEquals(collectionSize(ranges1), 5);
         assertTrue(ranges1.equals(generateRanges(97, 0, 70, 87, 50, 67, 87, 97, 67, 70)));
@@ -318,7 +318,7 @@ public class MoveTest
          *      /127.0.0.10=[(70,87], (87,97], (67,70]]
          *  }
          */
-        Multimap<InetAddress, Range<Token>> keyspace4ranges = tableStrategyMap.get("Keyspace4").getAddressRanges();
+        Multimap<InetAddress, Range<Token>> keyspace4ranges = keyspaceStrategyMap.get("Keyspace4").getAddressRanges();
         ranges1 = keyspace4ranges.get(InetAddress.getByName("127.0.0.1"));
         assertEquals(collectionSize(ranges1), 3);
         assertTrue(ranges1.equals(generateRanges(97, 0, 70, 87, 87, 97)));
@@ -397,16 +397,16 @@ public class MoveTest
         expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1"));
         expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2"));
 
-        for (Map.Entry<String, AbstractReplicationStrategy> tableStrategy : tableStrategyMap.entrySet())
+        for (Map.Entry<String, AbstractReplicationStrategy> keyspaceStrategy : keyspaceStrategyMap.entrySet())
         {
-            String table = tableStrategy.getKey();
-            AbstractReplicationStrategy strategy = tableStrategy.getValue();
+            String keyspaceName = keyspaceStrategy.getKey();
+            AbstractReplicationStrategy strategy = keyspaceStrategy.getValue();
 
             for (Token token : keyTokens)
             {
-                endpoints = tmd.getWriteEndpoints(token, table, strategy.getNaturalEndpoints(token));
-                assertTrue(expectedEndpoints.get(table).get(token).size() == endpoints.size());
-                assertTrue(expectedEndpoints.get(table).get(token).containsAll(endpoints));
+                endpoints = tmd.getWriteEndpoints(token, keyspaceName, strategy.getNaturalEndpoints(token));
+                assertTrue(expectedEndpoints.get(keyspaceName).get(token).size() == endpoints.size());
+                assertTrue(expectedEndpoints.get(keyspaceName).get(token).containsAll(endpoints));
             }
 
             // just to be sure that things still work according to the old tests, run them:
@@ -416,7 +416,7 @@ public class MoveTest
             // tokens 5, 15 and 25 should go three nodes
             for (int i = 0; i < 3; i++)
             {
-                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), table, strategy.getNaturalEndpoints(keyTokens.get(i)));
+                endpoints = tmd.getWriteEndpoints(keyTokens.get(i), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(i)));
                 assertTrue(endpoints.size() == 3);
                 assertTrue(endpoints.contains(hosts.get(i+1)));
                 assertTrue(endpoints.contains(hosts.get(i+2)));
@@ -424,7 +424,7 @@ public class MoveTest
             }
 
             // token 35 should go to nodes 4, 5, 6 and boot1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(3), table, strategy.getNaturalEndpoints(keyTokens.get(3)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(3), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(3)));
             assertTrue(endpoints.size() == 4);
             assertTrue(endpoints.contains(hosts.get(4)));
             assertTrue(endpoints.contains(hosts.get(5)));
@@ -432,7 +432,7 @@ public class MoveTest
             assertTrue(endpoints.contains(boot1));
 
             // token 45 should go to nodes 5, 6, 7 boot1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), table, strategy.getNaturalEndpoints(keyTokens.get(4)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(4), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(4)));
             assertTrue(endpoints.size() == 4);
             assertTrue(endpoints.contains(hosts.get(5)));
             assertTrue(endpoints.contains(hosts.get(6)));
@@ -440,7 +440,7 @@ public class MoveTest
             assertTrue(endpoints.contains(boot1));
 
             // token 55 should go to nodes 6, 7, 8 boot1 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), table, strategy.getNaturalEndpoints(keyTokens.get(5)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(5), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(5)));
             assertTrue(endpoints.size() == 5);
             assertTrue(endpoints.contains(hosts.get(6)));
             assertTrue(endpoints.contains(hosts.get(7)));
@@ -449,7 +449,7 @@ public class MoveTest
             assertTrue(endpoints.contains(boot2));
 
             // token 65 should go to nodes 6, 7, 8 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), table, strategy.getNaturalEndpoints(keyTokens.get(6)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(6), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(6)));
             assertTrue(endpoints.size() == 4);
             assertTrue(endpoints.contains(hosts.get(6)));
             assertTrue(endpoints.contains(hosts.get(7)));
@@ -457,7 +457,7 @@ public class MoveTest
             assertTrue(endpoints.contains(boot2));
 
             // token 75 should to go nodes 8, 9, 0 and boot2
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), table, strategy.getNaturalEndpoints(keyTokens.get(7)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(7), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(7)));
             assertTrue(endpoints.size() == 4);
             assertTrue(endpoints.contains(hosts.get(8)));
             assertTrue(endpoints.contains(hosts.get(9)));
@@ -465,14 +465,14 @@ public class MoveTest
             assertTrue(endpoints.contains(boot2));
 
             // token 85 should go to nodes 8, 9 and 0
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), table, strategy.getNaturalEndpoints(keyTokens.get(8)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(8), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(8)));
             assertTrue(endpoints.size() == 3);
             assertTrue(endpoints.contains(hosts.get(8)));
             assertTrue(endpoints.contains(hosts.get(9)));
             assertTrue(endpoints.contains(hosts.get(0)));
 
             // token 95 should go to nodes 9, 0 and 1
-            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), table, strategy.getNaturalEndpoints(keyTokens.get(9)));
+            endpoints = tmd.getWriteEndpoints(keyTokens.get(9), keyspaceName, strategy.getNaturalEndpoints(keyTokens.get(9)));
             assertTrue(endpoints.size() == 3);
             assertTrue(endpoints.contains(hosts.get(9)));
             assertTrue(endpoints.contains(hosts.get(0)));
@@ -538,11 +538,11 @@ public class MoveTest
         return addrs;
     }
 
-    private AbstractReplicationStrategy getStrategy(String table, TokenMetadata tmd) throws ConfigurationException
+    private AbstractReplicationStrategy getStrategy(String keyspaceName, TokenMetadata tmd) throws ConfigurationException
     {
-        KSMetaData ksmd = Schema.instance.getKSMetaData(table);
+        KSMetaData ksmd = Schema.instance.getKSMetaData(keyspaceName);
         return AbstractReplicationStrategy.createReplicationStrategy(
-                table,
+                keyspaceName,
                 ksmd.strategyClass,
                 tmd,
                 new SimpleSnitch(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/service/QueryPagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/QueryPagerTest.java b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
index 1d18321..cf1a083 100644
--- a/test/unit/org/apache/cassandra/service/QueryPagerTest.java
+++ b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
@@ -86,7 +86,7 @@ public class QueryPagerTest extends SchemaLoader
 
     private static ColumnFamilyStore cfs()
     {
-        return Table.open(KS).getColumnFamilyStore(CF);
+        return Keyspace.open(KS).getColumnFamilyStore(CF);
     }
 
     private static String toString(List<Row> rows)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/service/RelocateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/RelocateTest.java b/test/unit/org/apache/cassandra/service/RelocateTest.java
index 2791f18..21dfb96 100644
--- a/test/unit/org/apache/cassandra/service/RelocateTest.java
+++ b/test/unit/org/apache/cassandra/service/RelocateTest.java
@@ -33,7 +33,7 @@ import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.db.SystemTable;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
@@ -114,11 +114,11 @@ public class RelocateTest
     }
 
     // Copy-pasta from MoveTest.java
-    private AbstractReplicationStrategy getStrategy(String table, TokenMetadata tmd) throws ConfigurationException
+    private AbstractReplicationStrategy getStrategy(String keyspaceName, TokenMetadata tmd) throws ConfigurationException
     {
-        KSMetaData ksmd = Schema.instance.getKSMetaData(table);
+        KSMetaData ksmd = Schema.instance.getKSMetaData(keyspaceName);
         return AbstractReplicationStrategy.createReplicationStrategy(
-                table,
+                keyspaceName,
                 ksmd.strategyClass,
                 tmd,
                 new SimpleSnitch(),
@@ -156,14 +156,14 @@ public class RelocateTest
         assertTrue(tmd.isRelocating(relocateToken));
 
         AbstractReplicationStrategy strategy;
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            strategy = getStrategy(table, tmd);
+            strategy = getStrategy(keyspaceName, tmd);
             for (Token token : tokenMap.keySet())
             {
                 BigIntegerToken keyToken = new BigIntegerToken(((BigInteger)token.token).add(new BigInteger("5")));
 
-                HashSet<InetAddress> actual = new HashSet<InetAddress>(tmd.getWriteEndpoints(keyToken, table, strategy.calculateNaturalEndpoints(keyToken, tmd.cloneOnlyTokenMap())));
+                HashSet<InetAddress> actual = new HashSet<InetAddress>(tmd.getWriteEndpoints(keyToken, keyspaceName, strategy.calculateNaturalEndpoints(keyToken, tmd.cloneOnlyTokenMap())));
                 HashSet<InetAddress> expected = new HashSet<InetAddress>();
 
                 for (int i = 0; i < actual.size(); i++)
@@ -190,7 +190,7 @@ public class RelocateTest
 
         // Create a list of the endpoint's existing tokens, and add the relocatee to it.
         List<Token> tokens = new ArrayList<Token>(tmd.getTokens(relocator));
-        SystemTable.updateTokens(tokens);
+        SystemKeyspace.updateTokens(tokens);
         tokens.add(relocatee);
 
         // Send a normal status, then ensure all is copesetic.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/service/RowResolverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/RowResolverTest.java b/test/unit/org/apache/cassandra/service/RowResolverTest.java
index d19677c..72cafd1 100644
--- a/test/unit/org/apache/cassandra/service/RowResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/RowResolverTest.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.db.TreeMapBackedSortedColumns;
 
 import static junit.framework.Assert.*;
 import static org.apache.cassandra.Util.column;
-import static org.apache.cassandra.db.TableTest.*;
+import static org.apache.cassandra.db.KeyspaceTest.*;
 
 public class RowResolverTest extends SchemaLoader
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index d28bd8d..503a730 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -33,12 +33,12 @@ import org.junit.runner.RunWith;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.SystemTable;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.StringToken;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.Table;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.locator.IEndpointSnitch;
@@ -94,7 +94,7 @@ public class StorageServiceServerTest
     public void testColumnFamilySnapshot() throws IOException
     {
         // no need to insert extra data, even an "empty" database will have a little information in the system keyspace
-        StorageService.instance.takeColumnFamilySnapshot(Table.SYSTEM_KS, SystemTable.SCHEMA_KEYSPACES_CF, "cf_snapshot");
+        StorageService.instance.takeColumnFamilySnapshot(Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_KEYSPACES_CF, "cf_snapshot");
     }
 
     @Test
@@ -113,9 +113,9 @@ public class StorageServiceServerTest
         configOptions.put("DC1", "1");
         configOptions.put("DC2", "1");
 
-        Table.clear("Keyspace1");
+        Keyspace.clear("Keyspace1");
         KSMetaData meta = KSMetaData.newKeyspace("Keyspace1", "NetworkTopologyStrategy", configOptions, false);
-        Schema.instance.setTableDefinition(meta);
+        Schema.instance.setKeyspaceDefinition(meta);
 
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
         assert primaryRanges.size() == 1;
@@ -149,9 +149,9 @@ public class StorageServiceServerTest
         Map<String, String> configOptions = new HashMap<String, String>();
         configOptions.put("DC2", "2");
 
-        Table.clear("Keyspace1");
+        Keyspace.clear("Keyspace1");
         KSMetaData meta = KSMetaData.newKeyspace("Keyspace1", "NetworkTopologyStrategy", configOptions, false);
-        Schema.instance.setTableDefinition(meta);
+        Schema.instance.setKeyspaceDefinition(meta);
 
         // endpoints in DC1 should not have primary range
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
@@ -199,9 +199,9 @@ public class StorageServiceServerTest
         Map<String, String> configOptions = new HashMap<String, String>();
         configOptions.put("DC2", "2");
 
-        Table.clear("Keyspace1");
+        Keyspace.clear("Keyspace1");
         KSMetaData meta = KSMetaData.newKeyspace("Keyspace1", "NetworkTopologyStrategy", configOptions, false);
-        Schema.instance.setTableDefinition(meta);
+        Schema.instance.setKeyspaceDefinition(meta);
 
         // endpoints in DC1 should not have primary range
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
@@ -247,9 +247,9 @@ public class StorageServiceServerTest
         Map<String, String> configOptions = new HashMap<String, String>();
         configOptions.put("replication_factor", "2");
 
-        Table.clear("Keyspace1");
+        Keyspace.clear("Keyspace1");
         KSMetaData meta = KSMetaData.newKeyspace("Keyspace1", "SimpleStrategy", configOptions, false);
-        Schema.instance.setTableDefinition(meta);
+        Schema.instance.setKeyspaceDefinition(meta);
 
         Collection<Range<Token>> primaryRanges = StorageService.instance.getPrimaryRangesForEndpoint(meta.name, InetAddress.getByName("127.0.0.1"));
         assert primaryRanges.size() == 1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index 3e9eeb1..6919d7f 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -194,8 +194,8 @@ public class StreamingTransferTest extends SchemaLoader
     {
         String ks = "Keyspace1";
         String cfname = "StandardInteger1";
-        Table table = Table.open(ks);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfname);
+        Keyspace keyspace = Keyspace.open(ks);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
 
         String key = "key1";
         RowMutation rm = new RowMutation(ks, ByteBufferUtil.bytes(key));
@@ -223,15 +223,15 @@ public class StreamingTransferTest extends SchemaLoader
     @Test
     public void testTransferTable() throws Exception
     {
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfs = table.getColumnFamilyStore("Indexed1");
+        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed1");
 
         List<String> keys = createAndTransfer(cfs, new Mutator()
         {
             public void mutate(String key, String col, long timestamp) throws Exception
             {
                 long val = key.hashCode();
-                ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(table.getName(), cfs.name);
+                ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(keyspace.getName(), cfs.name);
                 cf.addColumn(column(col, "v", timestamp));
                 cf.addColumn(new Column(ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(val), timestamp));
                 RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
@@ -259,8 +259,8 @@ public class StreamingTransferTest extends SchemaLoader
     @Test
     public void testTransferTableCounter() throws Exception
     {
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfs = table.getColumnFamilyStore("Counter1");
+        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Counter1");
         final CounterContext cc = new CounterContext();
 
         final Map<String, ColumnFamily> cleanedEntries = new HashMap<>();
@@ -288,7 +288,7 @@ public class StreamingTransferTest extends SchemaLoader
                 entries.put(key, cf);
                 cleanedEntries.put(key, cfCleaned);
                 cfs.addSSTable(SSTableUtils.prepare()
-                    .ks(table.getName())
+                    .ks(keyspace.getName())
                     .cf(cfs.name)
                     .generation(0)
                     .write(entries));
@@ -298,7 +298,7 @@ public class StreamingTransferTest extends SchemaLoader
         // filter pre-cleaned entries locally, and ensure that the end result is equal
         cleanedEntries.keySet().retainAll(keys);
         SSTableReader cleaned = SSTableUtils.prepare()
-            .ks(table.getName())
+            .ks(keyspace.getName())
             .cf(cfs.name)
             .generation(0)
             .write(cleanedEntries);
@@ -321,7 +321,7 @@ public class StreamingTransferTest extends SchemaLoader
         content.add("test2");
         content.add("test3");
         SSTableReader sstable = SSTableUtils.prepare().write(content);
-        String tablename = sstable.getTableName();
+        String keyspaceName = sstable.getKeyspaceName();
         String cfname = sstable.getColumnFamilyName();
 
         content = new HashSet<>();
@@ -341,7 +341,7 @@ public class StreamingTransferTest extends SchemaLoader
         new StreamPlan("StreamingTransferTest").transferFiles(LOCAL, ranges, Arrays.asList(sstable, sstable2)).execute().get();
 
         // confirm that the sstables were transferred and registered and that 2 keys arrived
-        ColumnFamilyStore cfstore = Table.open(tablename).getColumnFamilyStore(cfname);
+        ColumnFamilyStore cfstore = Keyspace.open(keyspaceName).getColumnFamilyStore(cfname);
         List<Row> rows = Util.getRangeSlice(cfstore);
         assertEquals(2, rows.size());
         assert rows.get(0).key.key.equals(ByteBufferUtil.bytes("test"));
@@ -397,7 +397,7 @@ public class StreamingTransferTest extends SchemaLoader
         // check that only two keys were transferred
         for (Map.Entry<DecoratedKey,String> entry : Arrays.asList(first, last))
         {
-            ColumnFamilyStore store = Table.open(keyspace).getColumnFamilyStore(entry.getValue());
+            ColumnFamilyStore store = Keyspace.open(keyspace).getColumnFamilyStore(entry.getValue());
             List<Row> rows = Util.getRangeSlice(store);
             assertEquals(rows.toString(), 1, rows.size());
             assertEquals(entry.getKey(), rows.get(0).key);
@@ -407,13 +407,13 @@ public class StreamingTransferTest extends SchemaLoader
     @Test
     public void testRandomSSTableTransfer() throws Exception
     {
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
         Mutator mutator = new Mutator()
         {
             public void mutate(String key, String colName, long timestamp) throws Exception
             {
-                ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(table.getName(), cfs.name);
+                ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(keyspace.getName(), cfs.name);
                 cf.addColumn(column(colName, "value", timestamp));
                 cf.addColumn(new Column(ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(new Date(timestamp).toString()), timestamp));
                 RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes(key), cf);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java b/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
index bcc5ba6..7032d73 100644
--- a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
+++ b/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
@@ -38,7 +38,7 @@ import org.junit.Test;
 
 public class EncodedStreamsTest extends SchemaLoader
 {
-    private String tableName = "Keyspace1";
+    private String keyspaceName = "Keyspace1";
     private String standardCFName = "Standard1";
     private String counterCFName = "Counter1";
     private String superCFName = "Super1";
@@ -97,7 +97,7 @@ public class EncodedStreamsTest extends SchemaLoader
 
     private ColumnFamily createCF()
     {
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(tableName, standardCFName);
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(keyspaceName, standardCFName);
         cf.addColumn(column("vijay", "try", 1));
         cf.addColumn(column("to", "be_nice", 1));
         return cf;
@@ -105,7 +105,7 @@ public class EncodedStreamsTest extends SchemaLoader
 
     private ColumnFamily createCounterCF()
     {
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(tableName, counterCFName);
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(keyspaceName, counterCFName);
         cf.addColumn(counterColumn("vijay", 1L, 1));
         cf.addColumn(counterColumn("wants", 1000000, 1));
         return cf;


[07/11] Rename Table to Keyspace

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/dht/RangeStreamer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java
index 2196f4d..f398862 100644
--- a/src/java/org/apache/cassandra/dht/RangeStreamer.java
+++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java
@@ -28,7 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.gms.IFailureDetector;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
@@ -112,24 +112,24 @@ public class RangeStreamer
         sourceFilters.add(filter);
     }
 
-    public void addRanges(String table, Collection<Range<Token>> ranges)
+    public void addRanges(String keyspaceName, Collection<Range<Token>> ranges)
     {
-        Multimap<Range<Token>, InetAddress> rangesForTable = getAllRangesWithSourcesFor(table, ranges);
+        Multimap<Range<Token>, InetAddress> rangesForKeyspace = getAllRangesWithSourcesFor(keyspaceName, ranges);
 
         if (logger.isDebugEnabled())
         {
-            for (Map.Entry<Range<Token>, InetAddress> entry: rangesForTable.entries())
+            for (Map.Entry<Range<Token>, InetAddress> entry: rangesForKeyspace.entries())
                 logger.debug(String.format("%s: range %s exists on %s", description, entry.getKey(), entry.getValue()));
         }
 
-        for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : getRangeFetchMap(rangesForTable, sourceFilters).asMap().entrySet())
+        for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : getRangeFetchMap(rangesForKeyspace, sourceFilters).asMap().entrySet())
         {
             if (logger.isDebugEnabled())
             {
                 for (Range r : entry.getValue())
-                    logger.debug(String.format("%s: range %s from source %s for table %s", description, r, entry.getKey(), table));
+                    logger.debug(String.format("%s: range %s from source %s for keyspace %s", description, r, entry.getKey(), keyspaceName));
             }
-            toFetch.put(table, entry);
+            toFetch.put(keyspaceName, entry);
         }
     }
 
@@ -137,9 +137,9 @@ public class RangeStreamer
      * Get a map of all ranges and their respective sources that are candidates for streaming the given ranges
      * to us. For each range, the list of sources is sorted by proximity relative to the given destAddress.
      */
-    private Multimap<Range<Token>, InetAddress> getAllRangesWithSourcesFor(String table, Collection<Range<Token>> desiredRanges)
+    private Multimap<Range<Token>, InetAddress> getAllRangesWithSourcesFor(String keyspaceName, Collection<Range<Token>> desiredRanges)
     {
-        AbstractReplicationStrategy strat = Table.open(table).getReplicationStrategy();
+        AbstractReplicationStrategy strat = Keyspace.open(keyspaceName).getReplicationStrategy();
         Multimap<Range<Token>, InetAddress> rangeAddresses = strat.getRangeAddresses(metadata.cloneOnlyTokenMap());
 
         Multimap<Range<Token>, InetAddress> rangeSources = ArrayListMultimap.create();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index a80db89..b2316e8 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -306,7 +306,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
 
         private Column unthriftifyCounter(CounterColumn column)
         {
-            //CounterColumns read the counterID from the System table, so need the StorageService running and access
+            //CounterColumns read the counterID from the System keyspace, so need the StorageService running and access
             //to cassandra.yaml. To avoid a Hadoop needing access to yaml return a regular Column.
             return new Column(column.name, ByteBufferUtil.bytes(column.value), 0);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/io/compress/SnappyCompressor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/SnappyCompressor.java b/src/java/org/apache/cassandra/io/compress/SnappyCompressor.java
index ac3affb..7ceaa95 100644
--- a/src/java/org/apache/cassandra/io/compress/SnappyCompressor.java
+++ b/src/java/org/apache/cassandra/io/compress/SnappyCompressor.java
@@ -35,7 +35,7 @@ public class SnappyCompressor implements ICompressor
     static
     {
         if (!isAvailable())
-            logger.warn("Cannot initialize native Snappy library. Compression on new tables will be disabled.");
+            logger.warn("Cannot initialize native Snappy library. Compression on new sstables will be disabled.");
     }
 
     public static SnappyCompressor create(Map<String, String> compressionOptions)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/io/sstable/SSTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index 1112510..eaa4522 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -180,7 +180,7 @@ public abstract class SSTable
         return descriptor.cfname;
     }
 
-    public String getTableName()
+    public String getKeyspaceName()
     {
         return descriptor.ksname;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
index 6c10d95..b1af7c2 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -61,7 +61,7 @@ import org.apache.cassandra.utils.*;
 import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR;
 
 /**
- * SSTableReaders are open()ed by Table.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
  * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
  */
 public class SSTableReader extends SSTable
@@ -79,7 +79,7 @@ public class SSTableReader extends SSTable
      * The field is not serialized to disk, so relying on it for more than what truncate does is not advised.
      *
      * When a new sstable is flushed, maxDataAge is set to the time of creation.
-     * When a sstable is created from compaction, maxDataAge is set to max of all merged tables.
+     * When a sstable is created from compaction, maxDataAge is set to max of all merged sstables.
      *
      * The age is in milliseconds since epoc and is local to this host.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
index 879c9bc..bc98beb 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
@@ -307,7 +307,7 @@ public class SSTableWriter extends SSTable
                                                                                     metadata.getBloomFilterFpChance());
         writeMetadata(descriptor, sstableMetadata);
 
-        // save the table of components
+        // save the talbe of components
         SSTable.appendTOC(descriptor, components);
 
         // remove the 'tmp' marker from all components

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/io/util/FileUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index 005d3bb..a0be773 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.BlacklistedDirectories;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
@@ -398,7 +398,7 @@ public class FileUtils
                 {
                     File directory = BlacklistedDirectories.maybeMarkUnreadable(e.path);
                     if (directory != null)
-                        Table.removeUnreadableSSTables(directory);
+                        Keyspace.removeUnreadableSSTables(directory);
                 }
                 break;
             case ignore:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index 87099c4..2a01fed 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -28,7 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.RingPosition;
@@ -49,24 +49,24 @@ public abstract class AbstractReplicationStrategy
     private static final Logger logger = LoggerFactory.getLogger(AbstractReplicationStrategy.class);
 
     @VisibleForTesting
-    final String tableName;
-    private Table table;
+    final String keyspaceName;
+    private Keyspace keyspace;
     public final Map<String, String> configOptions;
     private final TokenMetadata tokenMetadata;
 
     public IEndpointSnitch snitch;
 
-    AbstractReplicationStrategy(String tableName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
+    AbstractReplicationStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
     {
-        assert tableName != null;
+        assert keyspaceName != null;
         assert snitch != null;
         assert tokenMetadata != null;
         this.tokenMetadata = tokenMetadata;
         this.snitch = snitch;
         this.tokenMetadata.register(this);
         this.configOptions = configOptions == null ? Collections.<String, String>emptyMap() : configOptions;
-        this.tableName = tableName;
-        // lazy-initialize table itself since we don't create them until after the replication strategies
+        this.keyspaceName = keyspaceName;
+        // lazy-initialize keyspace itself since we don't create them until after the replication strategies
     }
 
     private final Map<Token, ArrayList<InetAddress>> cachedEndpoints = new NonBlockingHashMap<Token, ArrayList<InetAddress>>();
@@ -125,20 +125,20 @@ public abstract class AbstractReplicationStrategy
         if (consistency_level == ConsistencyLevel.LOCAL_QUORUM)
         {
             // block for in this context will be localnodes block.
-            return new DatacenterWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, getTable(), callback, writeType);
+            return new DatacenterWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType);
         }
         else if (consistency_level == ConsistencyLevel.EACH_QUORUM)
         {
-            return new DatacenterSyncWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, getTable(), callback, writeType);
+            return new DatacenterSyncWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType);
         }
-        return new WriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, getTable(), callback, writeType);
+        return new WriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType);
     }
 
-    private Table getTable()
+    private Keyspace getKeyspace()
     {
-        if (table == null)
-            table = Table.open(tableName);
-        return table;
+        if (keyspace == null)
+            keyspace = Keyspace.open(keyspaceName);
+        return keyspace;
     }
 
     /**
@@ -222,7 +222,7 @@ public abstract class AbstractReplicationStrategy
         return null;
     }
 
-    private static AbstractReplicationStrategy createInternal(String table,
+    private static AbstractReplicationStrategy createInternal(String keyspaceName,
                                                               Class<? extends AbstractReplicationStrategy> strategyClass,
                                                               TokenMetadata tokenMetadata,
                                                               IEndpointSnitch snitch,
@@ -234,7 +234,7 @@ public abstract class AbstractReplicationStrategy
         try
         {
             Constructor<? extends AbstractReplicationStrategy> constructor = strategyClass.getConstructor(parameterTypes);
-            strategy = constructor.newInstance(table, tokenMetadata, snitch, strategyOptions);
+            strategy = constructor.newInstance(keyspaceName, tokenMetadata, snitch, strategyOptions);
         }
         catch (Exception e)
         {
@@ -243,7 +243,7 @@ public abstract class AbstractReplicationStrategy
         return strategy;
     }
 
-    public static AbstractReplicationStrategy createReplicationStrategy(String table,
+    public static AbstractReplicationStrategy createReplicationStrategy(String keyspaceName,
                                                                         Class<? extends AbstractReplicationStrategy> strategyClass,
                                                                         TokenMetadata tokenMetadata,
                                                                         IEndpointSnitch snitch,
@@ -251,7 +251,7 @@ public abstract class AbstractReplicationStrategy
     {
         try
         {
-            AbstractReplicationStrategy strategy = createInternal(table, strategyClass, tokenMetadata, snitch, strategyOptions);
+            AbstractReplicationStrategy strategy = createInternal(keyspaceName, strategyClass, tokenMetadata, snitch, strategyOptions);
 
             // Because we used to not properly validate unrecognized options, we only log a warning if we find one.
             try
@@ -273,13 +273,13 @@ public abstract class AbstractReplicationStrategy
         }
     }
 
-    public static void validateReplicationStrategy(String table,
+    public static void validateReplicationStrategy(String keyspaceName,
                                                    Class<? extends AbstractReplicationStrategy> strategyClass,
                                                    TokenMetadata tokenMetadata,
                                                    IEndpointSnitch snitch,
                                                    Map<String, String> strategyOptions) throws ConfigurationException
     {
-        AbstractReplicationStrategy strategy = createInternal(table, strategyClass, tokenMetadata, snitch, strategyOptions);
+        AbstractReplicationStrategy strategy = createInternal(keyspaceName, strategyClass, tokenMetadata, snitch, strategyOptions);
         strategy.validateExpectedOptions();
         strategy.validateOptions();
     }
@@ -319,7 +319,7 @@ public abstract class AbstractReplicationStrategy
         for (String key : configOptions.keySet())
         {
             if (!expectedOptions.contains(key))
-                throw new ConfigurationException(String.format("Unrecognized strategy option {%s} passed to %s for keyspace %s", key, getClass().getSimpleName(), tableName));
+                throw new ConfigurationException(String.format("Unrecognized strategy option {%s} passed to %s for keyspace %s", key, getClass().getSimpleName(), keyspaceName));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java b/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
index e00239e..dd1637d 100644
--- a/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
+++ b/src/java/org/apache/cassandra/locator/GossipingPropertyFileSnitch.java
@@ -21,10 +21,10 @@ package org.apache.cassandra.locator;
 import java.net.InetAddress;
 import java.util.Map;
 
-import org.apache.cassandra.db.SystemTable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.EndpointState;
@@ -83,7 +83,7 @@ public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch//
             if (psnitch == null)
             {
                 if (savedEndpoints == null)
-                    savedEndpoints = SystemTable.loadDcRackInfo();
+                    savedEndpoints = SystemKeyspace.loadDcRackInfo();
                 if (savedEndpoints.containsKey(endpoint))
                     return savedEndpoints.get(endpoint).get("data_center");
                 return DEFAULT_DC;
@@ -111,7 +111,7 @@ public class GossipingPropertyFileSnitch extends AbstractNetworkTopologySnitch//
             if (psnitch == null)
             {
                 if (savedEndpoints == null)
-                    savedEndpoints = SystemTable.loadDcRackInfo();
+                    savedEndpoints = SystemKeyspace.loadDcRackInfo();
                 if (savedEndpoints.containsKey(endpoint))
                     return savedEndpoints.get(endpoint).get("rack");
                 return DEFAULT_RACK;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/locator/LocalStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/LocalStrategy.java b/src/java/org/apache/cassandra/locator/LocalStrategy.java
index 39a391d..ae58203 100644
--- a/src/java/org/apache/cassandra/locator/LocalStrategy.java
+++ b/src/java/org/apache/cassandra/locator/LocalStrategy.java
@@ -31,9 +31,9 @@ import org.apache.cassandra.utils.FBUtilities;
 
 public class LocalStrategy extends AbstractReplicationStrategy
 {
-    public LocalStrategy(String table, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
+    public LocalStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
     {
-        super(table, tokenMetadata, snitch, configOptions);
+        super(keyspaceName, tokenMetadata, snitch, configOptions);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index d354019..5e5ecb7 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -50,9 +50,9 @@ public class NetworkTopologyStrategy extends AbstractReplicationStrategy
     private final Map<String, Integer> datacenters;
     private static final Logger logger = LoggerFactory.getLogger(NetworkTopologyStrategy.class);
 
-    public NetworkTopologyStrategy(String table, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions) throws ConfigurationException
+    public NetworkTopologyStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions) throws ConfigurationException
     {
-        super(table, tokenMetadata, snitch, configOptions);
+        super(keyspaceName, tokenMetadata, snitch, configOptions);
         this.snitch = snitch;
 
         Map<String, Integer> newDatacenters = new HashMap<String, Integer>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
index e184d24..b9bd767 100644
--- a/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/OldNetworkTopologyStrategy.java
@@ -37,9 +37,9 @@ import org.apache.cassandra.dht.Token;
  */
 public class OldNetworkTopologyStrategy extends AbstractReplicationStrategy
 {
-    public OldNetworkTopologyStrategy(String table, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
+    public OldNetworkTopologyStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
     {
-        super(table, tokenMetadata, snitch, configOptions);
+        super(keyspaceName, tokenMetadata, snitch, configOptions);
     }
 
     public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/locator/SimpleStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/SimpleStrategy.java b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
index b19b1be..9a5062b 100644
--- a/src/java/org/apache/cassandra/locator/SimpleStrategy.java
+++ b/src/java/org/apache/cassandra/locator/SimpleStrategy.java
@@ -37,9 +37,9 @@ import org.apache.cassandra.dht.Token;
  */
 public class SimpleStrategy extends AbstractReplicationStrategy
 {
-    public SimpleStrategy(String table, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
+    public SimpleStrategy(String keyspaceName, TokenMetadata tokenMetadata, IEndpointSnitch snitch, Map<String, String> configOptions)
     {
-        super(table, tokenMetadata, snitch, configOptions);
+        super(keyspaceName, tokenMetadata, snitch, configOptions);
     }
 
     public List<InetAddress> calculateNaturalEndpoints(Token token, TokenMetadata metadata)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/locator/TokenMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java
index 1f6682b..cf76718 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -656,13 +656,13 @@ public class TokenMetadata
         return sortedTokens;
     }
 
-    private Multimap<Range<Token>, InetAddress> getPendingRangesMM(String table)
+    private Multimap<Range<Token>, InetAddress> getPendingRangesMM(String keyspaceName)
     {
-        Multimap<Range<Token>, InetAddress> map = pendingRanges.get(table);
+        Multimap<Range<Token>, InetAddress> map = pendingRanges.get(keyspaceName);
         if (map == null)
         {
             map = HashMultimap.create();
-            Multimap<Range<Token>, InetAddress> priorMap = pendingRanges.putIfAbsent(table, map);
+            Multimap<Range<Token>, InetAddress> priorMap = pendingRanges.putIfAbsent(keyspaceName, map);
             if (priorMap != null)
                 map = priorMap;
         }
@@ -670,15 +670,15 @@ public class TokenMetadata
     }
 
     /** a mutable map may be returned but caller should not modify it */
-    public Map<Range<Token>, Collection<InetAddress>> getPendingRanges(String table)
+    public Map<Range<Token>, Collection<InetAddress>> getPendingRanges(String keyspaceName)
     {
-        return getPendingRangesMM(table).asMap();
+        return getPendingRangesMM(keyspaceName).asMap();
     }
 
-    public List<Range<Token>> getPendingRanges(String table, InetAddress endpoint)
+    public List<Range<Token>> getPendingRanges(String keyspaceName, InetAddress endpoint)
     {
         List<Range<Token>> ranges = new ArrayList<Range<Token>>();
-        for (Map.Entry<Range<Token>, InetAddress> entry : getPendingRangesMM(table).entries())
+        for (Map.Entry<Range<Token>, InetAddress> entry : getPendingRangesMM(keyspaceName).entries())
         {
             if (entry.getValue().equals(endpoint))
             {
@@ -688,9 +688,9 @@ public class TokenMetadata
         return ranges;
     }
 
-    public void setPendingRanges(String table, Multimap<Range<Token>, InetAddress> rangeMap)
+    public void setPendingRanges(String keyspaceName, Multimap<Range<Token>, InetAddress> rangeMap)
     {
-        pendingRanges.put(table, rangeMap);
+        pendingRanges.put(keyspaceName, rangeMap);
     }
 
     public Token getPredecessor(Token token)
@@ -926,9 +926,9 @@ public class TokenMetadata
         subscribers.remove(subscriber);
     }
 
-    public Collection<InetAddress> pendingEndpointsFor(Token token, String table)
+    public Collection<InetAddress> pendingEndpointsFor(Token token, String keyspaceName)
     {
-        Map<Range<Token>, Collection<InetAddress>> ranges = getPendingRanges(table);
+        Map<Range<Token>, Collection<InetAddress>> ranges = getPendingRanges(keyspaceName);
         if (ranges.isEmpty())
             return Collections.emptyList();
 
@@ -945,10 +945,10 @@ public class TokenMetadata
     /**
      * @deprecated retained for benefit of old tests
      */
-    public Collection<InetAddress> getWriteEndpoints(Token token, String table, Collection<InetAddress> naturalEndpoints)
+    public Collection<InetAddress> getWriteEndpoints(Token token, String keyspaceName, Collection<InetAddress> naturalEndpoints)
     {
         ArrayList<InetAddress> endpoints = new ArrayList<InetAddress>();
-        Iterables.addAll(endpoints, Iterables.concat(naturalEndpoints, pendingEndpointsFor(token, table)));
+        Iterables.addAll(endpoints, Iterables.concat(naturalEndpoints, pendingEndpointsFor(token, keyspaceName)));
         return endpoints;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
index 7c7cfa6..bc27d6f 100644
--- a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
@@ -24,7 +24,7 @@ import com.yammer.metrics.core.Histogram;
 import com.yammer.metrics.core.MetricName;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.sstable.SSTableMetadata;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.EstimatedHistogram;
@@ -162,7 +162,7 @@ public class ColumnFamilyMetrics
             public Integer value()
             {
                 // TODO this actually isn't a good measure of pending tasks
-                return Table.switchLock.getQueueLength();
+                return Keyspace.switchLock.getQueueLength();
             }
         });
         liveSSTableCount = Metrics.newGauge(factory.createMetricName("LiveSSTableCount"), new Gauge<Integer>()
@@ -322,7 +322,7 @@ public class ColumnFamilyMetrics
 
         ColumnFamilyMetricNameFactory(ColumnFamilyStore cfs)
         {
-            this.keyspaceName = cfs.table.getName();
+            this.keyspaceName = cfs.keyspace.getName();
             this.columnFamilyName = cfs.name;
             isIndex = cfs.isIndex();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
index ae098ca..dd3803f 100644
--- a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
@@ -29,7 +29,7 @@ import com.yammer.metrics.core.MetricName;
 
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
 
@@ -60,9 +60,9 @@ public class CompactionMetrics implements CompactionManager.CompactionExecutorSt
             public Integer value()
             {
                 int n = 0;
-                for (String tableName : Schema.instance.getTables())
+                for (String keyspaceName : Schema.instance.getKeyspaces())
                 {
-                    for (ColumnFamilyStore cfs : Table.open(tableName).getColumnFamilyStores())
+                    for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
                         n += cfs.getCompactionStrategy().getEstimatedRemainingTasks();
                 }
                 for (ThreadPoolExecutor collector : collectors)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
index a12be19..1585342 100644
--- a/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
@@ -22,7 +22,7 @@ import java.util.Map.Entry;
 import java.util.concurrent.ExecutionException;
 
 import org.apache.cassandra.db.HintedHandOffManager;
-import org.apache.cassandra.db.SystemTable;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.utils.UUIDGen;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -73,7 +73,7 @@ public class HintedHandoffMetrics
             if (diffrence == 0)
                 continue;
             logger.warn("{} has {} dropped hints, because node is down past configured hint window.", entry.getKey(), diffrence);
-            SystemTable.updateHintsDropped(entry.getKey(), UUIDGen.getTimeUUID(), (int) diffrence);
+            SystemKeyspace.updateHintsDropped(entry.getKey(), UUIDGen.getTimeUUID(), (int) diffrence);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/repair/RepairJob.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index be3744d..277075c 100644
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.db.SnapshotCommand;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.net.IAsyncCallback;
@@ -109,7 +109,7 @@ public class RepairJob
         if (isSequential)
             makeSnapshots(endpoints);
 
-        this.gcBefore = Table.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).gcBefore(System.currentTimeMillis());
+        this.gcBefore = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).gcBefore(System.currentTimeMillis());
 
         for (InetAddress endpoint : allEndpoints)
             treeRequests.add(endpoint);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
index 3057a41..acc8aab 100644
--- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
+++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
@@ -18,7 +18,7 @@
 package org.apache.cassandra.repair;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
@@ -43,7 +43,7 @@ public class RepairMessageVerbHandler implements IVerbHandler<RepairMessage>
             case VALIDATION_REQUEST:
                 ValidationRequest validationRequest = (ValidationRequest) message.payload;
                 // trigger read-only compaction
-                ColumnFamilyStore store = Table.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily);
+                ColumnFamilyStore store = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily);
                 Validator validator = new Validator(desc, message.from, validationRequest.gcBefore);
                 CompactionManager.instance.submitValidation(store, validator);
                 break;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
index 4670ce6..134894d 100644
--- a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
+++ b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
@@ -25,7 +25,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.SyncComplete;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
index f944507..051085b 100644
--- a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
+++ b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.ReadResponse;
 import org.apache.cassandra.db.Row;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.UnavailableException;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
@@ -61,7 +61,7 @@ public abstract class AbstractReadExecutor
     {
         unfiltered = allReplicas;
         this.endpoints = queryTargets;
-        this.resolver = new RowDigestResolver(command.table, command.key);
+        this.resolver = new RowDigestResolver(command.ksName, command.key);
         this.handler = new ReadCallback<ReadResponse, Row>(resolver, consistency_level, command, this.endpoints);
         this.command = command;
         this.cfs = cfs;
@@ -124,9 +124,9 @@ public abstract class AbstractReadExecutor
 
     public static AbstractReadExecutor getReadExecutor(ReadCommand command, ConsistencyLevel consistency_level) throws UnavailableException
     {
-        Table table = Table.open(command.table);
-        List<InetAddress> allReplicas = StorageProxy.getLiveSortedEndpoints(table, command.key);
-        CFMetaData metaData = Schema.instance.getCFMetaData(command.table, command.cfName);
+        Keyspace keyspace = Keyspace.open(command.ksName);
+        List<InetAddress> allReplicas = StorageProxy.getLiveSortedEndpoints(keyspace, command.key);
+        CFMetaData metaData = Schema.instance.getCFMetaData(command.ksName, command.cfName);
 
         ReadRepairDecision rrDecision = metaData.newReadRepairDecision();
          
@@ -134,14 +134,14 @@ public abstract class AbstractReadExecutor
             ReadRepairMetrics.attempted.mark();
         }
 
-        List<InetAddress> queryTargets = consistency_level.filterForQuery(table, allReplicas, rrDecision);
+        List<InetAddress> queryTargets = consistency_level.filterForQuery(keyspace, allReplicas, rrDecision);
 
         if (StorageService.instance.isClientMode())
         {
             return new DefaultReadExecutor(null, command, consistency_level, allReplicas, queryTargets);
         }
 
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(command.cfName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(command.cfName);
 
         switch (metaData.getSpeculativeRetry().type)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/AbstractRowResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractRowResolver.java b/src/java/org/apache/cassandra/service/AbstractRowResolver.java
index b6204e2..2ebaaf1 100644
--- a/src/java/org/apache/cassandra/service/AbstractRowResolver.java
+++ b/src/java/org/apache/cassandra/service/AbstractRowResolver.java
@@ -33,14 +33,14 @@ public abstract class AbstractRowResolver implements IResponseResolver<ReadRespo
 {
     protected static final Logger logger = LoggerFactory.getLogger(AbstractRowResolver.class);
 
-    protected final String table;
+    protected final String keyspaceName;
     protected final Set<MessageIn<ReadResponse>> replies = new NonBlockingHashSet<MessageIn<ReadResponse>>();
     protected final DecoratedKey key;
 
-    public AbstractRowResolver(ByteBuffer key, String table)
+    public AbstractRowResolver(ByteBuffer key, String keyspaceName)
     {
         this.key = StorageService.getPartitioner().decorateKey(key);
-        this.table = table;
+        this.keyspaceName = keyspaceName;
     }
 
     public boolean preprocess(MessageIn<ReadResponse> message)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
index 1740ee2..95f9ac4 100644
--- a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
@@ -25,7 +25,7 @@ import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.net.IAsyncCallback;
@@ -35,7 +35,7 @@ import org.apache.cassandra.utils.SimpleCondition;
 public abstract class AbstractWriteResponseHandler implements IAsyncCallback
 {
     private final SimpleCondition condition = new SimpleCondition();
-    protected final Table table;
+    protected final Keyspace keyspace;
     protected final long start;
     protected final Collection<InetAddress> naturalEndpoints;
     protected final ConsistencyLevel consistencyLevel;
@@ -47,14 +47,14 @@ public abstract class AbstractWriteResponseHandler implements IAsyncCallback
      * @param pendingEndpoints
      * @param callback A callback to be called when the write is successful.
      */
-    protected AbstractWriteResponseHandler(Table table,
+    protected AbstractWriteResponseHandler(Keyspace keyspace,
                                            Collection<InetAddress> naturalEndpoints,
                                            Collection<InetAddress> pendingEndpoints,
                                            ConsistencyLevel consistencyLevel,
                                            Runnable callback,
                                            WriteType writeType)
     {
-        this.table = table;
+        this.keyspace = keyspace;
         this.pendingEndpoints = pendingEndpoints;
         this.start = System.nanoTime();
         this.consistencyLevel = consistencyLevel;
@@ -85,7 +85,7 @@ public abstract class AbstractWriteResponseHandler implements IAsyncCallback
     {
         // During bootstrap, we have to include the pending endpoints or we may fail the consistency level
         // guarantees (see #833)
-        return consistencyLevel.blockFor(table) + pendingEndpoints.size();
+        return consistencyLevel.blockFor(keyspace) + pendingEndpoints.size();
     }
 
     protected abstract int ackCount();
@@ -95,7 +95,7 @@ public abstract class AbstractWriteResponseHandler implements IAsyncCallback
 
     public void assureSufficientLiveNodes() throws UnavailableException
     {
-        consistencyLevel.assureSufficientLiveNodes(table, Iterables.filter(Iterables.concat(naturalEndpoints, pendingEndpoints), isAlive));
+        consistencyLevel.assureSufficientLiveNodes(keyspace, Iterables.filter(Iterables.concat(naturalEndpoints, pendingEndpoints), isAlive));
     }
 
     protected void signal()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index 53460b9..2f16b31 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -136,18 +136,18 @@ public class ActiveRepairService
     /**
      * Return all of the neighbors with whom we share the provided range.
      *
-     * @param table table to repair
+     * @param keyspaceName keyspace to repair
      * @param toRepair token to repair
      * @param isLocal need to use only nodes from local datacenter
      *
      * @return neighbors with whom we share the provided range
      */
-    public static Set<InetAddress> getNeighbors(String table, Range<Token> toRepair, boolean isLocal)
+    public static Set<InetAddress> getNeighbors(String keyspaceName, Range<Token> toRepair, boolean isLocal)
     {
         StorageService ss = StorageService.instance;
-        Map<Range<Token>, List<InetAddress>> replicaSets = ss.getRangeToAddressMap(table);
+        Map<Range<Token>, List<InetAddress>> replicaSets = ss.getRangeToAddressMap(keyspaceName);
         Range<Token> rangeSuperSet = null;
-        for (Range<Token> range : ss.getLocalRanges(table))
+        for (Range<Token> range : ss.getLocalRanges(keyspaceName))
         {
             if (range.contains(toRepair))
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 7d22840..f49d549 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -227,14 +227,14 @@ public class CassandraDaemon
         if (CacheService.instance == null) // should never happen
             throw new RuntimeException("Failed to initialize Cache Service.");
 
-        // check the system table to keep user from shooting self in foot by changing partitioner, cluster name, etc.
-        // we do a one-off scrub of the system table first; we can't load the list of the rest of the tables,
-        // until system table is opened.
-        for (CFMetaData cfm : Schema.instance.getTableMetaData(Table.SYSTEM_KS).values())
-            ColumnFamilyStore.scrubDataDirectories(Table.SYSTEM_KS, cfm.cfName);
+        // check the system keyspace to keep user from shooting self in foot by changing partitioner, cluster name, etc.
+        // we do a one-off scrub of the system keyspace first; we can't load the list of the rest of the keyspaces,
+        // until system keyspace is opened.
+        for (CFMetaData cfm : Schema.instance.getKeyspaceMetaData(Keyspace.SYSTEM_KS).values())
+            ColumnFamilyStore.scrubDataDirectories(Keyspace.SYSTEM_KS, cfm.cfName);
         try
         {
-            SystemTable.checkHealth();
+            SystemKeyspace.checkHealth();
         }
         catch (ConfigurationException e)
         {
@@ -245,16 +245,16 @@ public class CassandraDaemon
         // load keyspace descriptions.
         DatabaseDescriptor.loadSchemas();
 
-        // clean up debris in the rest of the tables
-        for (String table : Schema.instance.getTables())
+        // clean up debris in the rest of the keyspaces
+        for (String keyspaceName : Schema.instance.getKeyspaces())
         {
-            for (CFMetaData cfm : Schema.instance.getTableMetaData(table).values())
+            for (CFMetaData cfm : Schema.instance.getKeyspaceMetaData(keyspaceName).values())
             {
-                if (LegacyLeveledManifest.manifestNeedsMigration(table,cfm.cfName))
+                if (LegacyLeveledManifest.manifestNeedsMigration(keyspaceName,cfm.cfName))
                 {
                     try
                     {
-                        LegacyLeveledManifest.migrateManifests(table, cfm.cfName);
+                        LegacyLeveledManifest.migrateManifests(keyspaceName, cfm.cfName);
                     }
                     catch (IOException e)
                     {
@@ -263,24 +263,24 @@ public class CassandraDaemon
                     }
                 }
 
-                ColumnFamilyStore.scrubDataDirectories(table, cfm.cfName);
+                ColumnFamilyStore.scrubDataDirectories(keyspaceName, cfm.cfName);
             }
         }
         // clean up compaction leftovers
-        SetMultimap<Pair<String, String>, Integer> unfinishedCompactions = SystemTable.getUnfinishedCompactions();
+        SetMultimap<Pair<String, String>, Integer> unfinishedCompactions = SystemKeyspace.getUnfinishedCompactions();
         for (Pair<String, String> kscf : unfinishedCompactions.keySet())
         {
             ColumnFamilyStore.removeUnfinishedCompactionLeftovers(kscf.left, kscf.right, unfinishedCompactions.get(kscf));
         }
-        SystemTable.discardCompactionsInProgress();
+        SystemKeyspace.discardCompactionsInProgress();
 
         // initialize keyspaces
-        for (String table : Schema.instance.getTables())
+        for (String keyspaceName : Schema.instance.getKeyspaces())
         {
             if (logger.isDebugEnabled())
-                logger.debug("opening keyspace " + table);
+                logger.debug("opening keyspace " + keyspaceName);
             // disable auto compaction until commit log replay ends
-            for (ColumnFamilyStore cfs : Table.open(table).getColumnFamilyStores())
+            for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
             {
                 for (ColumnFamilyStore store : cfs.concatWithIndexes())
                 {
@@ -315,9 +315,9 @@ public class CassandraDaemon
         }
 
         // enable auto compaction
-        for (Table table : Table.all())
+        for (Keyspace keyspace : Keyspace.all())
         {
-            for (ColumnFamilyStore cfs : table.getColumnFamilyStores())
+            for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
             {
                 for (final ColumnFamilyStore store : cfs.concatWithIndexes())
                 {
@@ -330,9 +330,9 @@ public class CassandraDaemon
         {
             public void run()
             {
-                for (Table table : Table.all())
+                for (Keyspace keyspaceName : Keyspace.all())
                 {
-                    for (ColumnFamilyStore cf : table.getColumnFamilyStores())
+                    for (ColumnFamilyStore cf : keyspaceName.getColumnFamilyStores())
                     {
                         for (ColumnFamilyStore store : cf.concatWithIndexes())
                             CompactionManager.instance.submitBackground(store);
@@ -345,7 +345,7 @@ public class CassandraDaemon
         // MeteredFlusher can block if flush queue fills up, so don't put on scheduledTasks
         StorageService.optionalTasks.scheduleWithFixedDelay(new MeteredFlusher(), 1000, 1000, TimeUnit.MILLISECONDS);
 
-        SystemTable.finishStartup();
+        SystemKeyspace.finishStartup();
 
         // start server internals
         StorageService.instance.registerDaemon(this);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/ClientState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index e911a3e..5f238c5 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -31,8 +31,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.SystemTable;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
@@ -56,13 +56,13 @@ public class ClientState
     static
     {
         // We want these system cfs to be always readable since many tools rely on them (nodetool, cqlsh, bulkloader, etc.)
-        String[] cfs =  new String[] { SystemTable.LOCAL_CF,
-                                       SystemTable.PEERS_CF,
-                                       SystemTable.SCHEMA_KEYSPACES_CF,
-                                       SystemTable.SCHEMA_COLUMNFAMILIES_CF,
-                                       SystemTable.SCHEMA_COLUMNS_CF };
+        String[] cfs =  new String[] { SystemKeyspace.LOCAL_CF,
+                                       SystemKeyspace.PEERS_CF,
+                                       SystemKeyspace.SCHEMA_KEYSPACES_CF,
+                                       SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF,
+                                       SystemKeyspace.SCHEMA_COLUMNS_CF };
         for (String cf : cfs)
-            READABLE_SYSTEM_RESOURCES.add(DataResource.columnFamily(Table.SYSTEM_KS, cf));
+            READABLE_SYSTEM_RESOURCES.add(DataResource.columnFamily(Keyspace.SYSTEM_KS, cf));
 
         PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthenticator().protectedResources());
         PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthorizer().protectedResources());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
index 4941d04..81ae1f3 100644
--- a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
@@ -23,7 +23,7 @@ import java.util.HashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
 import org.apache.cassandra.net.MessageIn;
@@ -44,15 +44,15 @@ public class DatacenterSyncWriteResponseHandler extends AbstractWriteResponseHan
     public DatacenterSyncWriteResponseHandler(Collection<InetAddress> naturalEndpoints,
                                               Collection<InetAddress> pendingEndpoints,
                                               ConsistencyLevel consistencyLevel,
-                                              Table table,
+                                              Keyspace keyspace,
                                               Runnable callback,
                                               WriteType writeType)
     {
         // Response is been managed by the map so make it 1 for the superclass.
-        super(table, naturalEndpoints, pendingEndpoints, consistencyLevel, callback, writeType);
+        super(keyspace, naturalEndpoints, pendingEndpoints, consistencyLevel, callback, writeType);
         assert consistencyLevel == ConsistencyLevel.EACH_QUORUM;
 
-        strategy = (NetworkTopologyStrategy) table.getReplicationStrategy();
+        strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
 
         for (String dc : strategy.getDatacenters())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
index e63ebd1..2455655 100644
--- a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
@@ -21,7 +21,7 @@ import java.net.InetAddress;
 import java.util.Collection;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.db.ConsistencyLevel;
@@ -37,11 +37,11 @@ public class DatacenterWriteResponseHandler extends WriteResponseHandler
     public DatacenterWriteResponseHandler(Collection<InetAddress> naturalEndpoints,
                                           Collection<InetAddress> pendingEndpoints,
                                           ConsistencyLevel consistencyLevel,
-                                          Table table,
+                                          Keyspace keyspace,
                                           Runnable callback,
                                           WriteType writeType)
     {
-        super(naturalEndpoints, pendingEndpoints, consistencyLevel, table, callback, writeType);
+        super(naturalEndpoints, pendingEndpoints, consistencyLevel, keyspace, callback, writeType);
         assert consistencyLevel == ConsistencyLevel.LOCAL_QUORUM;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java b/src/java/org/apache/cassandra/service/MigrationManager.java
index cce674f..b24c3c1 100644
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ b/src/java/org/apache/cassandra/service/MigrationManager.java
@@ -292,7 +292,7 @@ public class MigrationManager implements IEndpointStateChangeSubscriber
         {
             protected void runMayThrow() throws IOException, ConfigurationException
             {
-                DefsTable.mergeSchema(schema);
+                DefsTables.mergeSchema(schema);
             }
         });
 
@@ -332,9 +332,9 @@ public class MigrationManager implements IEndpointStateChangeSubscriber
             logger.debug("Truncating schema tables...");
 
         // truncate schema tables
-        SystemTable.schemaCFS(SystemTable.SCHEMA_KEYSPACES_CF).truncateBlocking();
-        SystemTable.schemaCFS(SystemTable.SCHEMA_COLUMNFAMILIES_CF).truncateBlocking();
-        SystemTable.schemaCFS(SystemTable.SCHEMA_COLUMNS_CF).truncateBlocking();
+        SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_KEYSPACES_CF).truncateBlocking();
+        SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF).truncateBlocking();
+        SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_COLUMNS_CF).truncateBlocking();
 
         if (logger.isDebugEnabled())
             logger.debug("Clearing local schema keyspace definitions...");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/MigrationTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationTask.java b/src/java/org/apache/cassandra/service/MigrationTask.java
index ac7c6be..df216ea 100644
--- a/src/java/org/apache/cassandra/service/MigrationTask.java
+++ b/src/java/org/apache/cassandra/service/MigrationTask.java
@@ -25,7 +25,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.DefsTable;
+import org.apache.cassandra.db.DefsTables;
 import org.apache.cassandra.db.RowMutation;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.net.IAsyncCallback;
@@ -63,7 +63,7 @@ class MigrationTask extends WrappedRunnable
             {
                 try
                 {
-                    DefsTable.mergeSchema(message.payload);
+                    DefsTables.mergeSchema(message.payload);
                 }
                 catch (IOException e)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/RangeSliceResponseResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/RangeSliceResponseResolver.java b/src/java/org/apache/cassandra/service/RangeSliceResponseResolver.java
index 4aab87d..72ea69c 100644
--- a/src/java/org/apache/cassandra/service/RangeSliceResponseResolver.java
+++ b/src/java/org/apache/cassandra/service/RangeSliceResponseResolver.java
@@ -47,15 +47,15 @@ public class RangeSliceResponseResolver implements IResponseResolver<RangeSliceR
         }
     };
 
-    private final String table;
+    private final String keyspaceName;
     private final long timestamp;
     private List<InetAddress> sources;
     protected final Collection<MessageIn<RangeSliceReply>> responses = new ConcurrentLinkedQueue<MessageIn<RangeSliceReply>>();
     public final List<AsyncOneResponse> repairResults = new ArrayList<AsyncOneResponse>();
 
-    public RangeSliceResponseResolver(String table, long timestamp)
+    public RangeSliceResponseResolver(String keyspaceName, long timestamp)
     {
-        this.table = table;
+        this.keyspaceName = keyspaceName;
         this.timestamp = timestamp;
     }
 
@@ -160,7 +160,7 @@ public class RangeSliceResponseResolver implements IResponseResolver<RangeSliceR
             }
             // resolved can be null even if versions doesn't have all nulls because of the call to removeDeleted in resolveSuperSet
             if (resolved != null)
-                repairResults.addAll(RowDataResolver.scheduleRepairs(resolved, table, key, versions, versionSources));
+                repairResults.addAll(RowDataResolver.scheduleRepairs(resolved, keyspaceName, key, versions, versionSources));
             versions.clear();
             versionSources.clear();
             return new Row(key, resolved);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/ReadCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ReadCallback.java b/src/java/org/apache/cassandra/service/ReadCallback.java
index bd8b025..e50a1db 100644
--- a/src/java/org/apache/cassandra/service/ReadCallback.java
+++ b/src/java/org/apache/cassandra/service/ReadCallback.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ReadCommand;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.UnavailableException;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
@@ -55,22 +55,22 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag
     private final IReadCommand command;
     private final ConsistencyLevel consistencyLevel;
     private final AtomicInteger received = new AtomicInteger(0);
-    private final Table table; // TODO push this into ConsistencyLevel?
+    private final Keyspace keyspace; // TODO push this into ConsistencyLevel?
 
     /**
      * Constructor when response count has to be calculated and blocked for.
      */
     public ReadCallback(IResponseResolver<TMessage, TResolved> resolver, ConsistencyLevel consistencyLevel, IReadCommand command, List<InetAddress> filteredEndpoints)
     {
-        this(resolver, consistencyLevel, consistencyLevel.blockFor(Table.open(command.getKeyspace())), command, Table.open(command.getKeyspace()), filteredEndpoints);
+        this(resolver, consistencyLevel, consistencyLevel.blockFor(Keyspace.open(command.getKeyspace())), command, Keyspace.open(command.getKeyspace()), filteredEndpoints);
         if (logger.isTraceEnabled())
             logger.trace(String.format("Blockfor is %s; setting up requests to %s", blockfor, StringUtils.join(this.endpoints, ",")));
     }
 
-    private ReadCallback(IResponseResolver<TMessage, TResolved> resolver, ConsistencyLevel consistencyLevel, int blockfor, IReadCommand command, Table table, List<InetAddress> endpoints)
+    private ReadCallback(IResponseResolver<TMessage, TResolved> resolver, ConsistencyLevel consistencyLevel, int blockfor, IReadCommand command, Keyspace keyspace, List<InetAddress> endpoints)
     {
         this.command = command;
-        this.table = table;
+        this.keyspace = keyspace;
         this.blockfor = blockfor;
         this.consistencyLevel = consistencyLevel;
         this.resolver = resolver;
@@ -80,7 +80,7 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag
 
     public ReadCallback<TMessage, TResolved> withNewResolver(IResponseResolver<TMessage, TResolved> newResolver)
     {
-        return new ReadCallback<TMessage, TResolved>(newResolver, consistencyLevel, blockfor, command, table, endpoints);
+        return new ReadCallback<TMessage, TResolved>(newResolver, consistencyLevel, blockfor, command, keyspace, endpoints);
     }
 
     public boolean await(long timePastStart, TimeUnit unit)
@@ -157,7 +157,7 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag
 
     public void assureSufficientLiveNodes() throws UnavailableException
     {
-        consistencyLevel.assureSufficientLiveNodes(table, endpoints);
+        consistencyLevel.assureSufficientLiveNodes(keyspace, endpoints);
     }
 
     public boolean isLatencyForSnitch()
@@ -186,7 +186,7 @@ public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessag
                 ReadRepairMetrics.repairedBackground.mark();
                 
                 ReadCommand readCommand = (ReadCommand) command;
-                final RowDataResolver repairResolver = new RowDataResolver(readCommand.table, readCommand.key, readCommand.filter(), readCommand.timestamp);
+                final RowDataResolver repairResolver = new RowDataResolver(readCommand.ksName, readCommand.key, readCommand.filter(), readCommand.timestamp);
                 AsyncRepairCallback repairHandler = new AsyncRepairCallback(repairResolver, endpoints.size());
 
                 MessageOut<ReadCommand> message = ((ReadCommand) command).createMessage();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/RowDataResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/RowDataResolver.java b/src/java/org/apache/cassandra/service/RowDataResolver.java
index 69cd381..00f8753 100644
--- a/src/java/org/apache/cassandra/service/RowDataResolver.java
+++ b/src/java/org/apache/cassandra/service/RowDataResolver.java
@@ -41,9 +41,9 @@ public class RowDataResolver extends AbstractRowResolver
     private final IDiskAtomFilter filter;
     private final long timestamp;
 
-    public RowDataResolver(String table, ByteBuffer key, IDiskAtomFilter qFilter, long timestamp)
+    public RowDataResolver(String keyspaceName, ByteBuffer key, IDiskAtomFilter qFilter, long timestamp)
     {
-        super(key, table);
+        super(key, keyspaceName);
         this.filter = qFilter;
         this.timestamp = timestamp;
     }
@@ -88,7 +88,7 @@ public class RowDataResolver extends AbstractRowResolver
             // send updates to any replica that was missing part of the full row
             // (resolved can be null even if versions doesn't have all nulls because of the call to removeDeleted in resolveSuperSet)
             if (resolved != null)
-                repairResults = scheduleRepairs(resolved, table, key, versions, endpoints);
+                repairResults = scheduleRepairs(resolved, keyspaceName, key, versions, endpoints);
         }
         else
         {
@@ -105,7 +105,7 @@ public class RowDataResolver extends AbstractRowResolver
      * For each row version, compare with resolved (the superset of all row versions);
      * if it is missing anything, send a mutation to the endpoint it come from.
      */
-    public static List<AsyncOneResponse> scheduleRepairs(ColumnFamily resolved, String table, DecoratedKey key, List<ColumnFamily> versions, List<InetAddress> endpoints)
+    public static List<AsyncOneResponse> scheduleRepairs(ColumnFamily resolved, String keyspaceName, DecoratedKey key, List<ColumnFamily> versions, List<InetAddress> endpoints)
     {
         List<AsyncOneResponse> results = new ArrayList<AsyncOneResponse>(versions.size());
 
@@ -116,7 +116,7 @@ public class RowDataResolver extends AbstractRowResolver
                 continue;
 
             // create and send the row mutation message based on the diff
-            RowMutation rowMutation = new RowMutation(table, key.key, diffCf);
+            RowMutation rowMutation = new RowMutation(keyspaceName, key.key, diffCf);
             MessageOut repairMessage;
             // use a separate verb here because we don't want these to be get the white glove hint-
             // on-timeout behavior that a "real" mutation gets

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/RowDigestResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/RowDigestResolver.java b/src/java/org/apache/cassandra/service/RowDigestResolver.java
index 7c4e394..bc4cf49 100644
--- a/src/java/org/apache/cassandra/service/RowDigestResolver.java
+++ b/src/java/org/apache/cassandra/service/RowDigestResolver.java
@@ -27,9 +27,9 @@ import org.apache.cassandra.net.MessageIn;
 
 public class RowDigestResolver extends AbstractRowResolver
 {
-    public RowDigestResolver(String table, ByteBuffer key)
+    public RowDigestResolver(String keyspaceName, ByteBuffer key)
     {
-        super(key, table);
+        super(key, keyspaceName);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/ScheduledRangeTransferExecutorService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ScheduledRangeTransferExecutorService.java b/src/java/org/apache/cassandra/service/ScheduledRangeTransferExecutorService.java
index ca8ea02..a231737 100644
--- a/src/java/org/apache/cassandra/service/ScheduledRangeTransferExecutorService.java
+++ b/src/java/org/apache/cassandra/service/ScheduledRangeTransferExecutorService.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.SystemTable;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.slf4j.Logger;
@@ -74,7 +74,7 @@ class RangeTransfer implements Runnable
 
     public void run()
     {
-        UntypedResultSet res = processInternal("SELECT * FROM system." + SystemTable.RANGE_XFERS_CF);
+        UntypedResultSet res = processInternal("SELECT * FROM system." + SystemKeyspace.RANGE_XFERS_CF);
 
         if (res.size() < 1)
         {
@@ -104,7 +104,7 @@ class RangeTransfer implements Runnable
         {
             LOG.debug("Removing queued entry for transfer of {}", token);
             processInternal(String.format("DELETE FROM system.%s WHERE token_bytes = '%s'",
-                                          SystemTable.RANGE_XFERS_CF,
+                                          SystemKeyspace.RANGE_XFERS_CF,
                                           ByteBufferUtil.bytesToHex(tokenBytes)));
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
index a24164f..718c79f 100644
--- a/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
+++ b/src/java/org/apache/cassandra/service/SnapshotVerbHandler.java
@@ -21,7 +21,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.SnapshotCommand;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
@@ -36,9 +36,9 @@ public class SnapshotVerbHandler implements IVerbHandler<SnapshotCommand>
     {
         SnapshotCommand command = message.payload;
         if (command.clear_snapshot)
-            Table.open(command.keyspace).clearSnapshot(command.snapshot_name);
+            Keyspace.open(command.keyspace).clearSnapshot(command.snapshot_name);
         else
-            Table.open(command.keyspace).getColumnFamilyStore(command.column_family).snapshot(command.snapshot_name);
+            Keyspace.open(command.keyspace).getColumnFamilyStore(command.column_family).snapshot(command.snapshot_name);
         Tracing.trace("Enqueuing response to snapshot request {} to {}", command.snapshot_name, message.from);
         MessagingService.instance().sendReply(new MessageOut(MessagingService.Verb.REQUEST_RESPONSE), id, message.from);
     }


[08/11] Rename Table to Keyspace

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/SystemTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemTable.java b/src/java/org/apache/cassandra/db/SystemTable.java
deleted file mode 100644
index 8528515..0000000
--- a/src/java/org/apache/cassandra/db/SystemTable.java
+++ /dev/null
@@ -1,814 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.base.Function;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.SetMultimap;
-import com.google.common.collect.Sets;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.service.paxos.Commit;
-import org.apache.cassandra.service.paxos.PaxosState;
-import org.apache.cassandra.thrift.cassandraConstants;
-import org.apache.cassandra.utils.*;
-
-import static org.apache.cassandra.cql3.QueryProcessor.processInternal;
-
-public class SystemTable
-{
-    private static final Logger logger = LoggerFactory.getLogger(SystemTable.class);
-
-    // see CFMetaData for schema definitions
-    public static final String PEERS_CF = "peers";
-    public static final String PEER_EVENTS_CF = "peer_events";
-    public static final String LOCAL_CF = "local";
-    public static final String INDEX_CF = "IndexInfo";
-    public static final String COUNTER_ID_CF = "NodeIdInfo";
-    public static final String HINTS_CF = "hints";
-    public static final String RANGE_XFERS_CF = "range_xfers";
-    public static final String BATCHLOG_CF = "batchlog";
-    // see layout description in the DefsTable class header
-    public static final String SCHEMA_KEYSPACES_CF = "schema_keyspaces";
-    public static final String SCHEMA_COLUMNFAMILIES_CF = "schema_columnfamilies";
-    public static final String SCHEMA_COLUMNS_CF = "schema_columns";
-    public static final String SCHEMA_TRIGGERS_CF = "schema_triggers";
-    public static final String COMPACTION_LOG = "compactions_in_progress";
-    public static final String PAXOS_CF = "paxos";
-
-    private static final String LOCAL_KEY = "local";
-    private static final ByteBuffer ALL_LOCAL_NODE_ID_KEY = ByteBufferUtil.bytes("Local");
-
-    public enum BootstrapState
-    {
-        NEEDS_BOOTSTRAP,
-        COMPLETED,
-        IN_PROGRESS
-    }
-
-    private static DecoratedKey decorate(ByteBuffer key)
-    {
-        return StorageService.getPartitioner().decorateKey(key);
-    }
-
-    public static void finishStartup()
-    {
-        setupVersion();
-
-        // add entries to system schema columnfamilies for the hardcoded system definitions
-        for (String ksname : Schema.systemKeyspaceNames)
-        {
-            KSMetaData ksmd = Schema.instance.getKSMetaData(ksname);
-
-            // delete old, possibly obsolete entries in schema columnfamilies
-            for (String cfname : Arrays.asList(SystemTable.SCHEMA_KEYSPACES_CF, SystemTable.SCHEMA_COLUMNFAMILIES_CF, SystemTable.SCHEMA_COLUMNS_CF))
-            {
-                String req = String.format("DELETE FROM system.%s WHERE keyspace_name = '%s'", cfname, ksmd.name);
-                processInternal(req);
-            }
-
-            // (+1 to timestamp to make sure we don't get shadowed by the tombstones we just added)
-            ksmd.toSchema(FBUtilities.timestampMicros() + 1).apply();
-        }
-    }
-
-    private static void setupVersion()
-    {
-        String req = "INSERT INTO system.%s (key, release_version, cql_version, thrift_version, data_center, rack, partitioner) VALUES ('%s', '%s', '%s', '%s', '%s', '%s', '%s')";
-        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        processInternal(String.format(req, LOCAL_CF,
-                                         LOCAL_KEY,
-                                         FBUtilities.getReleaseVersionString(),
-                                         QueryProcessor.CQL_VERSION.toString(),
-                                         cassandraConstants.VERSION,
-                                         snitch.getDatacenter(FBUtilities.getBroadcastAddress()),
-                                         snitch.getRack(FBUtilities.getBroadcastAddress()),
-                                         DatabaseDescriptor.getPartitioner().getClass().getName()));
-    }
-
-    /**
-     * Write compaction log, except columfamilies under system keyspace.
-     *
-     * @param cfs
-     * @param toCompact sstables to compact
-     * @return compaction task id or null if cfs is under system keyspace
-     */
-    public static UUID startCompaction(ColumnFamilyStore cfs, Iterable<SSTableReader> toCompact)
-    {
-        if (Table.SYSTEM_KS.equals(cfs.table.getName()))
-            return null;
-
-        UUID compactionId = UUIDGen.getTimeUUID();
-        String req = "INSERT INTO system.%s (id, keyspace_name, columnfamily_name, inputs) VALUES (%s, '%s', '%s', {%s})";
-        Iterable<Integer> generations = Iterables.transform(toCompact, new Function<SSTableReader, Integer>()
-        {
-            public Integer apply(SSTableReader sstable)
-            {
-                return sstable.descriptor.generation;
-            }
-        });
-        processInternal(String.format(req, COMPACTION_LOG, compactionId, cfs.table.getName(), cfs.name, StringUtils.join(Sets.newHashSet(generations), ',')));
-        forceBlockingFlush(COMPACTION_LOG);
-        return compactionId;
-    }
-
-    public static void finishCompaction(UUID taskId)
-    {
-        assert taskId != null;
-
-        String req = "DELETE FROM system.%s WHERE id = %s";
-        processInternal(String.format(req, COMPACTION_LOG, taskId));
-        forceBlockingFlush(COMPACTION_LOG);
-    }
-
-    /**
-     * @return unfinished compactions, grouped by keyspace/columnfamily pair.
-     */
-    public static SetMultimap<Pair<String, String>, Integer> getUnfinishedCompactions()
-    {
-        String req = "SELECT * FROM system.%s";
-        UntypedResultSet resultSet = processInternal(String.format(req, COMPACTION_LOG));
-
-        SetMultimap<Pair<String, String>, Integer> unfinishedCompactions = HashMultimap.create();
-        for (UntypedResultSet.Row row : resultSet)
-        {
-            String keyspace = row.getString("keyspace_name");
-            String columnfamily = row.getString("columnfamily_name");
-            Set<Integer> inputs = row.getSet("inputs", Int32Type.instance);
-
-            unfinishedCompactions.putAll(Pair.create(keyspace, columnfamily), inputs);
-        }
-        return unfinishedCompactions;
-    }
-
-    public static void discardCompactionsInProgress()
-    {
-        ColumnFamilyStore compactionLog = Table.open(Table.SYSTEM_KS).getColumnFamilyStore(COMPACTION_LOG);
-        compactionLog.truncateBlocking();
-    }
-
-    public static void saveTruncationRecord(ColumnFamilyStore cfs, long truncatedAt, ReplayPosition position)
-    {
-        String req = "UPDATE system.%s SET truncated_at = truncated_at + %s WHERE key = '%s'";
-        processInternal(String.format(req, LOCAL_CF, truncationAsMapEntry(cfs, truncatedAt, position), LOCAL_KEY));
-        forceBlockingFlush(LOCAL_CF);
-    }
-
-    /**
-     * This method is used to remove information about truncation time for specified column family
-     */
-    public static void removeTruncationRecord(UUID cfId)
-    {
-        String req = "DELETE truncated_at[%s] from system.%s WHERE key = '%s'";
-        processInternal(String.format(req, cfId, LOCAL_CF, LOCAL_KEY));
-        forceBlockingFlush(LOCAL_CF);
-    }
-
-    private static String truncationAsMapEntry(ColumnFamilyStore cfs, long truncatedAt, ReplayPosition position)
-    {
-        DataOutputBuffer out = new DataOutputBuffer();
-        try
-        {
-            ReplayPosition.serializer.serialize(position, out);
-            out.writeLong(truncatedAt);
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
-        return String.format("{%s: 0x%s}",
-                             cfs.metadata.cfId,
-                             ByteBufferUtil.bytesToHex(ByteBuffer.wrap(out.getData(), 0, out.getLength())));
-    }
-
-    public static Map<UUID, Pair<ReplayPosition, Long>> getTruncationRecords()
-    {
-        String req = "SELECT truncated_at FROM system.%s WHERE key = '%s'";
-        UntypedResultSet rows = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
-        if (rows.isEmpty())
-            return Collections.emptyMap();
-
-        UntypedResultSet.Row row = rows.one();
-        Map<UUID, ByteBuffer> rawMap = row.getMap("truncated_at", UUIDType.instance, BytesType.instance);
-        if (rawMap == null)
-            return Collections.emptyMap();
-
-        Map<UUID, Pair<ReplayPosition, Long>> positions = new HashMap<UUID, Pair<ReplayPosition, Long>>();
-        for (Map.Entry<UUID, ByteBuffer> entry : rawMap.entrySet())
-            positions.put(entry.getKey(), truncationRecordFromBlob(entry.getValue()));
-        return positions;
-    }
-
-    private static Pair<ReplayPosition, Long> truncationRecordFromBlob(ByteBuffer bytes)
-    {
-        try
-        {
-            DataInputStream in = new DataInputStream(ByteBufferUtil.inputStream(bytes));
-            return Pair.create(ReplayPosition.serializer.deserialize(in), in.available() > 0 ? in.readLong() : Long.MIN_VALUE);
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Record tokens being used by another node
-     */
-    public static synchronized void updateTokens(InetAddress ep, Collection<Token> tokens)
-    {
-        if (ep.equals(FBUtilities.getBroadcastAddress()))
-        {
-            removeEndpoint(ep);
-            return;
-        }
-
-        String req = "INSERT INTO system.%s (peer, tokens) VALUES ('%s', %s)";
-        processInternal(String.format(req, PEERS_CF, ep.getHostAddress(), tokensAsSet(tokens)));
-        forceBlockingFlush(PEERS_CF);
-    }
-
-    public static synchronized void updatePeerInfo(InetAddress ep, String columnName, String value)
-    {
-        if (ep.equals(FBUtilities.getBroadcastAddress()))
-            return;
-
-        String req = "INSERT INTO system.%s (peer, %s) VALUES ('%s', %s)";
-        processInternal(String.format(req, PEERS_CF, columnName, ep.getHostAddress(), value));
-    }
-
-    public static synchronized void updateHintsDropped(InetAddress ep, UUID timePeriod, int value)
-    {
-        // with 30 day TTL
-        String req = "UPDATE system.%s USING TTL 2592000 SET hints_dropped[ %s ] = %s WHERE peer = '%s'";
-        processInternal(String.format(req, PEER_EVENTS_CF, timePeriod.toString(), value, ep.getHostAddress()));
-    }
-
-    public static synchronized void updateSchemaVersion(UUID version)
-    {
-        String req = "INSERT INTO system.%s (key, schema_version) VALUES ('%s', %s)";
-        processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, version.toString()));
-    }
-
-    private static String tokensAsSet(Collection<Token> tokens)
-    {
-        Token.TokenFactory factory = StorageService.getPartitioner().getTokenFactory();
-        StringBuilder sb = new StringBuilder();
-        sb.append("{");
-        Iterator<Token> iter = tokens.iterator();
-        while (iter.hasNext())
-        {
-            sb.append("'").append(factory.toString(iter.next())).append("'");
-            if (iter.hasNext())
-                sb.append(",");
-        }
-        sb.append("}");
-        return sb.toString();
-    }
-
-    private static Collection<Token> deserializeTokens(Collection<String> tokensStrings)
-    {
-        Token.TokenFactory factory = StorageService.getPartitioner().getTokenFactory();
-        List<Token> tokens = new ArrayList<Token>(tokensStrings.size());
-        for (String tk : tokensStrings)
-            tokens.add(factory.fromString(tk));
-        return tokens;
-    }
-
-    /**
-     * Remove stored tokens being used by another node
-     */
-    public static synchronized void removeEndpoint(InetAddress ep)
-    {
-        String req = "DELETE FROM system.%s WHERE peer = '%s'";
-        processInternal(String.format(req, PEERS_CF, ep.getHostAddress()));
-        forceBlockingFlush(PEERS_CF);
-    }
-
-    /**
-     * This method is used to update the System Table with the new tokens for this node
-    */
-    public static synchronized void updateTokens(Collection<Token> tokens)
-    {
-        assert !tokens.isEmpty() : "removeEndpoint should be used instead";
-        String req = "INSERT INTO system.%s (key, tokens) VALUES ('%s', %s)";
-        processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, tokensAsSet(tokens)));
-        forceBlockingFlush(LOCAL_CF);
-    }
-
-    /**
-     * Convenience method to update the list of tokens in the local system table.
-     *
-     * @param addTokens tokens to add
-     * @param rmTokens tokens to remove
-     * @return the collection of persisted tokens
-     */
-    public static synchronized Collection<Token> updateLocalTokens(Collection<Token> addTokens, Collection<Token> rmTokens)
-    {
-        Collection<Token> tokens = getSavedTokens();
-        tokens.removeAll(rmTokens);
-        tokens.addAll(addTokens);
-        updateTokens(tokens);
-        return tokens;
-    }
-
-    private static void forceBlockingFlush(String cfname)
-    {
-        Table.open(Table.SYSTEM_KS).getColumnFamilyStore(cfname).forceBlockingFlush();
-    }
-
-    /**
-     * Return a map of stored tokens to IP addresses
-     *
-     */
-    public static SetMultimap<InetAddress, Token> loadTokens()
-    {
-        SetMultimap<InetAddress, Token> tokenMap = HashMultimap.create();
-        for (UntypedResultSet.Row row : processInternal("SELECT peer, tokens FROM system." + PEERS_CF))
-        {
-            InetAddress peer = row.getInetAddress("peer");
-            if (row.has("tokens"))
-                tokenMap.putAll(peer, deserializeTokens(row.getSet("tokens", UTF8Type.instance)));
-        }
-
-        return tokenMap;
-    }
-
-    /**
-     * Return a map of store host_ids to IP addresses
-     *
-     */
-    public static Map<InetAddress, UUID> loadHostIds()
-    {
-        Map<InetAddress, UUID> hostIdMap = new HashMap<InetAddress, UUID>();
-        for (UntypedResultSet.Row row : processInternal("SELECT peer, host_id FROM system." + PEERS_CF))
-        {
-            InetAddress peer = row.getInetAddress("peer");
-            if (row.has("host_id"))
-            {
-                hostIdMap.put(peer, row.getUUID("host_id"));
-            }
-        }
-        return hostIdMap;
-    }
-
-    /**
-     * Return a map of IP addresses containing a map of dc and rack info
-     */
-    public static Map<InetAddress, Map<String,String>> loadDcRackInfo()
-    {
-        Map<InetAddress, Map<String, String>> result = new HashMap<InetAddress, Map<String, String>>();
-        for (UntypedResultSet.Row row : processInternal("SELECT peer, data_center, rack from system." + PEERS_CF))
-        {
-            InetAddress peer = row.getInetAddress("peer");
-            if (row.has("data_center") && row.has("rack"))
-            {
-                Map<String, String> dcRack = new HashMap<String, String>();
-                dcRack.put("data_center", row.getString("data_center"));
-                dcRack.put("rack", row.getString("rack"));
-                result.put(peer, dcRack);
-            }
-        }
-        return result;
-    }
-
-    /**
-     * One of three things will happen if you try to read the system table:
-     * 1. files are present and you can read them: great
-     * 2. no files are there: great (new node is assumed)
-     * 3. files are present but you can't read them: bad
-     * @throws ConfigurationException
-     */
-    public static void checkHealth() throws ConfigurationException
-    {
-        Table table;
-        try
-        {
-            table = Table.open(Table.SYSTEM_KS);
-        }
-        catch (AssertionError err)
-        {
-            // this happens when a user switches from OPP to RP.
-            ConfigurationException ex = new ConfigurationException("Could not read system table!");
-            ex.initCause(err);
-            throw ex;
-        }
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(LOCAL_CF);
-
-        String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
-
-        if (result.isEmpty() || !result.one().has("cluster_name"))
-        {
-            // this is a brand new node
-            if (!cfs.getSSTables().isEmpty())
-                throw new ConfigurationException("Found system table files, but they couldn't be loaded!");
-
-            // no system files.  this is a new node.
-            req = "INSERT INTO system.%s (key, cluster_name) VALUES ('%s', '%s')";
-            processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, DatabaseDescriptor.getClusterName()));
-            return;
-        }
-
-        String savedClusterName = result.one().getString("cluster_name");
-        if (!DatabaseDescriptor.getClusterName().equals(savedClusterName))
-            throw new ConfigurationException("Saved cluster name " + savedClusterName + " != configured name " + DatabaseDescriptor.getClusterName());
-    }
-
-    public static Collection<Token> getSavedTokens()
-    {
-        String req = "SELECT tokens FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
-        return result.isEmpty() || !result.one().has("tokens")
-             ? Collections.<Token>emptyList()
-             : deserializeTokens(result.one().<String>getSet("tokens", UTF8Type.instance));
-    }
-
-    public static int incrementAndGetGeneration()
-    {
-        String req = "SELECT gossip_generation FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
-
-        int generation;
-        if (result.isEmpty() || !result.one().has("gossip_generation"))
-        {
-            // seconds-since-epoch isn't a foolproof new generation
-            // (where foolproof is "guaranteed to be larger than the last one seen at this ip address"),
-            // but it's as close as sanely possible
-            generation = (int) (System.currentTimeMillis() / 1000);
-        }
-        else
-        {
-            // Other nodes will ignore gossip messages about a node that have a lower generation than previously seen.
-            final int storedGeneration = result.one().getInt("gossip_generation") + 1;
-            final int now = (int) (System.currentTimeMillis() / 1000);
-            if (storedGeneration >= now)
-            {
-                logger.warn("Using stored Gossip Generation {} as it is greater than current system time {}.  See CASSANDRA-3654 if you experience problems",
-                            storedGeneration, now);
-                generation = storedGeneration;
-            }
-            else
-            {
-                generation = now;
-            }
-        }
-
-        req = "INSERT INTO system.%s (key, gossip_generation) VALUES ('%s', %d)";
-        processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, generation));
-        forceBlockingFlush(LOCAL_CF);
-
-        return generation;
-    }
-
-    public static BootstrapState getBootstrapState()
-    {
-        String req = "SELECT bootstrapped FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
-
-        if (result.isEmpty() || !result.one().has("bootstrapped"))
-            return BootstrapState.NEEDS_BOOTSTRAP;
-
-        return BootstrapState.valueOf(result.one().getString("bootstrapped"));
-    }
-
-    public static boolean bootstrapComplete()
-    {
-        return getBootstrapState() == BootstrapState.COMPLETED;
-    }
-
-    public static boolean bootstrapInProgress()
-    {
-        return getBootstrapState() == BootstrapState.IN_PROGRESS;
-    }
-
-    public static void setBootstrapState(BootstrapState state)
-    {
-        String req = "INSERT INTO system.%s (key, bootstrapped) VALUES ('%s', '%s')";
-        processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, state.name()));
-        forceBlockingFlush(LOCAL_CF);
-    }
-
-    public static boolean isIndexBuilt(String table, String indexName)
-    {
-        ColumnFamilyStore cfs = Table.open(Table.SYSTEM_KS).getColumnFamilyStore(INDEX_CF);
-        QueryFilter filter = QueryFilter.getNamesFilter(decorate(ByteBufferUtil.bytes(table)),
-                                                        INDEX_CF,
-                                                        ByteBufferUtil.bytes(indexName),
-                                                        System.currentTimeMillis());
-        return ColumnFamilyStore.removeDeleted(cfs.getColumnFamily(filter), Integer.MAX_VALUE) != null;
-    }
-
-    public static void setIndexBuilt(String table, String indexName)
-    {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Table.SYSTEM_KS, INDEX_CF);
-        cf.addColumn(new Column(ByteBufferUtil.bytes(indexName), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()));
-        RowMutation rm = new RowMutation(Table.SYSTEM_KS, ByteBufferUtil.bytes(table), cf);
-        rm.apply();
-        forceBlockingFlush(INDEX_CF);
-    }
-
-    public static void setIndexRemoved(String table, String indexName)
-    {
-        RowMutation rm = new RowMutation(Table.SYSTEM_KS, ByteBufferUtil.bytes(table));
-        rm.delete(INDEX_CF, ByteBufferUtil.bytes(indexName), FBUtilities.timestampMicros());
-        rm.apply();
-        forceBlockingFlush(INDEX_CF);
-    }
-
-    /**
-     * Read the host ID from the system table, creating (and storing) one if
-     * none exists.
-     */
-    public static UUID getLocalHostId()
-    {
-        UUID hostId = null;
-
-        String req = "SELECT host_id FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
-
-        // Look up the Host UUID (return it if found)
-        if (!result.isEmpty() && result.one().has("host_id"))
-        {
-            return result.one().getUUID("host_id");
-        }
-
-        // ID not found, generate a new one, persist, and then return it.
-        hostId = UUID.randomUUID();
-        logger.warn("No host ID found, created {} (Note: This should happen exactly once per node).", hostId);
-
-        req = "INSERT INTO system.%s (key, host_id) VALUES ('%s', %s)";
-        processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, hostId));
-        return hostId;
-    }
-
-    /**
-     * Read the current local node id from the system table or null if no
-     * such node id is recorded.
-     */
-    public static CounterId getCurrentLocalCounterId()
-    {
-        Table table = Table.open(Table.SYSTEM_KS);
-
-        // Get the last CounterId (since CounterId are timeuuid is thus ordered from the older to the newer one)
-        QueryFilter filter = QueryFilter.getSliceFilter(decorate(ALL_LOCAL_NODE_ID_KEY),
-                                                        COUNTER_ID_CF,
-                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                        true,
-                                                        1,
-                                                        System.currentTimeMillis());
-        ColumnFamily cf = table.getColumnFamilyStore(COUNTER_ID_CF).getColumnFamily(filter);
-        if (cf != null && cf.getColumnCount() != 0)
-            return CounterId.wrap(cf.iterator().next().name());
-        else
-            return null;
-    }
-
-    /**
-     * Write a new current local node id to the system table.
-     *
-     * @param oldCounterId the previous local node id (that {@code newCounterId}
-     * replace) or null if no such node id exists (new node or removed system
-     * table)
-     * @param newCounterId the new current local node id to record
-     * @param now microsecond time stamp.
-     */
-    public static void writeCurrentLocalCounterId(CounterId oldCounterId, CounterId newCounterId, long now)
-    {
-        ByteBuffer ip = ByteBuffer.wrap(FBUtilities.getBroadcastAddress().getAddress());
-
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Table.SYSTEM_KS, COUNTER_ID_CF);
-        cf.addColumn(new Column(newCounterId.bytes(), ip, now));
-        RowMutation rm = new RowMutation(Table.SYSTEM_KS, ALL_LOCAL_NODE_ID_KEY, cf);
-        rm.apply();
-        forceBlockingFlush(COUNTER_ID_CF);
-    }
-
-    public static List<CounterId.CounterIdRecord> getOldLocalCounterIds()
-    {
-        List<CounterId.CounterIdRecord> l = new ArrayList<CounterId.CounterIdRecord>();
-
-        Table table = Table.open(Table.SYSTEM_KS);
-        QueryFilter filter = QueryFilter.getIdentityFilter(decorate(ALL_LOCAL_NODE_ID_KEY), COUNTER_ID_CF, System.currentTimeMillis());
-        ColumnFamily cf = table.getColumnFamilyStore(COUNTER_ID_CF).getColumnFamily(filter);
-
-        CounterId previous = null;
-        for (Column c : cf)
-        {
-            if (previous != null)
-                l.add(new CounterId.CounterIdRecord(previous, c.timestamp()));
-
-            // this will ignore the last column on purpose since it is the
-            // current local node id
-            previous = CounterId.wrap(c.name());
-        }
-        return l;
-    }
-
-    /**
-     * @param cfName The name of the ColumnFamily responsible for part of the schema (keyspace, ColumnFamily, columns)
-     * @return CFS responsible to hold low-level serialized schema
-     */
-    public static ColumnFamilyStore schemaCFS(String cfName)
-    {
-        return Table.open(Table.SYSTEM_KS).getColumnFamilyStore(cfName);
-    }
-
-    public static List<Row> serializedSchema()
-    {
-        List<Row> schema = new ArrayList<Row>(3);
-
-        schema.addAll(serializedSchema(SCHEMA_KEYSPACES_CF));
-        schema.addAll(serializedSchema(SCHEMA_COLUMNFAMILIES_CF));
-        schema.addAll(serializedSchema(SCHEMA_COLUMNS_CF));
-
-        return schema;
-    }
-
-    /**
-     * @param schemaCfName The name of the ColumnFamily responsible for part of the schema (keyspace, ColumnFamily, columns)
-     * @return low-level schema representation (each row represents individual Keyspace or ColumnFamily)
-     */
-    public static List<Row> serializedSchema(String schemaCfName)
-    {
-        Token minToken = StorageService.getPartitioner().getMinimumToken();
-
-        return schemaCFS(schemaCfName).getRangeSlice(new Range<RowPosition>(minToken.minKeyBound(), minToken.maxKeyBound()),
-                                                     null,
-                                                     new IdentityQueryFilter(),
-                                                     Integer.MAX_VALUE,
-                                                     System.currentTimeMillis());
-    }
-
-    public static Collection<RowMutation> serializeSchema()
-    {
-        Map<DecoratedKey, RowMutation> mutationMap = new HashMap<DecoratedKey, RowMutation>();
-
-        serializeSchema(mutationMap, SCHEMA_KEYSPACES_CF);
-        serializeSchema(mutationMap, SCHEMA_COLUMNFAMILIES_CF);
-        serializeSchema(mutationMap, SCHEMA_COLUMNS_CF);
-
-        return mutationMap.values();
-    }
-
-    private static void serializeSchema(Map<DecoratedKey, RowMutation> mutationMap, String schemaCfName)
-    {
-        for (Row schemaRow : serializedSchema(schemaCfName))
-        {
-            if (Schema.ignoredSchemaRow(schemaRow))
-                continue;
-
-            RowMutation mutation = mutationMap.get(schemaRow.key);
-            if (mutation == null)
-            {
-                mutation = new RowMutation(Table.SYSTEM_KS, schemaRow.key.key);
-                mutationMap.put(schemaRow.key, mutation);
-            }
-
-            mutation.add(schemaRow.cf);
-        }
-    }
-
-    public static Map<DecoratedKey, ColumnFamily> getSchema(String cfName)
-    {
-        Map<DecoratedKey, ColumnFamily> schema = new HashMap<DecoratedKey, ColumnFamily>();
-
-        for (Row schemaEntity : SystemTable.serializedSchema(cfName))
-            schema.put(schemaEntity.key, schemaEntity.cf);
-
-        return schema;
-    }
-
-    public static ByteBuffer getSchemaKSKey(String ksName)
-    {
-        return AsciiType.instance.fromString(ksName);
-    }
-
-    public static Row readSchemaRow(String ksName)
-    {
-        DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
-
-        ColumnFamilyStore schemaCFS = SystemTable.schemaCFS(SCHEMA_KEYSPACES_CF);
-        ColumnFamily result = schemaCFS.getColumnFamily(QueryFilter.getIdentityFilter(key, SCHEMA_KEYSPACES_CF, System.currentTimeMillis()));
-
-        return new Row(key, result);
-    }
-
-    public static Row readSchemaRow(String ksName, String cfName)
-    {
-        DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
-
-        ColumnFamilyStore schemaCFS = SystemTable.schemaCFS(SCHEMA_COLUMNFAMILIES_CF);
-        ColumnFamily result = schemaCFS.getColumnFamily(key,
-                                                        DefsTable.searchComposite(cfName, true),
-                                                        DefsTable.searchComposite(cfName, false),
-                                                        false,
-                                                        Integer.MAX_VALUE,
-                                                        System.currentTimeMillis());
-
-        return new Row(key, result);
-    }
-
-    public static PaxosState loadPaxosState(ByteBuffer key, CFMetaData metadata)
-    {
-        String req = "SELECT * FROM system.%s WHERE row_key = 0x%s AND cf_id = %s";
-        UntypedResultSet results = processInternal(String.format(req, PAXOS_CF, ByteBufferUtil.bytesToHex(key), metadata.cfId));
-        if (results.isEmpty())
-            return new PaxosState(key, metadata);
-        UntypedResultSet.Row row = results.one();
-        Commit inProgress = new Commit(key,
-                                       row.getUUID("in_progress_ballot"),
-                                       row.has("proposal") ? ColumnFamily.fromBytes(row.getBytes("proposal")) : EmptyColumns.factory.create(metadata));
-        // either most_recent_commit and most_recent_commit_at will both be set, or neither
-        Commit mostRecent = row.has("most_recent_commit")
-                          ? new Commit(key, row.getUUID("most_recent_commit_at"), ColumnFamily.fromBytes(row.getBytes("most_recent_commit")))
-                          : Commit.emptyCommit(key, metadata);
-        return new PaxosState(inProgress, mostRecent);
-    }
-
-    public static void savePaxosPromise(Commit promise)
-    {
-        String req = "UPDATE %s USING TIMESTAMP %d AND TTL %d SET in_progress_ballot = %s WHERE row_key = 0x%s AND cf_id = %s";
-        processInternal(String.format(req,
-                                      PAXOS_CF,
-                                      UUIDGen.microsTimestamp(promise.ballot),
-                                      paxosTtl(promise.update.metadata),
-                                      promise.ballot,
-                                      ByteBufferUtil.bytesToHex(promise.key),
-                                      promise.update.id()));
-    }
-
-    public static void savePaxosProposal(Commit commit)
-    {
-        processInternal(String.format("UPDATE %s USING TIMESTAMP %d AND TTL %d SET proposal = 0x%s WHERE row_key = 0x%s AND cf_id = %s",
-                                      PAXOS_CF,
-                                      UUIDGen.microsTimestamp(commit.ballot),
-                                      paxosTtl(commit.update.metadata),
-                                      ByteBufferUtil.bytesToHex(commit.update.toBytes()),
-                                      ByteBufferUtil.bytesToHex(commit.key),
-                                      commit.update.id()));
-    }
-
-    private static int paxosTtl(CFMetaData metadata)
-    {
-        // keep paxos state around for at least 3h
-        return Math.max(3 * 3600, metadata.getGcGraceSeconds());
-    }
-
-    public static void savePaxosCommit(Commit commit, boolean eraseInProgressProposal)
-    {
-        String preserveCql = "UPDATE %s USING TIMESTAMP %d AND TTL %d SET most_recent_commit_at = %s, most_recent_commit = 0x%s WHERE row_key = 0x%s AND cf_id = %s";
-        // identical except adds proposal = null
-        String eraseCql = "UPDATE %s USING TIMESTAMP %d AND TTL %d SET proposal = null, most_recent_commit_at = %s, most_recent_commit = 0x%s WHERE row_key = 0x%s AND cf_id = %s";
-        processInternal(String.format(eraseInProgressProposal ? eraseCql : preserveCql,
-                                      PAXOS_CF,
-                                      UUIDGen.microsTimestamp(commit.ballot),
-                                      paxosTtl(commit.update.metadata),
-                                      commit.ballot,
-                                      ByteBufferUtil.bytesToHex(commit.update.toBytes()),
-                                      ByteBufferUtil.bytesToHex(commit.key),
-                                      commit.update.id()));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/Table.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Table.java b/src/java/org/apache/cassandra/db/Table.java
deleted file mode 100644
index 409076f..0000000
--- a/src/java/org/apache/cassandra/db/Table.java
+++ /dev/null
@@ -1,455 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Future;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.index.SecondaryIndex;
-import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.locator.AbstractReplicationStrategy;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.service.pager.QueryPagers;
-import org.apache.cassandra.tracing.Tracing;
-
-/**
- * It represents a Keyspace.
- */
-public class Table
-{
-    public static final String SYSTEM_KS = "system";
-    private static final int DEFAULT_PAGE_SIZE = 10000;
-
-    private static final Logger logger = LoggerFactory.getLogger(Table.class);
-
-    /**
-     * accesses to CFS.memtable should acquire this for thread safety.
-     * CFS.maybeSwitchMemtable should aquire the writeLock; see that method for the full explanation.
-     * <p/>
-     * (Enabling fairness in the RRWL is observed to decrease throughput, so we leave it off.)
-     */
-    public static final ReentrantReadWriteLock switchLock = new ReentrantReadWriteLock();
-
-    // It is possible to call Table.open without a running daemon, so it makes sense to ensure
-    // proper directories here as well as in CassandraDaemon.
-    static
-    {
-        if (!StorageService.instance.isClientMode())
-            DatabaseDescriptor.createAllDirectories();
-    }
-
-    public final KSMetaData metadata;
-
-    /* ColumnFamilyStore per column family */
-    private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<UUID, ColumnFamilyStore>();
-    private volatile AbstractReplicationStrategy replicationStrategy;
-    public static final Function<String,Table> tableTransformer = new Function<String, Table>()
-    {
-        public Table apply(String tableName)
-        {
-            return Table.open(tableName);
-        }
-    };
-
-    public static Table open(String table)
-    {
-        return open(table, Schema.instance, true);
-    }
-
-    public static Table openWithoutSSTables(String table)
-    {
-        return open(table, Schema.instance, false);
-    }
-
-    private static Table open(String table, Schema schema, boolean loadSSTables)
-    {
-        Table tableInstance = schema.getTableInstance(table);
-
-        if (tableInstance == null)
-        {
-            // instantiate the Table.  we could use putIfAbsent but it's important to making sure it is only done once
-            // per keyspace, so we synchronize and re-check before doing it.
-            synchronized (Table.class)
-            {
-                tableInstance = schema.getTableInstance(table);
-                if (tableInstance == null)
-                {
-                    // open and store the table
-                    tableInstance = new Table(table, loadSSTables);
-                    schema.storeTableInstance(tableInstance);
-
-                    // table has to be constructed and in the cache before cacheRow can be called
-                    for (ColumnFamilyStore cfs : tableInstance.getColumnFamilyStores())
-                        cfs.initRowCache();
-                }
-            }
-        }
-        return tableInstance;
-    }
-
-    public static Table clear(String table)
-    {
-        return clear(table, Schema.instance);
-    }
-
-    public static Table clear(String table, Schema schema)
-    {
-        synchronized (Table.class)
-        {
-            Table t = schema.removeTableInstance(table);
-            if (t != null)
-            {
-                for (ColumnFamilyStore cfs : t.getColumnFamilyStores())
-                    t.unloadCf(cfs);
-            }
-            return t;
-        }
-    }
-
-    /**
-     * Removes every SSTable in the directory from the appropriate DataTracker's view.
-     * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
-     */
-    public static void removeUnreadableSSTables(File directory)
-    {
-        for (Table table : Table.all())
-        {
-            for (ColumnFamilyStore baseCfs : table.getColumnFamilyStores())
-            {
-                for (ColumnFamilyStore cfs : baseCfs.concatWithIndexes())
-                    cfs.maybeRemoveUnreadableSSTables(directory);
-            }
-        }
-    }
-
-    public Collection<ColumnFamilyStore> getColumnFamilyStores()
-    {
-        return Collections.unmodifiableCollection(columnFamilyStores.values());
-    }
-
-    public ColumnFamilyStore getColumnFamilyStore(String cfName)
-    {
-        UUID id = Schema.instance.getId(getName(), cfName);
-        if (id == null)
-            throw new IllegalArgumentException(String.format("Unknown table/cf pair (%s.%s)", getName(), cfName));
-        return getColumnFamilyStore(id);
-    }
-
-    public ColumnFamilyStore getColumnFamilyStore(UUID id)
-    {
-        ColumnFamilyStore cfs = columnFamilyStores.get(id);
-        if (cfs == null)
-            throw new IllegalArgumentException("Unknown CF " + id);
-        return cfs;
-    }
-
-    /**
-     * Take a snapshot of the specific column family, or the entire set of column families
-     * if columnFamily is null with a given timestamp
-     *
-     * @param snapshotName     the tag associated with the name of the snapshot.  This value may not be null
-     * @param columnFamilyName the column family to snapshot or all on null
-     * @throws IOException if the column family doesn't exist
-     */
-    public void snapshot(String snapshotName, String columnFamilyName) throws IOException
-    {
-        assert snapshotName != null;
-        boolean tookSnapShot = false;
-        for (ColumnFamilyStore cfStore : columnFamilyStores.values())
-        {
-            if (columnFamilyName == null || cfStore.name.equals(columnFamilyName))
-            {
-                tookSnapShot = true;
-                cfStore.snapshot(snapshotName);
-            }
-        }
-
-        if ((columnFamilyName != null) && !tookSnapShot)
-            throw new IOException("Failed taking snapshot. Column family " + columnFamilyName + " does not exist.");
-    }
-
-    /**
-     * @param clientSuppliedName may be null.
-     * @return the name of the snapshot
-     */
-    public static String getTimestampedSnapshotName(String clientSuppliedName)
-    {
-        String snapshotName = Long.toString(System.currentTimeMillis());
-        if (clientSuppliedName != null && !clientSuppliedName.equals(""))
-        {
-            snapshotName = snapshotName + "-" + clientSuppliedName;
-        }
-        return snapshotName;
-    }
-
-    /**
-     * Check whether snapshots already exists for a given name.
-     *
-     * @param snapshotName the user supplied snapshot name
-     * @return true if the snapshot exists
-     */
-    public boolean snapshotExists(String snapshotName)
-    {
-        assert snapshotName != null;
-        for (ColumnFamilyStore cfStore : columnFamilyStores.values())
-        {
-            if (cfStore.snapshotExists(snapshotName))
-                return true;
-        }
-        return false;
-    }
-
-    /**
-     * Clear all the snapshots for a given table.
-     *
-     * @param snapshotName the user supplied snapshot name. It empty or null,
-     *                     all the snapshots will be cleaned
-     */
-    public void clearSnapshot(String snapshotName)
-    {
-        for (ColumnFamilyStore cfStore : columnFamilyStores.values())
-        {
-            cfStore.clearSnapshot(snapshotName);
-        }
-    }
-
-    /**
-     * @return A list of open SSTableReaders
-     */
-    public List<SSTableReader> getAllSSTables()
-    {
-        List<SSTableReader> list = new ArrayList<SSTableReader>(columnFamilyStores.size());
-        for (ColumnFamilyStore cfStore : columnFamilyStores.values())
-            list.addAll(cfStore.getSSTables());
-        return list;
-    }
-
-    private Table(String table, boolean loadSSTables)
-    {
-        metadata = Schema.instance.getKSMetaData(table);
-        assert metadata != null : "Unknown keyspace " + table;
-        createReplicationStrategy(metadata);
-
-        for (CFMetaData cfm : new ArrayList<CFMetaData>(metadata.cfMetaData().values()))
-        {
-            logger.debug("Initializing {}.{}", getName(), cfm.cfName);
-            initCf(cfm.cfId, cfm.cfName, loadSSTables);
-        }
-    }
-
-    public void createReplicationStrategy(KSMetaData ksm)
-    {
-        if (replicationStrategy != null)
-            StorageService.instance.getTokenMetadata().unregister(replicationStrategy);
-
-        replicationStrategy = AbstractReplicationStrategy.createReplicationStrategy(ksm.name,
-                                                                                    ksm.strategyClass,
-                                                                                    StorageService.instance.getTokenMetadata(),
-                                                                                    DatabaseDescriptor.getEndpointSnitch(),
-                                                                                    ksm.strategyOptions);
-    }
-
-    // best invoked on the compaction mananger.
-    public void dropCf(UUID cfId)
-    {
-        assert columnFamilyStores.containsKey(cfId);
-        ColumnFamilyStore cfs = columnFamilyStores.remove(cfId);
-        if (cfs == null)
-            return;
-
-        unloadCf(cfs);
-    }
-
-    // disassociate a cfs from this table instance.
-    private void unloadCf(ColumnFamilyStore cfs)
-    {
-        cfs.forceBlockingFlush();
-        cfs.invalidate();
-    }
-
-    /**
-     * adds a cf to internal structures, ends up creating disk files).
-     */
-    public void initCf(UUID cfId, String cfName, boolean loadSSTables)
-    {
-        ColumnFamilyStore cfs = columnFamilyStores.get(cfId);
-
-        if (cfs == null)
-        {
-            // CFS being created for the first time, either on server startup or new CF being added.
-            // We don't worry about races here; startup is safe, and adding multiple idential CFs
-            // simultaneously is a "don't do that" scenario.
-            ColumnFamilyStore oldCfs = columnFamilyStores.putIfAbsent(cfId, ColumnFamilyStore.createColumnFamilyStore(this, cfName, loadSSTables));
-            // CFS mbean instantiation will error out before we hit this, but in case that changes...
-            if (oldCfs != null)
-                throw new IllegalStateException("added multiple mappings for cf id " + cfId);
-        }
-        else
-        {
-            // re-initializing an existing CF.  This will happen if you cleared the schema
-            // on this node and it's getting repopulated from the rest of the cluster.
-            assert cfs.name.equals(cfName);
-            cfs.metadata.reload();
-            cfs.reload();
-        }
-    }
-
-    public Row getRow(QueryFilter filter)
-    {
-        ColumnFamilyStore cfStore = getColumnFamilyStore(filter.getColumnFamilyName());
-        ColumnFamily columnFamily = cfStore.getColumnFamily(filter);
-        return new Row(filter.key, columnFamily);
-    }
-
-    public void apply(RowMutation mutation, boolean writeCommitLog)
-    {
-        apply(mutation, writeCommitLog, true);
-    }
-
-    /**
-     * This method appends a row to the global CommitLog, then updates memtables and indexes.
-     *
-     * @param mutation       the row to write.  Must not be modified after calling apply, since commitlog append
-     *                       may happen concurrently, depending on the CL Executor type.
-     * @param writeCommitLog false to disable commitlog append entirely
-     * @param updateIndexes  false to disable index updates (used by CollationController "defragmenting")
-     */
-    public void apply(RowMutation mutation, boolean writeCommitLog, boolean updateIndexes)
-    {
-        // write the mutation to the commitlog and memtables
-        Tracing.trace("Acquiring switchLock read lock");
-        switchLock.readLock().lock();
-        try
-        {
-            if (writeCommitLog)
-            {
-                Tracing.trace("Appending to commitlog");
-                CommitLog.instance.add(mutation);
-            }
-
-            DecoratedKey key = StorageService.getPartitioner().decorateKey(mutation.key());
-            for (ColumnFamily cf : mutation.getColumnFamilies())
-            {
-                ColumnFamilyStore cfs = columnFamilyStores.get(cf.id());
-                if (cfs == null)
-                {
-                    logger.error("Attempting to mutate non-existant column family " + cf.id());
-                    continue;
-                }
-
-                Tracing.trace("Adding to {} memtable", cf.metadata().cfName);
-                cfs.apply(key, cf, updateIndexes ? cfs.indexManager.updaterFor(key) : SecondaryIndexManager.nullUpdater);
-            }
-        }
-        finally
-        {
-            switchLock.readLock().unlock();
-        }
-    }
-
-    public AbstractReplicationStrategy getReplicationStrategy()
-    {
-        return replicationStrategy;
-    }
-
-    /**
-     * @param key row to index
-     * @param cfs ColumnFamily to index row in
-     * @param idxNames columns to index, in comparator order
-     */
-    public static void indexRow(DecoratedKey key, ColumnFamilyStore cfs, Set<String> idxNames)
-    {
-        if (logger.isDebugEnabled())
-            logger.debug("Indexing row {} ", cfs.metadata.getKeyValidator().getString(key.key));
-
-        Collection<SecondaryIndex> indexes = cfs.indexManager.getIndexesByNames(idxNames);
-
-        switchLock.readLock().lock();
-        try
-        {
-            Iterator<ColumnFamily> pager = QueryPagers.pageRowLocally(cfs, key.key, DEFAULT_PAGE_SIZE);
-            while (pager.hasNext())
-            {
-                ColumnFamily cf = pager.next();
-                ColumnFamily cf2 = cf.cloneMeShallow();
-                for (Column column : cf)
-                {
-                    if (cfs.indexManager.indexes(column.name(), indexes))
-                        cf2.addColumn(column);
-                }
-                cfs.indexManager.indexRow(key.key, cf2);
-            }
-        }
-        finally
-        {
-            switchLock.readLock().unlock();
-        }
-    }
-
-    public List<Future<?>> flush()
-    {
-        List<Future<?>> futures = new ArrayList<Future<?>>(columnFamilyStores.size());
-        for (UUID cfId : columnFamilyStores.keySet())
-            futures.add(columnFamilyStores.get(cfId).forceFlush());
-        return futures;
-    }
-
-    public static Iterable<Table> all()
-    {
-        return Iterables.transform(Schema.instance.getTables(), tableTransformer);
-    }
-
-    public static Iterable<Table> nonSystem()
-    {
-        return Iterables.transform(Schema.instance.getNonSystemTables(), tableTransformer);
-    }
-
-    public static Iterable<Table> system()
-    {
-        return Iterables.transform(Schema.systemKeyspaceNames, tableTransformer);
-    }
-
-    @Override
-    public String toString()
-    {
-        return getClass().getSimpleName() + "(name='" + getName() + "')";
-    }
-
-    public String getName()
-    {
-        return metadata.name;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/TruncateVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/TruncateVerbHandler.java b/src/java/org/apache/cassandra/db/TruncateVerbHandler.java
index 6ebedc7..226262c 100644
--- a/src/java/org/apache/cassandra/db/TruncateVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/TruncateVerbHandler.java
@@ -36,7 +36,7 @@ public class TruncateVerbHandler implements IVerbHandler<Truncation>
         Tracing.trace("Applying truncation of {}.{}", t.keyspace, t.columnFamily);
         try
         {
-            ColumnFamilyStore cfs = Table.open(t.keyspace).getColumnFamilyStore(t.columnFamily);
+            ColumnFamilyStore cfs = Keyspace.open(t.keyspace).getColumnFamilyStore(t.columnFamily);
             cfs.truncateBlocking();
         }
         catch (Exception e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
index 5cfdd27..4faa651 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
@@ -44,7 +44,7 @@ public class SSTableSliceIterator implements OnDiskAtomIterator
 
     /**
      * An iterator for a slice within an SSTable
-     * @param sstable Table for the CFS we are reading from
+     * @param sstable Keyspace for the CFS we are reading from
      * @param file Optional parameter that input is read from.  If null is passed, this class creates an appropriate one automatically.
      * If this class creates, it will close the underlying file when #close() is called.
      * If a caller passes a non-null argument, this class will NOT close the underlying file when the iterator is closed (i.e. the caller is responsible for closing the file)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/commitlog/CommitLogAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogAllocator.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogAllocator.java
index 2855979..d56bf7a 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogAllocator.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogAllocator.java
@@ -35,7 +35,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
@@ -132,7 +132,7 @@ public class CommitLogAllocator
         assert !activeSegments.contains(next);
         activeSegments.add(next);
         if (isCapExceeded())
-            flushOldestTables();
+            flushOldestKeyspaces();
 
         return next;
     }
@@ -288,7 +288,7 @@ public class CommitLogAllocator
     /**
      * Force a flush on all dirty CFs represented in the oldest commitlog segment
      */
-    private void flushOldestTables()
+    private void flushOldestKeyspaces()
     {
         CommitLogSegment oldestSegment = activeSegments.peek();
 
@@ -297,8 +297,8 @@ public class CommitLogAllocator
             for (UUID dirtyCFId : oldestSegment.getDirtyCFIDs())
             {
                 String keypace = Schema.instance.getCF(dirtyCFId).left;
-                final ColumnFamilyStore cfs = Table.open(keypace).getColumnFamilyStore(dirtyCFId);
-                // flush shouldn't run on the commitlog executor, since it acquires Table.switchLock,
+                final ColumnFamilyStore cfs = Keyspace.open(keypace).getColumnFamilyStore(dirtyCFId);
+                // flush shouldn't run on the commitlog executor, since it acquires Keyspace.switchLock,
                 // which may already be held by a thread waiting for the CL executor (via getContext),
                 // causing deadlock
                 Runnable runnable = new Runnable()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index a351f7f..a32da86 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -48,7 +48,7 @@ public class CommitLogReplayer
     private static final Logger logger = LoggerFactory.getLogger(CommitLogReplayer.class);
     private static final int MAX_OUTSTANDING_REPLAY_COUNT = 1024;
 
-    private final Set<Table> tablesRecovered;
+    private final Set<Keyspace> keyspacesRecovered;
     private final List<Future<?>> futures;
     private final Map<UUID, AtomicInteger> invalidMutations;
     private final AtomicInteger replayedCount;
@@ -59,7 +59,7 @@ public class CommitLogReplayer
 
     public CommitLogReplayer()
     {
-        this.tablesRecovered = new NonBlockingHashSet<Table>();
+        this.keyspacesRecovered = new NonBlockingHashSet<Keyspace>();
         this.futures = new ArrayList<Future<?>>();
         this.buffer = new byte[4096];
         this.invalidMutations = new HashMap<UUID, AtomicInteger>();
@@ -70,7 +70,7 @@ public class CommitLogReplayer
         // compute per-CF and global replay positions
         cfPositions = new HashMap<UUID, ReplayPosition>();
         Ordering<ReplayPosition> replayPositionOrdering = Ordering.from(ReplayPosition.comparator);
-        Map<UUID,Pair<ReplayPosition,Long>> truncationPositions = SystemTable.getTruncationRecords();
+        Map<UUID,Pair<ReplayPosition,Long>> truncationPositions = SystemKeyspace.getTruncationRecords();
         for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
         {
             // it's important to call RP.gRP per-cf, before aggregating all the positions w/ the Ordering.min call
@@ -105,10 +105,10 @@ public class CommitLogReplayer
         FBUtilities.waitOnFutures(futures);
         logger.debug("Finished waiting on mutations from recovery");
 
-        // flush replayed tables
+        // flush replayed keyspaces
         futures.clear();
-        for (Table table : tablesRecovered)
-            futures.addAll(table.flush());
+        for (Keyspace keyspace : keyspacesRecovered)
+            futures.addAll(keyspace.flush());
         FBUtilities.waitOnFutures(futures);
         return replayedCount.get();
     }
@@ -161,7 +161,7 @@ public class CommitLogReplayer
                     }
 
                     // RowMutation must be at LEAST 10 bytes:
-                    // 3 each for a non-empty Table and Key (including the
+                    // 3 each for a non-empty Keyspace and Key (including the
                     // 2-byte length from writeUTF/writeWithShortLength) and 4 bytes for column count.
                     // This prevents CRC by being fooled by special-case garbage in the file; see CASSANDRA-2128
                     if (serializedSize < 10)
@@ -221,7 +221,7 @@ public class CommitLogReplayer
                 }
 
                 if (logger.isDebugEnabled())
-                    logger.debug(String.format("replaying mutation for %s.%s: %s", rm.getTable(), ByteBufferUtil.bytesToHex(rm.key()), "{" + StringUtils.join(rm.getColumnFamilies().iterator(), ", ")
+                    logger.debug(String.format("replaying mutation for %s.%s: %s", rm.getKeyspaceName(), ByteBufferUtil.bytesToHex(rm.key()), "{" + StringUtils.join(rm.getColumnFamilies().iterator(), ", ")
                             + "}"));
 
                 final long entryLocation = reader.getFilePointer();
@@ -230,12 +230,12 @@ public class CommitLogReplayer
                 {
                     public void runMayThrow() throws IOException
                     {
-                        if (Schema.instance.getKSMetaData(frm.getTable()) == null)
+                        if (Schema.instance.getKSMetaData(frm.getKeyspaceName()) == null)
                             return;
                         if (pointInTimeExceeded(frm))
                             return;
 
-                        final Table table = Table.open(frm.getTable());
+                        final Keyspace keyspace = Keyspace.open(frm.getKeyspaceName());
 
                         // Rebuild the row mutation, omitting column families that 
                         // a) have already been flushed,
@@ -254,7 +254,7 @@ public class CommitLogReplayer
                             if (segment > rp.segment || (segment == rp.segment && entryLocation > rp.position))
                             {
                                 if (newRm == null)
-                                    newRm = new RowMutation(frm.getTable(), frm.key());
+                                    newRm = new RowMutation(frm.getKeyspaceName(), frm.key());
                                 newRm.add(columnFamily);
                                 replayedCount.incrementAndGet();
                             }
@@ -262,8 +262,8 @@ public class CommitLogReplayer
                         if (newRm != null)
                         {
                             assert !newRm.isEmpty();
-                            Table.open(newRm.getTable()).apply(newRm, false);
-                            tablesRecovered.add(table);
+                            Keyspace.open(newRm.getKeyspaceName()).apply(newRm, false);
+                            keyspacesRecovered.add(keyspace);
                         }
                     }
                 };

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/compaction/CompactionController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index 7c79368..4dbd91d 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -140,7 +140,7 @@ public class CompactionController
 
     public String getKeyspace()
     {
-        return cfs.table.getName();
+        return cfs.keyspace.getName();
     }
 
     public String getColumnFamily()
@@ -192,7 +192,7 @@ public class CompactionController
         {
             String keyString = cfs.metadata.getKeyValidator().getString(rows.get(0).getKey().key);
             logger.info(String.format("Compacting large row %s/%s:%s (%d bytes) incrementally",
-                                      cfs.table.getName(), cfs.name, keyString, rowSize));
+                                      cfs.keyspace.getName(), cfs.name, keyString, rowSize));
             return new LazilyCompactedRow(this, rows);
         }
         return new PrecompactedRow(this, rows);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 06dd95d..3641945 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -38,7 +38,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionInfo.Holder;
 import org.apache.cassandra.db.index.SecondaryIndexBuilder;
 import org.apache.cassandra.dht.Bounds;
@@ -137,12 +137,12 @@ public class CompactionManager implements CompactionManagerMBean
         if (count > 0 && executor.getActiveCount() >= executor.getMaximumPoolSize())
         {
             logger.debug("Background compaction is still running for {}.{} ({} remaining). Skipping",
-                         cfs.table.getName(), cfs.name, count);
+                         cfs.keyspace.getName(), cfs.name, count);
             return Collections.emptyList();
         }
 
         logger.debug("Scheduling a background task check for {}.{} with {}",
-                     cfs.table.getName(),
+                     cfs.keyspace.getName(),
                      cfs.name,
                      cfs.getCompactionStrategy().getClass().getSimpleName());
         List<Future<?>> futures = new ArrayList<Future<?>>();
@@ -180,7 +180,7 @@ public class CompactionManager implements CompactionManagerMBean
         {
             try
             {
-                logger.debug("Checking {}.{}", cfs.table.getName(), cfs.name);
+                logger.debug("Checking {}.{}", cfs.keyspace.getName(), cfs.name);
                 if (!cfs.isValid())
                 {
                     logger.debug("Aborting compaction for dropped CF");
@@ -322,7 +322,7 @@ public class CompactionManager implements CompactionManagerMBean
 
         for (Pair<String, String> key : descriptors.keySet())
         {
-            ColumnFamilyStore cfs = Table.open(key.left).getColumnFamilyStore(key.right);
+            ColumnFamilyStore cfs = Keyspace.open(key.left).getColumnFamilyStore(key.right);
             submitUserDefined(cfs, descriptors.get(key), getDefaultGcBefore(cfs));
         }
     }
@@ -409,9 +409,9 @@ public class CompactionManager implements CompactionManagerMBean
     /* Used in tests. */
     public void disableAutoCompaction()
     {
-        for (String ksname : Schema.instance.getNonSystemTables())
+        for (String ksname : Schema.instance.getNonSystemKeyspaces())
         {
-            for (ColumnFamilyStore cfs : Table.open(ksname).getColumnFamilyStores())
+            for (ColumnFamilyStore cfs : Keyspace.open(ksname).getColumnFamilyStores())
                 cfs.disableAutoCompaction();
         }
     }
@@ -464,8 +464,8 @@ public class CompactionManager implements CompactionManagerMBean
     private void doCleanupCompaction(ColumnFamilyStore cfs, Collection<SSTableReader> sstables, CounterId.OneShotRenewer renewer) throws IOException
     {
         assert !cfs.isIndex();
-        Table table = cfs.table;
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(table.getName());
+        Keyspace keyspace = cfs.keyspace;
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspace.getName());
         if (ranges.isEmpty())
         {
             logger.info("Cleanup cannot run before a node has joined the ring");
@@ -549,14 +549,14 @@ public class CompactionManager implements CompactionManagerMBean
                             if (indexedColumnsInRow != null && !indexedColumnsInRow.isEmpty())
                             {
                                 // acquire memtable lock here because secondary index deletion may cause a race. See CASSANDRA-3712
-                                Table.switchLock.readLock().lock();
+                                Keyspace.switchLock.readLock().lock();
                                 try
                                 {
                                     cfs.indexManager.deleteFromIndexes(row.getKey(), indexedColumnsInRow);
                                 }
                                 finally
                                 {
-                                    Table.switchLock.readLock().unlock();
+                                    Keyspace.switchLock.readLock().unlock();
                                 }
                             }
                         }
@@ -643,7 +643,7 @@ public class CompactionManager implements CompactionManagerMBean
         else
         {
             // flush first so everyone is validating data that is as similar as possible
-            StorageService.instance.forceTableFlush(cfs.table.getName(), cfs.name);
+            StorageService.instance.forceKeyspaceFlush(cfs.keyspace.getName(), cfs.name);
 
             // we don't mark validating sstables as compacting in DataTracker, so we have to mark them referenced
             // instead so they won't be cleaned up if they do get compacted during the validation
@@ -674,8 +674,8 @@ public class CompactionManager implements CompactionManagerMBean
         {
             SSTableReader.releaseReferences(sstables);
             iter.close();
-            if (cfs.table.snapshotExists(snapshotName))
-                cfs.table.clearSnapshot(snapshotName);
+            if (cfs.keyspace.snapshotExists(snapshotName))
+                cfs.keyspace.clearSnapshot(snapshotName);
 
             metrics.finishCompaction(ci);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index f07a1e7..a7b6c64 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -104,7 +104,7 @@ public class CompactionTask extends AbstractCompactionTask
         for (SSTableReader sstable : toCompact)
             assert sstable.descriptor.cfname.equals(cfs.name);
 
-        UUID taskId = SystemTable.startCompaction(cfs, toCompact);
+        UUID taskId = SystemKeyspace.startCompaction(cfs, toCompact);
 
         CompactionController controller = new CompactionController(cfs, toCompact, gcBefore);
         Set<SSTableReader> actuallyCompact = Sets.difference(toCompact, controller.getFullyExpiredSSTables());
@@ -222,7 +222,7 @@ public class CompactionTask extends AbstractCompactionTask
             // point of no return -- the new sstables are live on disk; next we'll start deleting the old ones
             // (in replaceCompactedSSTables)
             if (taskId != null)
-                SystemTable.finishCompaction(taskId);
+                SystemKeyspace.finishCompaction(taskId);
 
             if (collector != null)
                 collector.finishCompaction(ci);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/compaction/LegacyLeveledManifest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LegacyLeveledManifest.java b/src/java/org/apache/cassandra/db/compaction/LegacyLeveledManifest.java
index 1bb4619..8403e75 100644
--- a/src/java/org/apache/cassandra/db/compaction/LegacyLeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LegacyLeveledManifest.java
@@ -72,7 +72,7 @@ public class LegacyLeveledManifest
     /**
      * We need to migrate if there is a legacy leveledmanifest json-file
      * <p/>
-     * If there is no jsonfile, we can just start as normally, sstable level will be at 0 for all tables.
+     * If there is no jsonfile, we can just start as normally, sstable level will be at 0 for all sstables.
      *
      * @param keyspace
      * @param columnFamily

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index f43c13e..c0404e3 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -479,7 +479,7 @@ public class LeveledManifest
             }
         }
 
-        // look for a non-suspect table to compact with, starting with where we left off last time,
+        // look for a non-suspect keyspace to compact with, starting with where we left off last time,
         // and wrapping back to the beginning of the generation if necessary
         for (int i = 0; i < generations[level].size(); i++)
         {
@@ -541,7 +541,7 @@ public class LeveledManifest
         }
 
         logger.debug("Estimating {} compactions to do for {}.{}",
-                     Arrays.toString(estimated), cfs.table.getName(), cfs.name);
+                     Arrays.toString(estimated), cfs.keyspace.getName(), cfs.name);
         return Ints.checkedCast(tasks);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
index 3678184..cee5f97 100644
--- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
@@ -163,10 +163,10 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
 
     public static List<Pair<SSTableReader, Long>> createSSTableAndLengthPairs(Iterable<SSTableReader> sstables)
     {
-        List<Pair<SSTableReader, Long>> tableLengthPairs = new ArrayList<Pair<SSTableReader, Long>>(Iterables.size(sstables));
-        for(SSTableReader table: sstables)
-            tableLengthPairs.add(Pair.create(table, table.onDiskLength()));
-        return tableLengthPairs;
+        List<Pair<SSTableReader, Long>> sstableLengthPairs = new ArrayList<Pair<SSTableReader, Long>>(Iterables.size(sstables));
+        for(SSTableReader sstable : sstables)
+            sstableLengthPairs.add(Pair.create(sstable, sstable.onDiskLength()));
+        return sstableLengthPairs;
     }
 
     /*

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
index 4d0914a..73f818f 100644
--- a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
@@ -49,7 +49,7 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
 
         AbstractType indexComparator = SecondaryIndex.getIndexComparator(baseCfs.metadata, columnDef);
         CFMetaData indexedCfMetadata = CFMetaData.newIndexMetadata(baseCfs.metadata, columnDef, indexComparator);
-        indexCfs = ColumnFamilyStore.createColumnFamilyStore(baseCfs.table,
+        indexCfs = ColumnFamilyStore.createColumnFamilyStore(baseCfs.keyspace,
                                                              indexedCfMetadata.cfName,
                                                              new LocalPartitioner(columnDef.getValidator()),
                                                              indexedCfMetadata);
@@ -60,7 +60,7 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         {
             /*
              * # of index CF's key = cardinality of indexed column.
-             * if # of keys stored in index CF is more than average column counts (means tall table),
+             * if # of keys stored in index CF is more than average column counts (means tall keyspaceName),
              * then consider it as high cardinality.
              */
             double estimatedKeys = indexCfs.estimateKeys();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
index 991581d..e77bd0f 100644
--- a/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
@@ -52,7 +52,7 @@ public abstract class PerColumnSecondaryIndex extends SecondaryIndex
      */
     public abstract void update(ByteBuffer rowKey, Column col);
 
-    public String getNameForSystemTable(ByteBuffer column)
+    public String getNameForSystemKeyspace(ByteBuffer column)
     {
         return getIndexName();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
index 1dd2de7..f241447 100644
--- a/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
@@ -53,7 +53,7 @@ public abstract class PerRowSecondaryIndex extends SecondaryIndex
     public abstract void delete(DecoratedKey key);
 
     @Override
-    public String getNameForSystemTable(ByteBuffer columnName)
+    public String getNameForSystemKeyspace(ByteBuffer columnName)
     {
         try
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
index 40ff1cc..c712252 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.Column;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.SystemTable;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.index.keys.KeysIndex;
 import org.apache.cassandra.db.index.composites.CompositesIndex;
@@ -97,12 +97,12 @@ public abstract class SecondaryIndex
 
     /**
      * Return the unique name for this index and column
-     * to be stored in the SystemTable that tracks if each column is built
+     * to be stored in the SystemKeyspace that tracks if each column is built
      *
      * @param columnName the name of the column
      * @return the unique name
      */
-    abstract public String getNameForSystemTable(ByteBuffer columnName);
+    abstract public String getNameForSystemKeyspace(ByteBuffer columnName);
 
     /**
      * Checks if the index for specified column is fully built
@@ -112,19 +112,19 @@ public abstract class SecondaryIndex
      */
     public boolean isIndexBuilt(ByteBuffer columnName)
     {
-        return SystemTable.isIndexBuilt(baseCfs.table.getName(), getNameForSystemTable(columnName));
+        return SystemKeyspace.isIndexBuilt(baseCfs.keyspace.getName(), getNameForSystemKeyspace(columnName));
     }
 
     public void setIndexBuilt()
     {
         for (ColumnDefinition columnDef : columnDefs)
-            SystemTable.setIndexBuilt(baseCfs.table.getName(), getNameForSystemTable(columnDef.name));
+            SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), getNameForSystemKeyspace(columnDef.name));
     }
 
     public void setIndexRemoved()
     {
         for (ColumnDefinition columnDef : columnDefs)
-            SystemTable.setIndexRemoved(baseCfs.table.getName(), getNameForSystemTable(columnDef.name));
+            SystemKeyspace.setIndexRemoved(baseCfs.keyspace.getName(), getNameForSystemKeyspace(columnDef.name));
     }
 
     /**
@@ -204,7 +204,7 @@ public abstract class SecondaryIndex
         boolean allAreBuilt = true;
         for (ColumnDefinition cdef : columnDefs)
         {
-            if (!SystemTable.isIndexBuilt(baseCfs.table.getName(), getNameForSystemTable(cdef.name)))
+            if (!SystemKeyspace.isIndexBuilt(baseCfs.keyspace.getName(), getNameForSystemKeyspace(cdef.name)))
             {
                 allAreBuilt = false;
                 break;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java
index c418a9c..eb09e43 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java
@@ -22,7 +22,7 @@ import java.util.Set;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.compaction.CompactionInterruptedException;
@@ -59,7 +59,7 @@ public class SecondaryIndexBuilder extends CompactionInfo.Holder
             if (isStopRequested())
                 throw new CompactionInterruptedException(getCompactionInfo());
             DecoratedKey key = iter.next();
-            Table.indexRow(key, cfs, idxNames);
+            Keyspace.indexRow(key, cfs, idxNames);
         }
 
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index a40f4bd..4fab92b 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -225,7 +225,7 @@ public class SecondaryIndexManager
         }
 
         index.removeIndex(column);
-        SystemTable.setIndexRemoved(baseCfs.metadata.ksName, index.getNameForSystemTable(column));
+        SystemKeyspace.setIndexRemoved(baseCfs.metadata.ksName, index.getNameForSystemKeyspace(column));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
index d037518..d01889a 100644
--- a/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
@@ -191,7 +191,7 @@ public abstract class AbstractByteOrderedPartitioner extends AbstractPartitioner
             lastToken = node;
         }
 
-        for (String ks : Schema.instance.getTables())
+        for (String ks : Schema.instance.getKeyspaces())
         {
             for (CFMetaData cfmd : Schema.instance.getKSMetaData(ks).cfMetaData().values())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/dht/BootStrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index 55d82e1..57d8d94 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -29,7 +29,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.FailureDetector;
@@ -66,10 +66,10 @@ public class BootStrapper
         RangeStreamer streamer = new RangeStreamer(tokenMetadata, address, "Bootstrap");
         streamer.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(FailureDetector.instance));
 
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            AbstractReplicationStrategy strategy = Table.open(table).getReplicationStrategy();
-            streamer.addRanges(table, strategy.getPendingAddressRanges(tokenMetadata, tokens, address));
+            AbstractReplicationStrategy strategy = Keyspace.open(keyspaceName).getReplicationStrategy();
+            streamer.addRanges(keyspaceName, strategy.getPendingAddressRanges(tokenMetadata, tokens, address));
         }
 
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
index 57c7297..b37c924 100644
--- a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
@@ -184,7 +184,7 @@ public class OrderPreservingPartitioner extends AbstractPartitioner<StringToken>
             lastToken = node;
         }
 
-        for (String ks : Schema.instance.getTables())
+        for (String ks : Schema.instance.getKeyspaces())
         {
             for (CFMetaData cfmd : Schema.instance.getKSMetaData(ks).cfMetaData().values())
             {


[04/11] Rename Table to Keyspace

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index af8c138..4fe180c 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -23,7 +23,7 @@ import java.util.UUID;
 
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.Pair;
 
@@ -80,7 +80,7 @@ public class StreamReceiveTask extends StreamTask
         try
         {
             Pair<String, String> kscf = Schema.instance.getCF(cfId);
-            ColumnFamilyStore cfs = Table.open(kscf.left).getColumnFamilyStore(kscf.right);
+            ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
             // add sstables and build secondary indexes
             cfs.addSSTables(sstables);
             cfs.indexManager.maybeBuildSecondaryIndexes(sstables, cfs.indexManager.allIndexesNames());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index 7d96f43..9c3dfaa 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
@@ -187,12 +187,12 @@ public class StreamSession implements Runnable, IEndpointStateChangeSubscriber,
         // if columnfamilies are not specified, we add all cf under the keyspace
         if (columnFamilies.isEmpty())
         {
-            stores.addAll(Table.open(keyspace).getColumnFamilyStores());
+            stores.addAll(Keyspace.open(keyspace).getColumnFamilyStores());
         }
         else
         {
             for (String cf : columnFamilies)
-                stores.add(Table.open(keyspace).getColumnFamilyStore(cf));
+                stores.add(Keyspace.open(keyspace).getColumnFamilyStore(cf));
         }
 
         if (flushTables)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
index f3e511b..1e8308f 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -27,7 +27,7 @@ import com.google.common.base.Throwables;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
@@ -64,7 +64,7 @@ public class CompressedStreamReader extends StreamReader
         CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo);
 
         Pair<String, String> kscf = Schema.instance.getCF(cfId);
-        ColumnFamilyStore cfs = Table.open(kscf.left).getColumnFamilyStore(kscf.right);
+        ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
         Directories.DataDirectory localDir = cfs.directories.getLocationCapableOfSize(totalSize);
         if (localDir == null)
             throw new IOException("Insufficient disk space to store " + totalSize + " bytes");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index a15927e..0139e1f 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -522,8 +522,8 @@ public class CassandraServer implements Cassandra.Iface
             ThriftClientState cState = state();
             String keyspace = cState.getKeyspace();
             cState.hasColumnFamilyAccess(keyspace, column_parent.column_family, Permission.SELECT);
-            Table table = Table.open(keyspace);
-            ColumnFamilyStore cfs = table.getColumnFamilyStore(column_parent.column_family);
+            Keyspace keyspaceName = Keyspace.open(keyspace);
+            ColumnFamilyStore cfs = keyspaceName.getColumnFamilyStore(column_parent.column_family);
             long timestamp = System.currentTimeMillis();
 
             if (predicate.column_names != null)
@@ -873,7 +873,7 @@ public class CassandraServer implements Cassandra.Iface
         {
             for(ByteBuffer c : del.predicate.column_names)
             {
-                if (del.super_column == null && Schema.instance.getColumnFamilyType(rm.getTable(), cfName) == ColumnFamilyType.Super)
+                if (del.super_column == null && Schema.instance.getColumnFamilyType(rm.getKeyspaceName(), cfName) == ColumnFamilyType.Super)
                     rm.deleteRange(cfName, SuperColumns.startOf(c), SuperColumns.endOf(c), del.timestamp);
                 else if (del.super_column != null)
                     rm.delete(cfName, CompositeType.build(del.super_column, c), del.timestamp);
@@ -1056,11 +1056,11 @@ public class CassandraServer implements Cassandra.Iface
         }
     }
 
-    public KsDef describe_keyspace(String table) throws NotFoundException, InvalidRequestException
+    public KsDef describe_keyspace(String keyspaceName) throws NotFoundException, InvalidRequestException
     {
         validateLogin();
 
-        KSMetaData ksm = Schema.instance.getKSMetaData(table);
+        KSMetaData ksm = Schema.instance.getKSMetaData(keyspaceName);
         if (ksm == null)
             throw new NotFoundException();
 
@@ -1325,7 +1325,7 @@ public class CassandraServer implements Cassandra.Iface
     {
         validateLogin();
 
-        Set<String> keyspaces = Schema.instance.getTables();
+        Set<String> keyspaces = Schema.instance.getKeyspaces();
         List<KsDef> ksset = new ArrayList<KsDef>(keyspaces.size());
         for (String ks : keyspaces)
         {
@@ -1562,7 +1562,7 @@ public class CassandraServer implements Cassandra.Iface
         {
             ThriftValidation.validateKeyspaceNotSystem(ks_def.name);
             state().hasKeyspaceAccess(ks_def.name, Permission.ALTER);
-            ThriftValidation.validateTable(ks_def.name);
+            ThriftValidation.validateKeyspace(ks_def.name);
             if (ks_def.getCf_defs() != null && ks_def.getCf_defs().size() > 0)
                 throw new InvalidRequestException("Keyspace update must not contain any column family definitions.");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index 5dc1e44..6f83243 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -44,7 +44,7 @@ import org.apache.cassandra.utils.FBUtilities;
  *
  * The methods here mostly try to do just one part of the validation so they can be combined
  * for different needs -- supercolumns vs regular, range slices vs named, batch vs single-column.
- * (ValidateColumnPath is the main exception in that it includes table and CF validation.)
+ * (ValidateColumnPath is the main exception in that it includes keyspace and CF validation.)
  */
 public class ThriftValidation
 {
@@ -75,17 +75,17 @@ public class ThriftValidation
         }
     }
 
-    public static void validateTable(String tablename) throws KeyspaceNotDefinedException
+    public static void validateKeyspace(String keyspaceName) throws KeyspaceNotDefinedException
     {
-        if (!Schema.instance.getTables().contains(tablename))
+        if (!Schema.instance.getKeyspaces().contains(keyspaceName))
         {
-            throw new KeyspaceNotDefinedException("Keyspace " + tablename + " does not exist");
+            throw new KeyspaceNotDefinedException("Keyspace " + keyspaceName + " does not exist");
         }
     }
 
-    public static CFMetaData validateColumnFamily(String tablename, String cfName, boolean isCommutativeOp) throws org.apache.cassandra.exceptions.InvalidRequestException
+    public static CFMetaData validateColumnFamily(String keyspaceName, String cfName, boolean isCommutativeOp) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        CFMetaData metadata = validateColumnFamily(tablename, cfName);
+        CFMetaData metadata = validateColumnFamily(keyspaceName, cfName);
 
         if (isCommutativeOp)
         {
@@ -101,13 +101,13 @@ public class ThriftValidation
     }
 
     // To be used when the operation should be authorized whether this is a counter CF or not
-    public static CFMetaData validateColumnFamily(String tablename, String cfName) throws org.apache.cassandra.exceptions.InvalidRequestException
+    public static CFMetaData validateColumnFamily(String keyspaceName, String cfName) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        validateTable(tablename);
+        validateKeyspace(keyspaceName);
         if (cfName.isEmpty())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("non-empty columnfamily is required");
 
-        CFMetaData metadata = Schema.instance.getCFMetaData(tablename, cfName);
+        CFMetaData metadata = Schema.instance.getCFMetaData(keyspaceName, cfName);
         if (metadata == null)
             throw new org.apache.cassandra.exceptions.InvalidRequestException("unconfigured columnfamily " + cfName);
 
@@ -423,7 +423,7 @@ public class ThriftValidation
         }
 
         // Indexed column values cannot be larger than 64K.  See CASSANDRA-3057/4240 for more details
-        if (!Table.open(metadata.ksName).getColumnFamilyStore(metadata.cfName).indexManager.validate(asDBColumn(column)))
+        if (!Keyspace.open(metadata.ksName).getColumnFamilyStore(metadata.cfName).indexManager.validate(asDBColumn(column)))
                     throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("Can't index column value of size %d for index %s in CF %s of KS %s",
                                                                               column.value.remaining(),
                                                                               columnDef.getIndexName(),
@@ -536,7 +536,7 @@ public class ThriftValidation
             // no filter to apply
             return false;
 
-        SecondaryIndexManager idxManager = Table.open(metadata.ksName).getColumnFamilyStore(metadata.cfName).indexManager;
+        SecondaryIndexManager idxManager = Keyspace.open(metadata.ksName).getColumnFamilyStore(metadata.cfName).indexManager;
         AbstractType<?> nameValidator = SuperColumns.getComparatorFor(metadata, null);
 
         boolean isIndexed = false;
@@ -581,7 +581,7 @@ public class ThriftValidation
         // keyspace names must be unique case-insensitively because the keyspace name becomes the directory
         // where we store CF sstables.  Names that differ only in case would thus cause problems on
         // case-insensitive filesystems (NTFS, most installations of HFS+).
-        for (String ksName : Schema.instance.getTables())
+        for (String ksName : Schema.instance.getKeyspaces())
         {
             if (ksName.equalsIgnoreCase(newKsName))
                 throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("Keyspace names must be case-insensitively unique (\"%s\" conflicts with \"%s\")",
@@ -592,7 +592,7 @@ public class ThriftValidation
 
     public static void validateKeyspaceNotSystem(String modifiedKeyspace) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        if (modifiedKeyspace.equalsIgnoreCase(Table.SYSTEM_KS))
+        if (modifiedKeyspace.equalsIgnoreCase(Keyspace.SYSTEM_KS))
             throw new org.apache.cassandra.exceptions.InvalidRequestException("system keyspace is not user-modifiable");
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/tools/BulkLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java b/src/java/org/apache/cassandra/tools/BulkLoader.java
index 12650a2..45d298a 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -32,8 +32,8 @@ import org.apache.thrift.transport.TTransport;
 
 import org.apache.cassandra.auth.IAuthenticator;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.SystemTable;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.SSTableLoader;
@@ -188,8 +188,8 @@ public class BulkLoader
                     }
 
                     String query = String.format("SELECT columnfamily_name FROM %s.%s WHERE keyspace_name = '%s'",
-                                                 Table.SYSTEM_KS,
-                                                 SystemTable.SCHEMA_COLUMNFAMILIES_CF,
+                                                 Keyspace.SYSTEM_KS,
+                                                 SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF,
                                                  keyspace);
                     CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
                     for (CqlRow row : result.rows)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/tools/NodeCmd.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeCmd.java b/src/java/org/apache/cassandra/tools/NodeCmd.java
index 48ae998..bd3c027 100644
--- a/src/java/org/apache/cassandra/tools/NodeCmd.java
+++ b/src/java/org/apache/cassandra/tools/NodeCmd.java
@@ -37,7 +37,7 @@ import org.yaml.snakeyaml.constructor.Constructor;
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionManagerMBean;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.util.FileUtils;
@@ -506,7 +506,7 @@ public class NodeCmd
         }
     }
 
-    /** Writes a table of cluster-wide node information to a PrintStream
+    /** Writes a keyspaceName of cluster-wide node information to a PrintStream
      * @throws UnknownHostException */
     public void printClusterStatus(PrintStream outs, String keyspace) throws UnknownHostException
     {
@@ -743,33 +743,33 @@ public class NodeCmd
         while (cfamilies.hasNext())
         {
             Entry<String, ColumnFamilyStoreMBean> entry = cfamilies.next();
-            String tableName = entry.getKey();
+            String keyspaceName = entry.getKey();
             ColumnFamilyStoreMBean cfsProxy = entry.getValue();
 
-            if (!cfstoreMap.containsKey(tableName))
+            if (!cfstoreMap.containsKey(keyspaceName))
             {
                 List<ColumnFamilyStoreMBean> columnFamilies = new ArrayList<ColumnFamilyStoreMBean>();
                 columnFamilies.add(cfsProxy);
-                cfstoreMap.put(tableName, columnFamilies);
+                cfstoreMap.put(keyspaceName, columnFamilies);
             }
             else
             {
-                cfstoreMap.get(tableName).add(cfsProxy);
+                cfstoreMap.get(keyspaceName).add(cfsProxy);
             }
         }
 
-        // print out the table statistics
+        // print out the keyspace statistics
         for (Entry<String, List<ColumnFamilyStoreMBean>> entry : cfstoreMap.entrySet())
         {
-            String tableName = entry.getKey();
+            String keyspaceName = entry.getKey();
             List<ColumnFamilyStoreMBean> columnFamilies = entry.getValue();
-            long tableReadCount = 0;
-            long tableWriteCount = 0;
-            int tablePendingTasks = 0;
-            double tableTotalReadTime = 0.0f;
-            double tableTotalWriteTime = 0.0f;
+            long keyspaceReadCount = 0;
+            long keyspaceWriteCount = 0;
+            int keyspacePendingTasks = 0;
+            double keyspaceTotalReadTime = 0.0f;
+            double keyspaceTotalWriteTime = 0.0f;
 
-            outs.println("Keyspace: " + tableName);
+            outs.println("Keyspace: " + keyspaceName);
             for (ColumnFamilyStoreMBean cfstore : columnFamilies)
             {
                 long writeCount = cfstore.getWriteCount();
@@ -777,27 +777,27 @@ public class NodeCmd
 
                 if (readCount > 0)
                 {
-                    tableReadCount += readCount;
-                    tableTotalReadTime += cfstore.getTotalReadLatencyMicros();
+                    keyspaceReadCount += readCount;
+                    keyspaceTotalReadTime += cfstore.getTotalReadLatencyMicros();
                 }
                 if (writeCount > 0)
                 {
-                    tableWriteCount += writeCount;
-                    tableTotalWriteTime += cfstore.getTotalWriteLatencyMicros();
+                    keyspaceWriteCount += writeCount;
+                    keyspaceTotalWriteTime += cfstore.getTotalWriteLatencyMicros();
                 }
-                tablePendingTasks += cfstore.getPendingTasks();
+                keyspacePendingTasks += cfstore.getPendingTasks();
             }
 
-            double tableReadLatency = tableReadCount > 0 ? tableTotalReadTime / tableReadCount / 1000 : Double.NaN;
-            double tableWriteLatency = tableWriteCount > 0 ? tableTotalWriteTime / tableWriteCount / 1000 : Double.NaN;
+            double keyspaceReadLatency = keyspaceReadCount > 0 ? keyspaceTotalReadTime / keyspaceReadCount / 1000 : Double.NaN;
+            double keyspaceWriteLatency = keyspaceWriteCount > 0 ? keyspaceTotalWriteTime / keyspaceWriteCount / 1000 : Double.NaN;
 
-            outs.println("\tRead Count: " + tableReadCount);
-            outs.println("\tRead Latency: " + String.format("%s", tableReadLatency) + " ms.");
-            outs.println("\tWrite Count: " + tableWriteCount);
-            outs.println("\tWrite Latency: " + String.format("%s", tableWriteLatency) + " ms.");
-            outs.println("\tPending Tasks: " + tablePendingTasks);
+            outs.println("\tRead Count: " + keyspaceReadCount);
+            outs.println("\tRead Latency: " + String.format("%s", keyspaceReadLatency) + " ms.");
+            outs.println("\tWrite Count: " + keyspaceWriteCount);
+            outs.println("\tWrite Latency: " + String.format("%s", keyspaceWriteLatency) + " ms.");
+            outs.println("\tPending Tasks: " + keyspacePendingTasks);
 
-            // print out column family statistics for this table
+            // print out column family statistics for this keyspace
             for (ColumnFamilyStoreMBean cfstore : columnFamilies)
             {
                 String cfName = cfstore.getColumnFamilyName();
@@ -1364,16 +1364,16 @@ public class NodeCmd
                         probe.forceRepairAsync(System.out, keyspace, snapshot, localDC, primaryRange, columnFamilies);
                     break;
                 case FLUSH   :
-                    try { probe.forceTableFlush(keyspace, columnFamilies); }
+                    try { probe.forceKeyspaceFlush(keyspace, columnFamilies); }
                     catch (ExecutionException ee) { err(ee, "Error occurred during flushing"); }
                     break;
                 case COMPACT :
-                    try { probe.forceTableCompaction(keyspace, columnFamilies); }
+                    try { probe.forceKeyspaceCompaction(keyspace, columnFamilies); }
                     catch (ExecutionException ee) { err(ee, "Error occurred during compaction"); }
                     break;
                 case CLEANUP :
-                    if (keyspace.equals(Table.SYSTEM_KS)) { break; } // Skip cleanup on system cfs.
-                    try { probe.forceTableCleanup(keyspace, columnFamilies); }
+                    if (keyspace.equals(Keyspace.SYSTEM_KS)) { break; } // Skip cleanup on system cfs.
+                    try { probe.forceKeyspaceCleanup(keyspace, columnFamilies); }
                     catch (ExecutionException ee) { err(ee, "Error occurred during cleanup"); }
                     break;
                 case SCRUB :

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 452e865..13624a2 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -179,39 +179,39 @@ public class NodeProbe
         jmxc.close();
     }
 
-    public void forceTableCleanup(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        ssProxy.forceTableCleanup(tableName, columnFamilies);
+        ssProxy.forceKeyspaceCleanup(keyspaceName, columnFamilies);
     }
 
-    public void scrub(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        ssProxy.scrub(tableName, columnFamilies);
+        ssProxy.scrub(keyspaceName, columnFamilies);
     }
 
-    public void upgradeSSTables(String tableName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        ssProxy.upgradeSSTables(tableName, excludeCurrentVersion, columnFamilies);
+        ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, columnFamilies);
     }
 
-    public void forceTableCompaction(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceCompaction(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        ssProxy.forceTableCompaction(tableName, columnFamilies);
+        ssProxy.forceKeyspaceCompaction(keyspaceName, columnFamilies);
     }
 
-    public void forceTableFlush(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceFlush(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        ssProxy.forceTableFlush(tableName, columnFamilies);
+        ssProxy.forceKeyspaceFlush(keyspaceName, columnFamilies);
     }
 
-    public void forceTableRepair(String tableName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException
+    public void forceKeyspaceRepair(String keyspaceName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException
     {
-        ssProxy.forceTableRepair(tableName, isSequential, isLocal, columnFamilies);
+        ssProxy.forceKeyspaceRepair(keyspaceName, isSequential, isLocal, columnFamilies);
     }
 
-    public void forceRepairAsync(final PrintStream out, final String tableName, boolean isSequential, boolean isLocal, boolean primaryRange, String... columnFamilies) throws IOException
+    public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean isSequential, boolean isLocal, boolean primaryRange, String... columnFamilies) throws IOException
     {
-        RepairRunner runner = new RepairRunner(out, tableName, columnFamilies);
+        RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
         try
         {
             ssProxy.addNotificationListener(runner, null, null);
@@ -232,9 +232,9 @@ public class NodeProbe
         }
     }
 
-    public void forceRepairRangeAsync(final PrintStream out, final String tableName, boolean isSequential, boolean isLocal, final String startToken, final String endToken, String... columnFamilies) throws IOException
+    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, boolean isSequential, boolean isLocal, final String startToken, final String endToken, String... columnFamilies) throws IOException
     {
-        RepairRunner runner = new RepairRunner(out, tableName, columnFamilies);
+        RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
         try
         {
             ssProxy.addNotificationListener(runner, null, null);
@@ -255,14 +255,14 @@ public class NodeProbe
         }
     }
 
-    public void forceTableRepairPrimaryRange(String tableName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException
+    public void forceKeyspaceRepairPrimaryRange(String keyspaceName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException
     {
-        ssProxy.forceTableRepairPrimaryRange(tableName, isSequential, isLocal, columnFamilies);
+        ssProxy.forceKeyspaceRepairPrimaryRange(keyspaceName, isSequential, isLocal, columnFamilies);
     }
 
-    public void forceTableRepairRange(String beginToken, String endToken, String tableName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException
+    public void forceKeyspaceRepairRange(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException
     {
-        ssProxy.forceTableRepairRange(beginToken, endToken, tableName, isSequential, isLocal, columnFamilies);
+        ssProxy.forceKeyspaceRepairRange(beginToken, endToken, keyspaceName, isSequential, isLocal, columnFamilies);
     }
 
     public void invalidateKeyCache()
@@ -413,7 +413,7 @@ public class NodeProbe
     }
 
     /**
-     * Take a snapshot of all the tables, optionally specifying only a specific column family.
+     * Take a snapshot of all the keyspaces, optionally specifying only a specific column family.
      *
      * @param snapshotName the name of the snapshot.
      * @param columnFamily the column family to snapshot or all on null
@@ -555,11 +555,11 @@ public class NodeProbe
         return ssProxy.getOperationMode();
     }
 
-    public void truncate(String tableName, String cfName)
+    public void truncate(String keyspaceName, String cfName)
     {
         try
         {
-            ssProxy.truncate(tableName, cfName);
+            ssProxy.truncate(keyspaceName, cfName);
         }
         catch (TimeoutException e)
         {
@@ -846,9 +846,9 @@ class ColumnFamilyStoreMBeanIterator implements Iterator<Map.Entry<String, Colum
             public int compare(Entry<String, ColumnFamilyStoreMBean> e1, Entry<String, ColumnFamilyStoreMBean> e2)
             {
                 //compare keyspace, then CF name, then normal vs. index
-                int tableCmp = e1.getKey().compareTo(e2.getKey());
-                if(tableCmp != 0)
-                    return tableCmp;
+                int keyspaceNameCmp = e1.getKey().compareTo(e2.getKey());
+                if(keyspaceNameCmp != 0)
+                    return keyspaceNameCmp;
 
                 // get CF name and split it for index name
                 String e1CF[] = e1.getValue().getColumnFamilyName().split("\\.");
@@ -883,9 +883,9 @@ class ColumnFamilyStoreMBeanIterator implements Iterator<Map.Entry<String, Colum
         List<Entry<String, ColumnFamilyStoreMBean>> mbeans = new ArrayList<Entry<String, ColumnFamilyStoreMBean>>(cfObjects.size());
         for(ObjectName n : cfObjects)
         {
-            String tableName = n.getKeyProperty("keyspace");
+            String keyspaceName = n.getKeyProperty("keyspace");
             ColumnFamilyStoreMBean cfsProxy = JMX.newMBeanProxy(mbeanServerConn, n, ColumnFamilyStoreMBean.class);
-            mbeans.add(new AbstractMap.SimpleImmutableEntry<String, ColumnFamilyStoreMBean>(tableName, cfsProxy));
+            mbeans.add(new AbstractMap.SimpleImmutableEntry<String, ColumnFamilyStoreMBean>(keyspaceName, cfsProxy));
         }
         return mbeans;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/tools/SSTableExport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableExport.java b/src/java/org/apache/cassandra/tools/SSTableExport.java
index 84af3c4..047a51b 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -271,7 +271,7 @@ public class SSTableExport
     /**
      * Export specific rows from an SSTable and write the resulting JSON to a PrintStream.
      *
-     * @param desc     the descriptor of the sstable table to read from
+     * @param desc     the descriptor of the sstable to read from
      * @param outs     PrintStream to write the output to
      * @param toExport the keys corresponding to the rows to export
      * @param excludes keys to exclude from export
@@ -373,7 +373,7 @@ public class SSTableExport
     /**
      * Export an SSTable and write the resulting JSON to a PrintStream.
      *
-     * @param desc     the descriptor of the sstable table to read from
+     * @param desc     the descriptor of the sstable to read from
      * @param outs     PrintStream to write the output to
      * @param excludes keys to exclude from export
      * @throws IOException on failure to read/write input/output
@@ -386,7 +386,7 @@ public class SSTableExport
     /**
      * Export an SSTable and write the resulting JSON to standard out.
      *
-     * @param desc     the descriptor of the sstable table to read from
+     * @param desc     the descriptor of the sstable to read from
      * @param excludes keys to exclude from export
      * @throws IOException on failure to read/write SSTable/standard out
      */
@@ -442,7 +442,7 @@ public class SSTableExport
                                              ssTableFileName, descriptor.ksname));
             System.exit(1);
         }
-        Table table = Table.open(descriptor.ksname);
+        Keyspace keyspace = Keyspace.open(descriptor.ksname);
 
         // Make it work for indexes too - find parent cf if necessary
         String baseName = descriptor.cfname;
@@ -455,7 +455,7 @@ public class SSTableExport
         // IllegalArgumentException will be thrown here if ks/cf pair does not exist
         try
         {
-            table.getColumnFamilyStore(baseName);
+            keyspace.getColumnFamilyStore(baseName);
         }
         catch (IllegalArgumentException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index 5fddd65..ecdcdd5 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -534,9 +534,9 @@ public class SSTableImport
         }
 
         DatabaseDescriptor.loadSchemas();
-        if (Schema.instance.getNonSystemTables().size() < 1)
+        if (Schema.instance.getNonSystemKeyspaces().size() < 1)
         {
-            String msg = "no non-system tables are defined";
+            String msg = "no non-system keyspaces are defined";
             System.err.println(msg);
             throw new ConfigurationException(msg);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index abb60b1..4971bbd 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -27,7 +27,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.LeveledManifest;
 import org.apache.cassandra.db.compaction.Scrubber;
 import org.apache.cassandra.io.sstable.*;
@@ -57,14 +57,14 @@ public class StandaloneScrubber
             // load keyspace descriptions.
             DatabaseDescriptor.loadSchemas();
 
-            if (Schema.instance.getCFMetaData(options.tableName, options.cfName) == null)
+            if (Schema.instance.getCFMetaData(options.keyspaceName, options.cfName) == null)
                 throw new IllegalArgumentException(String.format("Unknown keyspace/columnFamily %s.%s",
-                                                                 options.tableName,
+                                                                 options.keyspaceName,
                                                                  options.cfName));
 
             // Do not load sstables since they might be broken
-            Table table = Table.openWithoutSSTables(options.tableName);
-            ColumnFamilyStore cfs = table.getColumnFamilyStore(options.cfName);
+            Keyspace keyspace = Keyspace.openWithoutSSTables(options.keyspaceName);
+            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(options.cfName);
             String snapshotName = "pre-scrub-" + System.currentTimeMillis();
 
             OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
@@ -173,16 +173,16 @@ public class StandaloneScrubber
 
     private static class Options
     {
-        public final String tableName;
+        public final String keyspaceName;
         public final String cfName;
 
         public boolean debug;
         public boolean verbose;
         public boolean manifestCheckOnly;
 
-        private Options(String tableName, String cfName)
+        private Options(String keyspaceName, String cfName)
         {
-            this.tableName = tableName;
+            this.keyspaceName = keyspaceName;
             this.cfName = cfName;
         }
 
@@ -209,10 +209,10 @@ public class StandaloneScrubber
                     System.exit(1);
                 }
 
-                String tableName = args[0];
+                String keyspaceName = args[0];
                 String cfName = args[1];
 
-                Options opts = new Options(tableName, cfName);
+                Options opts = new Options(keyspaceName, cfName);
 
                 opts.debug = cmd.hasOption(DEBUG_OPTION);
                 opts.verbose = cmd.hasOption(VERBOSE_OPTION);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
index 9329b0f..295de9a 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -29,7 +29,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.Upgrader;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.service.CassandraDaemon;
@@ -61,8 +61,8 @@ public class StandaloneUpgrader
                                                                  options.keyspace,
                                                                  options.cf));
 
-            Table table = Table.openWithoutSSTables(options.keyspace);
-            ColumnFamilyStore cfs = table.getColumnFamilyStore(options.cf);
+            Keyspace keyspace = Keyspace.openWithoutSSTables(options.keyspace);
+            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(options.cf);
 
             OutputHandler handler = new OutputHandler.SystemOutput(false, options.debug);
             Directories.SSTableLister lister = cfs.directories.sstableLister();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/tracing/Tracing.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index 815c550..22b0244 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -48,7 +48,7 @@ import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
 /**
  * A trace session context. Able to track and store trace sessions. A session is usually a user initiated query, and may
- * have multiple local and remote events before it is completed. All events and sessions are stored at table.
+ * have multiple local and remote events before it is completed. All events and sessions are stored at keyspace.
  */
 public class Tracing
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/utils/BloomCalculations.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/BloomCalculations.java b/src/java/org/apache/cassandra/utils/BloomCalculations.java
index b2477b4..17966e8 100644
--- a/src/java/org/apache/cassandra/utils/BloomCalculations.java
+++ b/src/java/org/apache/cassandra/utils/BloomCalculations.java
@@ -34,7 +34,7 @@ class BloomCalculations {
     private static final int EXCESS = 20;
 
     /**
-     * In the following table, the row 'i' shows false positive rates if i buckets
+     * In the following keyspaceName, the row 'i' shows false positive rates if i buckets
      * per element are used.  Column 'j' shows false positive rates if j hash
      * functions are used.  The first row is 'i=0', the first column is 'j=0'.
      * Each cell (i,j) the false positive rate determined by using i buckets per

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/utils/CounterId.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CounterId.java b/src/java/org/apache/cassandra/utils/CounterId.java
index f37c396..068a2ce 100644
--- a/src/java/org/apache/cassandra/utils/CounterId.java
+++ b/src/java/org/apache/cassandra/utils/CounterId.java
@@ -27,7 +27,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Objects;
 
 import org.apache.cassandra.db.CounterColumn;
-import org.apache.cassandra.db.SystemTable;
+import org.apache.cassandra.db.SystemKeyspace;
 
 public class CounterId implements Comparable<CounterId>
 {
@@ -35,7 +35,7 @@ public class CounterId implements Comparable<CounterId>
 
     public static final int LENGTH = 16; // we assume a fixed length size for all CounterIds
 
-    // Lazy holder because this opens the system table and we want to avoid
+    // Lazy holder because this opens the system keyspace and we want to avoid
     // having this triggered during class initialization
     private static class LocalIds
     {
@@ -191,13 +191,13 @@ public class CounterId implements Comparable<CounterId>
 
         LocalCounterIdHistory()
         {
-            CounterId id = SystemTable.getCurrentLocalCounterId();
+            CounterId id = SystemKeyspace.getCurrentLocalCounterId();
             if (id == null)
             {
                 // no recorded local counter id, generating a new one and saving it
                 id = generate();
                 logger.info("No saved local counter id, using newly generated: {}", id);
-                SystemTable.writeCurrentLocalCounterId(null, id, FBUtilities.timestampMicros());
+                SystemKeyspace.writeCurrentLocalCounterId(null, id, FBUtilities.timestampMicros());
                 current = new AtomicReference<CounterId>(id);
                 olds = new CopyOnWriteArrayList<CounterIdRecord>();
             }
@@ -205,7 +205,7 @@ public class CounterId implements Comparable<CounterId>
             {
                 logger.info("Saved local counter id: {}", id);
                 current = new AtomicReference<CounterId>(id);
-                olds = new CopyOnWriteArrayList<CounterIdRecord>(SystemTable.getOldLocalCounterIds());
+                olds = new CopyOnWriteArrayList<CounterIdRecord>(SystemKeyspace.getOldLocalCounterIds());
             }
         }
 
@@ -213,7 +213,7 @@ public class CounterId implements Comparable<CounterId>
         {
             CounterId newCounterId = generate();
             CounterId old = current.get();
-            SystemTable.writeCurrentLocalCounterId(old, newCounterId, now);
+            SystemKeyspace.writeCurrentLocalCounterId(old, newCounterId, now);
             current.set(newCounterId);
             olds.add(new CounterIdRecord(old, now));
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/utils/StatusLogger.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/StatusLogger.java b/src/java/org/apache/cassandra/utils/StatusLogger.java
index 94f045e..daf7c3b 100644
--- a/src/java/org/apache/cassandra/utils/StatusLogger.java
+++ b/src/java/org/apache/cassandra/utils/StatusLogger.java
@@ -109,7 +109,7 @@ public class StatusLogger
         for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
         {
             logger.info(String.format("%-25s%20s",
-                                      cfs.table.getName() + "." + cfs.name,
+                                      cfs.keyspace.getName() + "." + cfs.name,
                                       cfs.getMemtableColumnsCount() + "," + cfs.getMemtableDataSize()));
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/utils/obs/BitUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/obs/BitUtil.java b/src/java/org/apache/cassandra/utils/obs/BitUtil.java
index 1dd5661..200ffbf 100644
--- a/src/java/org/apache/cassandra/utils/obs/BitUtil.java
+++ b/src/java/org/apache/cassandra/utils/obs/BitUtil.java
@@ -683,7 +683,7 @@ final class BitUtil {
     return i
   print ','.join([ str(ntz(i)) for i in range(256) ])
   ***/
-  /** table of number of trailing zeros in a byte */
+  /** keyspaceName of number of trailing zeros in a byte */
   public static final byte[] ntzTable = {8,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,4,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,5,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,4,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,6,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,4,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,5,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,4,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,7,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,4,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,5,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,4,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,6,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,4,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,5,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0,4,0,1,0,2,0,1,0,3,0,1,0,2,0,1,0};
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
new file mode 100644
index 0000000..b1ef690
--- /dev/null
+++ b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db;
+
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.utils.WrappedRunnable;
+import static org.apache.cassandra.Util.column;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+
+public class LongKeyspaceTest extends SchemaLoader
+{
+    @Test
+    public void testGetRowMultiColumn() throws Throwable
+    {
+        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
+
+        for (int i = 1; i < 5000; i += 100)
+        {
+            RowMutation rm = new RowMutation("Keyspace1", Util.dk("key" + i).key);
+            ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+            for (int j = 0; j < i; j++)
+                cf.addColumn(column("c" + j, "v" + j, 1L));
+            rm.add(cf);
+            rm.applyUnsafe();
+        }
+
+        Runnable verify = new WrappedRunnable()
+        {
+            public void runMayThrow() throws Exception
+            {
+                ColumnFamily cf;
+                for (int i = 1; i < 5000; i += 100)
+                {
+                    for (int j = 0; j < i; j++)
+                    {
+                        cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("key" + i),
+                                                                                "Standard1",
+                                                                                ByteBufferUtil.bytes("c" + j),
+                                                                                System.currentTimeMillis()));
+                        KeyspaceTest.assertColumns(cf, "c" + j);
+                    }
+                }
+
+            }
+        };
+        KeyspaceTest.reTest(keyspace.getColumnFamilyStore("Standard1"), verify);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/long/org/apache/cassandra/db/LongTableTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/LongTableTest.java b/test/long/org/apache/cassandra/db/LongTableTest.java
deleted file mode 100644
index 70b40d2..0000000
--- a/test/long/org/apache/cassandra/db/LongTableTest.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.db;
-
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.utils.WrappedRunnable;
-import static org.apache.cassandra.Util.column;
-
-import org.apache.cassandra.Util;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-
-public class LongTableTest extends SchemaLoader
-{
-    @Test
-    public void testGetRowMultiColumn() throws Throwable
-    {
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard1");
-
-        for (int i = 1; i < 5000; i += 100)
-        {
-            RowMutation rm = new RowMutation("Keyspace1", Util.dk("key" + i).key);
-            ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-            for (int j = 0; j < i; j++)
-                cf.addColumn(column("c" + j, "v" + j, 1L));
-            rm.add(cf);
-            rm.applyUnsafe();
-        }
-
-        Runnable verify = new WrappedRunnable()
-        {
-            public void runMayThrow() throws Exception
-            {
-                ColumnFamily cf;
-                for (int i = 1; i < 5000; i += 100)
-                {
-                    for (int j = 0; j < i; j++)
-                    {
-                        cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("key" + i),
-                                                                                "Standard1",
-                                                                                ByteBufferUtil.bytes("c" + j),
-                                                                                System.currentTimeMillis()));
-                        TableTest.assertColumns(cf, "c" + j);
-                    }
-                }
-
-            }
-        };
-        TableTest.reTest(table.getColumnFamilyStore("Standard1"), verify);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/MeteredFlusherTest.java b/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
index bd905e0..5e25744 100644
--- a/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
+++ b/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
@@ -38,10 +38,10 @@ public class MeteredFlusherTest extends SchemaLoader
     @Test
     public void testManyMemtables() throws IOException, ConfigurationException
     {
-        Table table = Table.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
         for (int i = 0; i < 100; i++)
         {
-            CFMetaData metadata = new CFMetaData(table.getName(), "_CF" + i, ColumnFamilyType.Standard, UTF8Type.instance, null);
+            CFMetaData metadata = new CFMetaData(keyspace.getName(), "_CF" + i, ColumnFamilyType.Standard, UTF8Type.instance, null);
             MigrationManager.announceNewColumnFamily(metadata);
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
index a1323cb..48c64e3 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -38,7 +38,7 @@ import static junit.framework.Assert.assertEquals;
 
 public class LongCompactionsTest extends SchemaLoader
 {
-    public static final String TABLE1 = "Keyspace1";
+    public static final String KEYSPACE1 = "Keyspace1";
 
     /**
      * Test compaction with a very wide row.
@@ -71,8 +71,8 @@ public class LongCompactionsTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open(TABLE1);
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
 
         ArrayList<SSTableReader> sstables = new ArrayList<SSTableReader>();
         for (int k = 0; k < sstableCount; k++)
@@ -98,7 +98,7 @@ public class LongCompactionsTest extends SchemaLoader
         Thread.sleep(1000);
 
         long start = System.nanoTime();
-        final int gcBefore = (int) (System.currentTimeMillis() / 1000) - Schema.instance.getCFMetaData(TABLE1, "Standard1").getGcGraceSeconds();
+        final int gcBefore = (int) (System.currentTimeMillis() / 1000) - Schema.instance.getCFMetaData(KEYSPACE1, "Standard1").getGcGraceSeconds();
         new CompactionTask(store, sstables, gcBefore).execute(null);
         System.out.println(String.format("%s: sstables=%d rowsper=%d colsper=%d: %d ms",
                                          this.getClass().getName(),
@@ -112,8 +112,8 @@ public class LongCompactionsTest extends SchemaLoader
     public void testStandardColumnCompactions() throws IOException, ExecutionException, InterruptedException
     {
         // this test does enough rows to force multiple block indexes to be used
-        Table table = Table.open(TABLE1);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
         cfs.clearUnsafe();
 
         final int ROWS_PER_SSTABLE = 10;
@@ -127,7 +127,7 @@ public class LongCompactionsTest extends SchemaLoader
         for (int j = 0; j < SSTABLES; j++) {
             for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
                 DecoratedKey key = Util.dk(String.valueOf(i % 2));
-                RowMutation rm = new RowMutation(TABLE1, key.key);
+                RowMutation rm = new RowMutation(KEYSPACE1, key.key);
                 long timestamp = j * ROWS_PER_SSTABLE + i;
                 rm.add("Standard1", ByteBufferUtil.bytes(String.valueOf(i / 2)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
index f3be106..c6b6eb0 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.sstable.SSTable;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -43,8 +43,8 @@ public class LongLeveledCompactionStrategyTest extends SchemaLoader
     {
         String ksname = "Keyspace1";
         String cfname = "StandardLeveled";
-        Table table = Table.open(ksname);
-        ColumnFamilyStore store = table.getColumnFamilyStore(cfname);
+        Keyspace keyspace = Keyspace.open(ksname);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(cfname);
         store.disableAutoCompaction();
 
         LeveledCompactionStrategy lcs = (LeveledCompactionStrategy)store.getCompactionStrategy();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index a893576..1bd192b 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -392,7 +392,7 @@ public class SchemaLoader
 
         cleanupSavedCaches();
 
-        // clean up data directory which are stored as data directory/table/data files
+        // clean up data directory which are stored as data directory/keyspace/data files
         for (String dirName : DatabaseDescriptor.getAllDataFileLocations())
         {
             File dir = new File(dirName);
@@ -421,7 +421,7 @@ public class SchemaLoader
     /* usually used to populate the cache */
     protected void readData(String keyspace, String columnFamily, int offset, int numberOfRows) throws IOException
     {
-        ColumnFamilyStore store = Table.open(keyspace).getColumnFamilyStore(columnFamily);
+        ColumnFamilyStore store = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily);
         for (int i = offset; i < offset + numberOfRows; i++)
         {
             DecoratedKey key = Util.dk("key" + i);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 053fa7d..b4a375a 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -156,26 +156,26 @@ public class Util
     /**
      * Writes out a bunch of mutations for a single column family.
      *
-     * @param mutations A group of RowMutations for the same table and column family.
+     * @param mutations A group of RowMutations for the same keyspace and column family.
      * @return The ColumnFamilyStore that was used.
      */
     public static ColumnFamilyStore writeColumnFamily(List<IMutation> mutations) throws IOException, ExecutionException, InterruptedException
     {
         IMutation first = mutations.get(0);
-        String tablename = first.getTable();
+        String keyspaceName = first.getKeyspaceName();
         UUID cfid = first.getColumnFamilyIds().iterator().next();
 
         for (IMutation rm : mutations)
             rm.apply();
 
-        ColumnFamilyStore store = Table.open(tablename).getColumnFamilyStore(cfid);
+        ColumnFamilyStore store = Keyspace.open(keyspaceName).getColumnFamilyStore(cfid);
         store.forceBlockingFlush();
         return store;
     }
 
-    public static ColumnFamily getColumnFamily(Table table, DecoratedKey key, String cfName) throws IOException
+    public static ColumnFamily getColumnFamily(Keyspace keyspace, DecoratedKey key, String cfName) throws IOException
     {
-        ColumnFamilyStore cfStore = table.getColumnFamilyStore(cfName);
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(cfName);
         assert cfStore != null : "Column family " + cfName + " has not been defined";
         return cfStore.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
index 326f62e..ce36e48 100644
--- a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
+++ b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
@@ -45,7 +45,7 @@ public class CacheProviderTest extends SchemaLoader
     MeasureableString key4 = new MeasureableString("key4");
     MeasureableString key5 = new MeasureableString("key5");
     private static final long CAPACITY = 4;
-    private String tableName = "Keyspace1";
+    private String keyspaceName = "Keyspace1";
     private String cfName = "Standard1";
 
     private void simpleCase(ColumnFamily cf, ICache<MeasureableString, IRowCacheEntry> cache)
@@ -100,7 +100,7 @@ public class CacheProviderTest extends SchemaLoader
 
     private ColumnFamily createCF()
     {
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(tableName, cfName);
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(keyspaceName, cfName);
         cf.addColumn(column("vijay", "great", 1));
         cf.addColumn(column("awesome", "vijay", 1));
         return cf;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
index 73ba0a6..ce105c4 100644
--- a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
@@ -26,13 +26,7 @@ import java.util.HashMap;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Row;
-import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.db.SystemTable;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.io.compress.*;
@@ -104,9 +98,9 @@ public class CFMetaDataTest extends SchemaLoader
     @Test
     public void testConversionsInverses() throws Exception
     {
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            for (ColumnFamilyStore cfs : Table.open(table).getColumnFamilyStores())
+            for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
             {
                 CFMetaData cfm = cfs.metadata;
                 checkInverses(cfm);
@@ -148,8 +142,8 @@ public class CFMetaDataTest extends SchemaLoader
 
         // Test schema conversion
         RowMutation rm = cfm.toSchema(System.currentTimeMillis());
-        ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(Table.SYSTEM_KS, SystemTable.SCHEMA_COLUMNFAMILIES_CF));
-        ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(Table.SYSTEM_KS, SystemTable.SCHEMA_COLUMNS_CF));
+        ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF));
+        ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNS_CF));
         UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", new Row(k, serializedCf)).one();
         CFMetaData newCfm = CFMetaData.addColumnDefinitionSchema(CFMetaData.fromSchemaNoColumns(result), new Row(k, serializedCD));
         assert cfm.equals(newCfm) : String.format("\n%s\n!=\n%s", cfm, newCfm);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index ee14ab5..7d1c82b 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -41,9 +41,9 @@ public class DatabaseDescriptorTest
     public void testCFMetaDataSerialization() throws IOException, ConfigurationException, InvalidRequestException
     {
         // test serialization of all defined test CFs.
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            for (CFMetaData cfm : Schema.instance.getTableMetaData(table).values())
+            for (CFMetaData cfm : Schema.instance.getKeyspaceMetaData(keyspaceName).values())
             {
                 CFMetaData cfmDupe = CFMetaData.fromThrift(cfm.toThrift());
                 assert cfmDupe != null;
@@ -55,7 +55,7 @@ public class DatabaseDescriptorTest
     @Test
     public void testKSMetaDataSerialization() throws IOException, ConfigurationException
     {
-        for (KSMetaData ksm : Schema.instance.getTableDefinitions())
+        for (KSMetaData ksm : Schema.instance.getKeyspaceDefinitions())
         {
             // Not testing round-trip on the KsDef via serDe() because maps
             KSMetaData ksmDupe = KSMetaData.fromThrift(ksm.toThrift());
@@ -70,7 +70,7 @@ public class DatabaseDescriptorTest
     {
         SchemaLoader.cleanupAndLeaveDirs();
         DatabaseDescriptor.loadSchemas();
-        assert Schema.instance.getNonSystemTables().size() == 0;
+        assert Schema.instance.getNonSystemKeyspaces().size() == 0;
 
         Gossiper.instance.start((int)(System.currentTimeMillis() / 1000));
 
@@ -83,8 +83,8 @@ public class DatabaseDescriptorTest
             assert Schema.instance.getKSMetaData("ks0") != null;
             assert Schema.instance.getKSMetaData("ks1") != null;
 
-            Schema.instance.clearTableDefinition(Schema.instance.getKSMetaData("ks0"));
-            Schema.instance.clearTableDefinition(Schema.instance.getKSMetaData("ks1"));
+            Schema.instance.clearKeyspaceDefinition(Schema.instance.getKSMetaData("ks0"));
+            Schema.instance.clearKeyspaceDefinition(Schema.instance.getKSMetaData("ks1"));
 
             assert Schema.instance.getKSMetaData("ks0") == null;
             assert Schema.instance.getKSMetaData("ks1") == null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/config/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DefsTest.java b/test/unit/org/apache/cassandra/config/DefsTest.java
index 9864398..79c9b33 100644
--- a/test/unit/org/apache/cassandra/config/DefsTest.java
+++ b/test/unit/org/apache/cassandra/config/DefsTest.java
@@ -112,21 +112,21 @@ public class DefsTest extends SchemaLoader
         /*
         // verify dump and reload.
         UUID first = UUIDGen.makeType1UUIDFromHost(FBUtilities.getBroadcastAddress());
-        DefsTable.dumpToStorage(first);
-        List<KSMetaData> defs = new ArrayList<KSMetaData>(DefsTable.loadFromStorage(first));
+        DefsTables.dumpToStorage(first);
+        List<KSMetaData> defs = new ArrayList<KSMetaData>(DefsTables.loadFromStorage(first));
 
         assert defs.size() > 0;
-        assert defs.size() == Schema.instance.getNonSystemTables().size();
+        assert defs.size() == Schema.instance.getNonSystemKeyspaces().size();
         for (KSMetaData loaded : defs)
         {
-            KSMetaData defined = Schema.instance.getTableDefinition(loaded.name);
+            KSMetaData defined = Schema.instance.getKeyspaceDefinition(loaded.name);
             assert defined.equals(loaded) : String.format("%s != %s", loaded, defined);
         }
         */
     }
 
     @Test
-    public void addNewCfToBogusTable() throws InterruptedException
+    public void addNewCfToBogusKeyspace() throws InterruptedException
     {
         CFMetaData newCf = addTestCF("MadeUpKeyspace", "NewCF", "new cf");
         try
@@ -175,7 +175,7 @@ public class DefsTest extends SchemaLoader
         RowMutation rm = new RowMutation(ks, dk.key);
         rm.add(cf, ByteBufferUtil.bytes("col0"), ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
-        ColumnFamilyStore store = Table.open(ks).getColumnFamilyStore(cf);
+        ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
         assert store != null;
         store.forceBlockingFlush();
 
@@ -200,7 +200,7 @@ public class DefsTest extends SchemaLoader
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, ByteBufferUtil.bytes(("col" + i)), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
-        ColumnFamilyStore store = Table.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
         assert store != null;
         store.forceBlockingFlush();
         assert store.directories.sstableLister().list().size() > 0;
@@ -248,7 +248,7 @@ public class DefsTest extends SchemaLoader
         RowMutation rm = new RowMutation(newCf.ksName, dk.key);
         rm.add(newCf.cfName, ByteBufferUtil.bytes("col0"), ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
-        ColumnFamilyStore store = Table.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
+        ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
         assert store != null;
         store.forceBlockingFlush();
 
@@ -273,7 +273,7 @@ public class DefsTest extends SchemaLoader
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, ByteBufferUtil.bytes(("col" + i)), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
-        ColumnFamilyStore store = Table.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
         assert store != null;
         store.forceBlockingFlush();
         assert store.directories.sstableLister().list().size() > 0;
@@ -300,7 +300,7 @@ public class DefsTest extends SchemaLoader
         boolean threw = false;
         try
         {
-            Table.open(ks.name);
+            Keyspace.open(ks.name);
         }
         catch (Throwable th)
         {
@@ -356,7 +356,7 @@ public class DefsTest extends SchemaLoader
         RowMutation rm = new RowMutation(newKs.name, dk.key);
         rm.add(newCf.cfName, ByteBufferUtil.bytes("col0"), ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
-        ColumnFamilyStore store = Table.open(newKs.name).getColumnFamilyStore(newCf.cfName);
+        ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
         assert store != null;
         store.forceBlockingFlush();
 
@@ -498,7 +498,7 @@ public class DefsTest extends SchemaLoader
     @Test
     public void testDropIndex() throws IOException, ExecutionException, InterruptedException, ConfigurationException
     {
-        // persist keyspace definition in the system table
+        // persist keyspace definition in the system keyspace
         Schema.instance.getKSMetaData("Keyspace6").toSchema(System.currentTimeMillis()).apply();
 
         // insert some data.  save the sstable descriptor so we can make sure it's marked for delete after the drop
@@ -506,7 +506,7 @@ public class DefsTest extends SchemaLoader
         rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
-        ColumnFamilyStore cfs = Table.open("Keyspace6").getColumnFamilyStore("Indexed1");
+        ColumnFamilyStore cfs = Keyspace.open("Keyspace6").getColumnFamilyStore("Indexed1");
         cfs.forceBlockingFlush();
         ColumnFamilyStore indexedCfs = cfs.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
         Descriptor desc = indexedCfs.getSSTables().iterator().next().descriptor;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 4c02a75..562df1e 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -50,7 +50,7 @@ import org.junit.Test;
 public class CleanupTest extends SchemaLoader
 {
     public static final int LOOPS = 200;
-    public static final String TABLE1 = "Keyspace1";
+    public static final String KEYSPACE1 = "Keyspace1";
     public static final String CF1 = "Indexed1";
     public static final String CF2 = "Standard1";
     public static final ByteBuffer COLUMN = ByteBufferUtil.bytes("birthdate");
@@ -66,8 +66,8 @@ public class CleanupTest extends SchemaLoader
     {
         StorageService.instance.initServer(0);
 
-        Table table = Table.open(TABLE1);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(CF2);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF2);
 
         List<Row> rows;
 
@@ -94,8 +94,8 @@ public class CleanupTest extends SchemaLoader
     @Test
     public void testCleanupWithIndexes() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open(TABLE1);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(CF1);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF1);
 
         List<Row> rows;
 
@@ -115,7 +115,7 @@ public class CleanupTest extends SchemaLoader
         IDiskAtomFilter filter = new IdentityQueryFilter();
         IPartitioner p = StorageService.getPartitioner();
         Range<RowPosition> range = Util.range("", "");
-        rows = table.getColumnFamilyStore(CF1).search(range, clause, filter, Integer.MAX_VALUE);
+        rows = keyspace.getColumnFamilyStore(CF1).search(range, clause, filter, Integer.MAX_VALUE);
         assertEquals(LOOPS, rows.size());
 
         // we don't allow cleanup when the local host has no range to avoid wipping up all data when a node has not join the ring.
@@ -150,7 +150,7 @@ public class CleanupTest extends SchemaLoader
             String key = String.valueOf(i);
             // create a row and update the birthdate value, test that the index query fetches the new version
             RowMutation rm;
-            rm = new RowMutation(TABLE1, ByteBufferUtil.bytes(key));
+            rm = new RowMutation(KEYSPACE1, ByteBufferUtil.bytes(key));
             rm.add(cfs.name, COLUMN, VALUE, System.currentTimeMillis());
             rm.applyUnsafe();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/CollationControllerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CollationControllerTest.java b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
index ca0ec6e..0c23272 100644
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
@@ -37,8 +37,8 @@ public class CollationControllerTest extends SchemaLoader
     public void getTopLevelColumnsSkipsSSTablesModifiedBeforeRowDelete() 
             throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
         


[05/11] Rename Table to Keyspace

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index a67889a..dbd15da 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -55,7 +55,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.dht.*;
@@ -145,9 +145,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return DatabaseDescriptor.getPartitioner();
     }
 
-    public Collection<Range<Token>> getLocalRanges(String table)
+    public Collection<Range<Token>> getLocalRanges(String keyspaceName)
     {
-        return getRangesForEndpoint(table, FBUtilities.getBroadcastAddress());
+        return getRangesForEndpoint(keyspaceName, FBUtilities.getBroadcastAddress());
     }
 
     public Collection<Range<Token>> getLocalPrimaryRanges(String keyspace)
@@ -166,7 +166,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     /* we bootstrap but do NOT join the ring unless told to do so */
     private boolean isSurveyMode= Boolean.parseBoolean(System.getProperty("cassandra.write_survey", "false"));
 
-    /* when intialized as a client, we shouldn't write to the system table. */
+    /* when intialized as a client, we shouldn't write to the system keyspace. */
     private boolean isClientMode;
     private boolean initialized;
     private volatile boolean joined = false;
@@ -202,7 +202,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         if (logger.isDebugEnabled())
             logger.debug("Setting tokens to {}", tokens);
-        SystemTable.updateTokens(tokens);
+        SystemKeyspace.updateTokens(tokens);
         tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddress());
         // order is important here, the gossiper can fire in between adding these two states.  It's ok to send TOKENS without STATUS, but *not* vice versa.
         Gossiper.instance.addLocalApplicationState(ApplicationState.TOKENS, valueFactory.tokens(getLocalTokens()));
@@ -454,14 +454,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (Boolean.parseBoolean(System.getProperty("cassandra.load_ring_state", "true")))
         {
             logger.info("Loading persisted ring state");
-            Multimap<InetAddress, Token> loadedTokens = SystemTable.loadTokens();
-            Map<InetAddress, UUID> loadedHostIds = SystemTable.loadHostIds();
+            Multimap<InetAddress, Token> loadedTokens = SystemKeyspace.loadTokens();
+            Map<InetAddress, UUID> loadedHostIds = SystemKeyspace.loadHostIds();
             for (InetAddress ep : loadedTokens.keySet())
             {
                 if (ep.equals(FBUtilities.getBroadcastAddress()))
                 {
                     // entry has been mistakenly added, delete it
-                    SystemTable.removeEndpoint(ep);
+                    SystemKeyspace.removeEndpoint(ep);
                 }
                 else
                 {
@@ -501,12 +501,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 StorageProxy.instance.verifyNoHintsInProgress();
 
                 List<Future<?>> flushes = new ArrayList<Future<?>>();
-                for (Table table : Table.all())
+                for (Keyspace keyspace : Keyspace.all())
                 {
-                    KSMetaData ksm = Schema.instance.getKSMetaData(table.getName());
+                    KSMetaData ksm = Schema.instance.getKSMetaData(keyspace.getName());
                     if (!ksm.durableWrites)
                     {
-                        for (ColumnFamilyStore cfs : table.getColumnFamilyStores())
+                        for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
                             flushes.add(cfs.forceFlush());
                     }
                 }
@@ -538,21 +538,21 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         joined = true;
 
         // Seed the host ID-to-endpoint map with our own ID.
-        getTokenMetadata().updateHostId(SystemTable.getLocalHostId(), FBUtilities.getBroadcastAddress());
+        getTokenMetadata().updateHostId(SystemKeyspace.getLocalHostId(), FBUtilities.getBroadcastAddress());
 
         // have to start the gossip service before we can see any info on other nodes.  this is necessary
         // for bootstrap to get the load info it needs.
         // (we won't be part of the storage ring though until we add a counterId to our state, below.)
         Map<ApplicationState, VersionedValue> appStates = new HashMap<ApplicationState, VersionedValue>();
         appStates.put(ApplicationState.NET_VERSION, valueFactory.networkVersion());
-        appStates.put(ApplicationState.HOST_ID, valueFactory.hostId(SystemTable.getLocalHostId()));
+        appStates.put(ApplicationState.HOST_ID, valueFactory.hostId(SystemKeyspace.getLocalHostId()));
         appStates.put(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(DatabaseDescriptor.getRpcAddress()));
         if (0 != DatabaseDescriptor.getReplaceTokens().size())
             appStates.put(ApplicationState.STATUS, valueFactory.hibernate(true));
         appStates.put(ApplicationState.RELEASE_VERSION, valueFactory.releaseVersion());
         Gossiper.instance.register(this);
         Gossiper.instance.register(migrationManager);
-        Gossiper.instance.start(SystemTable.incrementAndGetGeneration(), appStates); // needed for node-ring gathering.
+        Gossiper.instance.start(SystemKeyspace.incrementAndGetGeneration(), appStates); // needed for node-ring gathering.
         // gossip snitch infos (local DC and rack)
         gossipSnitchInfo();
         // gossip Schema.emptyVersion forcing immediate check for schema updates (see MigrationManager#maybeScheduleSchemaPull)
@@ -578,17 +578,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         Collection<Token> tokens;
         logger.debug("Bootstrap variables: {} {} {} {}",
                      DatabaseDescriptor.isAutoBootstrap(),
-                     SystemTable.bootstrapInProgress(),
-                     SystemTable.bootstrapComplete(),
+                     SystemKeyspace.bootstrapInProgress(),
+                     SystemKeyspace.bootstrapComplete(),
                      DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()));
         if (DatabaseDescriptor.isAutoBootstrap()
-            && !SystemTable.bootstrapComplete()
+            && !SystemKeyspace.bootstrapComplete()
             && !DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()))
         {
-            if (SystemTable.bootstrapInProgress())
+            if (SystemKeyspace.bootstrapInProgress())
                 logger.warn("Detected previous bootstrap failure; retrying");
             else
-                SystemTable.setBootstrapState(SystemTable.BootstrapState.IN_PROGRESS);
+                SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.IN_PROGRESS);
             setMode(Mode.JOINING, "waiting for ring information", true);
             // first sleep the delay to make sure we see all our peers
             for (int i = 0; i < delay; i += 1000)
@@ -653,7 +653,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
         else
         {
-            tokens = SystemTable.getSavedTokens();
+            tokens = SystemKeyspace.getSavedTokens();
             if (tokens.isEmpty())
             {
                 Collection<String> initialTokens = DatabaseDescriptor.getInitialTokens();
@@ -726,7 +726,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (!isSurveyMode)
         {
             // start participating in the ring.
-            SystemTable.setBootstrapState(SystemTable.BootstrapState.COMPLETED);
+            SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED);
             setTokens(tokens);
             // remove the existing info about the replaced node.
             if (!current.isEmpty())
@@ -768,8 +768,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }
         else if (isSurveyMode)
         {
-            setTokens(SystemTable.getSavedTokens());
-            SystemTable.setBootstrapState(SystemTable.BootstrapState.COMPLETED);
+            setTokens(SystemKeyspace.getSavedTokens());
+            SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED);
             isSurveyMode = false;
             logger.info("Leaving write survey mode and joining ring at operator request");
             assert tokenMetadata.sortedTokens().size() > 0;
@@ -792,8 +792,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (sourceDc != null)
             streamer.addSourceFilter(new RangeStreamer.SingleDatacenterFilter(DatabaseDescriptor.getEndpointSnitch(), sourceDc));
 
-        for (String table : Schema.instance.getNonSystemTables())
-            streamer.addRanges(table, getLocalRanges(table));
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
+            streamer.addRanges(keyspaceName, getLocalRanges(keyspaceName));
 
         try
         {
@@ -860,7 +860,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     private void bootstrap(Collection<Token> tokens)
     {
         isBootstrapMode = true;
-        SystemTable.updateTokens(tokens); // DON'T use setToken, that makes us part of the ring locally which is incorrect until we are done bootstrapping
+        SystemKeyspace.updateTokens(tokens); // DON'T use setToken, that makes us part of the ring locally which is incorrect until we are done bootstrapping
         if (0 == DatabaseDescriptor.getReplaceTokens().size())
         {
             // if not an existing token then bootstrap
@@ -959,9 +959,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     public Map<List<String>, List<String>> getPendingRangeToEndpointMap(String keyspace)
     {
         // some people just want to get a visual representation of things. Allow null and set it to the first
-        // non-system table.
+        // non-system keyspace.
         if (keyspace == null)
-            keyspace = Schema.instance.getNonSystemTables().get(0);
+            keyspace = Schema.instance.getNonSystemKeyspaces().get(0);
 
         Map<List<String>, List<String>> map = new HashMap<List<String>, List<String>>();
         for (Map.Entry<Range<Token>, Collection<InetAddress>> entry : tokenMetadata.getPendingRanges(keyspace).entrySet())
@@ -975,9 +975,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     public Map<Range<Token>, List<InetAddress>> getRangeToAddressMap(String keyspace)
     {
         // some people just want to get a visual representation of things. Allow null and set it to the first
-        // non-system table.
+        // non-system keyspace.
         if (keyspace == null)
-            keyspace = Schema.instance.getNonSystemTables().get(0);
+            keyspace = Schema.instance.getNonSystemKeyspaces().get(0);
 
         List<Range<Token>> ranges = getAllRanges(tokenMetadata.sortedTokens());
         return constructRangeToEndpointMap(keyspace, ranges);
@@ -1020,7 +1020,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public List<TokenRange> describeRing(String keyspace) throws InvalidRequestException
     {
-        if (keyspace == null || Table.open(keyspace).getReplicationStrategy() instanceof LocalStrategy)
+        if (keyspace == null || Keyspace.open(keyspace).getReplicationStrategy() instanceof LocalStrategy)
             throw new InvalidRequestException("There is no ring for the keyspace: " + keyspace);
 
         List<TokenRange> ranges = new ArrayList<TokenRange>();
@@ -1095,7 +1095,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         Map<Range<Token>, List<InetAddress>> rangeToEndpointMap = new HashMap<Range<Token>, List<InetAddress>>();
         for (Range<Token> range : ranges)
         {
-            rangeToEndpointMap.put(range, Table.open(keyspace).getReplicationStrategy().getNaturalEndpoints(range.right));
+            rangeToEndpointMap.put(range, Keyspace.open(keyspace).getReplicationStrategy().getNaturalEndpoints(range.right));
         }
         return rangeToEndpointMap;
     }
@@ -1161,22 +1161,22 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     handleStateRelocating(endpoint, pieces);
                 break;
             case RELEASE_VERSION:
-                SystemTable.updatePeerInfo(endpoint, "release_version", quote(value.value));
+                SystemKeyspace.updatePeerInfo(endpoint, "release_version", quote(value.value));
                 break;
             case DC:
-                SystemTable.updatePeerInfo(endpoint, "data_center", quote(value.value));
+                SystemKeyspace.updatePeerInfo(endpoint, "data_center", quote(value.value));
                 break;
             case RACK:
-                SystemTable.updatePeerInfo(endpoint, "rack", quote(value.value));
+                SystemKeyspace.updatePeerInfo(endpoint, "rack", quote(value.value));
                 break;
             case RPC_ADDRESS:
-                SystemTable.updatePeerInfo(endpoint, "rpc_address", quote(value.value));
+                SystemKeyspace.updatePeerInfo(endpoint, "rpc_address", quote(value.value));
                 break;
             case SCHEMA:
-                SystemTable.updatePeerInfo(endpoint, "schema_version", value.value);
+                SystemKeyspace.updatePeerInfo(endpoint, "schema_version", value.value);
                 break;
             case HOST_ID:
-                SystemTable.updatePeerInfo(endpoint, "host_id", value.value);
+                SystemKeyspace.updatePeerInfo(endpoint, "host_id", value.value);
                 break;
         }
     }
@@ -1294,7 +1294,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             tokenMetadata.updateHostId(Gossiper.instance.getHostId(endpoint), endpoint);
 
         Set<Token> tokensToUpdateInMetadata = new HashSet<Token>();
-        Set<Token> tokensToUpdateInSystemTable = new HashSet<Token>();
+        Set<Token> tokensToUpdateInSystemKeyspace = new HashSet<Token>();
         Set<Token> localTokensToRemove = new HashSet<Token>();
         Set<InetAddress> endpointsToRemove = new HashSet<InetAddress>();
         Multimap<InetAddress, Token> epToTokenCopy = getTokenMetadata().getEndpointToTokenMapForReading();
@@ -1308,7 +1308,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 logger.debug("New node " + endpoint + " at token " + token);
                 tokensToUpdateInMetadata.add(token);
                 if (!isClientMode)
-                    tokensToUpdateInSystemTable.add(token);
+                    tokensToUpdateInSystemKeyspace.add(token);
             }
             else if (endpoint.equals(currentOwner))
             {
@@ -1321,7 +1321,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 // Token was relocating, this is the bookkeeping that makes it official.
                 tokensToUpdateInMetadata.add(token);
                 if (!isClientMode)
-                    tokensToUpdateInSystemTable.add(token);
+                    tokensToUpdateInSystemKeyspace.add(token);
 
                 optionalTasks.schedule(new Runnable()
                 {
@@ -1347,7 +1347,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             {
                 tokensToUpdateInMetadata.add(token);
                 if (!isClientMode)
-                    tokensToUpdateInSystemTable.add(token);
+                    tokensToUpdateInSystemKeyspace.add(token);
 
                 // currentOwner is no longer current, endpoint is.  Keep track of these moves, because when
                 // a host no longer has any tokens, we'll want to remove it.
@@ -1378,10 +1378,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         tokenMetadata.updateNormalTokens(tokensToUpdateInMetadata, endpoint);
         for (InetAddress ep : endpointsToRemove)
             removeEndpoint(ep);
-        if (!tokensToUpdateInSystemTable.isEmpty())
-            SystemTable.updateTokens(endpoint, tokensToUpdateInSystemTable);
+        if (!tokensToUpdateInSystemKeyspace.isEmpty())
+            SystemKeyspace.updateTokens(endpoint, tokensToUpdateInSystemKeyspace);
         if (!localTokensToRemove.isEmpty())
-            SystemTable.updateLocalTokens(Collections.<Token>emptyList(), localTokensToRemove);
+            SystemKeyspace.updateLocalTokens(Collections.<Token>emptyList(), localTokensToRemove);
 
         if (tokenMetadata.isMoving(endpoint)) // if endpoint was moving to a new token
         {
@@ -1570,7 +1570,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         Gossiper.instance.removeEndpoint(endpoint);
         if (!isClientMode)
-            SystemTable.removeEndpoint(endpoint);
+            SystemKeyspace.removeEndpoint(endpoint);
     }
 
     protected void addExpireTimeIfFound(InetAddress endpoint, long expireTime)
@@ -1614,12 +1614,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     private void calculatePendingRanges()
     {
-        for (String table : Schema.instance.getNonSystemTables())
-            calculatePendingRanges(Table.open(table).getReplicationStrategy(), table);
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
+            calculatePendingRanges(Keyspace.open(keyspaceName).getReplicationStrategy(), keyspaceName);
     }
 
     // public & static for testing purposes
-    public static void calculatePendingRanges(AbstractReplicationStrategy strategy, String table)
+    public static void calculatePendingRanges(AbstractReplicationStrategy strategy, String keyspaceName)
     {
         TokenMetadata tm = StorageService.instance.getTokenMetadata();
         Multimap<Range<Token>, InetAddress> pendingRanges = HashMultimap.create();
@@ -1629,8 +1629,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && tm.getMovingEndpoints().isEmpty() && tm.getRelocatingRanges().isEmpty())
         {
             if (logger.isDebugEnabled())
-                logger.debug("No bootstrapping, leaving or moving nodes, and no relocating tokens -> empty pending ranges for {}", table);
-            tm.setPendingRanges(table, pendingRanges);
+                logger.debug("No bootstrapping, leaving or moving nodes, and no relocating tokens -> empty pending ranges for {}", keyspaceName);
+            tm.setPendingRanges(keyspaceName, pendingRanges);
             return;
         }
 
@@ -1702,7 +1702,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             allLeftMetadata.removeEndpoint(endpoint);
         }
 
-        tm.setPendingRanges(table, pendingRanges);
+        tm.setPendingRanges(keyspaceName, pendingRanges);
 
         if (logger.isDebugEnabled())
             logger.debug("Pending ranges:\n" + (pendingRanges.isEmpty() ? "<empty>" : tm.printPendingRanges()));
@@ -1711,14 +1711,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     /**
      * Finds living endpoints responsible for the given ranges
      *
-     * @param table the table ranges belong to
+     * @param keyspaceName the keyspace ranges belong to
      * @param ranges the ranges to find sources for
      * @return multimap of addresses to ranges the address is responsible for
      */
-    private Multimap<InetAddress, Range<Token>> getNewSourceRanges(String table, Set<Range<Token>> ranges)
+    private Multimap<InetAddress, Range<Token>> getNewSourceRanges(String keyspaceName, Set<Range<Token>> ranges)
     {
         InetAddress myAddress = FBUtilities.getBroadcastAddress();
-        Multimap<Range<Token>, InetAddress> rangeAddresses = Table.open(table).getReplicationStrategy().getRangeAddresses(tokenMetadata.cloneOnlyTokenMap());
+        Multimap<Range<Token>, InetAddress> rangeAddresses = Keyspace.open(keyspaceName).getReplicationStrategy().getRangeAddresses(tokenMetadata.cloneOnlyTokenMap());
         Multimap<InetAddress, Range<Token>> sourceRanges = HashMultimap.create();
         IFailureDetector failureDetector = FailureDetector.instance;
 
@@ -1786,32 +1786,32 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         final InetAddress myAddress = FBUtilities.getBroadcastAddress();
 
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            Multimap<Range<Token>, InetAddress> changedRanges = getChangedRangesForLeaving(table, endpoint);
+            Multimap<Range<Token>, InetAddress> changedRanges = getChangedRangesForLeaving(keyspaceName, endpoint);
             Set<Range<Token>> myNewRanges = new HashSet<Range<Token>>();
             for (Map.Entry<Range<Token>, InetAddress> entry : changedRanges.entries())
             {
                 if (entry.getValue().equals(myAddress))
                     myNewRanges.add(entry.getKey());
             }
-            Multimap<InetAddress, Range<Token>> sourceRanges = getNewSourceRanges(table, myNewRanges);
+            Multimap<InetAddress, Range<Token>> sourceRanges = getNewSourceRanges(keyspaceName, myNewRanges);
             for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : sourceRanges.asMap().entrySet())
             {
-                rangesToFetch.put(table, entry);
+                rangesToFetch.put(keyspaceName, entry);
             }
         }
 
         StreamPlan stream = new StreamPlan("Restore replica count");
-        for (final String table : rangesToFetch.keySet())
+        for (final String keyspaceName : rangesToFetch.keySet())
         {
-            for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : rangesToFetch.get(table))
+            for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : rangesToFetch.get(keyspaceName))
             {
                 final InetAddress source = entry.getKey();
                 Collection<Range<Token>> ranges = entry.getValue();
                 if (logger.isDebugEnabled())
                     logger.debug("Requesting from " + source + " ranges " + StringUtils.join(ranges, ", "));
-                stream.requestRanges(source, table, ranges);
+                stream.requestRanges(source, keyspaceName, ranges);
             }
         }
         StreamResultFuture future = stream.execute();
@@ -1831,11 +1831,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         });
     }
 
-    // needs to be modified to accept either a table or ARS.
-    private Multimap<Range<Token>, InetAddress> getChangedRangesForLeaving(String table, InetAddress endpoint)
+    // needs to be modified to accept either a keyspace or ARS.
+    private Multimap<Range<Token>, InetAddress> getChangedRangesForLeaving(String keyspaceName, InetAddress endpoint)
     {
         // First get all ranges the leaving endpoint is responsible for
-        Collection<Range<Token>> ranges = getRangesForEndpoint(table, endpoint);
+        Collection<Range<Token>> ranges = getRangesForEndpoint(keyspaceName, endpoint);
 
         if (logger.isDebugEnabled())
             logger.debug("Node " + endpoint + " ranges [" + StringUtils.join(ranges, ", ") + "]");
@@ -1844,7 +1844,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         // Find (for each range) all nodes that store replicas for these ranges as well
         for (Range<Token> range : ranges)
-            currentReplicaEndpoints.put(range, Table.open(table).getReplicationStrategy().calculateNaturalEndpoints(range.right, tokenMetadata.cloneOnlyTokenMap()));
+            currentReplicaEndpoints.put(range, Keyspace.open(keyspaceName).getReplicationStrategy().calculateNaturalEndpoints(range.right, tokenMetadata.cloneOnlyTokenMap()));
 
         TokenMetadata temp = tokenMetadata.cloneAfterAllLeft();
 
@@ -1862,7 +1862,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         // range.
         for (Range<Token> range : ranges)
         {
-            Collection<InetAddress> newReplicaEndpoints = Table.open(table).getReplicationStrategy().calculateNaturalEndpoints(range.right, temp);
+            Collection<InetAddress> newReplicaEndpoints = Keyspace.open(keyspaceName).getReplicationStrategy().calculateNaturalEndpoints(range.right, temp);
             newReplicaEndpoints.removeAll(currentReplicaEndpoints.get(range));
             if (logger.isDebugEnabled())
                 if (newReplicaEndpoints.isEmpty())
@@ -1916,10 +1916,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     public double getLoad()
     {
         double bytes = 0;
-        for (String tableName : Schema.instance.getTables())
+        for (String keyspaceName : Schema.instance.getKeyspaces())
         {
-            Table table = Table.open(tableName);
-            for (ColumnFamilyStore cfs : table.getColumnFamilyStores())
+            Keyspace keyspace = Keyspace.open(keyspaceName);
+            for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
                 bytes += cfs.getLiveDiskSpaceUsed();
         }
         return bytes;
@@ -1949,7 +1949,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public Collection<Token> getLocalTokens()
     {
-        Collection<Token> tokens = SystemTable.getSavedTokens();
+        Collection<Token> tokens = SystemKeyspace.getSavedTokens();
         assert tokens != null && !tokens.isEmpty(); // should not be called before initServer sets this
         return tokens;
     }
@@ -2049,83 +2049,83 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return Gossiper.instance.getCurrentGenerationNumber(FBUtilities.getBroadcastAddress());
     }
 
-    public void forceTableCleanup(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (tableName.equals(Table.SYSTEM_KS))
-            throw new RuntimeException("Cleanup of the system table is neither necessary nor wise");
+        if (keyspaceName.equals(Keyspace.SYSTEM_KS))
+            throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
 
         CounterId.OneShotRenewer counterIdRenewer = new CounterId.OneShotRenewer();
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, tableName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
             cfStore.forceCleanup(counterIdRenewer);
         }
     }
 
-    public void scrub(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, tableName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
             cfStore.scrub();
     }
 
-    public void upgradeSSTables(String tableName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, tableName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, true, keyspaceName, columnFamilies))
             cfStore.sstablesRewrite(excludeCurrentVersion);
     }
 
-    public void forceTableCompaction(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceCompaction(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, tableName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
             cfStore.forceMajorCompaction();
         }
     }
 
     /**
-     * Takes the snapshot for the given tables. A snapshot name must be specified.
+     * Takes the snapshot for the given keyspaces. A snapshot name must be specified.
      *
      * @param tag the tag given to the snapshot; may not be null or empty
-     * @param tableNames the name of the tables to snapshot; empty means "all."
+     * @param keyspaceNames the names of the keyspaces to snapshot; empty means "all."
      */
-    public void takeSnapshot(String tag, String... tableNames) throws IOException
+    public void takeSnapshot(String tag, String... keyspaceNames) throws IOException
     {
         if (tag == null || tag.equals(""))
             throw new IOException("You must supply a snapshot name.");
 
-        Iterable<Table> tables;
-        if (tableNames.length == 0)
+        Iterable<Keyspace> keyspaces;
+        if (keyspaceNames.length == 0)
         {
-            tables = Table.all();
+            keyspaces = Keyspace.all();
         }
         else
         {
-            ArrayList<Table> t = new ArrayList<Table>(tableNames.length);
-            for (String table : tableNames)
-                t.add(getValidTable(table));
-            tables = t;
+            ArrayList<Keyspace> t = new ArrayList<Keyspace>(keyspaceNames.length);
+            for (String keyspaceName : keyspaceNames)
+                t.add(getValidKeyspace(keyspaceName));
+            keyspaces = t;
         }
 
         // Do a check to see if this snapshot exists before we actually snapshot
-        for (Table table : tables)
-            if (table.snapshotExists(tag))
+        for (Keyspace keyspace : keyspaces)
+            if (keyspace.snapshotExists(tag))
                 throw new IOException("Snapshot " + tag + " already exists.");
 
 
-        for (Table table : tables)
-            table.snapshot(tag, null);
+        for (Keyspace keyspace : keyspaces)
+            keyspace.snapshot(tag, null);
     }
 
     /**
      * Takes the snapshot of a specific column family. A snapshot name must be specified.
      *
-     * @param tableName the keyspace which holds the specified column family
+     * @param keyspaceName the keyspace which holds the specified column family
      * @param columnFamilyName the column family to snapshot
      * @param tag the tag given to the snapshot; may not be null or empty
      */
-    public void takeColumnFamilySnapshot(String tableName, String columnFamilyName, String tag) throws IOException
+    public void takeColumnFamilySnapshot(String keyspaceName, String columnFamilyName, String tag) throws IOException
     {
-        if (tableName == null)
-            throw new IOException("You must supply a table name");
+        if (keyspaceName == null)
+            throw new IOException("You must supply a keyspace name");
 
         if (columnFamilyName == null)
             throw new IOException("You must supply a column family name");
@@ -2135,46 +2135,46 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (tag == null || tag.equals(""))
             throw new IOException("You must supply a snapshot name.");
 
-        Table table = getValidTable(tableName);
-        if (table.snapshotExists(tag))
+        Keyspace keyspace = getValidKeyspace(keyspaceName);
+        if (keyspace.snapshotExists(tag))
             throw new IOException("Snapshot " + tag + " already exists.");
 
-        table.snapshot(tag, columnFamilyName);
+        keyspace.snapshot(tag, columnFamilyName);
     }
 
-    private Table getValidTable(String tableName) throws IOException
+    private Keyspace getValidKeyspace(String keyspaceName) throws IOException
     {
-        if (!Schema.instance.getTables().contains(tableName))
+        if (!Schema.instance.getKeyspaces().contains(keyspaceName))
         {
-            throw new IOException("Table " + tableName + " does not exist");
+            throw new IOException("Keyspace " + keyspaceName + " does not exist");
         }
-        return Table.open(tableName);
+        return Keyspace.open(keyspaceName);
     }
 
     /**
-     * Remove the snapshot with the given name from the given tables.
+     * Remove the snapshot with the given name from the given keyspaces.
      * If no tag is specified we will remove all snapshots.
      */
-    public void clearSnapshot(String tag, String... tableNames) throws IOException
+    public void clearSnapshot(String tag, String... keyspaceNames) throws IOException
     {
         if(tag == null)
             tag = "";
 
-        Iterable<Table> tables;
-        if (tableNames.length == 0)
+        Iterable<Keyspace> keyspaces;
+        if (keyspaceNames.length == 0)
         {
-            tables = Table.all();
+            keyspaces = Keyspace.all();
         }
         else
         {
-            ArrayList<Table> tempTables = new ArrayList<Table>(tableNames.length);
-            for(String table : tableNames)
-                tempTables.add(getValidTable(table));
-            tables = tempTables;
+            ArrayList<Keyspace> tempKeyspaces = new ArrayList<Keyspace>(keyspaceNames.length);
+            for(String keyspaceName : keyspaceNames)
+                tempKeyspaces.add(getValidKeyspace(keyspaceName));
+            keyspaces = tempKeyspaces;
         }
 
-        for (Table table : tables)
-            table.clearSnapshot(tag);
+        for (Keyspace keyspace : keyspaces)
+            keyspace.clearSnapshot(tag);
 
         if (logger.isDebugEnabled())
             logger.debug("Cleared out snapshot directories");
@@ -2183,16 +2183,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     /**
      * @param allowIndexes Allow index CF names to be passed in
      * @param autoAddIndexes Automatically add secondary indexes if a CF has them
-     * @param tableName keyspace
+     * @param keyspaceName keyspace
      * @param cfNames CFs
      */
-    public Iterable<ColumnFamilyStore> getValidColumnFamilies(boolean allowIndexes, boolean autoAddIndexes, String tableName, String... cfNames) throws IOException
+    public Iterable<ColumnFamilyStore> getValidColumnFamilies(boolean allowIndexes, boolean autoAddIndexes, String keyspaceName, String... cfNames) throws IOException
     {
-        Table table = getValidTable(tableName);
+        Keyspace keyspace = getValidKeyspace(keyspaceName);
 
         if (cfNames.length == 0)
             // all stores are interesting
-            return table.getColumnFamilyStores();
+            return keyspace.getColumnFamilyStores();
 
         // filter out interesting stores
         Set<ColumnFamilyStore> valid = new HashSet<ColumnFamilyStore>();
@@ -2214,7 +2214,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 idxName = parts[1];
             }
 
-            ColumnFamilyStore cfStore = table.getColumnFamilyStore(baseCfName);
+            ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(baseCfName);
             if (cfStore == null)
             {
                 // this means there was a cf passed in that is not recognized in the keyspace. report it and continue.
@@ -2246,16 +2246,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     }
 
     /**
-     * Flush all memtables for a table and column families.
-     * @param tableName
+     * Flush all memtables for a keyspace and column families.
+     * @param keyspaceName
      * @param columnFamilies
      * @throws IOException
      */
-    public void forceTableFlush(final String tableName, final String... columnFamilies) throws IOException
+    public void forceKeyspaceFlush(final String keyspaceName, final String... columnFamilies) throws IOException
     {
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, tableName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
         {
-            logger.debug("Forcing flush on keyspace " + tableName + ", CF " + cfStore.name);
+            logger.debug("Forcing flush on keyspace " + keyspaceName + ", CF " + cfStore.name);
             cfStore.forceBlockingFlush();
         }
     }
@@ -2281,7 +2281,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int forceRepairAsync(final String keyspace, final boolean isSequential, final boolean isLocal, final Collection<Range<Token>> ranges, final String... columnFamilies)
     {
-        if (Table.SYSTEM_KS.equals(keyspace) || Tracing.TRACE_KS.equals(keyspace) || ranges.isEmpty())
+        if (Keyspace.SYSTEM_KS.equals(keyspace) || Tracing.TRACE_KS.equals(keyspace) || ranges.isEmpty())
             return 0;
 
         final int cmd = nextRepairCommand.incrementAndGet();
@@ -2292,48 +2292,48 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return cmd;
     }
 
-    public int forceRepairRangeAsync(String beginToken, String endToken, final String tableName, boolean isSequential, boolean isLocal, final String... columnFamilies)
+    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies)
     {
         Token parsedBeginToken = getPartitioner().getTokenFactory().fromString(beginToken);
         Token parsedEndToken = getPartitioner().getTokenFactory().fromString(endToken);
 
         logger.info("starting user-requested repair of range ({}, {}] for keyspace {} and column families {}",
-                    parsedBeginToken, parsedEndToken, tableName, columnFamilies);
-        return forceRepairAsync(tableName, isSequential, isLocal, Collections.singleton(new Range<Token>(parsedBeginToken, parsedEndToken)), columnFamilies);
+                    parsedBeginToken, parsedEndToken, keyspaceName, columnFamilies);
+        return forceRepairAsync(keyspaceName, isSequential, isLocal, Collections.singleton(new Range<Token>(parsedBeginToken, parsedEndToken)), columnFamilies);
     }
 
 
     /**
-     * Trigger proactive repair for a table and column families.
-     * @param tableName
+     * Trigger proactive repair for a keyspace and column families.
+     * @param keyspaceName
      * @param columnFamilies
      * @throws IOException
      */
-    public void forceTableRepair(final String tableName, boolean isSequential, boolean  isLocal, final String... columnFamilies) throws IOException
+    public void forceKeyspaceRepair(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
     {
-        forceTableRepairRange(tableName, getLocalRanges(tableName), isSequential, isLocal, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, getLocalRanges(keyspaceName), isSequential, isLocal, columnFamilies);
     }
 
-    public void forceTableRepairPrimaryRange(final String tableName, boolean isSequential, boolean  isLocal, final String... columnFamilies) throws IOException
+    public void forceKeyspaceRepairPrimaryRange(final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
     {
-        forceTableRepairRange(tableName, getLocalPrimaryRanges(tableName), isSequential, isLocal, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, getLocalPrimaryRanges(keyspaceName), isSequential, isLocal, columnFamilies);
     }
 
-    public void forceTableRepairRange(String beginToken, String endToken, final String tableName, boolean isSequential, boolean  isLocal, final String... columnFamilies) throws IOException
+    public void forceKeyspaceRepairRange(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
     {
         Token parsedBeginToken = getPartitioner().getTokenFactory().fromString(beginToken);
         Token parsedEndToken = getPartitioner().getTokenFactory().fromString(endToken);
 
         logger.info("starting user-requested repair of range ({}, {}] for keyspace {} and column families {}",
-                    parsedBeginToken, parsedEndToken, tableName, columnFamilies);
-        forceTableRepairRange(tableName, Collections.singleton(new Range<Token>(parsedBeginToken, parsedEndToken)), isSequential, isLocal, columnFamilies);
+                    parsedBeginToken, parsedEndToken, keyspaceName, columnFamilies);
+        forceKeyspaceRepairRange(keyspaceName, Collections.singleton(new Range<Token>(parsedBeginToken, parsedEndToken)), isSequential, isLocal, columnFamilies);
     }
 
-    public void forceTableRepairRange(final String tableName, final Collection<Range<Token>> ranges, boolean isSequential, boolean  isLocal, final String... columnFamilies) throws IOException
+    public void forceKeyspaceRepairRange(final String keyspaceName, final Collection<Range<Token>> ranges, boolean isSequential, boolean isLocal, final String... columnFamilies) throws IOException
     {
-        if (Schema.systemKeyspaceNames.contains(tableName))
+        if (Schema.systemKeyspaceNames.contains(keyspaceName))
             return;
-        createRepairTask(nextRepairCommand.incrementAndGet(), tableName, ranges, isSequential, isLocal, columnFamilies).run();
+        createRepairTask(nextRepairCommand.incrementAndGet(), keyspaceName, ranges, isSequential, isLocal, columnFamilies).run();
     }
 
     private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final Collection<Range<Token>> ranges, final boolean isSequential, final boolean isLocal, final String... columnFamilies)
@@ -2352,7 +2352,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     RepairFuture future;
                     try
                     {
-                        future = forceTableRepair(range, keyspace, isSequential, isLocal, columnFamilies);
+                        future = forceKeyspaceRepair(range, keyspace, isSequential, isLocal, columnFamilies);
                     }
                     catch (IllegalArgumentException e)
                     {
@@ -2402,21 +2402,21 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         }, null);
     }
 
-    public RepairFuture forceTableRepair(final Range<Token> range, final String tableName, boolean isSequential, boolean  isLocal, final String... columnFamilies) throws IOException
+    public RepairFuture forceKeyspaceRepair(final Range<Token> range, final String keyspaceName, boolean isSequential, boolean  isLocal, final String... columnFamilies) throws IOException
     {
         ArrayList<String> names = new ArrayList<String>();
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, tableName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
             names.add(cfStore.name);
         }
 
         if (names.isEmpty())
         {
-            logger.info("No column family to repair for keyspace " + tableName);
+            logger.info("No column family to repair for keyspace " + keyspaceName);
             return null;
         }
 
-        return ActiveRepairService.instance.submitRepairSession(range, tableName, isSequential, isLocal, names.toArray(new String[names.size()]));
+        return ActiveRepairService.instance.submitRepairSession(range, keyspaceName, isSequential, isLocal, names.toArray(new String[names.size()]));
     }
 
     public void forceTerminateAllRepairSessions() {
@@ -2437,7 +2437,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public Collection<Range<Token>> getPrimaryRangesForEndpoint(String keyspace, InetAddress ep)
     {
-        AbstractReplicationStrategy strategy = Table.open(keyspace).getReplicationStrategy();
+        AbstractReplicationStrategy strategy = Keyspace.open(keyspace).getReplicationStrategy();
         Collection<Range<Token>> primaryRanges = new HashSet<Range<Token>>();
         TokenMetadata metadata = tokenMetadata.cloneOnlyTokenMap();
         for (Token token : metadata.sortedTokens())
@@ -2467,13 +2467,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     }
 
     /**
-     * Get all ranges an endpoint is responsible for (by table)
+     * Get all ranges an endpoint is responsible for (by keyspace)
      * @param ep endpoint we are interested in.
      * @return ranges for the specified endpoint.
      */
-    Collection<Range<Token>> getRangesForEndpoint(String table, InetAddress ep)
+    Collection<Range<Token>> getRangesForEndpoint(String keyspaceName, InetAddress ep)
     {
-        return Table.open(table).getReplicationStrategy().getAddressRanges().get(ep);
+        return Keyspace.open(keyspaceName).getReplicationStrategy().getAddressRanges().get(ep);
     }
 
     /**
@@ -2506,51 +2506,51 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * This method returns the N endpoints that are responsible for storing the
      * specified key i.e for replication.
      *
-     * @param table keyspace name also known as table
+     * @param keyspaceName keyspace name also known as keyspace
      * @param cf Column family name
      * @param key key for which we need to find the endpoint
      * @return the endpoint responsible for this key
      */
-    public List<InetAddress> getNaturalEndpoints(String table, String cf, String key)
+    public List<InetAddress> getNaturalEndpoints(String keyspaceName, String cf, String key)
     {
-        CFMetaData cfMetaData = Schema.instance.getKSMetaData(table).cfMetaData().get(cf);
-        return getNaturalEndpoints(table, getPartitioner().getToken(cfMetaData.getKeyValidator().fromString(key)));
+        CFMetaData cfMetaData = Schema.instance.getKSMetaData(keyspaceName).cfMetaData().get(cf);
+        return getNaturalEndpoints(keyspaceName, getPartitioner().getToken(cfMetaData.getKeyValidator().fromString(key)));
     }
 
-    public List<InetAddress> getNaturalEndpoints(String table, ByteBuffer key)
+    public List<InetAddress> getNaturalEndpoints(String keyspaceName, ByteBuffer key)
     {
-        return getNaturalEndpoints(table, getPartitioner().getToken(key));
+        return getNaturalEndpoints(keyspaceName, getPartitioner().getToken(key));
     }
 
     /**
      * This method returns the N endpoints that are responsible for storing the
      * specified key i.e for replication.
      *
-     * @param table keyspace name also known as table
+     * @param keyspaceName keyspace name also known as keyspace
      * @param pos position for which we need to find the endpoint
      * @return the endpoint responsible for this token
      */
-    public List<InetAddress> getNaturalEndpoints(String table, RingPosition pos)
+    public List<InetAddress> getNaturalEndpoints(String keyspaceName, RingPosition pos)
     {
-        return Table.open(table).getReplicationStrategy().getNaturalEndpoints(pos);
+        return Keyspace.open(keyspaceName).getReplicationStrategy().getNaturalEndpoints(pos);
     }
 
     /**
      * This method attempts to return N endpoints that are responsible for storing the
      * specified key i.e for replication.
      *
-     * @param table keyspace name also known as table
+     * @param keyspace keyspace name also known as keyspace
      * @param key key for which we need to find the endpoint
      * @return the endpoint responsible for this key
      */
-    public List<InetAddress> getLiveNaturalEndpoints(Table table, ByteBuffer key)
+    public List<InetAddress> getLiveNaturalEndpoints(Keyspace keyspace, ByteBuffer key)
     {
-        return getLiveNaturalEndpoints(table, getPartitioner().decorateKey(key));
+        return getLiveNaturalEndpoints(keyspace, getPartitioner().decorateKey(key));
     }
 
-    public List<InetAddress> getLiveNaturalEndpoints(Table table, RingPosition pos)
+    public List<InetAddress> getLiveNaturalEndpoints(Keyspace keyspace, RingPosition pos)
     {
-        List<InetAddress> endpoints = table.getReplicationStrategy().getNaturalEndpoints(pos);
+        List<InetAddress> endpoints = keyspace.getReplicationStrategy().getNaturalEndpoints(pos);
         List<InetAddress> liveEps = new ArrayList<InetAddress>(endpoints.size());
 
         for (InetAddress endpoint : endpoints)
@@ -2573,9 +2573,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * @return list of Token ranges (_not_ keys!) together with estimated key count,
      *      breaking up the data this node is responsible for into pieces of roughly keysPerSplit
      */
-    public List<Pair<Range<Token>, Long>> getSplits(String table, String cfName, Range<Token> range, int keysPerSplit, CFMetaData metadata)
+    public List<Pair<Range<Token>, Long>> getSplits(String keyspaceName, String cfName, Range<Token> range, int keysPerSplit, CFMetaData metadata)
     {
-        Table t = Table.open(table);
+        Keyspace t = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = t.getColumnFamilyStore(cfName);
         List<DecoratedKey> keys = keySamples(Collections.singleton(cfs), range);
 
@@ -2643,9 +2643,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new UnsupportedOperationException("local node is not a member of the token ring yet");
         if (tokenMetadata.cloneAfterAllLeft().sortedTokens().size() < 2)
             throw new UnsupportedOperationException("no other normal nodes in the ring; decommission would be pointless");
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            if (tokenMetadata.getPendingRanges(table, FBUtilities.getBroadcastAddress()).size() > 0)
+            if (tokenMetadata.getPendingRanges(keyspaceName, FBUtilities.getBroadcastAddress()).size() > 0)
                 throw new UnsupportedOperationException("data is currently moving to this node; unable to leave the ring");
         }
 
@@ -2672,7 +2672,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     private void leaveRing()
     {
-        SystemTable.setBootstrapState(SystemTable.BootstrapState.NEEDS_BOOTSTRAP);
+        SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.NEEDS_BOOTSTRAP);
         tokenMetadata.removeEndpoint(FBUtilities.getBroadcastAddress());
         calculatePendingRanges();
 
@@ -2686,14 +2686,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         Map<String, Multimap<Range<Token>, InetAddress>> rangesToStream = new HashMap<String, Multimap<Range<Token>, InetAddress>>();
 
-        for (final String table : Schema.instance.getNonSystemTables())
+        for (final String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            Multimap<Range<Token>, InetAddress> rangesMM = getChangedRangesForLeaving(table, FBUtilities.getBroadcastAddress());
+            Multimap<Range<Token>, InetAddress> rangesMM = getChangedRangesForLeaving(keyspaceName, FBUtilities.getBroadcastAddress());
 
             if (logger.isDebugEnabled())
                 logger.debug("Ranges needing transfer are [" + StringUtils.join(rangesMM.keySet(), ",") + "]");
 
-            rangesToStream.put(table, rangesMM);
+            rangesToStream.put(keyspaceName, rangesMM);
         }
 
         setMode(Mode.LEAVING, "streaming data to other nodes", true);
@@ -2748,9 +2748,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             List<Range<Token>> ranges = Collections.singletonList(new Range<Token>(token, token));
 
             return new StreamPlan("Hints").transferRanges(hintsDestinationHost,
-                                                                      Table.SYSTEM_KS,
+                                                                      Keyspace.SYSTEM_KS,
                                                                       ranges,
-                                                                      SystemTable.HINTS_CF)
+                                                                      SystemKeyspace.HINTS_CF)
                                                       .execute();
         }
     }
@@ -2793,12 +2793,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new UnsupportedOperationException("This node has more than one token and cannot be moved thusly.");
         }
 
-        List<String> tablesToProcess = Schema.instance.getNonSystemTables();
+        List<String> keyspacesToProcess = Schema.instance.getNonSystemKeyspaces();
 
         // checking if data is moving to this node
-        for (String table : tablesToProcess)
+        for (String keyspaceName : keyspacesToProcess)
         {
-            if (tokenMetadata.getPendingRanges(table, localAddress).size() > 0)
+            if (tokenMetadata.getPendingRanges(keyspaceName, localAddress).size() > 0)
                 throw new UnsupportedOperationException("data is currently moving to this node; unable to leave the ring");
         }
 
@@ -2808,7 +2808,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         setMode(Mode.MOVING, String.format("Sleeping %s ms before start streaming/fetching ranges", RING_DELAY), true);
         Uninterruptibles.sleepUninterruptibly(RING_DELAY, TimeUnit.MILLISECONDS);
 
-        RangeRelocator relocator = new RangeRelocator(Collections.singleton(newToken), tablesToProcess);
+        RangeRelocator relocator = new RangeRelocator(Collections.singleton(newToken), keyspacesToProcess);
 
         if (relocator.streamsNeeded())
         {
@@ -2837,12 +2837,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         private StreamPlan streamPlan = new StreamPlan("Bootstrap");
 
-        private RangeRelocator(Collection<Token> tokens, List<String> tables)
+        private RangeRelocator(Collection<Token> tokens, List<String> keyspaceNames)
         {
-            calculateToFromStreams(tokens, tables);
+            calculateToFromStreams(tokens, keyspaceNames);
         }
 
-        private void calculateToFromStreams(Collection<Token> newTokens, List<String> tables)
+        private void calculateToFromStreams(Collection<Token> newTokens, List<String> keyspaceNames)
         {
             InetAddress localAddress = FBUtilities.getBroadcastAddress();
             IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
@@ -2850,12 +2850,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             // clone to avoid concurrent modification in calculateNaturalEndpoints
             TokenMetadata tokenMetaClone = tokenMetadata.cloneOnlyTokenMap();
 
-            for (String keyspace : tables)
+            for (String keyspace : keyspaceNames)
             {
                 for (Token newToken : newTokens)
                 {
                     // replication strategy of the current keyspace (aka table)
-                    AbstractReplicationStrategy strategy = Table.open(keyspace).getReplicationStrategy();
+                    AbstractReplicationStrategy strategy = Keyspace.open(keyspace).getReplicationStrategy();
 
                     // getting collection of the currently used ranges by this keyspace
                     Collection<Range<Token>> currentRanges = getRangesForEndpoint(keyspace, localAddress);
@@ -2867,14 +2867,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     Multimap<Range<Token>, InetAddress> rangeAddresses = strategy.getRangeAddresses(tokenMetaClone);
 
                     // calculated parts of the ranges to request/stream from/to nodes in the ring
-                    Pair<Set<Range<Token>>, Set<Range<Token>>> rangesPerTable = calculateStreamAndFetchRanges(currentRanges, updatedRanges);
+                    Pair<Set<Range<Token>>, Set<Range<Token>>> rangesPerKeyspace = calculateStreamAndFetchRanges(currentRanges, updatedRanges);
 
                     /**
                      * In this loop we are going through all ranges "to fetch" and determining
                      * nodes in the ring responsible for data we are interested in
                      */
                     Multimap<Range<Token>, InetAddress> rangesToFetchWithPreferredEndpoints = ArrayListMultimap.create();
-                    for (Range<Token> toFetch : rangesPerTable.right)
+                    for (Range<Token> toFetch : rangesPerKeyspace.right)
                     {
                         for (Range<Token> range : rangeAddresses.keySet())
                         {
@@ -2890,7 +2890,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                     // calculating endpoints to stream current ranges to if needed
                     // in some situations node will handle current ranges as part of the new ranges
                     Multimap<InetAddress, Range<Token>> endpointRanges = HashMultimap.create();
-                    for (Range<Token> toStream : rangesPerTable.left)
+                    for (Range<Token> toStream : rangesPerKeyspace.left)
                     {
                         Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetaClone));
                         Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetaCloneAllSettled));
@@ -2909,7 +2909,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                         streamPlan.requestRanges(address, keyspace, workMap.get(address));
 
                     if (logger.isDebugEnabled())
-                        logger.debug("Table {}: work map {}.", keyspace, workMap);
+                        logger.debug("Keyspace {}: work map {}.", keyspace, workMap);
                 }
             }
         }
@@ -2967,12 +2967,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.relocating(tokens));
         setMode(Mode.RELOCATING, String.format("relocating %s to %s", tokens, localAddress.getHostAddress()), true);
 
-        List<String> tables = Schema.instance.getNonSystemTables();
+        List<String> keyspaceNames = Schema.instance.getNonSystemKeyspaces();
 
         setMode(Mode.RELOCATING, String.format("Sleeping %s ms before start streaming/fetching ranges", RING_DELAY), true);
         Uninterruptibles.sleepUninterruptibly(RING_DELAY, TimeUnit.MILLISECONDS);
 
-        RangeRelocator relocator = new RangeRelocator(tokens, tables);
+        RangeRelocator relocator = new RangeRelocator(tokens, keyspaceNames);
 
         if (relocator.streamsNeeded())
         {
@@ -2991,7 +2991,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             setMode(Mode.RELOCATING, "no new ranges to stream/fetch", true);
         }
 
-        Collection<Token> currentTokens = SystemTable.updateLocalTokens(tokens, Collections.<Token>emptyList());
+        Collection<Token> currentTokens = SystemKeyspace.updateLocalTokens(tokens, Collections.<Token>emptyList());
         tokenMetadata.updateNormalTokens(currentTokens, FBUtilities.getBroadcastAddress());
         Gossiper.instance.addLocalApplicationState(ApplicationState.TOKENS, valueFactory.tokens(currentTokens));
         Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.normal(currentTokens));
@@ -3071,15 +3071,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             throw new UnsupportedOperationException("This node is already processing a removal. Wait for it to complete, or use 'removetoken force' if this has failed.");
 
         // Find the endpoints that are going to become responsible for data
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
             // if the replication factor is 1 the data is lost so we shouldn't wait for confirmation
-            if (Table.open(table).getReplicationStrategy().getReplicationFactor() == 1)
+            if (Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor() == 1)
                 continue;
 
             // get all ranges that change ownership (that is, a node needs
             // to take responsibility for new range)
-            Multimap<Range<Token>, InetAddress> changedRanges = getChangedRangesForLeaving(table, endpoint);
+            Multimap<Range<Token>, InetAddress> changedRanges = getChangedRangesForLeaving(keyspaceName, endpoint);
             IFailureDetector failureDetector = FailureDetector.instance;
             for (InetAddress ep : changedRanges.values())
             {
@@ -3199,14 +3199,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         setMode(Mode.DRAINING, "flushing column families", false);
         // count CFs first, since forceFlush could block for the flushWriter to get a queue slot empty
         totalCFs = 0;
-        for (Table table : Table.nonSystem())
-            totalCFs += table.getColumnFamilyStores().size();
+        for (Keyspace keyspace : Keyspace.nonSystem())
+            totalCFs += keyspace.getColumnFamilyStores().size();
         remainingCFs = totalCFs;
         // flush
         List<Future<?>> flushes = new ArrayList<Future<?>>();
-        for (Table table : Table.nonSystem())
+        for (Keyspace keyspace : Keyspace.nonSystem())
         {
-            for (ColumnFamilyStore cfs : table.getColumnFamilyStores())
+            for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
                 flushes.add(cfs.forceFlush());
         }
         // wait for the flushes.
@@ -3220,9 +3220,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         // flush the system ones after all the rest are done, just in case flushing modifies any system state
         // like CASSANDRA-5151. don't bother with progress tracking since system data is tiny.
         flushes.clear();
-        for (Table table : Table.system())
+        for (Keyspace keyspace : Keyspace.system())
         {
-            for (ColumnFamilyStore cfs : table.getColumnFamilyStores())
+            for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
                 flushes.add(cfs.forceFlush());
         }
         FBUtilities.waitOnFutures(flushes);
@@ -3297,15 +3297,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     public LinkedHashMap<InetAddress, Float> effectiveOwnership(String keyspace) throws IllegalStateException
     {
-        if (Schema.instance.getNonSystemTables().size() <= 0)
+        if (Schema.instance.getNonSystemKeyspaces().size() <= 0)
             throw new IllegalStateException("Couldn't find any Non System Keyspaces to infer replication topology");
-        if (keyspace == null && !hasSameReplication(Schema.instance.getNonSystemTables()))
+        if (keyspace == null && !hasSameReplication(Schema.instance.getNonSystemKeyspaces()))
             throw new IllegalStateException("Non System keyspaces doesnt have the same topology");
 
         TokenMetadata metadata = tokenMetadata.cloneOnlyTokenMap();
 
         if (keyspace == null)
-            keyspace = Schema.instance.getNonSystemTables().get(0);
+            keyspace = Schema.instance.getNonSystemKeyspaces().get(0);
 
         Collection<Collection<InetAddress>> endpointsGroupedByDc = new ArrayList<Collection<InetAddress>>();
         // mapping of dc's to nodes, use sorted map so that we get dcs sorted
@@ -3355,8 +3355,8 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public List<String> getKeyspaces()
     {
-        List<String> tableslist = new ArrayList<String>(Schema.instance.getTables());
-        return Collections.unmodifiableList(tableslist);
+        List<String> keyspaceNamesList = new ArrayList<String>(Schema.instance.getKeyspaces());
+        return Collections.unmodifiableList(keyspaceNamesList);
     }
 
     public void updateSnitch(String epSnitchClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException
@@ -3383,9 +3383,9 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         // point snitch references to the new instance
         DatabaseDescriptor.setEndpointSnitch(newSnitch);
-        for (String ks : Schema.instance.getTables())
+        for (String ks : Schema.instance.getKeyspaces())
         {
-            Table.open(ks).getReplicationStrategy().snitch = newSnitch;
+            Keyspace.open(ks).getReplicationStrategy().snitch = newSnitch;
         }
 
         if (oldSnitch instanceof DynamicEndpointSnitch)
@@ -3395,14 +3395,14 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     /**
      * Seed data to the endpoints that will be responsible for it at the future
      *
-     * @param rangesToStreamByTable tables and data ranges with endpoints included for each
+     * @param rangesToStreamByKeyspace keyspaces and data ranges with endpoints included for each
      * @return async Future for whether stream was success
      */
-    private Future<StreamState> streamRanges(final Map<String, Multimap<Range<Token>, InetAddress>> rangesToStreamByTable)
+    private Future<StreamState> streamRanges(final Map<String, Multimap<Range<Token>, InetAddress>> rangesToStreamByKeyspace)
     {
         // First, we build a list of ranges to stream to each host, per table
-        final Map<String, Map<InetAddress, List<Range<Token>>>> sessionsToStreamByTable = new HashMap<String, Map<InetAddress, List<Range<Token>>>>();
-        for (Map.Entry<String, Multimap<Range<Token>, InetAddress>> entry : rangesToStreamByTable.entrySet())
+        final Map<String, Map<InetAddress, List<Range<Token>>>> sessionsToStreamByKeyspace = new HashMap<String, Map<InetAddress, List<Range<Token>>>>();
+        for (Map.Entry<String, Multimap<Range<Token>, InetAddress>> entry : rangesToStreamByKeyspace.entrySet())
         {
             String keyspace = entry.getKey();
             Multimap<Range<Token>, InetAddress> rangesWithEndpoints = entry.getValue();
@@ -3425,13 +3425,13 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 curRanges.add(range);
             }
 
-            sessionsToStreamByTable.put(keyspace, rangesPerEndpoint);
+            sessionsToStreamByKeyspace.put(keyspace, rangesPerEndpoint);
         }
 
         StreamPlan streamPlan = new StreamPlan("Unbootstrap");
-        for (Map.Entry<String, Map<InetAddress, List<Range<Token>>>> entry : sessionsToStreamByTable.entrySet())
+        for (Map.Entry<String, Map<InetAddress, List<Range<Token>>>> entry : sessionsToStreamByKeyspace.entrySet())
         {
-            final String table = entry.getKey();
+            final String keyspaceName = entry.getKey();
             final Map<InetAddress, List<Range<Token>>> rangesPerEndpoint = entry.getValue();
 
             for (final Map.Entry<InetAddress, List<Range<Token>>> rangesEntry : rangesPerEndpoint.entrySet())
@@ -3440,7 +3440,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 final InetAddress newEndpoint = rangesEntry.getKey();
 
                 // TODO each call to transferRanges re-flushes, this is potentially a lot of waste
-                streamPlan.transferRanges(newEndpoint, table, ranges);
+                streamPlan.transferRanges(newEndpoint, keyspaceName, ranges);
             }
         }
         return streamPlan.execute();
@@ -3448,7 +3448,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     /**
      * Calculate pair of ranges to stream/fetch for given two range collections
-     * (current ranges for table and ranges after move to new token)
+     * (current ranges for keyspace and ranges after move to new token)
      *
      * @param current collection of the ranges by current token
      * @param updated collection of the ranges after token is changed
@@ -3569,7 +3569,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     public List<String> sampleKeyRange() // do not rename to getter - see CASSANDRA-4452 for details
     {
         List<DecoratedKey> keys = new ArrayList<DecoratedKey>();
-        for (Table keyspace : Table.nonSystem())
+        for (Keyspace keyspace : Keyspace.nonSystem())
         {
             for (Range<Token> range : getPrimaryRangesForEndpoint(keyspace.getName(), FBUtilities.getBroadcastAddress()))
                 keys.addAll(keySamples(keyspace.getColumnFamilyStores(), range));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index ef619c5..a84bcf4 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -184,46 +184,46 @@ public interface StorageServiceMBean extends NotificationEmitter
      * This method returns the N endpoints that are responsible for storing the
      * specified key i.e for replication.
      *
-     * @param table keyspace name also known as table
+     * @param keyspaceName keyspace name
      * @param cf Column family name
      * @param key - key for which we need to find the endpoint return value -
      * the endpoint responsible for this key
      */
-    public List<InetAddress> getNaturalEndpoints(String table, String cf, String key);
-    public List<InetAddress> getNaturalEndpoints(String table, ByteBuffer key);
+    public List<InetAddress> getNaturalEndpoints(String keyspaceName, String cf, String key);
+    public List<InetAddress> getNaturalEndpoints(String keyspaceName, ByteBuffer key);
 
     /**
-     * Takes the snapshot for the given tables. A snapshot name must be specified.
+     * Takes the snapshot for the given keyspaces. A snapshot name must be specified.
      *
      * @param tag the tag given to the snapshot; may not be null or empty
-     * @param tableNames the name of the tables to snapshot; empty means "all."
+     * @param keyspaceNames the name of the keyspaces to snapshot; empty means "all."
      */
-    public void takeSnapshot(String tag, String... tableNames) throws IOException;
+    public void takeSnapshot(String tag, String... keyspaceNames) throws IOException;
 
     /**
      * Takes the snapshot of a specific column family. A snapshot name must be specified.
      *
-     * @param tableName the keyspace which holds the specified column family
+     * @param keyspaceName the keyspace which holds the specified column family
      * @param columnFamilyName the column family to snapshot
      * @param tag the tag given to the snapshot; may not be null or empty
      */
-    public void takeColumnFamilySnapshot(String tableName, String columnFamilyName, String tag) throws IOException;
+    public void takeColumnFamilySnapshot(String keyspaceName, String columnFamilyName, String tag) throws IOException;
 
     /**
-     * Remove the snapshot with the given name from the given tables.
+     * Remove the snapshot with the given name from the given keyspaces.
      * If no tag is specified we will remove all snapshots.
      */
-    public void clearSnapshot(String tag, String... tableNames) throws IOException;
+    public void clearSnapshot(String tag, String... keyspaceNames) throws IOException;
 
     /**
      * Forces major compaction of a single keyspace
      */
-    public void forceTableCompaction(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public void forceKeyspaceCompaction(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Trigger a cleanup of keys on a single keyspace
      */
-    public void forceTableCleanup(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public void forceKeyspaceCleanup(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Scrub (deserialize + reserialize at the latest version, skipping bad rows if any) the given keyspace.
@@ -231,22 +231,22 @@ public interface StorageServiceMBean extends NotificationEmitter
      *
      * Scrubbed CFs will be snapshotted first.
      */
-    public void scrub(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public void scrub(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
      */
-    public void upgradeSSTables(String tableName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public void upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
-     * Flush all memtables for the given column families, or all columnfamilies for the given table
+     * Flush all memtables for the given column families, or all columnfamilies for the given keyspace
      * if none are explicitly listed.
-     * @param tableName
+     * @param keyspaceName
      * @param columnFamilies
      * @throws IOException
      */
-    public void forceTableFlush(String tableName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public void forceKeyspaceFlush(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Invoke repair asynchronously.
@@ -256,28 +256,28 @@ public interface StorageServiceMBean extends NotificationEmitter
      *   userObject: int array of length 2, [0]=command number, [1]=ordinal of AntiEntropyService.Status
      *
      * @return Repair command number, or 0 if nothing to repair
-     * @see #forceTableRepair(String, boolean, boolean, String...)
+     * @see #forceKeyspaceRepair(String, boolean, boolean, String...)
      */
     public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, boolean primaryRange, String... columnFamilies);
 
     /**
      * Same as forceRepairAsync, but handles a specified range
      */
-    public int forceRepairRangeAsync(String beginToken, String endToken, final String tableName, boolean isSequential, boolean isLocal, final String... columnFamilies);
+    public int forceRepairRangeAsync(String beginToken, String endToken, final String keyspaceName, boolean isSequential, boolean isLocal, final String... columnFamilies);
 
     /**
-     * Triggers proactive repair for given column families, or all columnfamilies for the given table
+     * Triggers proactive repair for given column families, or all columnfamilies for the given keyspace
      * if none are explicitly listed.
-     * @param tableName
+     * @param keyspaceName
      * @param columnFamilies
      * @throws IOException
      */
-    public void forceTableRepair(String tableName, boolean isSequential, boolean  isLocal, String... columnFamilies) throws IOException;
+    public void forceKeyspaceRepair(String keyspaceName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException;
 
     /**
      * Triggers proactive repair but only for the node primary range.
      */
-    public void forceTableRepairPrimaryRange(String tableName, boolean isSequential, boolean  isLocal, String... columnFamilies) throws IOException;
+    public void forceKeyspaceRepairPrimaryRange(String keyspaceName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException;
 
     /**
      * Perform repair of a specific range.
@@ -285,7 +285,7 @@ public interface StorageServiceMBean extends NotificationEmitter
      * This allows incremental repair to be performed by having an external controller submitting repair jobs.
      * Note that the provided range much be a subset of one of the node local range.
      */
-    public void forceTableRepairRange(String beginToken, String endToken, String tableName, boolean isSequential, boolean  isLocal, String... columnFamilies) throws IOException;
+    public void forceKeyspaceRepairRange(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, String... columnFamilies) throws IOException;
 
     public void forceTerminateAllRepairSessions();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/WriteResponseHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/WriteResponseHandler.java b/src/java/org/apache/cassandra/service/WriteResponseHandler.java
index 34248e6..826ae01 100644
--- a/src/java/org/apache/cassandra/service/WriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/WriteResponseHandler.java
@@ -26,7 +26,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.WriteType;
@@ -43,11 +43,11 @@ public class WriteResponseHandler extends AbstractWriteResponseHandler
     public WriteResponseHandler(Collection<InetAddress> writeEndpoints,
                                 Collection<InetAddress> pendingEndpoints,
                                 ConsistencyLevel consistencyLevel,
-                                Table table,
+                                Keyspace keyspace,
                                 Runnable callback,
                                 WriteType writeType)
     {
-        super(table, writeEndpoints, pendingEndpoints, consistencyLevel, callback, writeType);
+        super(keyspace, writeEndpoints, pendingEndpoints, consistencyLevel, callback, writeType);
         responses = new AtomicInteger(totalBlockFor());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/pager/NamesQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/NamesQueryPager.java b/src/java/org/apache/cassandra/service/pager/NamesQueryPager.java
index 82e7376..0ac7079 100644
--- a/src/java/org/apache/cassandra/service/pager/NamesQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/NamesQueryPager.java
@@ -75,7 +75,7 @@ public class NamesQueryPager implements SinglePartitionPager
 
         queried = true;
         return localQuery
-             ? Collections.singletonList(command.getRow(Table.open(command.table)))
+             ? Collections.singletonList(command.getRow(Keyspace.open(command.ksName)))
              : StorageProxy.read(Collections.<ReadCommand>singletonList(command), consistencyLevel);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/pager/SliceQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/SliceQueryPager.java b/src/java/org/apache/cassandra/service/pager/SliceQueryPager.java
index 58ef3c4..1be1aed 100644
--- a/src/java/org/apache/cassandra/service/pager/SliceQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/SliceQueryPager.java
@@ -42,7 +42,7 @@ public class SliceQueryPager extends AbstractQueryPager implements SinglePartiti
     // Don't use directly, use QueryPagers method instead
     SliceQueryPager(SliceFromReadCommand command, ConsistencyLevel consistencyLevel, boolean localQuery)
     {
-        super(consistencyLevel, command.filter.count, localQuery, command.table, command.cfName, command.filter, command.timestamp);
+        super(consistencyLevel, command.filter.count, localQuery, command.ksName, command.cfName, command.filter, command.timestamp);
         this.command = command;
     }
 
@@ -55,7 +55,7 @@ public class SliceQueryPager extends AbstractQueryPager implements SinglePartiti
 
         ReadCommand pageCmd = command.withUpdatedFilter(filter);
         return localQuery
-             ? Collections.singletonList(pageCmd.getRow(Table.open(command.table)))
+             ? Collections.singletonList(pageCmd.getRow(Keyspace.open(command.ksName)))
              : StorageProxy.read(Collections.singletonList(pageCmd), consistencyLevel);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/paxos/PaxosState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/paxos/PaxosState.java b/src/java/org/apache/cassandra/service/paxos/PaxosState.java
index dc24661..ff0035e 100644
--- a/src/java/org/apache/cassandra/service/paxos/PaxosState.java
+++ b/src/java/org/apache/cassandra/service/paxos/PaxosState.java
@@ -28,8 +28,8 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.db.SystemTable;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.tracing.Tracing;
 
 public class PaxosState
@@ -69,11 +69,11 @@ public class PaxosState
     {
         synchronized (lockFor(toPrepare.key))
         {
-            PaxosState state = SystemTable.loadPaxosState(toPrepare.key, toPrepare.update.metadata());
+            PaxosState state = SystemKeyspace.loadPaxosState(toPrepare.key, toPrepare.update.metadata());
             if (toPrepare.isAfter(state.inProgressCommit))
             {
                 Tracing.trace("promising ballot {}", toPrepare.ballot);
-                SystemTable.savePaxosPromise(toPrepare);
+                SystemKeyspace.savePaxosPromise(toPrepare);
                 // return the pre-promise ballot so coordinator can pick the most recent in-progress value to resume
                 return new PrepareResponse(true, state.inProgressCommit, state.mostRecentCommit);
             }
@@ -89,11 +89,11 @@ public class PaxosState
     {
         synchronized (lockFor(proposal.key))
         {
-            PaxosState state = SystemTable.loadPaxosState(proposal.key, proposal.update.metadata());
+            PaxosState state = SystemKeyspace.loadPaxosState(proposal.key, proposal.update.metadata());
             if (proposal.hasBallot(state.inProgressCommit.ballot) || proposal.isAfter(state.inProgressCommit))
             {
                 Tracing.trace("accepting proposal {}", proposal);
-                SystemTable.savePaxosProposal(proposal);
+                SystemKeyspace.savePaxosProposal(proposal);
                 return true;
             }
 
@@ -111,12 +111,12 @@ public class PaxosState
         // erase the in-progress update.
         Tracing.trace("committing proposal {}", proposal);
         RowMutation rm = proposal.makeMutation();
-        Table.open(rm.getTable()).apply(rm, true);
+        Keyspace.open(rm.getKeyspaceName()).apply(rm, true);
 
         synchronized (lockFor(proposal.key))
         {
-            PaxosState state = SystemTable.loadPaxosState(proposal.key, proposal.update.metadata());
-            SystemTable.savePaxosCommit(proposal, !state.inProgressCommit.isAfter(proposal));
+            PaxosState state = SystemKeyspace.loadPaxosState(proposal.key, proposal.update.metadata());
+            SystemKeyspace.savePaxosCommit(proposal, !state.inProgressCommit.isAfter(proposal));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReader.java b/src/java/org/apache/cassandra/streaming/StreamReader.java
index a01de3c..bad8445 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
@@ -73,7 +73,7 @@ public class StreamReader
         long totalSize = totalSize();
 
         Pair<String, String> kscf = Schema.instance.getCF(cfId);
-        ColumnFamilyStore cfs = Table.open(kscf.left).getColumnFamilyStore(kscf.right);
+        ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
         Directories.DataDirectory localDir = cfs.directories.getLocationCapableOfSize(totalSize);
         if (localDir == null)
             throw new IOException("Insufficient disk space to store " + totalSize + " bytes");


[10/11] Rename Table to Keyspace

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c643e2b..0a40457 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -82,7 +82,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public static final ExecutorService postFlushExecutor = new JMXEnabledThreadPoolExecutor("MemtablePostFlusher");
 
-    public final Table table;
+    public final Keyspace keyspace;
     public final String name;
     public final CFMetaData metadata;
     public final IPartitioner partitioner;
@@ -217,7 +217,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         try
         {
-            for (SSTableReader sstable : table.getAllSSTables())
+            for (SSTableReader sstable : keyspace.getAllSSTables())
                 if (sstable.compression)
                     sstable.getCompressionMetadata().parameters.setCrcCheckChance(crcCheckChance);
         }
@@ -227,7 +227,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    private ColumnFamilyStore(Table table,
+    private ColumnFamilyStore(Keyspace keyspace,
                               String columnFamilyName,
                               IPartitioner partitioner,
                               int generation,
@@ -235,9 +235,9 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                               Directories directories,
                               boolean loadSSTables)
     {
-        assert metadata != null : "null metadata for " + table + ":" + columnFamilyName;
+        assert metadata != null : "null metadata for " + keyspace + ":" + columnFamilyName;
 
-        this.table = table;
+        this.keyspace = keyspace;
         name = columnFamilyName;
         this.metadata = metadata;
         this.minCompactionThreshold = new DefaultInteger(metadata.getMinCompactionThreshold());
@@ -284,7 +284,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         // register the mbean
         String type = this.partitioner instanceof LocalPartitioner ? "IndexColumnFamilies" : "ColumnFamilies";
-        mbeanName = "org.apache.cassandra.db:type=" + type + ",keyspace=" + this.table.getName() + ",columnfamily=" + name;
+        mbeanName = "org.apache.cassandra.db:type=" + type + ",keyspace=" + this.keyspace.getName() + ",columnfamily=" + name;
         try
         {
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
@@ -326,7 +326,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             valid = false;
             unregisterMBean();
 
-            SystemTable.removeTruncationRecord(metadata.cfId);
+            SystemKeyspace.removeTruncationRecord(metadata.cfId);
             data.unreferenceSSTables();
             indexManager.invalidate();
         }
@@ -377,24 +377,24 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return data.getMeanColumns();
     }
 
-    public static ColumnFamilyStore createColumnFamilyStore(Table table, String columnFamily, boolean loadSSTables)
+    public static ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace, String columnFamily, boolean loadSSTables)
     {
-        return createColumnFamilyStore(table, columnFamily, StorageService.getPartitioner(), Schema.instance.getCFMetaData(table.getName(), columnFamily), loadSSTables);
+        return createColumnFamilyStore(keyspace, columnFamily, StorageService.getPartitioner(), Schema.instance.getCFMetaData(keyspace.getName(), columnFamily), loadSSTables);
     }
 
-    public static ColumnFamilyStore createColumnFamilyStore(Table table, String columnFamily, IPartitioner partitioner, CFMetaData metadata)
+    public static ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace, String columnFamily, IPartitioner partitioner, CFMetaData metadata)
     {
-        return createColumnFamilyStore(table, columnFamily, partitioner, metadata, true);
+        return createColumnFamilyStore(keyspace, columnFamily, partitioner, metadata, true);
     }
 
-    private static synchronized ColumnFamilyStore createColumnFamilyStore(Table table,
+    private static synchronized ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace,
                                                                          String columnFamily,
                                                                          IPartitioner partitioner,
                                                                          CFMetaData metadata,
                                                                          boolean loadSSTables)
     {
         // get the max generation number, to prevent generation conflicts
-        Directories directories = Directories.create(table.getName(), columnFamily);
+        Directories directories = Directories.create(keyspace.getName(), columnFamily);
         Directories.SSTableLister lister = directories.sstableLister().includeBackups(true);
         List<Integer> generations = new ArrayList<Integer>();
         for (Map.Entry<Descriptor, Set<Component>> entry : lister.list().entrySet())
@@ -407,18 +407,18 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         Collections.sort(generations);
         int value = (generations.size() > 0) ? (generations.get(generations.size() - 1)) : 0;
 
-        return new ColumnFamilyStore(table, columnFamily, partitioner, value, metadata, directories, loadSSTables);
+        return new ColumnFamilyStore(keyspace, columnFamily, partitioner, value, metadata, directories, loadSSTables);
     }
 
     /**
      * Removes unnecessary files from the cf directory at startup: these include temp files, orphans, zero-length files
      * and compacted sstables. Files that cannot be recognized will be ignored.
      */
-    public static void scrubDataDirectories(String table, String columnFamily)
+    public static void scrubDataDirectories(String keyspaceName, String columnFamily)
     {
         logger.debug("Removing compacted SSTable files from {} (see http://wiki.apache.org/cassandra/MemtableSSTable)", columnFamily);
 
-        Directories directories = Directories.create(table, columnFamily);
+        Directories directories = Directories.create(keyspaceName, columnFamily);
         for (Map.Entry<Descriptor,Set<Component>> sstableFiles : directories.sstableLister().list().entrySet())
         {
             Descriptor desc = sstableFiles.getKey();
@@ -444,7 +444,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
 
         // cleanup incomplete saved caches
-        Pattern tmpCacheFilePattern = Pattern.compile(table + "-" + columnFamily + "-(Key|Row)Cache.*\\.tmp$");
+        Pattern tmpCacheFilePattern = Pattern.compile(keyspaceName + "-" + columnFamily + "-(Key|Row)Cache.*\\.tmp$");
         File dir = new File(DatabaseDescriptor.getSavedCachesLocation());
 
         if (dir.exists())
@@ -457,11 +457,11 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
 
         // also clean out any index leftovers.
-        CFMetaData cfm = Schema.instance.getCFMetaData(table, columnFamily);
+        CFMetaData cfm = Schema.instance.getCFMetaData(keyspaceName, columnFamily);
         if (cfm != null) // secondary indexes aren't stored in DD.
         {
             for (ColumnDefinition def : cfm.allColumns())
-                scrubDataDirectories(table, cfm.indexColumnFamilyName(def));
+                scrubDataDirectories(keyspaceName, cfm.indexColumnFamilyName(def));
         }
     }
 
@@ -544,7 +544,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             logger.info("completed loading ({} ms; {} keys) row cache for {}.{}",
                         TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start),
                         cachedRowsRead,
-                        table.getName(),
+                        keyspace.getName(),
                         name);
     }
 
@@ -557,8 +557,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     public static synchronized void loadNewSSTables(String ksName, String cfName)
     {
         /** ks/cf existence checks will be done by open and getCFS methods for us */
-        Table table = Table.open(ksName);
-        table.getColumnFamilyStore(cfName).loadNewSSTables();
+        Keyspace keyspace = Keyspace.open(ksName);
+        keyspace.getColumnFamilyStore(cfName).loadNewSSTables();
     }
 
     /**
@@ -566,7 +566,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     public synchronized void loadNewSSTables()
     {
-        logger.info("Loading new SSTables for " + table.getName() + "/" + name + "...");
+        logger.info("Loading new SSTables for " + keyspace.getName() + "/" + name + "...");
 
         Set<Descriptor> currentDescriptors = new HashSet<Descriptor>();
         for (SSTableReader sstable : data.getView().sstables)
@@ -627,11 +627,11 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         if (newSSTables.isEmpty())
         {
-            logger.info("No new SSTables were found for " + table.getName() + "/" + name);
+            logger.info("No new SSTables were found for " + keyspace.getName() + "/" + name);
             return;
         }
 
-        logger.info("Loading new SSTables and building secondary indexes for " + table.getName() + "/" + name + ": " + newSSTables);
+        logger.info("Loading new SSTables and building secondary indexes for " + keyspace.getName() + "/" + name + ": " + newSSTables);
         SSTableReader.acquireReferences(newSSTables);
         data.addSSTables(newSSTables);
         try
@@ -643,12 +643,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             SSTableReader.releaseReferences(newSSTables);
         }
 
-        logger.info("Done loading load new SSTables for " + table.getName() + "/" + name);
+        logger.info("Done loading load new SSTables for " + keyspace.getName() + "/" + name);
     }
 
     public static void rebuildSecondaryIndex(String ksName, String cfName, String... idxNames)
     {
-        ColumnFamilyStore cfs = Table.open(ksName).getColumnFamilyStore(cfName);
+        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(cfName);
 
         Set<String> indexes = new HashSet<String>(Arrays.asList(idxNames));
 
@@ -681,7 +681,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         Descriptor desc = new Descriptor(version,
                                          directory,
-                                         table.getName(),
+                                         keyspace.getName(),
                                          name,
                                          fileIndexGenerator.incrementAndGet(),
                                          true);
@@ -700,12 +700,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
          * all ongoing updates to memtables have completed. We can get the tail
          * of the log and use it as the starting position for log replay on recovery.
          *
-         * This is why we Table.switchLock needs to be global instead of per-Table:
+         * This is why we Keyspace.switchLock needs to be global instead of per-Keyspace:
          * we need to schedule discardCompletedSegments calls in the same order as their
          * contexts (commitlog position) were read, even though the flush executor
          * is multithreaded.
          */
-        Table.switchLock.writeLock().lock();
+        Keyspace.switchLock.writeLock().lock();
         try
         {
             final Future<ReplayPosition> ctx = writeCommitLog ? CommitLog.instance.getContext() : Futures.immediateFuture(ReplayPosition.NONE);
@@ -774,7 +774,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
         finally
         {
-            Table.switchLock.writeLock().unlock();
+            Keyspace.switchLock.writeLock().unlock();
         }
     }
 
@@ -820,7 +820,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     /**
      * Insert/Update the column family for this key.
-     * Caller is responsible for acquiring Table.flusherLock!
+     * Caller is responsible for acquiring Keyspace.switchLock
      * param @ lock - lock that needs to be used.
      * param @ key - key for update/insert
      * param @ columnFamily - columnFamily changes
@@ -1005,7 +1005,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         // cleanup size estimation only counts bytes for keys local to this node
         long expectedFileSize = 0;
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(table.getName());
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspace.getName());
         for (SSTableReader sstable : sstables)
         {
             List<Pair<Long, Long>> positions = sstable.getPositionsForRanges(ranges);
@@ -1262,10 +1262,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             int gcBefore = gcBefore(filter.timestamp);
             if (isRowCacheEnabled())
             {
-                UUID cfId = Schema.instance.getId(table.getName(), name);
+                UUID cfId = Schema.instance.getId(keyspace.getName(), name);
                 if (cfId == null)
                 {
-                    logger.trace("no id found for {}.{}", table.getName(), name);
+                    logger.trace("no id found for {}.{}", keyspace.getName(), name);
                     return null;
                 }
 
@@ -1710,7 +1710,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     File snapshotDirectory = Directories.getSnapshotDirectory(ssTable.descriptor, snapshotName);
                     ssTable.createLinks(snapshotDirectory.getPath()); // hard links
                     if (logger.isDebugEnabled())
-                        logger.debug("Snapshot for " + table + " keyspace data file " + ssTable.getFilename() +
+                        logger.debug("Snapshot for " + keyspace + " keyspace data file " + ssTable.getFilename() +
                                      " created in " + snapshotDirectory);
                 }
 
@@ -1811,7 +1811,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public void invalidateCachedRow(DecoratedKey key)
     {
-        UUID cfId = Schema.instance.getId(table.getName(), this.name);
+        UUID cfId = Schema.instance.getId(keyspace.getName(), this.name);
         if (cfId == null)
             return; // secondary index
 
@@ -1825,10 +1825,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public static Iterable<ColumnFamilyStore> all()
     {
-        List<Iterable<ColumnFamilyStore>> stores = new ArrayList<Iterable<ColumnFamilyStore>>(Schema.instance.getTables().size());
-        for (Table table : Table.all())
+        List<Iterable<ColumnFamilyStore>> stores = new ArrayList<Iterable<ColumnFamilyStore>>(Schema.instance.getKeyspaces().size());
+        for (Keyspace keyspace : Keyspace.all())
         {
-            stores.add(table.getColumnFamilyStores());
+            stores.add(keyspace.getColumnFamilyStores());
         }
         return Iterables.concat(stores);
     }
@@ -1896,7 +1896,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         else
         {
             // just nuke the memtable data w/o writing to disk first
-            Table.switchLock.writeLock().lock();
+            Keyspace.switchLock.writeLock().lock();
             try
             {
                 for (ColumnFamilyStore cfs : concatWithIndexes())
@@ -1910,7 +1910,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             }
             finally
             {
-                Table.switchLock.writeLock().unlock();
+                Keyspace.switchLock.writeLock().unlock();
             }
         }
 
@@ -1922,14 +1922,14 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
                 final long truncatedAt = System.currentTimeMillis();
                 if (DatabaseDescriptor.isAutoSnapshot())
-                    snapshot(Table.getTimestampedSnapshotName(name));
+                    snapshot(Keyspace.getTimestampedSnapshotName(name));
 
                 ReplayPosition replayAfter = discardSSTables(truncatedAt);
 
                 for (SecondaryIndex index : indexManager.getIndexes())
                     index.truncateBlocking(truncatedAt);
 
-                SystemTable.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
+                SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
 
                 logger.debug("cleaning out row cache");
                 for (RowCacheKey key : CacheService.instance.rowCache.getKeySet())
@@ -2055,7 +2055,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     public String toString()
     {
         return "CFS(" +
-               "Keyspace='" + table.getName() + '\'' +
+               "Keyspace='" + keyspace.getName() + '\'' +
                ", ColumnFamily='" + name + '\'' +
                ')';
     }
@@ -2309,7 +2309,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public long getTruncationTime()
     {
-        Pair<ReplayPosition, Long> truncationRecord = SystemTable.getTruncationRecords().get(metadata.cfId);
+        Pair<ReplayPosition, Long> truncationRecord = SystemKeyspace.getTruncationRecords().get(metadata.cfId);
         return truncationRecord == null ? Long.MIN_VALUE : truncationRecord.right;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/ConsistencyLevel.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
index e62da1b..d642d08 100644
--- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java
+++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
@@ -81,12 +81,12 @@ public enum ConsistencyLevel
         return codeIdx[code];
     }
 
-    private int localQuorumFor(Table table, String dc)
+    private int localQuorumFor(Keyspace keyspace, String dc)
     {
-        return (((NetworkTopologyStrategy) table.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1;
+        return (((NetworkTopologyStrategy) keyspace.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1;
     }
 
-    public int blockFor(Table table)
+    public int blockFor(Keyspace keyspace)
     {
         switch (this)
         {
@@ -99,16 +99,16 @@ public enum ConsistencyLevel
             case THREE:
                 return 3;
             case QUORUM:
-                return (table.getReplicationStrategy().getReplicationFactor() / 2) + 1;
+                return (keyspace.getReplicationStrategy().getReplicationFactor() / 2) + 1;
             case ALL:
-                return table.getReplicationStrategy().getReplicationFactor();
+                return keyspace.getReplicationStrategy().getReplicationFactor();
             case LOCAL_QUORUM:
-                return localQuorumFor(table, DatabaseDescriptor.getLocalDataCenter());
+                return localQuorumFor(keyspace, DatabaseDescriptor.getLocalDataCenter());
             case EACH_QUORUM:
-                NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) table.getReplicationStrategy();
+                NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
                 int n = 0;
                 for (String dc : strategy.getDatacenters())
-                    n += localQuorumFor(table, dc);
+                    n += localQuorumFor(keyspace, dc);
                 return n;
             default:
                 throw new UnsupportedOperationException("Invalid consistency level: " + toString());
@@ -129,9 +129,9 @@ public enum ConsistencyLevel
         return count;
     }
 
-    private Map<String, Integer> countPerDCEndpoints(Table table, Iterable<InetAddress> liveEndpoints)
+    private Map<String, Integer> countPerDCEndpoints(Keyspace keyspace, Iterable<InetAddress> liveEndpoints)
     {
-        NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) table.getReplicationStrategy();
+        NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
 
         Map<String, Integer> dcEndpoints = new HashMap<String, Integer>();
         for (String dc: strategy.getDatacenters())
@@ -145,12 +145,12 @@ public enum ConsistencyLevel
         return dcEndpoints;
     }
 
-    public List<InetAddress> filterForQuery(Table table, List<InetAddress> liveEndpoints)
+    public List<InetAddress> filterForQuery(Keyspace keyspace, List<InetAddress> liveEndpoints)
     {
-        return filterForQuery(table, liveEndpoints, ReadRepairDecision.NONE);
+        return filterForQuery(keyspace, liveEndpoints, ReadRepairDecision.NONE);
     }
 
-    public List<InetAddress> filterForQuery(Table table, List<InetAddress> liveEndpoints, ReadRepairDecision readRepair)
+    public List<InetAddress> filterForQuery(Keyspace keyspace, List<InetAddress> liveEndpoints, ReadRepairDecision readRepair)
     {
         /*
          * Endpoints are expected to be restricted to live replicas, sorted by snitch preference.
@@ -164,7 +164,7 @@ public enum ConsistencyLevel
         switch (readRepair)
         {
             case NONE:
-                return liveEndpoints.subList(0, Math.min(liveEndpoints.size(), blockFor(table)));
+                return liveEndpoints.subList(0, Math.min(liveEndpoints.size(), blockFor(keyspace)));
             case GLOBAL:
                 return liveEndpoints;
             case DC_LOCAL:
@@ -178,7 +178,7 @@ public enum ConsistencyLevel
                         other.add(add);
                 }
                 // check if blockfor more than we have localep's
-                int blockFor = blockFor(table);
+                int blockFor = blockFor(keyspace);
                 if (local.size() < blockFor)
                     local.addAll(other.subList(0, Math.min(blockFor - local.size(), other.size())));
                 return local;
@@ -187,7 +187,7 @@ public enum ConsistencyLevel
         }
     }
 
-    public boolean isSufficientLiveNodes(Table table, Iterable<InetAddress> liveEndpoints)
+    public boolean isSufficientLiveNodes(Keyspace keyspace, Iterable<InetAddress> liveEndpoints)
     {
         switch (this)
         {
@@ -195,22 +195,22 @@ public enum ConsistencyLevel
                 // local hint is acceptable, and local node is always live
                 return true;
             case LOCAL_QUORUM:
-                return countLocalEndpoints(liveEndpoints) >= blockFor(table);
+                return countLocalEndpoints(liveEndpoints) >= blockFor(keyspace);
             case EACH_QUORUM:
-                for (Map.Entry<String, Integer> entry : countPerDCEndpoints(table, liveEndpoints).entrySet())
+                for (Map.Entry<String, Integer> entry : countPerDCEndpoints(keyspace, liveEndpoints).entrySet())
                 {
-                    if (entry.getValue() < localQuorumFor(table, entry.getKey()))
+                    if (entry.getValue() < localQuorumFor(keyspace, entry.getKey()))
                         return false;
                 }
                 return true;
             default:
-                return Iterables.size(liveEndpoints) >= blockFor(table);
+                return Iterables.size(liveEndpoints) >= blockFor(keyspace);
         }
     }
 
-    public void assureSufficientLiveNodes(Table table, Iterable<InetAddress> liveEndpoints) throws UnavailableException
+    public void assureSufficientLiveNodes(Keyspace keyspace, Iterable<InetAddress> liveEndpoints) throws UnavailableException
     {
-        int blockFor = blockFor(table);
+        int blockFor = blockFor(keyspace);
         switch (this)
         {
             case ANY:
@@ -235,9 +235,9 @@ public enum ConsistencyLevel
                 }
                 break;
             case EACH_QUORUM:
-                for (Map.Entry<String, Integer> entry : countPerDCEndpoints(table, liveEndpoints).entrySet())
+                for (Map.Entry<String, Integer> entry : countPerDCEndpoints(keyspace, liveEndpoints).entrySet())
                 {
-                    int dcBlockFor = localQuorumFor(table, entry.getKey());
+                    int dcBlockFor = localQuorumFor(keyspace, entry.getKey());
                     int dcLive = entry.getValue();
                     if (dcLive < dcBlockFor)
                         throw new UnavailableException(this, dcBlockFor, dcLive);
@@ -254,12 +254,12 @@ public enum ConsistencyLevel
         }
     }
 
-    public void validateForRead(String table) throws InvalidRequestException
+    public void validateForRead(String keyspaceName) throws InvalidRequestException
     {
         switch (this)
         {
             case LOCAL_QUORUM:
-                requireNetworkTopologyStrategy(table);
+                requireNetworkTopologyStrategy(keyspaceName);
                 break;
             case ANY:
                 throw new InvalidRequestException("ANY ConsistencyLevel is only supported for writes");
@@ -268,26 +268,26 @@ public enum ConsistencyLevel
         }
     }
 
-    public void validateForWrite(String table) throws InvalidRequestException
+    public void validateForWrite(String keyspaceName) throws InvalidRequestException
     {
         switch (this)
         {
             case LOCAL_QUORUM:
             case EACH_QUORUM:
-                requireNetworkTopologyStrategy(table);
+                requireNetworkTopologyStrategy(keyspaceName);
                 break;
             case SERIAL:
                 throw new InvalidRequestException("You must use conditional updates for serializable writes");
         }
     }
 
-    public void validateForCas(String table) throws InvalidRequestException
+    public void validateForCas(String keyspaceName) throws InvalidRequestException
     {
         switch (this)
         {
             case LOCAL_QUORUM:
             case EACH_QUORUM:
-                requireNetworkTopologyStrategy(table);
+                requireNetworkTopologyStrategy(keyspaceName);
                 break;
             case ANY:
                 throw new InvalidRequestException("ANY is not supported with CAS. Use SERIAL if you mean, make sure it is accepted but I don't care how many replicas commit it for non-SERIAL reads");
@@ -310,9 +310,9 @@ public enum ConsistencyLevel
         }
     }
 
-    private void requireNetworkTopologyStrategy(String table) throws InvalidRequestException
+    private void requireNetworkTopologyStrategy(String keyspaceName) throws InvalidRequestException
     {
-        AbstractReplicationStrategy strategy = Table.open(table).getReplicationStrategy();
+        AbstractReplicationStrategy strategy = Keyspace.open(keyspaceName).getReplicationStrategy();
         if (!(strategy instanceof NetworkTopologyStrategy))
             throw new InvalidRequestException(String.format("consistency level %s not compatible with replication strategy (%s)", this, strategy.getClass().getName()));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/CounterMutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java
index 9ace314..fb363c2 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -51,9 +51,9 @@ public class CounterMutation implements IMutation
         this.consistency = consistency;
     }
 
-    public String getTable()
+    public String getKeyspaceName()
     {
-        return rowMutation.getTable();
+        return rowMutation.getKeyspaceName();
     }
 
     public Collection<UUID> getColumnFamilyIds()
@@ -89,15 +89,15 @@ public class CounterMutation implements IMutation
         {
             if (!columnFamily.metadata().getReplicateOnWrite())
                 continue;
-            addReadCommandFromColumnFamily(rowMutation.getTable(), rowMutation.key(), columnFamily, timestamp, readCommands);
+            addReadCommandFromColumnFamily(rowMutation.getKeyspaceName(), rowMutation.key(), columnFamily, timestamp, readCommands);
         }
 
         // create a replication RowMutation
-        RowMutation replicationMutation = new RowMutation(rowMutation.getTable(), rowMutation.key());
+        RowMutation replicationMutation = new RowMutation(rowMutation.getKeyspaceName(), rowMutation.key());
         for (ReadCommand readCommand : readCommands)
         {
-            Table table = Table.open(readCommand.table);
-            Row row = readCommand.getRow(table);
+            Keyspace keyspace = Keyspace.open(readCommand.ksName);
+            Row row = readCommand.getRow(keyspace);
             if (row == null || row.cf == null)
                 continue;
 
@@ -107,11 +107,11 @@ public class CounterMutation implements IMutation
         return replicationMutation;
     }
 
-    private void addReadCommandFromColumnFamily(String table, ByteBuffer key, ColumnFamily columnFamily, long timestamp, List<ReadCommand> commands)
+    private void addReadCommandFromColumnFamily(String keyspaceName, ByteBuffer key, ColumnFamily columnFamily, long timestamp, List<ReadCommand> commands)
     {
         SortedSet<ByteBuffer> s = new TreeSet<ByteBuffer>(columnFamily.metadata().comparator);
         Iterables.addAll(s, columnFamily.getColumnNames());
-        commands.add(new SliceByNamesReadCommand(table, key, columnFamily.metadata().cfName, timestamp, new NamesQueryFilter(s)));
+        commands.add(new SliceByNamesReadCommand(keyspaceName, key, columnFamily.metadata().cfName, timestamp, new NamesQueryFilter(s)));
     }
 
     public MessageOut<CounterMutation> makeMutationMessage()
@@ -130,13 +130,13 @@ public class CounterMutation implements IMutation
     public void apply()
     {
         // transform all CounterUpdateColumn to CounterColumn: accomplished by localCopy
-        RowMutation rm = new RowMutation(rowMutation.getTable(), ByteBufferUtil.clone(rowMutation.key()));
-        Table table = Table.open(rm.getTable());
+        RowMutation rm = new RowMutation(rowMutation.getKeyspaceName(), ByteBufferUtil.clone(rowMutation.key()));
+        Keyspace keyspace = Keyspace.open(rm.getKeyspaceName());
 
         for (ColumnFamily cf_ : rowMutation.getColumnFamilies())
         {
             ColumnFamily cf = cf_.cloneMeShallow();
-            ColumnFamilyStore cfs = table.getColumnFamilyStore(cf.id());
+            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cf.id());
             for (Column column : cf_)
             {
                 cf.addColumn(column.localCopy(cfs), HeapAllocator.instance);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/DataTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DataTracker.java b/src/java/org/apache/cassandra/db/DataTracker.java
index c5a9c2b..af22b1b 100644
--- a/src/java/org/apache/cassandra/db/DataTracker.java
+++ b/src/java/org/apache/cassandra/db/DataTracker.java
@@ -344,7 +344,7 @@ public class DataTracker
         {
             if (logger.isDebugEnabled())
                 logger.debug(String.format("adding %s to list of files tracked for %s.%s",
-                            sstable.descriptor, cfstore.table.getName(), cfstore.name));
+                            sstable.descriptor, cfstore.keyspace.getName(), cfstore.name));
             long size = sstable.bytesOnDisk();
             StorageMetrics.load.inc(size);
             cfstore.metric.liveDiskSpaceUsed.inc(size);
@@ -359,7 +359,7 @@ public class DataTracker
         {
             if (logger.isDebugEnabled())
                 logger.debug(String.format("removing %s from list of files tracked for %s.%s",
-                            sstable.descriptor, cfstore.table.getName(), cfstore.name));
+                            sstable.descriptor, cfstore.keyspace.getName(), cfstore.name));
             long size = sstable.bytesOnDisk();
             StorageMetrics.load.dec(size);
             cfstore.metric.liveDiskSpaceUsed.dec(size);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java b/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
index a7ad353..c4b9f84 100644
--- a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
@@ -46,7 +46,7 @@ public class DefinitionsUpdateVerbHandler implements IVerbHandler<Collection<Row
         {
             public void runMayThrow() throws Exception
             {
-                DefsTable.mergeSchema(message.payload);
+                DefsTables.mergeSchema(message.payload);
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/DefsTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefsTable.java b/src/java/org/apache/cassandra/db/DefsTable.java
deleted file mode 100644
index 6008e75..0000000
--- a/src/java/org/apache/cassandra/db/DefsTable.java
+++ /dev/null
@@ -1,470 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.MapDifference;
-import com.google.common.collect.Maps;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-
-/**
- * SCHEMA_{KEYSPACES, COLUMNFAMILIES, COLUMNS}_CF are used to store Keyspace/ColumnFamily attributes to make schema
- * load/distribution easy, it replaces old mechanism when local migrations where serialized, stored in system.Migrations
- * and used for schema distribution.
- *
- * SCHEMA_KEYSPACES_CF layout:
- *
- * <key (AsciiType)>
- *   ascii => json_serialized_value
- *   ...
- * </key>
- *
- * Where <key> is a name of keyspace e.g. "ks".
- *
- * SCHEMA_COLUMNFAMILIES_CF layout:
- *
- * <key (AsciiType)>
- *     composite(ascii, ascii) => json_serialized_value
- * </key>
- *
- * Where <key> is a name of keyspace e.g. "ks"., first component of the column name is name of the ColumnFamily, last
- * component is the name of the ColumnFamily attribute.
- *
- * SCHEMA_COLUMNS_CF layout:
- *
- * <key (AsciiType)>
- *     composite(ascii, ascii, ascii) => json_serialized value
- * </key>
- *
- * Where <key> is a name of keyspace e.g. "ks".
- *
- * Column names where made composite to support 3-level nesting which represents following structure:
- * "ColumnFamily name":"column name":"column attribute" => "value"
- *
- * Example of schema (using CLI):
- *
- * schema_keyspaces
- * ----------------
- * RowKey: ks
- *  => (column=durable_writes, value=true, timestamp=1327061028312185000)
- *  => (column=name, value="ks", timestamp=1327061028312185000)
- *  => (column=replication_factor, value=0, timestamp=1327061028312185000)
- *  => (column=strategy_class, value="org.apache.cassandra.locator.NetworkTopologyStrategy", timestamp=1327061028312185000)
- *  => (column=strategy_options, value={"datacenter1":"1"}, timestamp=1327061028312185000)
- *
- * schema_columnfamilies
- * ---------------------
- * RowKey: ks
- *  => (column=cf:bloom_filter_fp_chance, value=0.0, timestamp=1327061105833119000)
- *  => (column=cf:caching, value="NONE", timestamp=1327061105833119000)
- *  => (column=cf:column_type, value="Standard", timestamp=1327061105833119000)
- *  => (column=cf:comment, value="ColumnFamily", timestamp=1327061105833119000)
- *  => (column=cf:default_validation_class, value="org.apache.cassandra.db.marshal.BytesType", timestamp=1327061105833119000)
- *  => (column=cf:gc_grace_seconds, value=864000, timestamp=1327061105833119000)
- *  => (column=cf:id, value=1000, timestamp=1327061105833119000)
- *  => (column=cf:key_alias, value="S0VZ", timestamp=1327061105833119000)
- *  ... part of the output omitted.
- *
- * schema_columns
- * --------------
- * RowKey: ks
- *  => (column=cf:c:index_name, value=null, timestamp=1327061105833119000)
- *  => (column=cf:c:index_options, value=null, timestamp=1327061105833119000)
- *  => (column=cf:c:index_type, value=null, timestamp=1327061105833119000)
- *  => (column=cf:c:name, value="aGVsbG8=", timestamp=1327061105833119000)
- *  => (column=cf:c:validation_class, value="org.apache.cassandra.db.marshal.AsciiType", timestamp=1327061105833119000)
- */
-public class DefsTable
-{
-    private static final Logger logger = LoggerFactory.getLogger(DefsTable.class);
-
-    /* saves keyspace definitions to system schema columnfamilies */
-    public static synchronized void save(Collection<KSMetaData> keyspaces)
-    {
-        long timestamp = System.currentTimeMillis();
-
-        for (KSMetaData ksMetaData : keyspaces)
-            ksMetaData.toSchema(timestamp).apply();
-    }
-
-    /**
-     * Load keyspace definitions for the system keyspace (system.SCHEMA_KEYSPACES_CF)
-     *
-     * @return Collection of found keyspace definitions
-     */
-    public static Collection<KSMetaData> loadFromTable()
-    {
-        List<Row> serializedSchema = SystemTable.serializedSchema(SystemTable.SCHEMA_KEYSPACES_CF);
-
-        List<KSMetaData> keyspaces = new ArrayList<KSMetaData>(serializedSchema.size());
-
-        for (Row row : serializedSchema)
-        {
-            if (Schema.invalidSchemaRow(row) || Schema.ignoredSchemaRow(row))
-                continue;
-
-            keyspaces.add(KSMetaData.fromSchema(row, serializedColumnFamilies(row.key)));
-        }
-
-        return keyspaces;
-    }
-
-    public static ByteBuffer searchComposite(String name, boolean start)
-    {
-        assert name != null;
-        ByteBuffer nameBytes = UTF8Type.instance.decompose(name);
-        int length = nameBytes.remaining();
-        byte[] bytes = new byte[2 + length + 1];
-        bytes[0] = (byte)((length >> 8) & 0xFF);
-        bytes[1] = (byte)(length & 0xFF);
-        ByteBufferUtil.arrayCopy(nameBytes, 0, bytes, 2, length);
-        bytes[bytes.length - 1] = (byte)(start ? 0 : 1);
-        return ByteBuffer.wrap(bytes);
-    }
-
-    private static Row serializedColumnFamilies(DecoratedKey ksNameKey)
-    {
-        ColumnFamilyStore cfsStore = SystemTable.schemaCFS(SystemTable.SCHEMA_COLUMNFAMILIES_CF);
-        return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey,
-                                                                                         SystemTable.SCHEMA_COLUMNFAMILIES_CF,
-                                                                                         System.currentTimeMillis())));
-    }
-
-    /**
-     * Merge remote schema in form of row mutations with local and mutate ks/cf metadata objects
-     * (which also involves fs operations on add/drop ks/cf)
-     *
-     * @param mutations the schema changes to apply
-     *
-     * @throws ConfigurationException If one of metadata attributes has invalid value
-     * @throws IOException If data was corrupted during transportation or failed to apply fs operations
-     */
-    public static synchronized void mergeSchema(Collection<RowMutation> mutations) throws ConfigurationException, IOException
-    {
-        // current state of the schema
-        Map<DecoratedKey, ColumnFamily> oldKeyspaces = SystemTable.getSchema(SystemTable.SCHEMA_KEYSPACES_CF);
-        Map<DecoratedKey, ColumnFamily> oldColumnFamilies = SystemTable.getSchema(SystemTable.SCHEMA_COLUMNFAMILIES_CF);
-
-        for (RowMutation mutation : mutations)
-            mutation.apply();
-
-        if (!StorageService.instance.isClientMode())
-            flushSchemaCFs();
-
-        Schema.instance.updateVersionAndAnnounce();
-
-        // with new data applied
-        Map<DecoratedKey, ColumnFamily> newKeyspaces = SystemTable.getSchema(SystemTable.SCHEMA_KEYSPACES_CF);
-        Map<DecoratedKey, ColumnFamily> newColumnFamilies = SystemTable.getSchema(SystemTable.SCHEMA_COLUMNFAMILIES_CF);
-
-        Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
-        mergeColumnFamilies(oldColumnFamilies, newColumnFamilies);
-
-        // it is safe to drop a keyspace only when all nested ColumnFamilies where deleted
-        for (String keyspaceToDrop : keyspacesToDrop)
-            dropKeyspace(keyspaceToDrop);
-
-    }
-
-    private static Set<String> mergeKeyspaces(Map<DecoratedKey, ColumnFamily> old, Map<DecoratedKey, ColumnFamily> updated)
-    {
-        // calculate the difference between old and new states (note that entriesOnlyLeft() will be always empty)
-        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(old, updated);
-
-        /**
-         * At first step we check if any new keyspaces were added.
-         */
-        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
-        {
-            ColumnFamily ksAttrs = entry.getValue();
-
-            // we don't care about nested ColumnFamilies here because those are going to be processed separately
-            if (!(ksAttrs.getColumnCount() == 0))
-                addKeyspace(KSMetaData.fromSchema(new Row(entry.getKey(), entry.getValue()), Collections.<CFMetaData>emptyList()));
-        }
-
-        /**
-         * At second step we check if there were any keyspaces re-created, in this context
-         * re-created means that they were previously deleted but still exist in the low-level schema as empty keys
-         */
-
-        Map<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> modifiedEntries = diff.entriesDiffering();
-
-        // instead of looping over all modified entries and skipping processed keys all the time
-        // we would rather store "left to process" items and iterate over them removing already met keys
-        List<DecoratedKey> leftToProcess = new ArrayList<DecoratedKey>(modifiedEntries.size());
-
-        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : modifiedEntries.entrySet())
-        {
-            ColumnFamily prevValue = entry.getValue().leftValue();
-            ColumnFamily newValue = entry.getValue().rightValue();
-
-            if (prevValue.getColumnCount() == 0)
-            {
-                addKeyspace(KSMetaData.fromSchema(new Row(entry.getKey(), newValue), Collections.<CFMetaData>emptyList()));
-                continue;
-            }
-
-            leftToProcess.add(entry.getKey());
-        }
-
-        if (leftToProcess.size() == 0)
-            return Collections.emptySet();
-
-        /**
-         * At final step we updating modified keyspaces and saving keyspaces drop them later
-         */
-
-        Set<String> keyspacesToDrop = new HashSet<String>();
-
-        for (DecoratedKey key : leftToProcess)
-        {
-            MapDifference.ValueDifference<ColumnFamily> valueDiff = modifiedEntries.get(key);
-
-            ColumnFamily newState = valueDiff.rightValue();
-
-            if (newState.getColumnCount() == 0)
-                keyspacesToDrop.add(AsciiType.instance.getString(key.key));
-            else
-                updateKeyspace(KSMetaData.fromSchema(new Row(key, newState), Collections.<CFMetaData>emptyList()));
-        }
-
-        return keyspacesToDrop;
-    }
-
-    private static void mergeColumnFamilies(Map<DecoratedKey, ColumnFamily> old, Map<DecoratedKey, ColumnFamily> updated)
-    {
-        // calculate the difference between old and new states (note that entriesOnlyLeft() will be always empty)
-        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(old, updated);
-
-        // check if any new Keyspaces with ColumnFamilies were added.
-        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
-        {
-            ColumnFamily cfAttrs = entry.getValue();
-
-            if (!(cfAttrs.getColumnCount() == 0))
-            {
-               Map<String, CFMetaData> cfDefs = KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), cfAttrs));
-
-                for (CFMetaData cfDef : cfDefs.values())
-                    addColumnFamily(cfDef);
-            }
-        }
-
-        // deal with modified ColumnFamilies (remember that all of the keyspace nested ColumnFamilies are put to the single row)
-        Map<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> modifiedEntries = diff.entriesDiffering();
-
-        for (DecoratedKey keyspace : modifiedEntries.keySet())
-        {
-            MapDifference.ValueDifference<ColumnFamily> valueDiff = modifiedEntries.get(keyspace);
-
-            ColumnFamily prevValue = valueDiff.leftValue(); // state before external modification
-            ColumnFamily newValue = valueDiff.rightValue(); // updated state
-
-            Row newRow = new Row(keyspace, newValue);
-
-            if (prevValue.getColumnCount() == 0) // whole keyspace was deleted and now it's re-created
-            {
-                for (CFMetaData cfm : KSMetaData.deserializeColumnFamilies(newRow).values())
-                    addColumnFamily(cfm);
-            }
-            else if (newValue.getColumnCount() == 0) // whole keyspace is deleted
-            {
-                for (CFMetaData cfm : KSMetaData.deserializeColumnFamilies(new Row(keyspace, prevValue)).values())
-                    dropColumnFamily(cfm.ksName, cfm.cfName);
-            }
-            else // has modifications in the nested ColumnFamilies, need to perform nested diff to determine what was really changed
-            {
-                String ksName = AsciiType.instance.getString(keyspace.key);
-
-                Map<String, CFMetaData> oldCfDefs = new HashMap<String, CFMetaData>();
-                for (CFMetaData cfm : Schema.instance.getKSMetaData(ksName).cfMetaData().values())
-                    oldCfDefs.put(cfm.cfName, cfm);
-
-                Map<String, CFMetaData> newCfDefs = KSMetaData.deserializeColumnFamilies(newRow);
-
-                MapDifference<String, CFMetaData> cfDefDiff = Maps.difference(oldCfDefs, newCfDefs);
-
-                for (CFMetaData cfDef : cfDefDiff.entriesOnlyOnRight().values())
-                    addColumnFamily(cfDef);
-
-                for (CFMetaData cfDef : cfDefDiff.entriesOnlyOnLeft().values())
-                    dropColumnFamily(cfDef.ksName, cfDef.cfName);
-
-                for (MapDifference.ValueDifference<CFMetaData> cfDef : cfDefDiff.entriesDiffering().values())
-                    updateColumnFamily(cfDef.rightValue());
-            }
-        }
-    }
-
-    private static void addKeyspace(KSMetaData ksm)
-    {
-        assert Schema.instance.getKSMetaData(ksm.name) == null;
-        Schema.instance.load(ksm);
-
-        if (!StorageService.instance.isClientMode())
-        {
-            Table.open(ksm.name);
-            MigrationManager.instance.notifyCreateKeyspace(ksm);
-        }
-    }
-
-    private static void addColumnFamily(CFMetaData cfm)
-    {
-        assert Schema.instance.getCFMetaData(cfm.ksName, cfm.cfName) == null;
-        KSMetaData ksm = Schema.instance.getKSMetaData(cfm.ksName);
-        ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(cfm)));
-
-        logger.info("Loading " + cfm);
-
-        Schema.instance.load(cfm);
-
-        // make sure it's init-ed w/ the old definitions first,
-        // since we're going to call initCf on the new one manually
-        Table.open(cfm.ksName);
-
-        Schema.instance.setTableDefinition(ksm);
-
-        if (!StorageService.instance.isClientMode())
-        {
-            Table.open(ksm.name).initCf(cfm.cfId, cfm.cfName, true);
-            MigrationManager.instance.notifyCreateColumnFamily(cfm);
-        }
-    }
-
-    private static void updateKeyspace(KSMetaData newState)
-    {
-        KSMetaData oldKsm = Schema.instance.getKSMetaData(newState.name);
-        assert oldKsm != null;
-        KSMetaData newKsm = KSMetaData.cloneWith(oldKsm.reloadAttributes(), oldKsm.cfMetaData().values());
-
-        Schema.instance.setTableDefinition(newKsm);
-
-        if (!StorageService.instance.isClientMode())
-        {
-            Table.open(newState.name).createReplicationStrategy(newKsm);
-            MigrationManager.instance.notifyUpdateKeyspace(newKsm);
-        }
-    }
-
-    private static void updateColumnFamily(CFMetaData newState)
-    {
-        CFMetaData cfm = Schema.instance.getCFMetaData(newState.ksName, newState.cfName);
-        assert cfm != null;
-        cfm.reload();
-
-        if (!StorageService.instance.isClientMode())
-        {
-            Table table = Table.open(cfm.ksName);
-            table.getColumnFamilyStore(cfm.cfName).reload();
-            MigrationManager.instance.notifyUpdateColumnFamily(cfm);
-        }
-    }
-
-    private static void dropKeyspace(String ksName)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
-        String snapshotName = Table.getTimestampedSnapshotName(ksName);
-
-        CompactionManager.instance.interruptCompactionFor(ksm.cfMetaData().values(), true);
-
-        // remove all cfs from the table instance.
-        for (CFMetaData cfm : ksm.cfMetaData().values())
-        {
-            ColumnFamilyStore cfs = Table.open(ksm.name).getColumnFamilyStore(cfm.cfName);
-
-            Schema.instance.purge(cfm);
-
-            if (!StorageService.instance.isClientMode())
-            {
-                if (DatabaseDescriptor.isAutoSnapshot())
-                    cfs.snapshot(snapshotName);
-                Table.open(ksm.name).dropCf(cfm.cfId);
-            }
-        }
-
-        // remove the table from the static instances.
-        Table.clear(ksm.name);
-        Schema.instance.clearTableDefinition(ksm);
-        if (!StorageService.instance.isClientMode())
-        {
-            MigrationManager.instance.notifyDropKeyspace(ksm);
-        }
-    }
-
-    private static void dropColumnFamily(String ksName, String cfName)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
-        assert ksm != null;
-        ColumnFamilyStore cfs = Table.open(ksName).getColumnFamilyStore(cfName);
-        assert cfs != null;
-
-        // reinitialize the table.
-        CFMetaData cfm = ksm.cfMetaData().get(cfName);
-
-        Schema.instance.purge(cfm);
-        Schema.instance.setTableDefinition(makeNewKeyspaceDefinition(ksm, cfm));
-
-        CompactionManager.instance.interruptCompactionFor(Arrays.asList(cfm), true);
-
-        if (!StorageService.instance.isClientMode())
-        {
-            if (DatabaseDescriptor.isAutoSnapshot())
-                cfs.snapshot(Table.getTimestampedSnapshotName(cfs.name));
-            Table.open(ksm.name).dropCf(cfm.cfId);
-            MigrationManager.instance.notifyDropColumnFamily(cfm);
-        }
-    }
-
-    private static KSMetaData makeNewKeyspaceDefinition(KSMetaData ksm, CFMetaData toExclude)
-    {
-        // clone ksm but do not include the new def
-        List<CFMetaData> newCfs = new ArrayList<CFMetaData>(ksm.cfMetaData().values());
-        newCfs.remove(toExclude);
-        assert newCfs.size() == ksm.cfMetaData().size() - 1;
-        return KSMetaData.cloneWith(ksm, newCfs);
-    }
-
-    private static void flushSchemaCFs()
-    {
-        flushSchemaCF(SystemTable.SCHEMA_KEYSPACES_CF);
-        flushSchemaCF(SystemTable.SCHEMA_COLUMNFAMILIES_CF);
-        flushSchemaCF(SystemTable.SCHEMA_COLUMNS_CF);
-    }
-
-    private static void flushSchemaCF(String cfName)
-    {
-        FBUtilities.waitOnFuture(SystemTable.schemaCFS(cfName).forceFlush());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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
new file mode 100644
index 0000000..6f35ed8
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/DefsTables.java
@@ -0,0 +1,470 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * SCHEMA_{KEYSPACES, COLUMNFAMILIES, COLUMNS}_CF are used to store Keyspace/ColumnFamily attributes to make schema
+ * load/distribution easy, it replaces old mechanism when local migrations where serialized, stored in system.Migrations
+ * and used for schema distribution.
+ *
+ * SCHEMA_KEYSPACES_CF layout:
+ *
+ * <key (AsciiType)>
+ *   ascii => json_serialized_value
+ *   ...
+ * </key>
+ *
+ * Where <key> is a name of keyspace e.g. "ks".
+ *
+ * SCHEMA_COLUMNFAMILIES_CF layout:
+ *
+ * <key (AsciiType)>
+ *     composite(ascii, ascii) => json_serialized_value
+ * </key>
+ *
+ * Where <key> is a name of keyspace e.g. "ks"., first component of the column name is name of the ColumnFamily, last
+ * component is the name of the ColumnFamily attribute.
+ *
+ * SCHEMA_COLUMNS_CF layout:
+ *
+ * <key (AsciiType)>
+ *     composite(ascii, ascii, ascii) => json_serialized value
+ * </key>
+ *
+ * Where <key> is a name of keyspace e.g. "ks".
+ *
+ * Column names where made composite to support 3-level nesting which represents following structure:
+ * "ColumnFamily name":"column name":"column attribute" => "value"
+ *
+ * Example of schema (using CLI):
+ *
+ * schema_keyspaces
+ * ----------------
+ * RowKey: ks
+ *  => (column=durable_writes, value=true, timestamp=1327061028312185000)
+ *  => (column=name, value="ks", timestamp=1327061028312185000)
+ *  => (column=replication_factor, value=0, timestamp=1327061028312185000)
+ *  => (column=strategy_class, value="org.apache.cassandra.locator.NetworkTopologyStrategy", timestamp=1327061028312185000)
+ *  => (column=strategy_options, value={"datacenter1":"1"}, timestamp=1327061028312185000)
+ *
+ * schema_columnfamilies
+ * ---------------------
+ * RowKey: ks
+ *  => (column=cf:bloom_filter_fp_chance, value=0.0, timestamp=1327061105833119000)
+ *  => (column=cf:caching, value="NONE", timestamp=1327061105833119000)
+ *  => (column=cf:column_type, value="Standard", timestamp=1327061105833119000)
+ *  => (column=cf:comment, value="ColumnFamily", timestamp=1327061105833119000)
+ *  => (column=cf:default_validation_class, value="org.apache.cassandra.db.marshal.BytesType", timestamp=1327061105833119000)
+ *  => (column=cf:gc_grace_seconds, value=864000, timestamp=1327061105833119000)
+ *  => (column=cf:id, value=1000, timestamp=1327061105833119000)
+ *  => (column=cf:key_alias, value="S0VZ", timestamp=1327061105833119000)
+ *  ... part of the output omitted.
+ *
+ * schema_columns
+ * --------------
+ * RowKey: ks
+ *  => (column=cf:c:index_name, value=null, timestamp=1327061105833119000)
+ *  => (column=cf:c:index_options, value=null, timestamp=1327061105833119000)
+ *  => (column=cf:c:index_type, value=null, timestamp=1327061105833119000)
+ *  => (column=cf:c:name, value="aGVsbG8=", timestamp=1327061105833119000)
+ *  => (column=cf:c:validation_class, value="org.apache.cassandra.db.marshal.AsciiType", timestamp=1327061105833119000)
+ */
+public class DefsTables
+{
+    private static final Logger logger = LoggerFactory.getLogger(DefsTables.class);
+
+    /* saves keyspace definitions to system schema columnfamilies */
+    public static synchronized void save(Collection<KSMetaData> keyspaces)
+    {
+        long timestamp = System.currentTimeMillis();
+
+        for (KSMetaData ksMetaData : keyspaces)
+            ksMetaData.toSchema(timestamp).apply();
+    }
+
+    /**
+     * Load keyspace definitions for the system keyspace (system.SCHEMA_KEYSPACES_CF)
+     *
+     * @return Collection of found keyspace definitions
+     */
+    public static Collection<KSMetaData> loadFromKeyspace()
+    {
+        List<Row> serializedSchema = SystemKeyspace.serializedSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF);
+
+        List<KSMetaData> keyspaces = new ArrayList<KSMetaData>(serializedSchema.size());
+
+        for (Row row : serializedSchema)
+        {
+            if (Schema.invalidSchemaRow(row) || Schema.ignoredSchemaRow(row))
+                continue;
+
+            keyspaces.add(KSMetaData.fromSchema(row, serializedColumnFamilies(row.key)));
+        }
+
+        return keyspaces;
+    }
+
+    public static ByteBuffer searchComposite(String name, boolean start)
+    {
+        assert name != null;
+        ByteBuffer nameBytes = UTF8Type.instance.decompose(name);
+        int length = nameBytes.remaining();
+        byte[] bytes = new byte[2 + length + 1];
+        bytes[0] = (byte)((length >> 8) & 0xFF);
+        bytes[1] = (byte)(length & 0xFF);
+        ByteBufferUtil.arrayCopy(nameBytes, 0, bytes, 2, length);
+        bytes[bytes.length - 1] = (byte)(start ? 0 : 1);
+        return ByteBuffer.wrap(bytes);
+    }
+
+    private static Row serializedColumnFamilies(DecoratedKey ksNameKey)
+    {
+        ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF);
+        return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey,
+                                                                                         SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF,
+                                                                                         System.currentTimeMillis())));
+    }
+
+    /**
+     * Merge remote schema in form of row mutations with local and mutate ks/cf metadata objects
+     * (which also involves fs operations on add/drop ks/cf)
+     *
+     * @param mutations the schema changes to apply
+     *
+     * @throws ConfigurationException If one of metadata attributes has invalid value
+     * @throws IOException If data was corrupted during transportation or failed to apply fs operations
+     */
+    public static synchronized void mergeSchema(Collection<RowMutation> mutations) throws ConfigurationException, IOException
+    {
+        // current state of the schema
+        Map<DecoratedKey, ColumnFamily> oldKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF);
+        Map<DecoratedKey, ColumnFamily> oldColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF);
+
+        for (RowMutation mutation : mutations)
+            mutation.apply();
+
+        if (!StorageService.instance.isClientMode())
+            flushSchemaCFs();
+
+        Schema.instance.updateVersionAndAnnounce();
+
+        // with new data applied
+        Map<DecoratedKey, ColumnFamily> newKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF);
+        Map<DecoratedKey, ColumnFamily> newColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF);
+
+        Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
+        mergeColumnFamilies(oldColumnFamilies, newColumnFamilies);
+
+        // it is safe to drop a keyspace only when all nested ColumnFamilies where deleted
+        for (String keyspaceToDrop : keyspacesToDrop)
+            dropKeyspace(keyspaceToDrop);
+
+    }
+
+    private static Set<String> mergeKeyspaces(Map<DecoratedKey, ColumnFamily> old, Map<DecoratedKey, ColumnFamily> updated)
+    {
+        // calculate the difference between old and new states (note that entriesOnlyLeft() will be always empty)
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(old, updated);
+
+        /**
+         * At first step we check if any new keyspaces were added.
+         */
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+        {
+            ColumnFamily ksAttrs = entry.getValue();
+
+            // we don't care about nested ColumnFamilies here because those are going to be processed separately
+            if (!(ksAttrs.getColumnCount() == 0))
+                addKeyspace(KSMetaData.fromSchema(new Row(entry.getKey(), entry.getValue()), Collections.<CFMetaData>emptyList()));
+        }
+
+        /**
+         * At second step we check if there were any keyspaces re-created, in this context
+         * re-created means that they were previously deleted but still exist in the low-level schema as empty keys
+         */
+
+        Map<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> modifiedEntries = diff.entriesDiffering();
+
+        // instead of looping over all modified entries and skipping processed keys all the time
+        // we would rather store "left to process" items and iterate over them removing already met keys
+        List<DecoratedKey> leftToProcess = new ArrayList<DecoratedKey>(modifiedEntries.size());
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : modifiedEntries.entrySet())
+        {
+            ColumnFamily prevValue = entry.getValue().leftValue();
+            ColumnFamily newValue = entry.getValue().rightValue();
+
+            if (prevValue.getColumnCount() == 0)
+            {
+                addKeyspace(KSMetaData.fromSchema(new Row(entry.getKey(), newValue), Collections.<CFMetaData>emptyList()));
+                continue;
+            }
+
+            leftToProcess.add(entry.getKey());
+        }
+
+        if (leftToProcess.size() == 0)
+            return Collections.emptySet();
+
+        /**
+         * At final step we updating modified keyspaces and saving keyspaces drop them later
+         */
+
+        Set<String> keyspacesToDrop = new HashSet<String>();
+
+        for (DecoratedKey key : leftToProcess)
+        {
+            MapDifference.ValueDifference<ColumnFamily> valueDiff = modifiedEntries.get(key);
+
+            ColumnFamily newState = valueDiff.rightValue();
+
+            if (newState.getColumnCount() == 0)
+                keyspacesToDrop.add(AsciiType.instance.getString(key.key));
+            else
+                updateKeyspace(KSMetaData.fromSchema(new Row(key, newState), Collections.<CFMetaData>emptyList()));
+        }
+
+        return keyspacesToDrop;
+    }
+
+    private static void mergeColumnFamilies(Map<DecoratedKey, ColumnFamily> old, Map<DecoratedKey, ColumnFamily> updated)
+    {
+        // calculate the difference between old and new states (note that entriesOnlyLeft() will be always empty)
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(old, updated);
+
+        // check if any new Keyspaces with ColumnFamilies were added.
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+        {
+            ColumnFamily cfAttrs = entry.getValue();
+
+            if (!(cfAttrs.getColumnCount() == 0))
+            {
+               Map<String, CFMetaData> cfDefs = KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), cfAttrs));
+
+                for (CFMetaData cfDef : cfDefs.values())
+                    addColumnFamily(cfDef);
+            }
+        }
+
+        // deal with modified ColumnFamilies (remember that all of the keyspace nested ColumnFamilies are put to the single row)
+        Map<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> modifiedEntries = diff.entriesDiffering();
+
+        for (DecoratedKey keyspace : modifiedEntries.keySet())
+        {
+            MapDifference.ValueDifference<ColumnFamily> valueDiff = modifiedEntries.get(keyspace);
+
+            ColumnFamily prevValue = valueDiff.leftValue(); // state before external modification
+            ColumnFamily newValue = valueDiff.rightValue(); // updated state
+
+            Row newRow = new Row(keyspace, newValue);
+
+            if (prevValue.getColumnCount() == 0) // whole keyspace was deleted and now it's re-created
+            {
+                for (CFMetaData cfm : KSMetaData.deserializeColumnFamilies(newRow).values())
+                    addColumnFamily(cfm);
+            }
+            else if (newValue.getColumnCount() == 0) // whole keyspace is deleted
+            {
+                for (CFMetaData cfm : KSMetaData.deserializeColumnFamilies(new Row(keyspace, prevValue)).values())
+                    dropColumnFamily(cfm.ksName, cfm.cfName);
+            }
+            else // has modifications in the nested ColumnFamilies, need to perform nested diff to determine what was really changed
+            {
+                String ksName = AsciiType.instance.getString(keyspace.key);
+
+                Map<String, CFMetaData> oldCfDefs = new HashMap<String, CFMetaData>();
+                for (CFMetaData cfm : Schema.instance.getKSMetaData(ksName).cfMetaData().values())
+                    oldCfDefs.put(cfm.cfName, cfm);
+
+                Map<String, CFMetaData> newCfDefs = KSMetaData.deserializeColumnFamilies(newRow);
+
+                MapDifference<String, CFMetaData> cfDefDiff = Maps.difference(oldCfDefs, newCfDefs);
+
+                for (CFMetaData cfDef : cfDefDiff.entriesOnlyOnRight().values())
+                    addColumnFamily(cfDef);
+
+                for (CFMetaData cfDef : cfDefDiff.entriesOnlyOnLeft().values())
+                    dropColumnFamily(cfDef.ksName, cfDef.cfName);
+
+                for (MapDifference.ValueDifference<CFMetaData> cfDef : cfDefDiff.entriesDiffering().values())
+                    updateColumnFamily(cfDef.rightValue());
+            }
+        }
+    }
+
+    private static void addKeyspace(KSMetaData ksm)
+    {
+        assert Schema.instance.getKSMetaData(ksm.name) == null;
+        Schema.instance.load(ksm);
+
+        if (!StorageService.instance.isClientMode())
+        {
+            Keyspace.open(ksm.name);
+            MigrationManager.instance.notifyCreateKeyspace(ksm);
+        }
+    }
+
+    private static void addColumnFamily(CFMetaData cfm)
+    {
+        assert Schema.instance.getCFMetaData(cfm.ksName, cfm.cfName) == null;
+        KSMetaData ksm = Schema.instance.getKSMetaData(cfm.ksName);
+        ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(cfm)));
+
+        logger.info("Loading " + cfm);
+
+        Schema.instance.load(cfm);
+
+        // make sure it's init-ed w/ the old definitions first,
+        // since we're going to call initCf on the new one manually
+        Keyspace.open(cfm.ksName);
+
+        Schema.instance.setKeyspaceDefinition(ksm);
+
+        if (!StorageService.instance.isClientMode())
+        {
+            Keyspace.open(ksm.name).initCf(cfm.cfId, cfm.cfName, true);
+            MigrationManager.instance.notifyCreateColumnFamily(cfm);
+        }
+    }
+
+    private static void updateKeyspace(KSMetaData newState)
+    {
+        KSMetaData oldKsm = Schema.instance.getKSMetaData(newState.name);
+        assert oldKsm != null;
+        KSMetaData newKsm = KSMetaData.cloneWith(oldKsm.reloadAttributes(), oldKsm.cfMetaData().values());
+
+        Schema.instance.setKeyspaceDefinition(newKsm);
+
+        if (!StorageService.instance.isClientMode())
+        {
+            Keyspace.open(newState.name).createReplicationStrategy(newKsm);
+            MigrationManager.instance.notifyUpdateKeyspace(newKsm);
+        }
+    }
+
+    private static void updateColumnFamily(CFMetaData newState)
+    {
+        CFMetaData cfm = Schema.instance.getCFMetaData(newState.ksName, newState.cfName);
+        assert cfm != null;
+        cfm.reload();
+
+        if (!StorageService.instance.isClientMode())
+        {
+            Keyspace keyspace = Keyspace.open(cfm.ksName);
+            keyspace.getColumnFamilyStore(cfm.cfName).reload();
+            MigrationManager.instance.notifyUpdateColumnFamily(cfm);
+        }
+    }
+
+    private static void dropKeyspace(String ksName)
+    {
+        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
+        String snapshotName = Keyspace.getTimestampedSnapshotName(ksName);
+
+        CompactionManager.instance.interruptCompactionFor(ksm.cfMetaData().values(), true);
+
+        // remove all cfs from the keyspace instance.
+        for (CFMetaData cfm : ksm.cfMetaData().values())
+        {
+            ColumnFamilyStore cfs = Keyspace.open(ksm.name).getColumnFamilyStore(cfm.cfName);
+
+            Schema.instance.purge(cfm);
+
+            if (!StorageService.instance.isClientMode())
+            {
+                if (DatabaseDescriptor.isAutoSnapshot())
+                    cfs.snapshot(snapshotName);
+                Keyspace.open(ksm.name).dropCf(cfm.cfId);
+            }
+        }
+
+        // remove the keyspace from the static instances.
+        Keyspace.clear(ksm.name);
+        Schema.instance.clearKeyspaceDefinition(ksm);
+        if (!StorageService.instance.isClientMode())
+        {
+            MigrationManager.instance.notifyDropKeyspace(ksm);
+        }
+    }
+
+    private static void dropColumnFamily(String ksName, String cfName)
+    {
+        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
+        assert ksm != null;
+        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(cfName);
+        assert cfs != null;
+
+        // reinitialize the keyspace.
+        CFMetaData cfm = ksm.cfMetaData().get(cfName);
+
+        Schema.instance.purge(cfm);
+        Schema.instance.setKeyspaceDefinition(makeNewKeyspaceDefinition(ksm, cfm));
+
+        CompactionManager.instance.interruptCompactionFor(Arrays.asList(cfm), true);
+
+        if (!StorageService.instance.isClientMode())
+        {
+            if (DatabaseDescriptor.isAutoSnapshot())
+                cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name));
+            Keyspace.open(ksm.name).dropCf(cfm.cfId);
+            MigrationManager.instance.notifyDropColumnFamily(cfm);
+        }
+    }
+
+    private static KSMetaData makeNewKeyspaceDefinition(KSMetaData ksm, CFMetaData toExclude)
+    {
+        // clone ksm but do not include the new def
+        List<CFMetaData> newCfs = new ArrayList<CFMetaData>(ksm.cfMetaData().values());
+        newCfs.remove(toExclude);
+        assert newCfs.size() == ksm.cfMetaData().size() - 1;
+        return KSMetaData.cloneWith(ksm, newCfs);
+    }
+
+    private static void flushSchemaCFs()
+    {
+        flushSchemaCF(SystemKeyspace.SCHEMA_KEYSPACES_CF);
+        flushSchemaCF(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF);
+        flushSchemaCF(SystemKeyspace.SCHEMA_COLUMNS_CF);
+    }
+
+    private static void flushSchemaCF(String cfName)
+    {
+        FBUtilities.waitOnFuture(SystemKeyspace.schemaCFS(cfName).forceFlush());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 0dd544e..5a0ac22 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -79,27 +79,27 @@ public class Directories
             dataFileLocations[i] = new DataDirectory(new File(locations[i]));
     }
 
-    private final String tablename;
+    private final String keyspacename;
     private final String cfname;
     private final File[] sstableDirectories;
 
-    public static Directories create(String tablename, String cfname)
+    public static Directories create(String keyspacename, String cfname)
     {
         int idx = cfname.indexOf(SECONDARY_INDEX_NAME_SEPARATOR);
         if (idx > 0)
             // secondary index, goes in the same directory than the base cf
-            return new Directories(tablename, cfname, cfname.substring(0, idx));
+            return new Directories(keyspacename, cfname, cfname.substring(0, idx));
         else
-            return new Directories(tablename, cfname, cfname);
+            return new Directories(keyspacename, cfname, cfname);
     }
 
-    private Directories(String tablename, String cfname, String directoryName)
+    private Directories(String keyspacename, String cfname, String directoryName)
     {
-        this.tablename = tablename;
+        this.keyspacename = keyspacename;
         this.cfname = cfname;
         this.sstableDirectories = new File[dataFileLocations.length];
         for (int i = 0; i < dataFileLocations.length; ++i)
-            sstableDirectories[i] = new File(dataFileLocations[i].location, join(tablename, directoryName));
+            sstableDirectories[i] = new File(dataFileLocations[i].location, join(keyspacename, directoryName));
 
         if (!StorageService.instance.isClientMode())
         {
@@ -362,7 +362,7 @@ public class Directories
         private FileFilter getFilter()
         {
             // Note: the prefix needs to include cfname + separator to distinguish between a cfs and it's secondary indexes
-            final String sstablePrefix = tablename + Component.separator + cfname + Component.separator;
+            final String sstablePrefix = keyspacename + Component.separator + cfname + Component.separator;
             return new FileFilter()
             {
                 // This function always return false since accepts adds to the components map

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 3a30701..9f21451 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -113,7 +113,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
                                                                                  new NamedThreadFactory("HintedHandoff", Thread.MIN_PRIORITY),
                                                                                  "internal");
 
-    private final ColumnFamilyStore hintStore = Table.open(Table.SYSTEM_KS).getColumnFamilyStore(SystemTable.HINTS_CF);
+    private final ColumnFamilyStore hintStore = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(SystemKeyspace.HINTS_CF);
 
     /**
      * Returns a mutation representing a Hint to be sent to <code>targetId</code>
@@ -126,9 +126,9 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         // serialize the hint with id and version as a composite column name
         ByteBuffer name = comparator.decompose(hintId, MessagingService.current_version);
         ByteBuffer value = ByteBuffer.wrap(FBUtilities.serialize(mutation, RowMutation.serializer, MessagingService.current_version));
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Schema.instance.getCFMetaData(Table.SYSTEM_KS, SystemTable.HINTS_CF));
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Schema.instance.getCFMetaData(Keyspace.SYSTEM_KS, SystemKeyspace.HINTS_CF));
         cf.addColumn(name, value, System.currentTimeMillis(), ttl);
-        return new RowMutation(Table.SYSTEM_KS, UUIDType.instance.decompose(targetId), cf);
+        return new RowMutation(Keyspace.SYSTEM_KS, UUIDType.instance.decompose(targetId), cf);
     }
 
     /*
@@ -171,8 +171,8 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
 
     private static void deleteHint(ByteBuffer tokenBytes, ByteBuffer columnName, long timestamp)
     {
-        RowMutation rm = new RowMutation(Table.SYSTEM_KS, tokenBytes);
-        rm.delete(SystemTable.HINTS_CF, columnName, timestamp);
+        RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, tokenBytes);
+        rm.delete(SystemKeyspace.HINTS_CF, columnName, timestamp);
         rm.applyUnsafe(); // don't bother with commitlog since we're going to flush as soon as we're done with delivery
     }
 
@@ -196,8 +196,8 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
             return;
         UUID hostId = StorageService.instance.getTokenMetadata().getHostId(endpoint);
         ByteBuffer hostIdBytes = ByteBuffer.wrap(UUIDGen.decompose(hostId));
-        final RowMutation rm = new RowMutation(Table.SYSTEM_KS, hostIdBytes);
-        rm.delete(SystemTable.HINTS_CF, System.currentTimeMillis());
+        final RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, hostIdBytes);
+        rm.delete(SystemKeyspace.HINTS_CF, System.currentTimeMillis());
 
         // execute asynchronously to avoid blocking caller (which may be processing gossip)
         Runnable runnable = new Runnable()
@@ -250,7 +250,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         }
         waited = 0;
         // then wait for the correct schema version.
-        // usually we use DD.getDefsVersion, which checks the local schema uuid as stored in the system table.
+        // usually we use DD.getDefsVersion, which checks the local schema uuid as stored in the system keyspace.
         // here we check the one in gossip instead; this serves as a canary to warn us if we introduce a bug that
         // causes the two to diverge (see CASSANDRA-2946)
         while (!gossiper.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.SCHEMA).value.equals(
@@ -328,7 +328,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         {
             long now = System.currentTimeMillis();
             QueryFilter filter = QueryFilter.getSliceFilter(epkey,
-                                                            SystemTable.HINTS_CF,
+                                                            SystemKeyspace.HINTS_CF,
                                                             startColumn,
                                                             ByteBufferUtil.EMPTY_BYTE_BUFFER,
                                                             false,
@@ -393,7 +393,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
                     Long truncatedAt = truncationTimesCache.get(cfId);
                     if (truncatedAt == null)
                     {
-                        ColumnFamilyStore cfs = Table.open(rm.getTable()).getColumnFamilyStore(cfId);
+                        ColumnFamilyStore cfs = Keyspace.open(rm.getKeyspaceName()).getColumnFamilyStore(cfId);
                         truncatedAt = cfs.getTruncationTime();
                         truncationTimesCache.put(cfId, truncatedAt);
                     }
@@ -579,8 +579,8 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
 
         try
         {
-            RangeSliceCommand cmd = new RangeSliceCommand(Table.SYSTEM_KS,
-                                                          SystemTable.HINTS_CF,
+            RangeSliceCommand cmd = new RangeSliceCommand(Keyspace.SYSTEM_KS,
+                                                          SystemKeyspace.HINTS_CF,
                                                           System.currentTimeMillis(),
                                                           predicate,
                                                           range,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/IMutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/IMutation.java b/src/java/org/apache/cassandra/db/IMutation.java
index b15b144..70bd79c 100644
--- a/src/java/org/apache/cassandra/db/IMutation.java
+++ b/src/java/org/apache/cassandra/db/IMutation.java
@@ -23,7 +23,7 @@ import java.util.UUID;
 
 public interface IMutation
 {
-    public String getTable();
+    public String getKeyspaceName();
     public Collection<UUID> getColumnFamilyIds();
     public ByteBuffer key();
     public void apply();


[02/11] Rename Table to Keyspace

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnTest.java b/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
index eccbfbc..0544da7 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
@@ -38,8 +38,8 @@ public class RemoveColumnTest extends SchemaLoader
     @Test
     public void testRemoveColumn() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java b/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
index d88ee60..add7ccd 100644
--- a/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
@@ -41,8 +41,8 @@ public class RemoveSubColumnTest extends SchemaLoader
     @Test
     public void testRemoveSubColumn() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Super1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
 
@@ -66,8 +66,8 @@ public class RemoveSubColumnTest extends SchemaLoader
     @Test
     public void testRemoveSubColumnAndContainer()
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Super1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key2");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RowCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowCacheTest.java b/test/unit/org/apache/cassandra/db/RowCacheTest.java
index 11fe86c..85302ee 100644
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@ -45,8 +45,8 @@ public class RowCacheTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open(KEYSPACE);
-        ColumnFamilyStore cachedStore  = table.getColumnFamilyStore(COLUMN_FAMILY);
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cachedStore  = keyspace.getColumnFamilyStore(COLUMN_FAMILY);
 
         // empty the row cache
         CacheService.instance.invalidateRowCache();
@@ -150,7 +150,7 @@ public class RowCacheTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        ColumnFamilyStore store = Table.open(KEYSPACE).getColumnFamilyStore(COLUMN_FAMILY);
+        ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(COLUMN_FAMILY);
 
         // empty the cache
         CacheService.instance.invalidateRowCache();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RowIterationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowIterationTest.java b/test/unit/org/apache/cassandra/db/RowIterationTest.java
index a6c0495..c083b19 100644
--- a/test/unit/org/apache/cassandra/db/RowIterationTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIterationTest.java
@@ -38,20 +38,20 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class RowIterationTest extends SchemaLoader
 {
-    public static final String TABLE1 = "Keyspace2";
+    public static final String KEYSPACE1 = "Keyspace2";
     public static final InetAddress LOCAL = FBUtilities.getBroadcastAddress();
 
     @Test
     public void testRowIteration() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open(TABLE1);
-        ColumnFamilyStore store = table.getColumnFamilyStore("Super3");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super3");
 
         final int ROWS_PER_SSTABLE = 10;
         Set<DecoratedKey> inserted = new HashSet<DecoratedKey>();
         for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
             DecoratedKey key = Util.dk(String.valueOf(i));
-            RowMutation rm = new RowMutation(TABLE1, key.key);
+            RowMutation rm = new RowMutation(KEYSPACE1, key.key);
             rm.add("Super3", CompositeType.build(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes(String.valueOf(i))), ByteBuffer.wrap(new byte[ROWS_PER_SSTABLE * 10 - i * 2]), i);
             rm.apply();
             inserted.add(key);
@@ -63,13 +63,13 @@ public class RowIterationTest extends SchemaLoader
     @Test
     public void testRowIterationDeletionTime() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open(TABLE1);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         String CF_NAME = "Standard3";
-        ColumnFamilyStore store = table.getColumnFamilyStore(CF_NAME);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_NAME);
         DecoratedKey key = Util.dk("key");
 
         // Delete row in first sstable
-        RowMutation rm = new RowMutation(TABLE1, key.key);
+        RowMutation rm = new RowMutation(KEYSPACE1, key.key);
         rm.delete(CF_NAME, 0);
         rm.add(CF_NAME, ByteBufferUtil.bytes("c"), ByteBufferUtil.bytes("values"), 0L);
         DeletionInfo delInfo1 = rm.getColumnFamilies().iterator().next().deletionInfo();
@@ -77,7 +77,7 @@ public class RowIterationTest extends SchemaLoader
         store.forceBlockingFlush();
 
         // Delete row in second sstable with higher timestamp
-        rm = new RowMutation(TABLE1, key.key);
+        rm = new RowMutation(KEYSPACE1, key.key);
         rm.delete(CF_NAME, 1);
         rm.add(CF_NAME, ByteBufferUtil.bytes("c"), ByteBufferUtil.bytes("values"), 1L);
         DeletionInfo delInfo2 = rm.getColumnFamilies().iterator().next().deletionInfo();
@@ -92,13 +92,13 @@ public class RowIterationTest extends SchemaLoader
     @Test
     public void testRowIterationDeletion() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open(TABLE1);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         String CF_NAME = "Standard3";
-        ColumnFamilyStore store = table.getColumnFamilyStore(CF_NAME);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_NAME);
         DecoratedKey key = Util.dk("key");
 
         // Delete a row in first sstable
-        RowMutation rm = new RowMutation(TABLE1, key.key);
+        RowMutation rm = new RowMutation(KEYSPACE1, key.key);
         rm.delete(CF_NAME, 0);
         rm.apply();
         store.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SerializationsTest.java b/test/unit/org/apache/cassandra/db/SerializationsTest.java
index 728a03c..119b47a 100644
--- a/test/unit/org/apache/cassandra/db/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/db/SerializationsTest.java
@@ -246,9 +246,9 @@ public class SerializationsTest extends AbstractSerializationsTester
     @Test
     public void testRowMutationRead() throws IOException
     {
-        // row mutation deserialization requires being able to look up the table in the schema,
+        // row mutation deserialization requires being able to look up the keyspace in the schema,
         // so we need to rewrite this each time.  We can go back to testing on-disk data
-        // once we pull RM.table field out.
+        // once we pull RM.keyspace field out.
         testRowMutationWrite();
 
         DataInputStream in = getInput("db.RowMutation.bin");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
new file mode 100644
index 0000000..3eebe98
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -0,0 +1,80 @@
+package org.apache.cassandra.db;
+/*
+ *
+ * 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.
+ *
+ */
+
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.dht.BytesToken;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class SystemKeyspaceTest
+{
+    @Test
+    public void testLocalTokens()
+    {
+        // Remove all existing tokens
+        Collection<Token> current = SystemKeyspace.loadTokens().asMap().get(FBUtilities.getLocalAddress());
+        if (current != null && !current.isEmpty())
+            SystemKeyspace.updateTokens(current);
+
+        List<Token> tokens = new ArrayList<Token>()
+        {{
+            for (int i = 0; i < 9; i++)
+                add(new BytesToken(ByteBufferUtil.bytes(String.format("token%d", i))));
+        }};
+
+        SystemKeyspace.updateTokens(tokens);
+        int count = 0;
+
+        for (Token tok : SystemKeyspace.getSavedTokens())
+            assert tokens.get(count++).equals(tok);
+    }
+
+    @Test
+    public void testNonLocalToken() throws UnknownHostException
+    {
+        BytesToken token = new BytesToken(ByteBufferUtil.bytes("token3"));
+        InetAddress address = InetAddress.getByName("127.0.0.2");
+        SystemKeyspace.updateTokens(address, Collections.<Token>singletonList(token));
+        assert SystemKeyspace.loadTokens().get(address).contains(token);
+        SystemKeyspace.removeEndpoint(address);
+        assert !SystemKeyspace.loadTokens().containsValue(token);
+    }
+
+    @Test
+    public void testLocalHostID()
+    {
+        UUID firstId = SystemKeyspace.getLocalHostId();
+        UUID secondId = SystemKeyspace.getLocalHostId();
+        assert firstId.equals(secondId) : String.format("%s != %s%n", firstId.toString(), secondId.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/SystemTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SystemTableTest.java b/test/unit/org/apache/cassandra/db/SystemTableTest.java
deleted file mode 100644
index b202173..0000000
--- a/test/unit/org/apache/cassandra/db/SystemTableTest.java
+++ /dev/null
@@ -1,81 +0,0 @@
-package org.apache.cassandra.db;
-/*
- *
- * 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.
- *
- */
-
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Collection;
-import java.util.List;
-import java.util.UUID;
-
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.dht.BytesToken;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-
-public class SystemTableTest
-{
-    @Test
-    public void testLocalTokens()
-    {
-        // Remove all existing tokens
-        Collection<Token> current = SystemTable.loadTokens().asMap().get(FBUtilities.getLocalAddress());
-        if (current != null && !current.isEmpty())
-            SystemTable.updateTokens(current);
-
-        List<Token> tokens = new ArrayList<Token>()
-        {{
-            for (int i = 0; i < 9; i++)
-                add(new BytesToken(ByteBufferUtil.bytes(String.format("token%d", i))));
-        }};
-
-        SystemTable.updateTokens(tokens);
-        int count = 0;
-
-        for (Token tok : SystemTable.getSavedTokens())
-            assert tokens.get(count++).equals(tok);
-    }
-
-    @Test
-    public void testNonLocalToken() throws UnknownHostException
-    {
-        BytesToken token = new BytesToken(ByteBufferUtil.bytes("token3"));
-        InetAddress address = InetAddress.getByName("127.0.0.2");
-        SystemTable.updateTokens(address, Collections.<Token>singletonList(token));
-        assert SystemTable.loadTokens().get(address).contains(token);
-        SystemTable.removeEndpoint(address);
-        assert !SystemTable.loadTokens().containsValue(token);
-    }
-
-    @Test
-    public void testLocalHostID()
-    {
-        UUID firstId = SystemTable.getLocalHostId();
-        UUID secondId = SystemTable.getLocalHostId();
-        assert firstId.equals(secondId) : String.format("%s != %s%n", firstId.toString(), secondId.toString());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/TableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TableTest.java b/test/unit/org/apache/cassandra/db/TableTest.java
deleted file mode 100644
index 0c3b44d..0000000
--- a/test/unit/org/apache/cassandra/db/TableTest.java
+++ /dev/null
@@ -1,653 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.db;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
-import java.text.DecimalFormat;
-import java.text.NumberFormat;
-import java.util.*;
-import java.io.IOException;
-import java.util.concurrent.ExecutionException;
-
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-
-import org.apache.commons.lang.StringUtils;
-import org.junit.Test;
-
-import static junit.framework.Assert.*;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.Relation;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.CompositeType;
-import org.apache.cassandra.db.marshal.IntegerType;
-import org.apache.cassandra.utils.WrappedRunnable;
-import static org.apache.cassandra.Util.column;
-import static org.apache.cassandra.Util.expiringColumn;
-import static org.apache.cassandra.Util.getBytes;
-import static org.junit.Assert.assertEquals;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-
-public class TableTest extends SchemaLoader
-{
-    private static final DecoratedKey TEST_KEY = Util.dk("key1");
-    private static final DecoratedKey TEST_SLICE_KEY = Util.dk("key1-slicerange");
-
-    public static void reTest(ColumnFamilyStore cfs, Runnable verify) throws Exception
-    {
-        verify.run();
-        cfs.forceBlockingFlush();
-        verify.run();
-    }
-
-    @Test
-    public void testGetRowNoColumns() throws Throwable
-    {
-        final Table table = Table.open("Keyspace2");
-        final ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard3");
-
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace2", "Standard3");
-        cf.addColumn(column("col1","val1", 1L));
-        RowMutation rm = new RowMutation("Keyspace2", TEST_KEY.key, cf);
-        rm.apply();
-
-        Runnable verify = new WrappedRunnable()
-        {
-            public void runMayThrow() throws Exception
-            {
-                ColumnFamily cf;
-
-                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY,
-                                                                        "Standard3",
-                                                                        new TreeSet<ByteBuffer>(),
-                                                                        System.currentTimeMillis()));
-                assertColumns(cf);
-
-                cf = cfStore.getColumnFamily(QueryFilter.getSliceFilter(TEST_KEY,
-                                                                        "Standard3",
-                                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                        false,
-                                                                        0,
-                                                                        System.currentTimeMillis()));
-                assertColumns(cf);
-
-                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY,
-                                                                        "Standard3",
-                                                                        ByteBufferUtil.bytes("col99"),
-                                                                        System.currentTimeMillis()));
-                assertColumns(cf);
-            }
-        };
-        reTest(table.getColumnFamilyStore("Standard3"), verify);
-    }
-
-    @Test
-    public void testGetRowSingleColumn() throws Throwable
-    {
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard1");
-
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        cf.addColumn(column("col1","val1", 1L));
-        cf.addColumn(column("col2","val2", 1L));
-        cf.addColumn(column("col3","val3", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", TEST_KEY.key, cf);
-        rm.apply();
-
-        Runnable verify = new WrappedRunnable()
-        {
-            public void runMayThrow() throws Exception
-            {
-                ColumnFamily cf;
-
-                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY,
-                                                                        "Standard1",
-                                                                        ByteBufferUtil.bytes("col1"),
-                                                                        System.currentTimeMillis()));
-                assertColumns(cf, "col1");
-
-                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY,
-                                                                        "Standard1",
-                                                                        ByteBufferUtil.bytes("col3"),
-                                                                        System.currentTimeMillis()));
-                assertColumns(cf, "col3");
-            }
-        };
-        reTest(table.getColumnFamilyStore("Standard1"), verify);
-    }
-
-    @Test
-    public void testGetRowSliceByRange() throws Throwable
-    {
-    	DecoratedKey key = TEST_SLICE_KEY;
-    	Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard1");
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        // First write "a", "b", "c"
-        cf.addColumn(column("a", "val1", 1L));
-        cf.addColumn(column("b", "val2", 1L));
-        cf.addColumn(column("c", "val3", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", key.key, cf);
-        rm.apply();
-
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("c"), false, 100, System.currentTimeMillis());
-        assertEquals(2, cf.getColumnCount());
-
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("b"), false, 100, System.currentTimeMillis());
-        assertEquals(1, cf.getColumnCount());
-
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("c"), false, 1, System.currentTimeMillis());
-        assertEquals(1, cf.getColumnCount());
-    }
-
-    @Test
-    public void testGetSliceNoMatch() throws Throwable
-    {
-        Table table = Table.open("Keyspace1");
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard2");
-        cf.addColumn(column("col1", "val1", 1));
-        RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("row1000"), cf);
-        rm.apply();
-
-        validateGetSliceNoMatch(table);
-        table.getColumnFamilyStore("Standard2").forceBlockingFlush();
-        validateGetSliceNoMatch(table);
-
-        Collection<SSTableReader> ssTables = table.getColumnFamilyStore("Standard2").getSSTables();
-        assertEquals(1, ssTables.size());
-        ssTables.iterator().next().forceFilterFailures();
-        validateGetSliceNoMatch(table);
-    }
-
-    @Test
-    public void testGetSliceWithCutoff() throws Throwable
-    {
-        // tests slicing against data from one row in a memtable and then flushed to an sstable
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard1");
-        final DecoratedKey ROW = Util.dk("row4");
-        final NumberFormat fmt = new DecimalFormat("000");
-
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        // at this rate, we're getting 78-79 cos/block, assuming the blocks are set to be about 4k.
-        // so if we go to 300, we'll get at least 4 blocks, which is plenty for testing.
-        for (int i = 0; i < 300; i++)
-            cf.addColumn(column("col" + fmt.format(i), "omg!thisisthevalue!"+i, 1L));
-        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
-        rm.apply();
-
-        Runnable verify = new WrappedRunnable()
-        {
-            public void runMayThrow() throws Exception
-            {
-                ColumnFamily cf;
-
-                // blocks are partitioned like this: 000-097, 098-193, 194-289, 290-299, assuming a 4k column index size.
-                assert DatabaseDescriptor.getColumnIndexSize() == 4096 : "Unexpected column index size, block boundaries won't be where tests expect them.";
-
-                // test forward, spanning a segment.
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col096"), ByteBufferUtil.bytes("col099"), false, 4, System.currentTimeMillis());
-                assertColumns(cf, "col096", "col097", "col098", "col099");
-
-                // test reversed, spanning a segment.
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col099"), ByteBufferUtil.bytes("col096"), true, 4, System.currentTimeMillis());
-                assertColumns(cf, "col096", "col097", "col098", "col099");
-
-                // test forward, within a segment.
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col100"), ByteBufferUtil.bytes("col103"), false, 4, System.currentTimeMillis());
-                assertColumns(cf, "col100", "col101", "col102", "col103");
-
-                // test reversed, within a segment.
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col103"), ByteBufferUtil.bytes("col100"), true, 4, System.currentTimeMillis());
-                assertColumns(cf, "col100", "col101", "col102", "col103");
-
-                // test forward from beginning, spanning a segment.
-                String[] strCols = new String[100]; // col000-col099
-                for (int i = 0; i < 100; i++)
-                    strCols[i] = "col" + fmt.format(i);
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.bytes("col099"), false, 100, System.currentTimeMillis());
-                assertColumns(cf, strCols);
-
-                // test reversed, from end, spanning a segment.
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.bytes("col288"), true, 12, System.currentTimeMillis());
-                assertColumns(cf, "col288", "col289", "col290", "col291", "col292", "col293", "col294", "col295", "col296", "col297", "col298", "col299");
-            }
-        };
-
-        reTest(table.getColumnFamilyStore("Standard1"), verify);
-    }
-
-    @Test
-    public void testReversedWithFlushing() throws IOException, ExecutionException, InterruptedException
-    {
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfs = table.getColumnFamilyStore("StandardLong1");
-        final DecoratedKey ROW = Util.dk("row4");
-
-        for (int i = 0; i < 10; i++)
-        {
-            ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
-            cf.addColumn(new Column(ByteBufferUtil.bytes((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
-            RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
-            rm.apply();
-        }
-
-        cfs.forceBlockingFlush();
-
-        for (int i = 10; i < 20; i++)
-        {
-            ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
-            cf.addColumn(new Column(ByteBufferUtil.bytes((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
-            RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
-            rm.apply();
-
-            cf = cfs.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 1, System.currentTimeMillis());
-            assertEquals(1, Iterables.size(cf.getColumnNames()));
-            assertEquals(i, cf.getColumnNames().iterator().next().getLong());
-        }
-    }
-
-    private void validateGetSliceNoMatch(Table table) throws IOException
-    {
-        ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard2");
-        ColumnFamily cf;
-
-        // key before the rows that exists
-        cf = cfStore.getColumnFamily(Util.dk("a"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1, System.currentTimeMillis());
-        assertColumns(cf);
-
-        // key after the rows that exist
-        cf = cfStore.getColumnFamily(Util.dk("z"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1, System.currentTimeMillis());
-        assertColumns(cf);
-    }
-
-    @Test
-    public void testGetSliceFromBasic() throws Throwable
-    {
-        // tests slicing against data from one row in a memtable and then flushed to an sstable
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard1");
-        final DecoratedKey ROW = Util.dk("row1");
-
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        cf.addColumn(column("col1", "val1", 1L));
-        cf.addColumn(column("col3", "val3", 1L));
-        cf.addColumn(column("col4", "val4", 1L));
-        cf.addColumn(column("col5", "val5", 1L));
-        cf.addColumn(column("col7", "val7", 1L));
-        cf.addColumn(column("col9", "val9", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
-        rm.apply();
-
-        rm = new RowMutation("Keyspace1", ROW.key);
-        rm.delete("Standard1", ByteBufferUtil.bytes("col4"), 2L);
-        rm.apply();
-
-        Runnable verify = new WrappedRunnable()
-        {
-            public void runMayThrow() throws Exception
-            {
-                ColumnFamily cf;
-
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col5"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2, System.currentTimeMillis());
-                assertColumns(cf, "col5", "col7");
-
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col4"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2, System.currentTimeMillis());
-                assertColumns(cf, "col4", "col5", "col7");
-                assertColumns(ColumnFamilyStore.removeDeleted(cf, Integer.MAX_VALUE), "col5", "col7");
-
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col5"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2, System.currentTimeMillis());
-                assertColumns(cf, "col3", "col4", "col5");
-
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col6"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2, System.currentTimeMillis());
-                assertColumns(cf, "col3", "col4", "col5");
-
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2, System.currentTimeMillis());
-                assertColumns(cf, "col7", "col9");
-
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col95"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2, System.currentTimeMillis());
-                assertColumns(cf);
-
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2, System.currentTimeMillis());
-                assertColumns(cf);
-            }
-        };
-
-        reTest(table.getColumnFamilyStore("Standard1"), verify);
-    }
-
-    @Test
-    public void testGetSliceWithExpiration() throws Throwable
-    {
-        // tests slicing against data from one row with expiring column in a memtable and then flushed to an sstable
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard1");
-        final DecoratedKey ROW = Util.dk("row5");
-
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        cf.addColumn(column("col1", "val1", 1L));
-        cf.addColumn(expiringColumn("col2", "val2", 1L, 60)); // long enough not to be tombstoned
-        cf.addColumn(column("col3", "val3", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
-        rm.apply();
-
-        Runnable verify = new WrappedRunnable()
-        {
-            public void runMayThrow() throws Exception
-            {
-                ColumnFamily cf;
-
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2, System.currentTimeMillis());
-                assertColumns(cf, "col1", "col2");
-                assertColumns(ColumnFamilyStore.removeDeleted(cf, Integer.MAX_VALUE), "col1");
-
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1, System.currentTimeMillis());
-                assertColumns(cf, "col2");
-                assertColumns(ColumnFamilyStore.removeDeleted(cf, Integer.MAX_VALUE));
-            }
-        };
-
-        reTest(table.getColumnFamilyStore("Standard1"), verify);
-    }
-
-    @Test
-    public void testGetSliceFromAdvanced() throws Throwable
-    {
-        // tests slicing against data from one row spread across two sstables
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard1");
-        final DecoratedKey ROW = Util.dk("row2");
-
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        cf.addColumn(column("col1", "val1", 1L));
-        cf.addColumn(column("col2", "val2", 1L));
-        cf.addColumn(column("col3", "val3", 1L));
-        cf.addColumn(column("col4", "val4", 1L));
-        cf.addColumn(column("col5", "val5", 1L));
-        cf.addColumn(column("col6", "val6", 1L));
-        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
-        rm.apply();
-        cfStore.forceBlockingFlush();
-
-        cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        cf.addColumn(column("col1", "valx", 2L));
-        cf.addColumn(column("col2", "valx", 2L));
-        cf.addColumn(column("col3", "valx", 2L));
-        rm = new RowMutation("Keyspace1", ROW.key, cf);
-        rm.apply();
-
-        Runnable verify = new WrappedRunnable()
-        {
-            public void runMayThrow() throws Exception
-            {
-                ColumnFamily cf;
-
-                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3, System.currentTimeMillis());
-                assertColumns(cf, "col2", "col3", "col4");
-
-                ByteBuffer col = cf.getColumn(ByteBufferUtil.bytes("col2")).value();
-                assertEquals(ByteBufferUtil.string(col), "valx");
-
-                col = cf.getColumn(ByteBufferUtil.bytes("col3")).value();
-                assertEquals(ByteBufferUtil.string(col), "valx");
-
-                col = cf.getColumn(ByteBufferUtil.bytes("col4")).value();
-                assertEquals(ByteBufferUtil.string(col), "val4");
-            }
-        };
-
-        reTest(table.getColumnFamilyStore("Standard1"), verify);
-    }
-
-    @Test
-    public void testGetSliceFromLarge() throws Throwable
-    {
-        // tests slicing against 1000 columns in an sstable
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard1");
-        DecoratedKey key = Util.dk("row3");
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        for (int i = 1000; i < 2000; i++)
-            cf.addColumn(column("col" + i, ("v" + i), 1L));
-        RowMutation rm = new RowMutation("Keyspace1", key.key, cf);
-        rm.apply();
-        cfStore.forceBlockingFlush();
-
-        validateSliceLarge(cfStore);
-
-        // compact so we have a big row with more than the minimum index count
-        if (cfStore.getSSTables().size() > 1)
-        {
-            CompactionManager.instance.performMaximal(cfStore);
-        }
-        // verify that we do indeed have multiple index entries
-        SSTableReader sstable = cfStore.getSSTables().iterator().next();
-        RowIndexEntry indexEntry = sstable.getPosition(key, SSTableReader.Operator.EQ);
-        assert indexEntry.columnsIndex().size() > 2;
-
-        validateSliceLarge(cfStore);
-    }
-
-    @Test
-    public void testLimitSSTables() throws CharacterCodingException, InterruptedException
-    {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard1");
-        cfStore.disableAutoCompaction();
-        DecoratedKey key = Util.dk("row_maxmin");
-        for (int j = 0; j < 10; j++)
-        {
-            ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-            for (int i = 1000 + (j*100); i < 1000 + ((j+1)*100); i++)
-            {
-                cf.addColumn(column("col" + i, ("v" + i), i));
-            }
-            RowMutation rm = new RowMutation("Keyspace1", key.key, cf);
-            rm.apply();
-            cfStore.forceBlockingFlush();
-        }
-        cfStore.metric.sstablesPerReadHistogram.clear();
-        ColumnFamily cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes(""), ByteBufferUtil.bytes("col1499"), false, 1000, System.currentTimeMillis());
-        assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
-        int i = 0;
-        for (Column c : cf.getSortedColumns())
-        {
-            assertEquals(ByteBufferUtil.string(c.name), "col" + (1000 + i++));
-        }
-        assertEquals(i, 500);
-        cfStore.metric.sstablesPerReadHistogram.clear();
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1500"), ByteBufferUtil.bytes("col2000"), false, 1000, System.currentTimeMillis());
-        assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
-
-        for (Column c : cf.getSortedColumns())
-        {
-            assertEquals(ByteBufferUtil.string(c.name), "col"+(1000 + i++));
-        }
-        assertEquals(i, 1000);
-
-        // reverse
-        cfStore.metric.sstablesPerReadHistogram.clear();
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col2000"), ByteBufferUtil.bytes("col1500"), true, 1000, System.currentTimeMillis());
-        assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
-        i = 500;
-        for (Column c : cf.getSortedColumns())
-        {
-            assertEquals(ByteBufferUtil.string(c.name), "col"+(1000 + i++));
-        }
-        assertEquals(i, 1000);
-
-    }
-
-    @Test
-    public void testLimitSSTablesComposites() throws CharacterCodingException, ExecutionException, InterruptedException
-    {
-        /*
-        creates 10 sstables, composite columns like this:
-        ---------------------
-        k   |a0:0|a1:1|..|a9:9
-        ---------------------
-        ---------------------
-        k   |a0:10|a1:11|..|a9:19
-        ---------------------
-        ...
-        ---------------------
-        k   |a0:90|a1:91|..|a9:99
-        ---------------------
-        then we slice out col1 = a5 and col2 > 85 -> which should let us just check 2 sstables and get 2 columns
-         */
-        Table table = Table.open("Keyspace1");
-
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("StandardComposite2");
-        cfs.disableAutoCompaction();
-
-        CompositeType ct = CompositeType.getInstance(BytesType.instance, IntegerType.instance);
-        DecoratedKey key = Util.dk("k");
-        for (int j = 0; j < 10; j++)
-        {
-            for (int i = 0; i < 10; i++)
-            {
-                RowMutation rm = new RowMutation("Keyspace1", key.key);
-                ByteBuffer colName = ct.builder().add(ByteBufferUtil.bytes("a" + i)).add(ByteBufferUtil.bytes(j*10 + i)).build();
-                rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-                rm.apply();
-            }
-            cfs.forceBlockingFlush();
-        }
-        ByteBuffer start = ct.builder().add(ByteBufferUtil.bytes("a5")).add(ByteBufferUtil.bytes(85)).build();
-        ByteBuffer finish = ct.builder().add(ByteBufferUtil.bytes("a5")).buildAsEndOfRange();
-        cfs.metric.sstablesPerReadHistogram.clear();
-        ColumnFamily cf = cfs.getColumnFamily(key, start, finish, false, 1000, System.currentTimeMillis());
-        int colCount = 0;
-        for (Column c : cf)
-            colCount++;
-        assertEquals(2, colCount);
-        assertEquals(2, cfs.metric.sstablesPerReadHistogram.max(), 0.1);
-    }
-
-    private void validateSliceLarge(ColumnFamilyStore cfStore) throws IOException
-    {
-        DecoratedKey key = Util.dk("row3");
-        ColumnFamily cf;
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3, System.currentTimeMillis());
-        assertColumns(cf, "col1000", "col1001", "col1002");
-
-        ByteBuffer col;
-        col = cf.getColumn(ByteBufferUtil.bytes("col1000")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1000");
-        col = cf.getColumn(ByteBufferUtil.bytes("col1001")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1001");
-        col = cf.getColumn(ByteBufferUtil.bytes("col1002")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1002");
-
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1195"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3, System.currentTimeMillis());
-        assertColumns(cf, "col1195", "col1196", "col1197");
-
-        col = cf.getColumn(ByteBufferUtil.bytes("col1195")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1195");
-        col = cf.getColumn(ByteBufferUtil.bytes("col1196")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1196");
-        col = cf.getColumn(ByteBufferUtil.bytes("col1197")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1197");
-
-
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1996"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 1000, System.currentTimeMillis());
-        Column[] columns = cf.getSortedColumns().toArray(new Column[0]);
-        for (int i = 1000; i < 1996; i++)
-        {
-            String expectedName = "col" + i;
-            Column column = columns[i - 1000];
-            assertEquals(ByteBufferUtil.string(column.name()), expectedName);
-            assertEquals(ByteBufferUtil.string(column.value()), ("v" + i));
-        }
-
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1990"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3, System.currentTimeMillis());
-        assertColumns(cf, "col1990", "col1991", "col1992");
-        col = cf.getColumn(ByteBufferUtil.bytes("col1990")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1990");
-        col = cf.getColumn(ByteBufferUtil.bytes("col1991")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1991");
-        col = cf.getColumn(ByteBufferUtil.bytes("col1992")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1992");
-
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 3, System.currentTimeMillis());
-        assertColumns(cf, "col1997", "col1998", "col1999");
-        col = cf.getColumn(ByteBufferUtil.bytes("col1997")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1997");
-        col = cf.getColumn(ByteBufferUtil.bytes("col1998")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1998");
-        col = cf.getColumn(ByteBufferUtil.bytes("col1999")).value();
-        assertEquals(ByteBufferUtil.string(col), "v1999");
-
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col9000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 3, System.currentTimeMillis());
-        assertColumns(cf, "col1997", "col1998", "col1999");
-
-        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col9000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3, System.currentTimeMillis());
-        assertColumns(cf);
-    }
-
-    public static void assertColumns(ColumnFamily container, String... columnNames)
-    {
-        Collection<Column> columns = container == null ? new TreeSet<Column>() : container.getSortedColumns();
-        List<String> L = new ArrayList<String>();
-        for (Column column : columns)
-        {
-            try
-            {
-                L.add(ByteBufferUtil.string(column.name()));
-            }
-            catch (CharacterCodingException e)
-            {
-                throw new AssertionError(e);
-            }
-        }
-
-        List<String> names = new ArrayList<String>(columnNames.length);
-
-        names.addAll(Arrays.asList(columnNames));
-
-        String[] columnNames1 = names.toArray(new String[0]);
-        String[] la = L.toArray(new String[columns.size()]);
-
-        assert Arrays.equals(la, columnNames1)
-                : String.format("Columns [%s])] is not expected [%s]",
-                                ((container == null) ? "" : container.getComparator().getColumnsString(columns)),
-                                StringUtils.join(columnNames1, ","));
-    }
-
-    public static void assertColumn(ColumnFamily cf, String name, String value, long timestamp)
-    {
-        assertColumn(cf.getColumn(ByteBufferUtil.bytes(name)), value, timestamp);
-    }
-
-    public static void assertColumn(Column column, String value, long timestamp)
-    {
-        assertNotNull(column);
-        assertEquals(0, ByteBufferUtil.compareUnsigned(column.value(), ByteBufferUtil.bytes(value)));
-        assertEquals(timestamp, column.timestamp());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/TimeSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TimeSortTest.java b/test/unit/org/apache/cassandra/db/TimeSortTest.java
index 3ca996b..37966de 100644
--- a/test/unit/org/apache/cassandra/db/TimeSortTest.java
+++ b/test/unit/org/apache/cassandra/db/TimeSortTest.java
@@ -40,8 +40,8 @@ public class TimeSortTest extends SchemaLoader
     @Test
     public void testMixedSources() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfStore = table.getColumnFamilyStore("StandardLong1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("StandardLong1");
         RowMutation rm;
         DecoratedKey key = Util.dk("key0");
 
@@ -62,8 +62,8 @@ public class TimeSortTest extends SchemaLoader
     @Test
     public void testTimeSort() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfStore = table.getColumnFamilyStore("StandardLong1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("StandardLong1");
 
         for (int i = 900; i < 1000; ++i)
         {
@@ -75,10 +75,10 @@ public class TimeSortTest extends SchemaLoader
             rm.apply();
         }
 
-        validateTimeSort(table);
+        validateTimeSort(keyspace);
 
         cfStore.forceBlockingFlush();
-        validateTimeSort(table);
+        validateTimeSort(keyspace);
 
         // interleave some new data to test memtable + sstable
         DecoratedKey key = Util.dk("900");
@@ -113,19 +113,19 @@ public class TimeSortTest extends SchemaLoader
         assert "c".equals(ByteBufferUtil.string(cf.getColumn(getBytes(10)).value()));
     }
 
-    private void validateTimeSort(Table table) throws IOException
+    private void validateTimeSort(Keyspace keyspace) throws IOException
     {
         for (int i = 900; i < 1000; ++i)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
             for (int j = 0; j < 8; j += 3)
             {
-                ColumnFamily cf = table.getColumnFamilyStore("StandardLong1").getColumnFamily(key,
-                                                                                              getBytes(j * 2),
-                                                                                              ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                                              false,
-                                                                                              1000,
-                                                                                              System.currentTimeMillis());
+                ColumnFamily cf = keyspace.getColumnFamilyStore("StandardLong1").getColumnFamily(key,
+                                                                                                 getBytes(j * 2),
+                                                                                                 ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                                                                 false,
+                                                                                                 1000,
+                                                                                                 System.currentTimeMillis());
                 Collection<Column> columns = cf.getSortedColumns();
                 assert columns.size() == 8 - j;
                 int k = j;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
index 6fcc890..0b4c78d 100644
--- a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
@@ -34,7 +34,7 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -72,8 +72,8 @@ public class BlacklistingCompactionsTest extends SchemaLoader
     public void testBlacklisting(String compactionStrategy) throws Exception
     {
         // this test does enough rows to force multiple block indexes to be used
-        Table table = Table.open(KEYSPACE);
-        final ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         final int ROWS_PER_SSTABLE = 10;
         final int SSTABLES = cfs.metadata.getIndexInterval() * 2 / ROWS_PER_SSTABLE;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
index 3ac6418..54c9e4f 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
@@ -28,7 +28,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.Column;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowMutation;
 import org.apache.cassandra.db.ColumnFamily;
@@ -38,29 +38,29 @@ import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.Util;
 
 import static junit.framework.Assert.assertEquals;
-import static org.apache.cassandra.db.TableTest.assertColumns;
+import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
 public class CompactionsPurgeTest extends SchemaLoader
 {
-    public static final String TABLE1 = "Keyspace1";
-    public static final String TABLE2 = "Keyspace2";
+    public static final String KEYSPACE1 = "Keyspace1";
+    public static final String KEYSPACE2 = "Keyspace2";
 
     @Test
     public void testMajorCompactionPurge() throws IOException, ExecutionException, InterruptedException
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open(TABLE1);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         String cfName = "Standard1";
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 
         DecoratedKey key = Util.dk("key1");
         RowMutation rm;
 
         // inserts
-        rm = new RowMutation(TABLE1, key.key);
+        rm = new RowMutation(KEYSPACE1, key.key);
         for (int i = 0; i < 10; i++)
         {
             rm.add(cfName, ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
@@ -71,14 +71,14 @@ public class CompactionsPurgeTest extends SchemaLoader
         // deletes
         for (int i = 0; i < 10; i++)
         {
-            rm = new RowMutation(TABLE1, key.key);
+            rm = new RowMutation(KEYSPACE1, key.key);
             rm.delete(cfName, ByteBufferUtil.bytes(String.valueOf(i)), 1);
             rm.apply();
         }
         cfs.forceBlockingFlush();
 
         // resurrect one column
-        rm = new RowMutation(TABLE1, key.key);
+        rm = new RowMutation(KEYSPACE1, key.key);
         rm.add(cfName, ByteBufferUtil.bytes(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
         rm.apply();
         cfs.forceBlockingFlush();
@@ -96,16 +96,16 @@ public class CompactionsPurgeTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open(TABLE2);
+        Keyspace keyspace = Keyspace.open(KEYSPACE2);
         String cfName = "Standard1";
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 
         RowMutation rm;
         for (int k = 1; k <= 2; ++k) {
             DecoratedKey key = Util.dk("key" + k);
 
             // inserts
-            rm = new RowMutation(TABLE2, key.key);
+            rm = new RowMutation(KEYSPACE2, key.key);
             for (int i = 0; i < 10; i++)
             {
                 rm.add(cfName, ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
@@ -116,7 +116,7 @@ public class CompactionsPurgeTest extends SchemaLoader
             // deletes
             for (int i = 0; i < 10; i++)
             {
-                rm = new RowMutation(TABLE2, key.key);
+                rm = new RowMutation(KEYSPACE2, key.key);
                 rm.delete(cfName, ByteBufferUtil.bytes(String.valueOf(i)), 1);
                 rm.apply();
             }
@@ -130,7 +130,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         // for first key. Then submit minor compaction on remembered sstables.
         cfs.forceBlockingFlush();
         Collection<SSTableReader> sstablesIncomplete = cfs.getSSTables();
-        rm = new RowMutation(TABLE2, key1.key);
+        rm = new RowMutation(KEYSPACE2, key1.key);
         rm.add(cfName, ByteBufferUtil.bytes(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
         rm.apply();
         cfs.forceBlockingFlush();
@@ -152,27 +152,27 @@ public class CompactionsPurgeTest extends SchemaLoader
     {
         // verify that we don't drop tombstones during a minor compaction that might still be relevant
         CompactionManager.instance.disableAutoCompaction();
-        Table table = Table.open(TABLE2);
+        Keyspace keyspace = Keyspace.open(KEYSPACE2);
         String cfName = "Standard1";
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 
         RowMutation rm;
         DecoratedKey key3 = Util.dk("key3");
         // inserts
-        rm = new RowMutation(TABLE2, key3.key);
+        rm = new RowMutation(KEYSPACE2, key3.key);
         rm.add(cfName, ByteBufferUtil.bytes("c1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 8);
         rm.add(cfName, ByteBufferUtil.bytes("c2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 8);
         rm.apply();
         cfs.forceBlockingFlush();
         // deletes
-        rm = new RowMutation(TABLE2, key3.key);
+        rm = new RowMutation(KEYSPACE2, key3.key);
         rm.delete(cfName, ByteBufferUtil.bytes("c1"), 10);
         rm.apply();
         cfs.forceBlockingFlush();
         Collection<SSTableReader> sstablesIncomplete = cfs.getSSTables();
 
         // delete so we have new delete in a diffrent SST.
-        rm = new RowMutation(TABLE2, key3.key);
+        rm = new RowMutation(KEYSPACE2, key3.key);
         rm.delete(cfName, ByteBufferUtil.bytes("c2"), 9);
         rm.apply();
         cfs.forceBlockingFlush();
@@ -190,15 +190,15 @@ public class CompactionsPurgeTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open(TABLE1);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         String cfName = "Standard2";
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 
         DecoratedKey key = Util.dk("key1");
         RowMutation rm;
 
         // inserts
-        rm = new RowMutation(TABLE1, key.key);
+        rm = new RowMutation(KEYSPACE1, key.key);
         for (int i = 0; i < 5; i++)
         {
             rm.add(cfName, ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
@@ -208,7 +208,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         // deletes
         for (int i = 0; i < 5; i++)
         {
-            rm = new RowMutation(TABLE1, key.key);
+            rm = new RowMutation(KEYSPACE1, key.key);
             rm.delete(cfName, ByteBufferUtil.bytes(String.valueOf(i)), 1);
             rm.apply();
         }
@@ -218,7 +218,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         // compact and test that the row is completely gone
         Util.compactAll(cfs).get();
         assert cfs.getSSTables().isEmpty();
-        ColumnFamily cf = table.getColumnFamilyStore(cfName).getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
+        ColumnFamily cf = keyspace.getColumnFamilyStore(cfName).getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
         assert cf == null : cf;
     }
 
@@ -227,16 +227,16 @@ public class CompactionsPurgeTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        String tableName = "RowCacheSpace";
+        String keyspaceName = "RowCacheSpace";
         String cfName = "CachedCF";
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 
         DecoratedKey key = Util.dk("key3");
         RowMutation rm;
 
         // inserts
-        rm = new RowMutation(tableName, key.key);
+        rm = new RowMutation(keyspaceName, key.key);
         for (int i = 0; i < 10; i++)
         {
             rm.add(cfName, ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
@@ -247,7 +247,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
 
         // deletes row
-        rm = new RowMutation(tableName, key.key);
+        rm = new RowMutation(keyspaceName, key.key);
         rm.delete(cfName, 1);
         rm.apply();
 
@@ -256,7 +256,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         Util.compactAll(cfs).get();
 
         // re-inserts with timestamp lower than delete
-        rm = new RowMutation(tableName, key.key);
+        rm = new RowMutation(keyspaceName, key.key);
         for (int i = 0; i < 10; i++)
         {
             rm.add(cfName, ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
@@ -275,16 +275,16 @@ public class CompactionsPurgeTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        String tableName = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfName = "Standard1";
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 
         DecoratedKey key = Util.dk("key3");
         RowMutation rm;
 
         // inserts
-        rm = new RowMutation(tableName, key.key);
+        rm = new RowMutation(keyspaceName, key.key);
         for (int i = 0; i < 10; i++)
         {
             rm.add(cfName, ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
@@ -292,7 +292,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         rm.apply();
 
         // deletes row with timestamp such that not all columns are deleted
-        rm = new RowMutation(tableName, key.key);
+        rm = new RowMutation(keyspaceName, key.key);
         rm.delete(cfName, 4);
         rm.apply();
 
@@ -301,7 +301,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         Util.compactAll(cfs).get();
 
         // re-inserts with timestamp lower than delete
-        rm = new RowMutation(tableName, key.key);
+        rm = new RowMutation(keyspaceName, key.key);
         for (int i = 0; i < 5; i++)
         {
             rm.add(cfName, ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index 790163d..a775988 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -52,12 +52,12 @@ import static junit.framework.Assert.*;
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class CompactionsTest extends SchemaLoader
 {
-    public static final String TABLE1 = "Keyspace1";
+    public static final String KEYSPACE1 = "Keyspace1";
 
     public ColumnFamilyStore testSingleSSTableCompaction(String strategyClassName) throws Exception
     {
-        Table table = Table.open(TABLE1);
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         store.clearUnsafe();
         store.metadata.gcGraceSeconds(1);
         store.setCompactionStrategyClass(strategyClassName);
@@ -69,7 +69,7 @@ public class CompactionsTest extends SchemaLoader
         for (int i = 0; i < 10; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
-            RowMutation rm = new RowMutation(TABLE1, key.key);
+            RowMutation rm = new RowMutation(KEYSPACE1, key.key);
             for (int j = 0; j < 10; j++)
                 rm.add("Standard1", ByteBufferUtil.bytes(Integer.toString(j)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
@@ -122,15 +122,15 @@ public class CompactionsTest extends SchemaLoader
     @Test
     public void testSuperColumnTombstones() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open(TABLE1);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Super1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Super1");
         cfs.disableAutoCompaction();
 
         DecoratedKey key = Util.dk("tskey");
         ByteBuffer scName = ByteBufferUtil.bytes("TestSuperColumn");
 
         // a subcolumn
-        RowMutation rm = new RowMutation(TABLE1, key.key);
+        RowMutation rm = new RowMutation(KEYSPACE1, key.key);
         rm.add("Super1", CompositeType.build(scName, ByteBufferUtil.bytes(0)),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                FBUtilities.timestampMicros());
@@ -138,7 +138,7 @@ public class CompactionsTest extends SchemaLoader
         cfs.forceBlockingFlush();
 
         // shadow the subcolumn with a supercolumn tombstone
-        rm = new RowMutation(TABLE1, key.key);
+        rm = new RowMutation(KEYSPACE1, key.key);
         rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), FBUtilities.timestampMicros());
         rm.apply();
         cfs.forceBlockingFlush();
@@ -169,8 +169,8 @@ public class CompactionsTest extends SchemaLoader
     {
         // This test check that EchoedRow doesn't skipp rows: see CASSANDRA-2653
 
-        Table table = Table.open(TABLE1);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard2");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard2");
 
         // disable compaction while flushing
         cfs.disableAutoCompaction();
@@ -180,7 +180,7 @@ public class CompactionsTest extends SchemaLoader
         for (int i=1; i < 5; i++)
         {
             DecoratedKey key = Util.dk(String.valueOf(i));
-            RowMutation rm = new RowMutation(TABLE1, key.key);
+            RowMutation rm = new RowMutation(KEYSPACE1, key.key);
             rm.add("Standard2", ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
             rm.apply();
 
@@ -195,7 +195,7 @@ public class CompactionsTest extends SchemaLoader
         for (int i=1; i < 5; i++)
         {
             DecoratedKey key = Util.dk(String.valueOf(i));
-            RowMutation rm = new RowMutation(TABLE1, key.key);
+            RowMutation rm = new RowMutation(KEYSPACE1, key.key);
             rm.add("Standard2", ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
             rm.apply();
         }
@@ -230,9 +230,9 @@ public class CompactionsTest extends SchemaLoader
     @Test
     public void testUserDefinedCompaction() throws Exception
     {
-        Table table = Table.open(TABLE1);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final String cfname = "Standard3"; // use clean(no sstable) CF
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfname);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
 
         // disable compaction while flushing
         cfs.disableAutoCompaction();
@@ -240,7 +240,7 @@ public class CompactionsTest extends SchemaLoader
         final int ROWS_PER_SSTABLE = 10;
         for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
             DecoratedKey key = Util.dk(String.valueOf(i));
-            RowMutation rm = new RowMutation(TABLE1, key.key);
+            RowMutation rm = new RowMutation(KEYSPACE1, key.key);
             rm.add(cfname, ByteBufferUtil.bytes("col"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    System.currentTimeMillis());
@@ -270,11 +270,11 @@ public class CompactionsTest extends SchemaLoader
     @Test
     public void testCompactionLog() throws Exception
     {
-        SystemTable.discardCompactionsInProgress();
+        SystemKeyspace.discardCompactionsInProgress();
 
         String cf = "Standard4";
-        ColumnFamilyStore cfs = Table.open(TABLE1).getColumnFamilyStore(cf);
-        insertData(TABLE1, cf, 0, 1);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(cf);
+        insertData(KEYSPACE1, cf, 0, 1);
         cfs.forceBlockingFlush();
 
         Collection<SSTableReader> sstables = cfs.getSSTables();
@@ -286,21 +286,21 @@ public class CompactionsTest extends SchemaLoader
                 return sstable.descriptor.generation;
             }
         }));
-        UUID taskId = SystemTable.startCompaction(cfs, sstables);
-        SetMultimap<Pair<String, String>, Integer> compactionLogs = SystemTable.getUnfinishedCompactions();
-        Set<Integer> unfinishedCompactions = compactionLogs.get(Pair.create(TABLE1, cf));
+        UUID taskId = SystemKeyspace.startCompaction(cfs, sstables);
+        SetMultimap<Pair<String, String>, Integer> compactionLogs = SystemKeyspace.getUnfinishedCompactions();
+        Set<Integer> unfinishedCompactions = compactionLogs.get(Pair.create(KEYSPACE1, cf));
         assert unfinishedCompactions.containsAll(generations);
 
-        SystemTable.finishCompaction(taskId);
-        compactionLogs = SystemTable.getUnfinishedCompactions();
-        assert !compactionLogs.containsKey(Pair.create(TABLE1, cf));
+        SystemKeyspace.finishCompaction(taskId);
+        compactionLogs = SystemKeyspace.getUnfinishedCompactions();
+        assert !compactionLogs.containsKey(Pair.create(KEYSPACE1, cf));
     }
 
     private void testDontPurgeAccidentaly(String k, String cfname) throws IOException, ExecutionException, InterruptedException
     {
         // This test catches the regression of CASSANDRA-2786
-        Table table = Table.open(TABLE1);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfname);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
 
         // disable compaction while flushing
         cfs.clearUnsafe();
@@ -308,7 +308,7 @@ public class CompactionsTest extends SchemaLoader
 
         // Add test row
         DecoratedKey key = Util.dk(k);
-        RowMutation rm = new RowMutation(TABLE1, key.key);
+        RowMutation rm = new RowMutation(KEYSPACE1, key.key);
         rm.add(cfname, CompositeType.build(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes("c")), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
 
@@ -320,7 +320,7 @@ public class CompactionsTest extends SchemaLoader
         assert !(cfs.getColumnFamily(filter).getColumnCount() == 0);
 
         // Remove key
-        rm = new RowMutation(TABLE1, key.key);
+        rm = new RowMutation(KEYSPACE1, key.key);
         rm.delete(cfname, 2);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 9218dc9..3aac83e 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -35,7 +35,7 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.Component;
@@ -60,8 +60,8 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
     {
         String ksname = "Keyspace1";
         String cfname = "StandardLeveled";
-        Table table = Table.open(ksname);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfname);
+        Keyspace keyspace = Keyspace.open(ksname);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
 
         ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
 
@@ -89,7 +89,7 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         assert strategy.getLevelSize(2) > 0;
 
         Range<Token> range = new Range<Token>(Util.token(""), Util.token(""));
-        int gcBefore = table.getColumnFamilyStore(cfname).gcBefore(System.currentTimeMillis());
+        int gcBefore = keyspace.getColumnFamilyStore(cfname).gcBefore(System.currentTimeMillis());
         RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), ksname, cfname, range);
         Validator validator = new Validator(desc, FBUtilities.getBroadcastAddress(), gcBefore);
         CompactionManager.instance.submitValidation(cfs, validator).get();
@@ -111,8 +111,8 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
     {
         String ksname = "Keyspace1";
         String cfname = "StandardLeveled";
-        Table table = Table.open(ksname);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfname);
+        Keyspace keyspace = Keyspace.open(ksname);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
 
         // make sure we have SSTables in L1
         ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]);
@@ -152,8 +152,8 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
     {
         String ksname = "Keyspace1";
         String cfname = "StandardLeveled";
-        Table table = Table.open(ksname);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfname);
+        Keyspace keyspace = Keyspace.open(ksname);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
 
         ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
index fb5ee93..0cc669d 100644
--- a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
@@ -28,7 +28,7 @@ import org.apache.cassandra.Util;
 import org.junit.Test;
 
 import static junit.framework.Assert.assertEquals;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowMutation;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -43,8 +43,8 @@ public class OneCompactionTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore(columnFamilyName);
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(columnFamilyName);
 
         Set<DecoratedKey> inserted = new HashSet<DecoratedKey>();
         for (int j = 0; j < insertsPerTable; j++) {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
index 8c4c305..7666922 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
@@ -32,16 +32,15 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DataRange;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.RowMutation;
 import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.db.Table;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.SSTableScanner;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -51,7 +50,7 @@ public class TTLExpiryTest extends SchemaLoader
     @Test
     public void testSimpleExpire() throws ExecutionException, InterruptedException
     {
-        ColumnFamilyStore cfs = Table.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
         cfs.disableAutoCompaction();
         cfs.metadata.gcGraceSeconds(0);
         long timestamp = System.currentTimeMillis();
@@ -99,7 +98,7 @@ public class TTLExpiryTest extends SchemaLoader
     @Test
     public void testNoExpire() throws ExecutionException, InterruptedException
     {
-        ColumnFamilyStore cfs = Table.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
         cfs.disableAutoCompaction();
         cfs.metadata.gcGraceSeconds(0);
         long timestamp = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
index 8314af8..4fba73c 100644
--- a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
@@ -165,8 +165,8 @@ public class CompositeTypeTest extends SchemaLoader
     @Test
     public void testFullRound() throws Exception
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 
         ByteBuffer cname1 = createCompositeKey("test1", null, -1, false);
         ByteBuffer cname2 = createCompositeKey("test1", uuids[0], 24, false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
index 14ec219..4b39582 100644
--- a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
@@ -161,8 +161,8 @@ public class DynamicCompositeTypeTest extends SchemaLoader
     @Test
     public void testFullRound() throws Exception
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 
         ByteBuffer cname1 = createDynamicCompositeKey("test1", null, -1, false);
         ByteBuffer cname2 = createDynamicCompositeKey("test1", uuids[0], 24, false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
index 17fa3fd..8c3998d 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -31,7 +31,7 @@ import org.junit.runner.RunWith;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.gms.IFailureDetectionEventListener;
 import org.apache.cassandra.gms.IFailureDetector;
 import org.apache.cassandra.locator.TokenMetadata;
@@ -46,16 +46,16 @@ public class BootStrapperTest extends SchemaLoader
     public void testSourceTargetComputation() throws UnknownHostException
     {
         final int[] clusterSizes = new int[] { 1, 3, 5, 10, 100};
-        for (String table : Schema.instance.getNonSystemTables())
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            int replicationFactor = Table.open(table).getReplicationStrategy().getReplicationFactor();
+            int replicationFactor = Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor();
             for (int clusterSize : clusterSizes)
                 if (clusterSize >= replicationFactor)
-                    testSourceTargetComputation(table, clusterSize, replicationFactor);
+                    testSourceTargetComputation(keyspaceName, clusterSize, replicationFactor);
         }
     }
 
-    private RangeStreamer testSourceTargetComputation(String table, int numOldNodes, int replicationFactor) throws UnknownHostException
+    private RangeStreamer testSourceTargetComputation(String keyspaceName, int numOldNodes, int replicationFactor) throws UnknownHostException
     {
         StorageService ss = StorageService.instance;
 
@@ -82,9 +82,9 @@ public class BootStrapperTest extends SchemaLoader
             public void clear(InetAddress ep) { throw new UnsupportedOperationException(); }
         };
         s.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(mockFailureDetector));
-        s.addRanges(table, Table.open(table).getReplicationStrategy().getPendingAddressRanges(tmd, myToken, myEndpoint));
+        s.addRanges(keyspaceName, Keyspace.open(keyspaceName).getReplicationStrategy().getPendingAddressRanges(tmd, myToken, myEndpoint));
 
-        Collection<Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch = s.toFetch().get(table);
+        Collection<Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch = s.toFetch().get(keyspaceName);
 
         // Check we get get RF new ranges in total
         Set<Range<Token>> ranges = new HashSet<Range<Token>>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java b/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
index cce32df..f89e96a 100644
--- a/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
+++ b/test/unit/org/apache/cassandra/io/LazilyCompactedRowTest.java
@@ -163,8 +163,8 @@ public class LazilyCompactedRowTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         ByteBuffer key = ByteBufferUtil.bytes("k");
         RowMutation rm = new RowMutation("Keyspace1", key);
@@ -181,8 +181,8 @@ public class LazilyCompactedRowTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         ByteBuffer key = ByteBufferUtil.bytes("k");
         RowMutation rm = new RowMutation("Keyspace1", key);
@@ -200,8 +200,8 @@ public class LazilyCompactedRowTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         ByteBuffer key = ByteBuffer.wrap("k".getBytes());
         RowMutation rm = new RowMutation("Keyspace1", key);
@@ -222,8 +222,8 @@ public class LazilyCompactedRowTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         ByteBuffer key = ByteBufferUtil.bytes("k");
         RowMutation rm = new RowMutation("Keyspace1", key);
@@ -243,8 +243,8 @@ public class LazilyCompactedRowTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         ByteBuffer key = ByteBufferUtil.bytes("k");
         RowMutation rm = new RowMutation("Keyspace1", key);
@@ -265,8 +265,8 @@ public class LazilyCompactedRowTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         final int ROWS_PER_SSTABLE = 10;
         for (int j = 0; j < (cfs.metadata.getIndexInterval() * 3) / ROWS_PER_SSTABLE; j++)


[03/11] Rename Table to Keyspace

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index e5ee5f3..984c354 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -78,8 +78,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     // create two sstables, and verify that we only deserialize data from the most recent one
     public void testTimeSortedQuery() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
         cfs.truncateBlocking();
 
         RowMutation rm;
@@ -101,8 +101,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testGetColumnWithWrongBF() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
         cfs.truncateBlocking();
 
         List<IMutation> rms = new LinkedList<IMutation>();
@@ -113,7 +113,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rms.add(rm);
         Util.writeColumnFamily(rms);
 
-        List<SSTableReader> ssTables = table.getAllSSTables();
+        List<SSTableReader> ssTables = keyspace.getAllSSTables();
         assertEquals(1, ssTables.size());
         ssTables.get(0).forceFilterFailures();
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("key2"), "Standard1", System.currentTimeMillis()));
@@ -123,8 +123,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testEmptyRow() throws Exception
     {
-        Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore store = table.getColumnFamilyStore("Standard2");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
         RowMutation rm;
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
@@ -156,7 +156,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
             }
         };
 
-        TableTest.reTest(store, r);
+        KeyspaceTest.reTest(store, r);
     }
 
     @Test
@@ -204,7 +204,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         IDiskAtomFilter filter = new IdentityQueryFilter();
         IPartitioner p = StorageService.getPartitioner();
         Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        List<Row> rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
 
         assert rows != null;
         assert rows.size() == 2 : StringUtils.join(rows, ",");
@@ -221,14 +221,14 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // add a second expression
         IndexExpression expr2 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), IndexOperator.GTE, ByteBufferUtil.bytes(2L));
         clause = Arrays.asList(expr, expr2);
-        rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
 
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining());
         assert "k3".equals( key );
 
         // same query again, but with resultset not including the subordinate expression
-        rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, new NamesQueryFilter(ByteBufferUtil.bytes("birthdate")), 100);
+        rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, new NamesQueryFilter(ByteBufferUtil.bytes("birthdate")), 100);
 
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining());
@@ -238,7 +238,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // once more, this time with a slice rowset that needs to be expanded
         SliceQueryFilter emptyFilter = new SliceQueryFilter(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 0);
-        rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, emptyFilter, 100);
+        rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, emptyFilter, 100);
 
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining());
@@ -250,7 +250,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // doesn't tell the scan loop that it's done
         IndexExpression expr3 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(-1L));
         clause = Arrays.asList(expr, expr3);
-        rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
 
         assert rows.isEmpty();
     }
@@ -273,7 +273,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         IDiskAtomFilter filter = new IdentityQueryFilter();
         IPartitioner p = StorageService.getPartitioner();
         Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        List<Row> rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
 
         assert rows != null;
         assert rows.size() == 50 : rows.size();
@@ -287,7 +287,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testIndexDeletions() throws IOException
     {
-        ColumnFamilyStore cfs = Table.open("Keyspace3").getColumnFamilyStore("Indexed1");
+        ColumnFamilyStore cfs = Keyspace.open("Keyspace3").getColumnFamilyStore("Indexed1");
         RowMutation rm;
 
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
@@ -382,7 +382,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testIndexUpdate() throws IOException
     {
-        Table table = Table.open("Keyspace2");
+        Keyspace keyspace = Keyspace.open("Keyspace2");
 
         // create a row and update the birthdate value, test that the index query fetches the new version
         RowMutation rm;
@@ -398,12 +398,12 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         IDiskAtomFilter filter = new IdentityQueryFilter();
         IPartitioner p = StorageService.getPartitioner();
         Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = table.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        List<Row> rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         assert rows.size() == 0;
 
         expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(2L));
         clause = Arrays.asList(expr);
-        rows = table.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         String key = ByteBufferUtil.string(rows.get(0).key.key);
         assert "k1".equals( key );
 
@@ -412,7 +412,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(3L), 0);
         rm.apply();
 
-        rows = table.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         key = ByteBufferUtil.string(rows.get(0).key.key);
         assert "k1".equals( key );
 
@@ -424,8 +424,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         String keySpace = "Keyspace2";
         String cfName = "Indexed1";
 
-        Table table = Table.open(keySpace);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keySpace);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.truncateBlocking();
 
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
@@ -442,42 +442,42 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = table.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
+        List<Row> rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(1, rows.size());
 
         // force a flush, so our index isn't being read from a memtable
-        table.getColumnFamilyStore(cfName).forceBlockingFlush();
+        keyspace.getColumnFamilyStore(cfName).forceBlockingFlush();
 
         // now apply another update, but force the index update to be skipped
         rm = new RowMutation(keySpace, rowKey);
         rm.add(cfName, colName, val2, 1);
-        table.apply(rm, true, false);
+        keyspace.apply(rm, true, false);
 
         // Now searching the index for either the old or new value should return 0 rows
         // because the new value was not indexed and the old value should be ignored
         // (and in fact purged from the index cf).
         // first check for the old value
-        rows = table.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(0, rows.size());
         // now check for the updated value
         expr = new IndexExpression(colName, IndexOperator.EQ, val2);
         clause = Arrays.asList(expr);
         filter = new IdentityQueryFilter();
         range = Util.range("", "");
-        rows = table.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(0, rows.size());
 
         // now, reset back to the original value, still skipping the index update, to
         // make sure the value was expunged from the index when it was discovered to be inconsistent
         rm = new RowMutation(keySpace, rowKey);
         rm.add(cfName, colName, ByteBufferUtil.bytes(1L), 3);
-        table.apply(rm, true, false);
+        keyspace.apply(rm, true, false);
 
         expr = new IndexExpression(colName, IndexOperator.EQ, ByteBufferUtil.bytes(1L));
         clause = Arrays.asList(expr);
         filter = new IdentityQueryFilter();
         range = Util.range("", "");
-        rows = table.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(0, rows.size());
     }
 
@@ -487,8 +487,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         String keySpace = "Keyspace2";
         String cfName = "Indexed2";
 
-        Table table = Table.open(keySpace);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keySpace);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.truncateBlocking();
 
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
@@ -514,44 +514,44 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = table.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
+        List<Row> rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(1, rows.size());
 
         // force a flush and retry the query, so our index isn't being read from a memtable
-        table.getColumnFamilyStore(cfName).forceBlockingFlush();
-        rows = table.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
+        keyspace.getColumnFamilyStore(cfName).forceBlockingFlush();
+        rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(1, rows.size());
 
         // now apply another update, but force the index update to be skipped
         rm = new RowMutation(keySpace, rowKey);
         rm.add(cfName, compositeName, val2, 1);
-        table.apply(rm, true, false);
+        keyspace.apply(rm, true, false);
 
         // Now searching the index for either the old or new value should return 0 rows
         // because the new value was not indexed and the old value should be ignored
         // (and in fact purged from the index cf).
         // first check for the old value
-        rows = table.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(0, rows.size());
         // now check for the updated value
         expr = new IndexExpression(colName, IndexOperator.EQ, val2);
         clause = Arrays.asList(expr);
         filter = new IdentityQueryFilter();
         range = Util.range("", "");
-        rows = table.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(0, rows.size());
 
         // now, reset back to the original value, still skipping the index update, to
         // make sure the value was expunged from the index when it was discovered to be inconsistent
         rm = new RowMutation(keySpace, rowKey);
         rm.add(cfName, compositeName, val1, 2);
-        table.apply(rm, true, false);
+        keyspace.apply(rm, true, false);
 
         expr = new IndexExpression(colName, IndexOperator.EQ, val1);
         clause = Arrays.asList(expr);
         filter = new IdentityQueryFilter();
         range = Util.range("", "");
-        rows = table.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(0, rows.size());
     }
 
@@ -588,7 +588,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         IDiskAtomFilter filter = new IdentityQueryFilter();
         IPartitioner p = StorageService.getPartitioner();
         Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = Table.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 1);
+        List<Row> rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 1);
 
         assert rows != null;
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -597,7 +597,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testIndexCreate() throws IOException, ConfigurationException, InterruptedException, ExecutionException
     {
-        Table table = Table.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
 
         // create a row and update the birthdate value, test that the index query fetches the new version
         RowMutation rm;
@@ -605,7 +605,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.add("Indexed2", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 1);
         rm.apply();
 
-        ColumnFamilyStore cfs = table.getColumnFamilyStore("Indexed2");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed2");
         ColumnDefinition old = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("birthdate"));
         ColumnDefinition cd = ColumnDefinition.regularDef(old.name, old.getValidator(), null).setIndex("birthdate_index", IndexType.KEYS, null);
         Future<?> future = cfs.indexManager.addIndexedColumn(cd);
@@ -613,7 +613,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // we had a bug (CASSANDRA-2244) where index would get created but not flushed -- check for that
         assert cfs.indexManager.getIndexForColumn(cd.name).getIndexCfs().getSSTables().size() > 0;
 
-        queryBirthdate(table);
+        queryBirthdate(keyspace);
 
         // validate that drop clears it out & rebuild works (CASSANDRA-2320)
         SecondaryIndex indexedCfs = cfs.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate"));
@@ -623,16 +623,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // rebuild & re-query
         future = cfs.indexManager.addIndexedColumn(cd);
         future.get();
-        queryBirthdate(table);
+        queryBirthdate(keyspace);
     }
 
-    private void queryBirthdate(Table table) throws CharacterCodingException
+    private void queryBirthdate(Keyspace keyspace) throws CharacterCodingException
     {
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         IPartitioner p = StorageService.getPartitioner();
-        List<Row> rows = table.getColumnFamilyStore("Indexed2").search(Util.range("", ""), clause, filter, 100);
+        List<Row> rows = keyspace.getColumnFamilyStore("Indexed2").search(Util.range("", ""), clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         assertEquals("k1", ByteBufferUtil.string(rows.get(0).key.key));
     }
@@ -654,11 +654,11 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testDeleteSuperRowSticksAfterFlush() throws Throwable
     {
-        String tableName = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfName= "Super1";
         ByteBuffer scfName = ByteBufferUtil.bytes("SuperDuper");
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         DecoratedKey key = Util.dk("flush-resurrection");
 
         // create an isolated sstable.
@@ -684,7 +684,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertRowAndColCount(1, 6, scfName, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
 
         // delete
-        RowMutation rm = new RowMutation(table.getName(), key.key);
+        RowMutation rm = new RowMutation(keyspace.getName(), key.key);
         rm.deleteRange(cfName, SuperColumns.startOf(scfName), SuperColumns.endOf(scfName), 2);
         rm.apply();
 
@@ -735,19 +735,19 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
     private static void putColsSuper(ColumnFamilyStore cfs, DecoratedKey key, ByteBuffer scfName, Column... cols) throws Throwable
     {
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.table.getName(), cfs.name);
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.keyspace.getName(), cfs.name);
         for (Column col : cols)
             cf.addColumn(col.withUpdatedName(CompositeType.build(scfName, col.name())));
-        RowMutation rm = new RowMutation(cfs.table.getName(), key.key, cf);
+        RowMutation rm = new RowMutation(cfs.keyspace.getName(), key.key, cf);
         rm.apply();
     }
 
     private static void putColsStandard(ColumnFamilyStore cfs, DecoratedKey key, Column... cols) throws Throwable
     {
-        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.table.getName(), cfs.name);
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.keyspace.getName(), cfs.name);
         for (Column col : cols)
             cf.addColumn(col);
-        RowMutation rm = new RowMutation(cfs.table.getName(), key.key, cf);
+        RowMutation rm = new RowMutation(cfs.keyspace.getName(), key.key, cf);
         rm.apply();
     }
 
@@ -755,10 +755,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testDeleteStandardRowSticksAfterFlush() throws Throwable
     {
         // test to make sure flushing after a delete doesn't resurrect delted cols.
-        String tableName = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfName = "Standard1";
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         DecoratedKey key = Util.dk("f-flush-resurrection");
 
         SlicePredicate sp = new SlicePredicate();
@@ -779,7 +779,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertRowAndColCount(1, 4, null, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, null), 100));
 
         // delete (from sstable and memtable)
-        RowMutation rm = new RowMutation(table.getName(), key.key);
+        RowMutation rm = new RowMutation(keyspace.getName(), key.key);
         rm.delete(cfs.name, 2);
         rm.apply();
 
@@ -841,11 +841,11 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testSliceByNamesCommandOnUUIDTypeSCF() throws Throwable
     {
-        String tableName = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfName = "Super6";
         ByteBuffer superColName = LexicalUUIDType.instance.fromString("a4ed3562-0e8e-4b41-bdfd-c45a2774682d");
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         DecoratedKey key = Util.dk("slice-get-uuid-type");
 
         // Insert a row with one supercolumn and multiple subcolumns
@@ -861,8 +861,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         SortedSet<ByteBuffer> sliceColNames = new TreeSet<ByteBuffer>(cfs.metadata.comparator);
         sliceColNames.add(CompositeType.build(superColName, ByteBufferUtil.bytes("a")));
         sliceColNames.add(CompositeType.build(superColName, ByteBufferUtil.bytes("b")));
-        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(tableName, key.key, cfName, System.currentTimeMillis(), new NamesQueryFilter(sliceColNames));
-        ColumnFamily cfSliced = cmd.getRow(table).cf;
+        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(keyspaceName, key.key, cfName, System.currentTimeMillis(), new NamesQueryFilter(sliceColNames));
+        ColumnFamily cfSliced = cmd.getRow(keyspace).cf;
 
         // Make sure the slice returns the same as the straight get
         assertEquals(ByteBufferUtil.bytes("A"), cfSliced.getColumn(CompositeType.build(superColName, ByteBufferUtil.bytes("a"))).value());
@@ -872,12 +872,12 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testSliceByNamesCommandOldMetatada() throws Throwable
     {
-        String tableName = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfName= "Standard1";
         DecoratedKey key = Util.dk("slice-name-old-metadata");
         ByteBuffer cname = ByteBufferUtil.bytes("c1");
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
 
         // Create a column a 'high timestamp'
@@ -897,8 +897,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         putColsStandard(cfs, key, new Column(cname, ByteBufferUtil.bytes("b"), 1));
 
         // Test fetching the column by name returns the first column
-        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(tableName, key.key, cfName, System.currentTimeMillis(), new NamesQueryFilter(cname));
-        ColumnFamily cf = cmd.getRow(table).cf;
+        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(keyspaceName, key.key, cfName, System.currentTimeMillis(), new NamesQueryFilter(cname));
+        ColumnFamily cf = cmd.getRow(keyspace).cf;
         Column column = (Column) cf.getColumn(cname);
         assert column.value().equals(ByteBufferUtil.bytes("a")) : "expecting a, got " + ByteBufferUtil.string(column.value());
     }
@@ -917,10 +917,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testRangeSliceColumnsLimit() throws Throwable
     {
-        String tableName = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfName = "Standard1";
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
 
         Column[] cols = new Column[5];
@@ -1033,10 +1033,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testRangeSlicePaging() throws Throwable
     {
-        String tableName = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfName = "Standard1";
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
 
         Column[] cols = new Column[4];
@@ -1169,10 +1169,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testRangeSliceInclusionExclusion() throws Throwable
     {
-        String tableName = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfName = "Standard1";
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
 
         Column[] cols = new Column[5];
@@ -1223,8 +1223,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testKeysSearcher() throws Exception
     {
         // Create secondary index and flush to disk
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Indexed1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
 
         store.truncateBlocking();
 
@@ -1271,10 +1271,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 new ColumnSlice(bytes("colE"), bytes("colC")),
                 new ColumnSlice(bytes("colA"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
 
-        String tableName = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfName = "Standard1";
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
 
         String[] letters = new String[] { "a", "b", "c", "d", "e", "f", "g", "h", "i" };
@@ -1391,10 +1391,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
     private ColumnFamilyStore prepareMultiRangeSlicesTest(int valueSize, boolean flush) throws Throwable
     {
-        String tableName = "Keyspace1";
+        String keyspaceName = "Keyspace1";
         String cfName = "Standard1";
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
 
         String[] letters = new String[] { "a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l" };

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/CounterColumnTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterColumnTest.java b/test/unit/org/apache/cassandra/db/CounterColumnTest.java
index 90fe76f..7c2be9f 100644
--- a/test/unit/org/apache/cassandra/db/CounterColumnTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterColumnTest.java
@@ -60,7 +60,7 @@ public class CounterColumnTest extends SchemaLoader
     {
         long delta = 3L;
         CounterUpdateColumn cuc = new CounterUpdateColumn(ByteBufferUtil.bytes("x"), delta, 1L);
-        CounterColumn column = cuc.localCopy(Table.open("Keyspace5").getColumnFamilyStore("Counter1"));
+        CounterColumn column = cuc.localCopy(Keyspace.open("Keyspace5").getColumnFamilyStore("Counter1"));
 
         assert delta == column.total();
         assert 1 == column.value().getShort(0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/CounterMutationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterMutationTest.java b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
index 82141d8..3cbd030 100644
--- a/test/unit/org/apache/cassandra/db/CounterMutationTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
@@ -64,7 +64,7 @@ public class CounterMutationTest extends SchemaLoader
         cm.apply();
 
         DecoratedKey dk = Util.dk("key1");
-        ColumnFamily cf = Util.getColumnFamily(Table.open("Keyspace1"), dk, "Counter1");
+        ColumnFamily cf = Util.getColumnFamily(Keyspace.open("Keyspace1"), dk, "Counter1");
 
         // First merges old shards
         CounterColumn.mergeAndRemoveOldShards(dk, cf, Integer.MIN_VALUE, Integer.MAX_VALUE, false);
@@ -99,16 +99,16 @@ public class CounterMutationTest extends SchemaLoader
     }
 
     @Test
-    public void testGetOldShardFromSystemTable() throws IOException
+    public void testGetOldShardFromSystemKeyspace() throws IOException
     {
         // Renewing a bunch of times and checking we get the same thing from
-        // the system table that what is in memory
+        // the system keyspace that what is in memory
         CounterId.renewLocalId();
         CounterId.renewLocalId();
         CounterId.renewLocalId();
 
         List<CounterId.CounterIdRecord> inMem = CounterId.getOldLocalCounterIds();
-        List<CounterId.CounterIdRecord> onDisk = SystemTable.getOldLocalCounterIds();
+        List<CounterId.CounterIdRecord> onDisk = SystemKeyspace.getOldLocalCounterIds();
 
         assert inMem.equals(onDisk);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
index c19efcb..8bbf5a9 100644
--- a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
+++ b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
@@ -45,7 +45,7 @@ import static org.apache.cassandra.cql3.QueryProcessor.processInternal;
 public class HintedHandOffTest extends SchemaLoader
 {
 
-    public static final String TABLE4 = "Keyspace4";
+    public static final String KEYSPACE4 = "Keyspace4";
     public static final String STANDARD1_CF = "Standard1";
     public static final String COLUMN1 = "column1";
 
@@ -54,15 +54,15 @@ public class HintedHandOffTest extends SchemaLoader
     public void testCompactionOfHintsCF() throws Exception
     {
         // prepare hints column family
-        Table systemTable = Table.open("system");
-        ColumnFamilyStore hintStore = systemTable.getColumnFamilyStore(SystemTable.HINTS_CF);
+        Keyspace systemKeyspace = Keyspace.open("system");
+        ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS_CF);
         hintStore.clearUnsafe();
         hintStore.metadata.gcGraceSeconds(36000); // 10 hours
         hintStore.setCompactionStrategyClass(SizeTieredCompactionStrategy.class.getCanonicalName());
         hintStore.disableAutoCompaction();
 
         // insert 1 hint
-        RowMutation rm = new RowMutation(TABLE4, ByteBufferUtil.bytes(1));
+        RowMutation rm = new RowMutation(KEYSPACE4, ByteBufferUtil.bytes(1));
         rm.add(STANDARD1_CF, ByteBufferUtil.bytes(String.valueOf(COLUMN1)), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
 
         HintedHandOffManager.hintFor(rm, HintedHandOffManager.calculateHintTTL(rm), UUID.randomUUID()).apply();
@@ -88,7 +88,7 @@ public class HintedHandOffTest extends SchemaLoader
             HintedHandOffManager.instance.metrics.incrPastWindow(InetAddress.getLocalHost());
         HintedHandOffManager.instance.metrics.log();
 
-        UntypedResultSet rows = processInternal("SELECT hints_dropped FROM system." + SystemTable.PEER_EVENTS_CF);
+        UntypedResultSet rows = processInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS_CF);
         Map<UUID, Integer> returned = rows.one().getMap("hints_dropped", UUIDType.instance, Int32Type.instance);
         assertEquals(Iterators.getLast(returned.values().iterator()).intValue(), 99);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/KeyCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index 922e255..e8e524d 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -41,7 +41,7 @@ import static junit.framework.Assert.assertEquals;
 
 public class KeyCacheTest extends SchemaLoader
 {
-    private static final String TABLE1 = "KeyCacheSpace";
+    private static final String KEYSPACE1 = "KeyCacheSpace";
     private static final String COLUMN_FAMILY1 = "Standard1";
     private static final String COLUMN_FAMILY2 = "Standard2";
 
@@ -56,18 +56,18 @@ public class KeyCacheTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        ColumnFamilyStore store = Table.open(TABLE1).getColumnFamilyStore(COLUMN_FAMILY2);
+        ColumnFamilyStore store = Keyspace.open(KEYSPACE1).getColumnFamilyStore(COLUMN_FAMILY2);
 
         // empty the cache
         CacheService.instance.invalidateKeyCache();
         assert CacheService.instance.keyCache.size() == 0;
 
         // insert data and force to disk
-        insertData(TABLE1, COLUMN_FAMILY2, 0, 100);
+        insertData(KEYSPACE1, COLUMN_FAMILY2, 0, 100);
         store.forceBlockingFlush();
 
         // populate the cache
-        readData(TABLE1, COLUMN_FAMILY2, 0, 100);
+        readData(KEYSPACE1, COLUMN_FAMILY2, 0, 100);
         assertEquals(100, CacheService.instance.keyCache.size());
 
         // really? our caches don't implement the map interface? (hence no .addAll)
@@ -89,8 +89,8 @@ public class KeyCacheTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Table table = Table.open(TABLE1);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(COLUMN_FAMILY1);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COLUMN_FAMILY1);
 
         // just to make sure that everything is clean
         CacheService.instance.invalidateKeyCache();
@@ -103,10 +103,10 @@ public class KeyCacheTest extends SchemaLoader
         RowMutation rm;
 
         // inserts
-        rm = new RowMutation(TABLE1, key1.key);
+        rm = new RowMutation(KEYSPACE1, key1.key);
         rm.add(COLUMN_FAMILY1, ByteBufferUtil.bytes("1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
-        rm = new RowMutation(TABLE1, key2.key);
+        rm = new RowMutation(KEYSPACE1, key2.key);
         rm.add(COLUMN_FAMILY1, ByteBufferUtil.bytes("2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
index a6fbecf..345febd 100644
--- a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
@@ -63,8 +63,8 @@ public class KeyCollisionTest extends SchemaLoader
     @Test
     public void testGetSliceWithCollision() throws Exception
     {
-        Table table = Table.open(KEYSPACE);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(CF);
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
         cfs.clearUnsafe();
 
         insert("k1", "k2", "k3");       // token = 2
@@ -181,7 +181,7 @@ public class KeyCollisionTest extends SchemaLoader
                 lastToken = node;
             }
 
-            for (String ks : Schema.instance.getTables())
+            for (String ks : Schema.instance.getKeyspaces())
             {
                 for (CFMetaData cfmd : Schema.instance.getKSMetaData(ks).cfMetaData().values())
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/KeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
new file mode 100644
index 0000000..faee4e7
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
@@ -0,0 +1,649 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.db;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.text.DecimalFormat;
+import java.text.NumberFormat;
+import java.util.*;
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.collect.Iterables;
+import org.apache.commons.lang.StringUtils;
+import org.junit.Test;
+
+import static junit.framework.Assert.*;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.utils.WrappedRunnable;
+import static org.apache.cassandra.Util.column;
+import static org.apache.cassandra.Util.expiringColumn;
+import static org.apache.cassandra.Util.getBytes;
+import static org.junit.Assert.assertEquals;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+
+public class KeyspaceTest extends SchemaLoader
+{
+    private static final DecoratedKey TEST_KEY = Util.dk("key1");
+    private static final DecoratedKey TEST_SLICE_KEY = Util.dk("key1-slicerange");
+
+    public static void reTest(ColumnFamilyStore cfs, Runnable verify) throws Exception
+    {
+        verify.run();
+        cfs.forceBlockingFlush();
+        verify.run();
+    }
+
+    @Test
+    public void testGetRowNoColumns() throws Throwable
+    {
+        final Keyspace keyspace = Keyspace.open("Keyspace2");
+        final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard3");
+
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace2", "Standard3");
+        cf.addColumn(column("col1","val1", 1L));
+        RowMutation rm = new RowMutation("Keyspace2", TEST_KEY.key, cf);
+        rm.apply();
+
+        Runnable verify = new WrappedRunnable()
+        {
+            public void runMayThrow() throws Exception
+            {
+                ColumnFamily cf;
+
+                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY,
+                                                                        "Standard3",
+                                                                        new TreeSet<ByteBuffer>(),
+                                                                        System.currentTimeMillis()));
+                assertColumns(cf);
+
+                cf = cfStore.getColumnFamily(QueryFilter.getSliceFilter(TEST_KEY,
+                                                                        "Standard3",
+                                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                                        false,
+                                                                        0,
+                                                                        System.currentTimeMillis()));
+                assertColumns(cf);
+
+                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY,
+                                                                        "Standard3",
+                                                                        ByteBufferUtil.bytes("col99"),
+                                                                        System.currentTimeMillis()));
+                assertColumns(cf);
+            }
+        };
+        reTest(keyspace.getColumnFamilyStore("Standard3"), verify);
+    }
+
+    @Test
+    public void testGetRowSingleColumn() throws Throwable
+    {
+        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
+
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf.addColumn(column("col1","val1", 1L));
+        cf.addColumn(column("col2","val2", 1L));
+        cf.addColumn(column("col3","val3", 1L));
+        RowMutation rm = new RowMutation("Keyspace1", TEST_KEY.key, cf);
+        rm.apply();
+
+        Runnable verify = new WrappedRunnable()
+        {
+            public void runMayThrow() throws Exception
+            {
+                ColumnFamily cf;
+
+                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY,
+                                                                        "Standard1",
+                                                                        ByteBufferUtil.bytes("col1"),
+                                                                        System.currentTimeMillis()));
+                assertColumns(cf, "col1");
+
+                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY,
+                                                                        "Standard1",
+                                                                        ByteBufferUtil.bytes("col3"),
+                                                                        System.currentTimeMillis()));
+                assertColumns(cf, "col3");
+            }
+        };
+        reTest(keyspace.getColumnFamilyStore("Standard1"), verify);
+    }
+
+    @Test
+    public void testGetRowSliceByRange() throws Throwable
+    {
+    	DecoratedKey key = TEST_SLICE_KEY;
+    	Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        // First write "a", "b", "c"
+        cf.addColumn(column("a", "val1", 1L));
+        cf.addColumn(column("b", "val2", 1L));
+        cf.addColumn(column("c", "val3", 1L));
+        RowMutation rm = new RowMutation("Keyspace1", key.key, cf);
+        rm.apply();
+
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("c"), false, 100, System.currentTimeMillis());
+        assertEquals(2, cf.getColumnCount());
+
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("b"), false, 100, System.currentTimeMillis());
+        assertEquals(1, cf.getColumnCount());
+
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("c"), false, 1, System.currentTimeMillis());
+        assertEquals(1, cf.getColumnCount());
+    }
+
+    @Test
+    public void testGetSliceNoMatch() throws Throwable
+    {
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard2");
+        cf.addColumn(column("col1", "val1", 1));
+        RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("row1000"), cf);
+        rm.apply();
+
+        validateGetSliceNoMatch(keyspace);
+        keyspace.getColumnFamilyStore("Standard2").forceBlockingFlush();
+        validateGetSliceNoMatch(keyspace);
+
+        Collection<SSTableReader> ssTables = keyspace.getColumnFamilyStore("Standard2").getSSTables();
+        assertEquals(1, ssTables.size());
+        ssTables.iterator().next().forceFilterFailures();
+        validateGetSliceNoMatch(keyspace);
+    }
+
+    @Test
+    public void testGetSliceWithCutoff() throws Throwable
+    {
+        // tests slicing against data from one row in a memtable and then flushed to an sstable
+        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
+        final DecoratedKey ROW = Util.dk("row4");
+        final NumberFormat fmt = new DecimalFormat("000");
+
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        // at this rate, we're getting 78-79 cos/block, assuming the blocks are set to be about 4k.
+        // so if we go to 300, we'll get at least 4 blocks, which is plenty for testing.
+        for (int i = 0; i < 300; i++)
+            cf.addColumn(column("col" + fmt.format(i), "omg!thisisthevalue!"+i, 1L));
+        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+        rm.apply();
+
+        Runnable verify = new WrappedRunnable()
+        {
+            public void runMayThrow() throws Exception
+            {
+                ColumnFamily cf;
+
+                // blocks are partitioned like this: 000-097, 098-193, 194-289, 290-299, assuming a 4k column index size.
+                assert DatabaseDescriptor.getColumnIndexSize() == 4096 : "Unexpected column index size, block boundaries won't be where tests expect them.";
+
+                // test forward, spanning a segment.
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col096"), ByteBufferUtil.bytes("col099"), false, 4, System.currentTimeMillis());
+                assertColumns(cf, "col096", "col097", "col098", "col099");
+
+                // test reversed, spanning a segment.
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col099"), ByteBufferUtil.bytes("col096"), true, 4, System.currentTimeMillis());
+                assertColumns(cf, "col096", "col097", "col098", "col099");
+
+                // test forward, within a segment.
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col100"), ByteBufferUtil.bytes("col103"), false, 4, System.currentTimeMillis());
+                assertColumns(cf, "col100", "col101", "col102", "col103");
+
+                // test reversed, within a segment.
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col103"), ByteBufferUtil.bytes("col100"), true, 4, System.currentTimeMillis());
+                assertColumns(cf, "col100", "col101", "col102", "col103");
+
+                // test forward from beginning, spanning a segment.
+                String[] strCols = new String[100]; // col000-col099
+                for (int i = 0; i < 100; i++)
+                    strCols[i] = "col" + fmt.format(i);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.bytes("col099"), false, 100, System.currentTimeMillis());
+                assertColumns(cf, strCols);
+
+                // test reversed, from end, spanning a segment.
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.bytes("col288"), true, 12, System.currentTimeMillis());
+                assertColumns(cf, "col288", "col289", "col290", "col291", "col292", "col293", "col294", "col295", "col296", "col297", "col298", "col299");
+            }
+        };
+
+        reTest(keyspace.getColumnFamilyStore("Standard1"), verify);
+    }
+
+    @Test
+    public void testReversedWithFlushing() throws IOException, ExecutionException, InterruptedException
+    {
+        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardLong1");
+        final DecoratedKey ROW = Util.dk("row4");
+
+        for (int i = 0; i < 10; i++)
+        {
+            ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
+            cf.addColumn(new Column(ByteBufferUtil.bytes((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
+            RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+            rm.apply();
+        }
+
+        cfs.forceBlockingFlush();
+
+        for (int i = 10; i < 20; i++)
+        {
+            ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
+            cf.addColumn(new Column(ByteBufferUtil.bytes((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
+            RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+            rm.apply();
+
+            cf = cfs.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 1, System.currentTimeMillis());
+            assertEquals(1, Iterables.size(cf.getColumnNames()));
+            assertEquals(i, cf.getColumnNames().iterator().next().getLong());
+        }
+    }
+
+    private void validateGetSliceNoMatch(Keyspace keyspace) throws IOException
+    {
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard2");
+        ColumnFamily cf;
+
+        // key before the rows that exists
+        cf = cfStore.getColumnFamily(Util.dk("a"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1, System.currentTimeMillis());
+        assertColumns(cf);
+
+        // key after the rows that exist
+        cf = cfStore.getColumnFamily(Util.dk("z"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1, System.currentTimeMillis());
+        assertColumns(cf);
+    }
+
+    @Test
+    public void testGetSliceFromBasic() throws Throwable
+    {
+        // tests slicing against data from one row in a memtable and then flushed to an sstable
+        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
+        final DecoratedKey ROW = Util.dk("row1");
+
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf.addColumn(column("col1", "val1", 1L));
+        cf.addColumn(column("col3", "val3", 1L));
+        cf.addColumn(column("col4", "val4", 1L));
+        cf.addColumn(column("col5", "val5", 1L));
+        cf.addColumn(column("col7", "val7", 1L));
+        cf.addColumn(column("col9", "val9", 1L));
+        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+        rm.apply();
+
+        rm = new RowMutation("Keyspace1", ROW.key);
+        rm.delete("Standard1", ByteBufferUtil.bytes("col4"), 2L);
+        rm.apply();
+
+        Runnable verify = new WrappedRunnable()
+        {
+            public void runMayThrow() throws Exception
+            {
+                ColumnFamily cf;
+
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col5"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2, System.currentTimeMillis());
+                assertColumns(cf, "col5", "col7");
+
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col4"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2, System.currentTimeMillis());
+                assertColumns(cf, "col4", "col5", "col7");
+                assertColumns(ColumnFamilyStore.removeDeleted(cf, Integer.MAX_VALUE), "col5", "col7");
+
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col5"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2, System.currentTimeMillis());
+                assertColumns(cf, "col3", "col4", "col5");
+
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col6"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2, System.currentTimeMillis());
+                assertColumns(cf, "col3", "col4", "col5");
+
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2, System.currentTimeMillis());
+                assertColumns(cf, "col7", "col9");
+
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col95"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2, System.currentTimeMillis());
+                assertColumns(cf);
+
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2, System.currentTimeMillis());
+                assertColumns(cf);
+            }
+        };
+
+        reTest(keyspace.getColumnFamilyStore("Standard1"), verify);
+    }
+
+    @Test
+    public void testGetSliceWithExpiration() throws Throwable
+    {
+        // tests slicing against data from one row with expiring column in a memtable and then flushed to an sstable
+        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
+        final DecoratedKey ROW = Util.dk("row5");
+
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf.addColumn(column("col1", "val1", 1L));
+        cf.addColumn(expiringColumn("col2", "val2", 1L, 60)); // long enough not to be tombstoned
+        cf.addColumn(column("col3", "val3", 1L));
+        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+        rm.apply();
+
+        Runnable verify = new WrappedRunnable()
+        {
+            public void runMayThrow() throws Exception
+            {
+                ColumnFamily cf;
+
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2, System.currentTimeMillis());
+                assertColumns(cf, "col1", "col2");
+                assertColumns(ColumnFamilyStore.removeDeleted(cf, Integer.MAX_VALUE), "col1");
+
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1, System.currentTimeMillis());
+                assertColumns(cf, "col2");
+                assertColumns(ColumnFamilyStore.removeDeleted(cf, Integer.MAX_VALUE));
+            }
+        };
+
+        reTest(keyspace.getColumnFamilyStore("Standard1"), verify);
+    }
+
+    @Test
+    public void testGetSliceFromAdvanced() throws Throwable
+    {
+        // tests slicing against data from one row spread across two sstables
+        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
+        final DecoratedKey ROW = Util.dk("row2");
+
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf.addColumn(column("col1", "val1", 1L));
+        cf.addColumn(column("col2", "val2", 1L));
+        cf.addColumn(column("col3", "val3", 1L));
+        cf.addColumn(column("col4", "val4", 1L));
+        cf.addColumn(column("col5", "val5", 1L));
+        cf.addColumn(column("col6", "val6", 1L));
+        RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
+        rm.apply();
+        cfStore.forceBlockingFlush();
+
+        cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf.addColumn(column("col1", "valx", 2L));
+        cf.addColumn(column("col2", "valx", 2L));
+        cf.addColumn(column("col3", "valx", 2L));
+        rm = new RowMutation("Keyspace1", ROW.key, cf);
+        rm.apply();
+
+        Runnable verify = new WrappedRunnable()
+        {
+            public void runMayThrow() throws Exception
+            {
+                ColumnFamily cf;
+
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3, System.currentTimeMillis());
+                assertColumns(cf, "col2", "col3", "col4");
+
+                ByteBuffer col = cf.getColumn(ByteBufferUtil.bytes("col2")).value();
+                assertEquals(ByteBufferUtil.string(col), "valx");
+
+                col = cf.getColumn(ByteBufferUtil.bytes("col3")).value();
+                assertEquals(ByteBufferUtil.string(col), "valx");
+
+                col = cf.getColumn(ByteBufferUtil.bytes("col4")).value();
+                assertEquals(ByteBufferUtil.string(col), "val4");
+            }
+        };
+
+        reTest(keyspace.getColumnFamilyStore("Standard1"), verify);
+    }
+
+    @Test
+    public void testGetSliceFromLarge() throws Throwable
+    {
+        // tests slicing against 1000 columns in an sstable
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
+        DecoratedKey key = Util.dk("row3");
+        ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        for (int i = 1000; i < 2000; i++)
+            cf.addColumn(column("col" + i, ("v" + i), 1L));
+        RowMutation rm = new RowMutation("Keyspace1", key.key, cf);
+        rm.apply();
+        cfStore.forceBlockingFlush();
+
+        validateSliceLarge(cfStore);
+
+        // compact so we have a big row with more than the minimum index count
+        if (cfStore.getSSTables().size() > 1)
+        {
+            CompactionManager.instance.performMaximal(cfStore);
+        }
+        // verify that we do indeed have multiple index entries
+        SSTableReader sstable = cfStore.getSSTables().iterator().next();
+        RowIndexEntry indexEntry = sstable.getPosition(key, SSTableReader.Operator.EQ);
+        assert indexEntry.columnsIndex().size() > 2;
+
+        validateSliceLarge(cfStore);
+    }
+
+    @Test
+    public void testLimitSSTables() throws CharacterCodingException, InterruptedException
+    {
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
+        cfStore.disableAutoCompaction();
+        DecoratedKey key = Util.dk("row_maxmin");
+        for (int j = 0; j < 10; j++)
+        {
+            ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+            for (int i = 1000 + (j*100); i < 1000 + ((j+1)*100); i++)
+            {
+                cf.addColumn(column("col" + i, ("v" + i), i));
+            }
+            RowMutation rm = new RowMutation("Keyspace1", key.key, cf);
+            rm.apply();
+            cfStore.forceBlockingFlush();
+        }
+        cfStore.metric.sstablesPerReadHistogram.clear();
+        ColumnFamily cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes(""), ByteBufferUtil.bytes("col1499"), false, 1000, System.currentTimeMillis());
+        assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
+        int i = 0;
+        for (Column c : cf.getSortedColumns())
+        {
+            assertEquals(ByteBufferUtil.string(c.name), "col" + (1000 + i++));
+        }
+        assertEquals(i, 500);
+        cfStore.metric.sstablesPerReadHistogram.clear();
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1500"), ByteBufferUtil.bytes("col2000"), false, 1000, System.currentTimeMillis());
+        assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
+
+        for (Column c : cf.getSortedColumns())
+        {
+            assertEquals(ByteBufferUtil.string(c.name), "col"+(1000 + i++));
+        }
+        assertEquals(i, 1000);
+
+        // reverse
+        cfStore.metric.sstablesPerReadHistogram.clear();
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col2000"), ByteBufferUtil.bytes("col1500"), true, 1000, System.currentTimeMillis());
+        assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
+        i = 500;
+        for (Column c : cf.getSortedColumns())
+        {
+            assertEquals(ByteBufferUtil.string(c.name), "col"+(1000 + i++));
+        }
+        assertEquals(i, 1000);
+
+    }
+
+    @Test
+    public void testLimitSSTablesComposites() throws CharacterCodingException, ExecutionException, InterruptedException
+    {
+        /*
+        creates 10 sstables, composite columns like this:
+        ---------------------
+        k   |a0:0|a1:1|..|a9:9
+        ---------------------
+        ---------------------
+        k   |a0:10|a1:11|..|a9:19
+        ---------------------
+        ...
+        ---------------------
+        k   |a0:90|a1:91|..|a9:99
+        ---------------------
+        then we slice out col1 = a5 and col2 > 85 -> which should let us just check 2 sstables and get 2 columns
+         */
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardComposite2");
+        cfs.disableAutoCompaction();
+
+        CompositeType ct = CompositeType.getInstance(BytesType.instance, IntegerType.instance);
+        DecoratedKey key = Util.dk("k");
+        for (int j = 0; j < 10; j++)
+        {
+            for (int i = 0; i < 10; i++)
+            {
+                RowMutation rm = new RowMutation("Keyspace1", key.key);
+                ByteBuffer colName = ct.builder().add(ByteBufferUtil.bytes("a" + i)).add(ByteBufferUtil.bytes(j*10 + i)).build();
+                rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+                rm.apply();
+            }
+            cfs.forceBlockingFlush();
+        }
+        ByteBuffer start = ct.builder().add(ByteBufferUtil.bytes("a5")).add(ByteBufferUtil.bytes(85)).build();
+        ByteBuffer finish = ct.builder().add(ByteBufferUtil.bytes("a5")).buildAsEndOfRange();
+        cfs.metric.sstablesPerReadHistogram.clear();
+        ColumnFamily cf = cfs.getColumnFamily(key, start, finish, false, 1000, System.currentTimeMillis());
+        int colCount = 0;
+        for (Column c : cf)
+            colCount++;
+        assertEquals(2, colCount);
+        assertEquals(2, cfs.metric.sstablesPerReadHistogram.max(), 0.1);
+    }
+
+    private void validateSliceLarge(ColumnFamilyStore cfStore) throws IOException
+    {
+        DecoratedKey key = Util.dk("row3");
+        ColumnFamily cf;
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3, System.currentTimeMillis());
+        assertColumns(cf, "col1000", "col1001", "col1002");
+
+        ByteBuffer col;
+        col = cf.getColumn(ByteBufferUtil.bytes("col1000")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1000");
+        col = cf.getColumn(ByteBufferUtil.bytes("col1001")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1001");
+        col = cf.getColumn(ByteBufferUtil.bytes("col1002")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1002");
+
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1195"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3, System.currentTimeMillis());
+        assertColumns(cf, "col1195", "col1196", "col1197");
+
+        col = cf.getColumn(ByteBufferUtil.bytes("col1195")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1195");
+        col = cf.getColumn(ByteBufferUtil.bytes("col1196")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1196");
+        col = cf.getColumn(ByteBufferUtil.bytes("col1197")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1197");
+
+
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1996"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 1000, System.currentTimeMillis());
+        Column[] columns = cf.getSortedColumns().toArray(new Column[0]);
+        for (int i = 1000; i < 1996; i++)
+        {
+            String expectedName = "col" + i;
+            Column column = columns[i - 1000];
+            assertEquals(ByteBufferUtil.string(column.name()), expectedName);
+            assertEquals(ByteBufferUtil.string(column.value()), ("v" + i));
+        }
+
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1990"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3, System.currentTimeMillis());
+        assertColumns(cf, "col1990", "col1991", "col1992");
+        col = cf.getColumn(ByteBufferUtil.bytes("col1990")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1990");
+        col = cf.getColumn(ByteBufferUtil.bytes("col1991")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1991");
+        col = cf.getColumn(ByteBufferUtil.bytes("col1992")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1992");
+
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 3, System.currentTimeMillis());
+        assertColumns(cf, "col1997", "col1998", "col1999");
+        col = cf.getColumn(ByteBufferUtil.bytes("col1997")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1997");
+        col = cf.getColumn(ByteBufferUtil.bytes("col1998")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1998");
+        col = cf.getColumn(ByteBufferUtil.bytes("col1999")).value();
+        assertEquals(ByteBufferUtil.string(col), "v1999");
+
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col9000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 3, System.currentTimeMillis());
+        assertColumns(cf, "col1997", "col1998", "col1999");
+
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col9000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3, System.currentTimeMillis());
+        assertColumns(cf);
+    }
+
+    public static void assertColumns(ColumnFamily container, String... columnNames)
+    {
+        Collection<Column> columns = container == null ? new TreeSet<Column>() : container.getSortedColumns();
+        List<String> L = new ArrayList<String>();
+        for (Column column : columns)
+        {
+            try
+            {
+                L.add(ByteBufferUtil.string(column.name()));
+            }
+            catch (CharacterCodingException e)
+            {
+                throw new AssertionError(e);
+            }
+        }
+
+        List<String> names = new ArrayList<String>(columnNames.length);
+
+        names.addAll(Arrays.asList(columnNames));
+
+        String[] columnNames1 = names.toArray(new String[0]);
+        String[] la = L.toArray(new String[columns.size()]);
+
+        assert Arrays.equals(la, columnNames1)
+                : String.format("Columns [%s])] is not expected [%s]",
+                                ((container == null) ? "" : container.getComparator().getColumnsString(columns)),
+                                StringUtils.join(columnNames1, ","));
+    }
+
+    public static void assertColumn(ColumnFamily cf, String name, String value, long timestamp)
+    {
+        assertColumn(cf.getColumn(ByteBufferUtil.bytes(name)), value, timestamp);
+    }
+
+    public static void assertColumn(Column column, String value, long timestamp)
+    {
+        assertNotNull(column);
+        assertEquals(0, ByteBufferUtil.compareUnsigned(column.value(), ByteBufferUtil.bytes(value)));
+        assertEquals(timestamp, column.timestamp());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/MultitableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/MultitableTest.java b/test/unit/org/apache/cassandra/db/MultitableTest.java
index 606af98..3d0c8ac 100644
--- a/test/unit/org/apache/cassandra/db/MultitableTest.java
+++ b/test/unit/org/apache/cassandra/db/MultitableTest.java
@@ -27,7 +27,7 @@ import java.util.concurrent.ExecutionException;
 import org.apache.cassandra.Util;
 import org.junit.Test;
 
-import static org.apache.cassandra.db.TableTest.assertColumns;
+import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
 import org.apache.cassandra.SchemaLoader;
 import static org.apache.cassandra.Util.column;
 
@@ -36,8 +36,8 @@ public class MultitableTest extends SchemaLoader
     @Test
     public void testSameCFs() throws IOException, ExecutionException, InterruptedException
     {
-        Table table1 = Table.open("Keyspace1");
-        Table table2 = Table.open("Keyspace2");
+        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        Keyspace keyspace2 = Keyspace.open("Keyspace2");
 
         RowMutation rm;
         DecoratedKey dk = Util.dk("keymulti");
@@ -53,10 +53,10 @@ public class MultitableTest extends SchemaLoader
         rm = new RowMutation("Keyspace2", dk.key, cf);
         rm.apply();
 
-        table1.getColumnFamilyStore("Standard1").forceBlockingFlush();
-        table2.getColumnFamilyStore("Standard1").forceBlockingFlush();
+        keyspace1.getColumnFamilyStore("Standard1").forceBlockingFlush();
+        keyspace2.getColumnFamilyStore("Standard1").forceBlockingFlush();
 
-        assertColumns(Util.getColumnFamily(table1, dk, "Standard1"), "col1");
-        assertColumns(Util.getColumnFamily(table2, dk, "Standard1"), "col2");
+        assertColumns(Util.getColumnFamily(keyspace1, dk, "Standard1"), "col1");
+        assertColumns(Util.getColumnFamily(keyspace2, dk, "Standard1"), "col2");
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/NameSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NameSortTest.java b/test/unit/org/apache/cassandra/db/NameSortTest.java
index 97a4a8a..4babab2 100644
--- a/test/unit/org/apache/cassandra/db/NameSortTest.java
+++ b/test/unit/org/apache/cassandra/db/NameSortTest.java
@@ -56,7 +56,7 @@ public class NameSortTest extends SchemaLoader
 
     private void testNameSort(int N) throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
 
         for (int i = 0; i < N; ++i)
         {
@@ -85,21 +85,21 @@ public class NameSortTest extends SchemaLoader
             }
         }
 
-        validateNameSort(table, N);
+        validateNameSort(keyspace, N);
 
-        table.getColumnFamilyStore("Standard1").forceBlockingFlush();
-        table.getColumnFamilyStore("Super1").forceBlockingFlush();
-        validateNameSort(table, N);
+        keyspace.getColumnFamilyStore("Standard1").forceBlockingFlush();
+        keyspace.getColumnFamilyStore("Super1").forceBlockingFlush();
+        validateNameSort(keyspace, N);
     }
 
-    private void validateNameSort(Table table, int N) throws IOException
+    private void validateNameSort(Keyspace keyspace, int N) throws IOException
     {
         for (int i = 0; i < N; ++i)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
             ColumnFamily cf;
 
-            cf = Util.getColumnFamily(table, key, "Standard1");
+            cf = Util.getColumnFamily(keyspace, key, "Standard1");
             Collection<Column> columns = cf.getSortedColumns();
             for (Column column : columns)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 0ed0c4a..9d04cfd 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -38,8 +38,8 @@ public class RangeTombstoneTest extends SchemaLoader
     @Test
     public void simpleQueryWithRangeTombstoneTest() throws Exception
     {
-        Table table = Table.open(KSNAME);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(CFNAME);
+        Keyspace keyspace = Keyspace.open(KSNAME);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CFNAME);
 
         // Inserting data
         String key = "k1";
@@ -98,8 +98,8 @@ public class RangeTombstoneTest extends SchemaLoader
     public void overlappingRangeTest() throws Exception
     {
         CompactionManager.instance.disableAutoCompaction();
-        Table table = Table.open(KSNAME);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(CFNAME);
+        Keyspace keyspace = Keyspace.open(KSNAME);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CFNAME);
 
         // Inserting data
         String key = "k2";

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/ReadMessageTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ReadMessageTest.java b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
index cc15fe7..b16b921 100644
--- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
@@ -85,7 +85,7 @@ public class ReadMessageTest extends SchemaLoader
     @Test
     public void testGetColumn() throws IOException, ColumnFamilyNotDefinedException
     {
-        Table table = Table.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
 
@@ -95,7 +95,7 @@ public class ReadMessageTest extends SchemaLoader
         rm.apply();
 
         ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.key, "Standard1", System.currentTimeMillis(), new NamesQueryFilter(ByteBufferUtil.bytes("Column1")));
-        Row row = command.getRow(table);
+        Row row = command.getRow(keyspace);
         Column col = row.cf.getColumn(ByteBufferUtil.bytes("Column1"));
         assertEquals(col.value(), ByteBuffer.wrap("abcd".getBytes()));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java b/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
index 0e743f6..ac2d642 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
@@ -43,7 +43,7 @@ public class RecoveryManager2Test extends SchemaLoader
     public void testWithFlush() throws Exception
     {
         // Flush everything that may be in the commit log now to start fresh
-        FBUtilities.waitOnFutures(Table.open(Table.SYSTEM_KS).flush());
+        FBUtilities.waitOnFutures(Keyspace.open(Keyspace.SYSTEM_KS).flush());
 
         CompactionManager.instance.disableAutoCompaction();
 
@@ -56,8 +56,8 @@ public class RecoveryManager2Test extends SchemaLoader
             insertRow("Standard1", key);
         }
 
-        Table table1 = Table.open("Keyspace1");
-        ColumnFamilyStore cfs = table1.getColumnFamilyStore("Standard1");
+        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace1.getColumnFamilyStore("Standard1");
         logger.debug("forcing flush");
         cfs.forceBlockingFlush();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java b/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
index dcc0901..8258970 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
@@ -34,15 +34,15 @@ import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.io.util.FileUtils;
 
 import static org.apache.cassandra.Util.column;
-import static org.apache.cassandra.db.TableTest.assertColumns;
+import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
 
 public class RecoveryManager3Test extends SchemaLoader
 {
     @Test
     public void testMissingHeader() throws IOException, ExecutionException, InterruptedException
     {
-        Table table1 = Table.open("Keyspace1");
-        Table table2 = Table.open("Keyspace2");
+        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        Keyspace keyspace2 = Keyspace.open("Keyspace2");
 
         RowMutation rm;
         DecoratedKey dk = Util.dk("keymulti");
@@ -58,8 +58,8 @@ public class RecoveryManager3Test extends SchemaLoader
         rm = new RowMutation("Keyspace2", dk.key, cf);
         rm.apply();
 
-        table1.getColumnFamilyStore("Standard1").clearUnsafe();
-        table2.getColumnFamilyStore("Standard3").clearUnsafe();
+        keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();
+        keyspace2.getColumnFamilyStore("Standard3").clearUnsafe();
 
         // nuke the header
         for (File file : new File(DatabaseDescriptor.getCommitLogLocation()).listFiles())
@@ -71,7 +71,7 @@ public class RecoveryManager3Test extends SchemaLoader
         CommitLog.instance.resetUnsafe(); // disassociate segments from live CL
         CommitLog.instance.recover();
 
-        assertColumns(Util.getColumnFamily(table1, dk, "Standard1"), "col1");
-        assertColumns(Util.getColumnFamily(table2, dk, "Standard3"), "col2");
+        assertColumns(Util.getColumnFamily(keyspace1, dk, "Standard1"), "col1");
+        assertColumns(Util.getColumnFamily(keyspace2, dk, "Standard3"), "col2");
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
index a98a401..bd15bbf 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
@@ -29,7 +29,7 @@ import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.apache.cassandra.Util.column;
-import static org.apache.cassandra.db.TableTest.assertColumns;
+import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
 
 public class RecoveryManagerTest extends SchemaLoader
 {
@@ -41,8 +41,8 @@ public class RecoveryManagerTest extends SchemaLoader
     @Test
     public void testOne() throws IOException, ExecutionException, InterruptedException
     {
-        Table table1 = Table.open("Keyspace1");
-        Table table2 = Table.open("Keyspace2");
+        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        Keyspace keyspace2 = Keyspace.open("Keyspace2");
 
         RowMutation rm;
         DecoratedKey dk = Util.dk("keymulti");
@@ -58,20 +58,20 @@ public class RecoveryManagerTest extends SchemaLoader
         rm = new RowMutation("Keyspace2", dk.key, cf);
         rm.apply();
 
-        table1.getColumnFamilyStore("Standard1").clearUnsafe();
-        table2.getColumnFamilyStore("Standard3").clearUnsafe();
+        keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();
+        keyspace2.getColumnFamilyStore("Standard3").clearUnsafe();
 
         CommitLog.instance.resetUnsafe(); // disassociate segments from live CL
         CommitLog.instance.recover();
 
-        assertColumns(Util.getColumnFamily(table1, dk, "Standard1"), "col1");
-        assertColumns(Util.getColumnFamily(table2, dk, "Standard3"), "col2");
+        assertColumns(Util.getColumnFamily(keyspace1, dk, "Standard1"), "col1");
+        assertColumns(Util.getColumnFamily(keyspace2, dk, "Standard3"), "col2");
     }
 
     @Test
     public void testRecoverCounter() throws IOException, ExecutionException, InterruptedException
     {
-        Table table1 = Table.open("Keyspace1");
+        Keyspace keyspace1 = Keyspace.open("Keyspace1");
 
         RowMutation rm;
         DecoratedKey dk = Util.dk("key");
@@ -85,12 +85,12 @@ public class RecoveryManagerTest extends SchemaLoader
             rm.apply();
         }
 
-        table1.getColumnFamilyStore("Counter1").clearUnsafe();
+        keyspace1.getColumnFamilyStore("Counter1").clearUnsafe();
 
         CommitLog.instance.resetUnsafe(); // disassociate segments from live CL
         CommitLog.instance.recover();
 
-        cf = Util.getColumnFamily(table1, dk, "Counter1");
+        cf = Util.getColumnFamily(keyspace1, dk, "Counter1");
 
         assert cf.getColumnCount() == 1;
         Column c = cf.getColumn(ByteBufferUtil.bytes("col"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
index 5e34e91..b98a8b0 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
@@ -40,8 +40,8 @@ public class RecoveryManagerTruncateTest extends SchemaLoader
 	@Test
 	public void testTruncate() throws IOException, ExecutionException, InterruptedException
 	{
-		Table table = Table.open("Keyspace1");
-		ColumnFamilyStore cfs = table.getColumnFamilyStore("Standard1");
+		Keyspace keyspace = Keyspace.open("Keyspace1");
+		ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
 		RowMutation rm;
 		ColumnFamily cf;
@@ -53,7 +53,7 @@ public class RecoveryManagerTruncateTest extends SchemaLoader
 		rm.apply();
 
 		// Make sure data was written
-		assertNotNull(getFromTable(table, "Standard1", "keymulti", "col1"));
+		assertNotNull(getFromTable(keyspace, "Standard1", "keymulti", "col1"));
 
 		// and now truncate it
 		cfs.truncateBlocking();
@@ -61,13 +61,13 @@ public class RecoveryManagerTruncateTest extends SchemaLoader
 		CommitLog.instance.recover();
 
 		// and validate truncation.
-		assertNull(getFromTable(table, "Standard1", "keymulti", "col1"));
+		assertNull(getFromTable(keyspace, "Standard1", "keymulti", "col1"));
 	}
 
-	private Column getFromTable(Table table, String cfName, String keyName, String columnName)
+	private Column getFromTable(Keyspace keyspace, String cfName, String keyName, String columnName)
 	{
 		ColumnFamily cf;
-		ColumnFamilyStore cfStore = table.getColumnFamilyStore(cfName);
+		ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(cfName);
 		if (cfStore == null)
 		{
 			return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
index 345f967..f937709 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
@@ -36,8 +36,8 @@ public class RemoveColumnFamilyTest extends SchemaLoader
     @Test
     public void testRemoveColumnFamily() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
index bd97e00..08913c6 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
@@ -36,8 +36,8 @@ public class RemoveColumnFamilyWithFlush1Test extends SchemaLoader
     @Test
     public void testRemoveColumnFamilyWithFlush1() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
index 037c5dd..b7fc640 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
@@ -36,8 +36,8 @@ public class RemoveColumnFamilyWithFlush2Test extends SchemaLoader
     @Test
     public void testRemoveColumnFamilyWithFlush2() throws IOException, ExecutionException, InterruptedException
     {
-        Table table = Table.open("Keyspace1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
 


[09/11] Rename Table to Keyspace

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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
new file mode 100644
index 0000000..667a656
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -0,0 +1,454 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Future;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.pager.QueryPagers;
+import org.apache.cassandra.tracing.Tracing;
+
+/**
+ * It represents a Keyspace.
+ */
+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);
+
+    /**
+     * accesses to CFS.memtable should acquire this for thread safety.
+     * CFS.maybeSwitchMemtable should aquire the writeLock; see that method for the full explanation.
+     * <p/>
+     * (Enabling fairness in the RRWL is observed to decrease throughput, so we leave it off.)
+     */
+    public static final ReentrantReadWriteLock switchLock = new ReentrantReadWriteLock();
+
+    // It is possible to call Keyspace.open without a running daemon, so it makes sense to ensure
+    // proper directories here as well as in CassandraDaemon.
+    static
+    {
+        if (!StorageService.instance.isClientMode())
+            DatabaseDescriptor.createAllDirectories();
+    }
+
+    public final KSMetaData metadata;
+
+    /* ColumnFamilyStore per column family */
+    private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<UUID, ColumnFamilyStore>();
+    private volatile AbstractReplicationStrategy replicationStrategy;
+    public static final Function<String,Keyspace> keyspaceTransformer = new Function<String, Keyspace>()
+    {
+        public Keyspace apply(String keyspaceName)
+        {
+            return Keyspace.open(keyspaceName);
+        }
+    };
+
+    public static Keyspace open(String keyspaceName)
+    {
+        return open(keyspaceName, Schema.instance, true);
+    }
+
+    public static Keyspace openWithoutSSTables(String keyspaceName)
+    {
+        return open(keyspaceName, Schema.instance, false);
+    }
+
+    private static Keyspace open(String keyspaceName, Schema schema, boolean loadSSTables)
+    {
+        Keyspace keyspaceInstance = schema.getKeyspaceInstance(keyspaceName);
+
+        if (keyspaceInstance == null)
+        {
+            // instantiate the Keyspace.  we could use putIfAbsent but it's important to making sure it is only done once
+            // per keyspace, so we synchronize and re-check before doing it.
+            synchronized (Keyspace.class)
+            {
+                keyspaceInstance = schema.getKeyspaceInstance(keyspaceName);
+                if (keyspaceInstance == null)
+                {
+                    // open and store the keyspace
+                    keyspaceInstance = new Keyspace(keyspaceName, loadSSTables);
+                    schema.storeKeyspaceInstance(keyspaceInstance);
+
+                    // keyspace has to be constructed and in the cache before cacheRow can be called
+                    for (ColumnFamilyStore cfs : keyspaceInstance.getColumnFamilyStores())
+                        cfs.initRowCache();
+                }
+            }
+        }
+        return keyspaceInstance;
+    }
+
+    public static Keyspace clear(String keyspaceName)
+    {
+        return clear(keyspaceName, Schema.instance);
+    }
+
+    public static Keyspace clear(String keyspaceName, Schema schema)
+    {
+        synchronized (Keyspace.class)
+        {
+            Keyspace t = schema.removeKeyspaceInstance(keyspaceName);
+            if (t != null)
+            {
+                for (ColumnFamilyStore cfs : t.getColumnFamilyStores())
+                    t.unloadCf(cfs);
+            }
+            return t;
+        }
+    }
+
+    /**
+     * Removes every SSTable in the directory from the appropriate DataTracker's view.
+     * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
+     */
+    public static void removeUnreadableSSTables(File directory)
+    {
+        for (Keyspace keyspace : Keyspace.all())
+        {
+            for (ColumnFamilyStore baseCfs : keyspace.getColumnFamilyStores())
+            {
+                for (ColumnFamilyStore cfs : baseCfs.concatWithIndexes())
+                    cfs.maybeRemoveUnreadableSSTables(directory);
+            }
+        }
+    }
+
+    public Collection<ColumnFamilyStore> getColumnFamilyStores()
+    {
+        return Collections.unmodifiableCollection(columnFamilyStores.values());
+    }
+
+    public ColumnFamilyStore getColumnFamilyStore(String cfName)
+    {
+        UUID id = Schema.instance.getId(getName(), cfName);
+        if (id == null)
+            throw new IllegalArgumentException(String.format("Unknown keyspace/cf pair (%s.%s)", getName(), cfName));
+        return getColumnFamilyStore(id);
+    }
+
+    public ColumnFamilyStore getColumnFamilyStore(UUID id)
+    {
+        ColumnFamilyStore cfs = columnFamilyStores.get(id);
+        if (cfs == null)
+            throw new IllegalArgumentException("Unknown CF " + id);
+        return cfs;
+    }
+
+    /**
+     * Take a snapshot of the specific column family, or the entire set of column families
+     * if columnFamily is null with a given timestamp
+     *
+     * @param snapshotName     the tag associated with the name of the snapshot.  This value may not be null
+     * @param columnFamilyName the column family to snapshot or all on null
+     * @throws IOException if the column family doesn't exist
+     */
+    public void snapshot(String snapshotName, String columnFamilyName) throws IOException
+    {
+        assert snapshotName != null;
+        boolean tookSnapShot = false;
+        for (ColumnFamilyStore cfStore : columnFamilyStores.values())
+        {
+            if (columnFamilyName == null || cfStore.name.equals(columnFamilyName))
+            {
+                tookSnapShot = true;
+                cfStore.snapshot(snapshotName);
+            }
+        }
+
+        if ((columnFamilyName != null) && !tookSnapShot)
+            throw new IOException("Failed taking snapshot. Column family " + columnFamilyName + " does not exist.");
+    }
+
+    /**
+     * @param clientSuppliedName may be null.
+     * @return the name of the snapshot
+     */
+    public static String getTimestampedSnapshotName(String clientSuppliedName)
+    {
+        String snapshotName = Long.toString(System.currentTimeMillis());
+        if (clientSuppliedName != null && !clientSuppliedName.equals(""))
+        {
+            snapshotName = snapshotName + "-" + clientSuppliedName;
+        }
+        return snapshotName;
+    }
+
+    /**
+     * Check whether snapshots already exists for a given name.
+     *
+     * @param snapshotName the user supplied snapshot name
+     * @return true if the snapshot exists
+     */
+    public boolean snapshotExists(String snapshotName)
+    {
+        assert snapshotName != null;
+        for (ColumnFamilyStore cfStore : columnFamilyStores.values())
+        {
+            if (cfStore.snapshotExists(snapshotName))
+                return true;
+        }
+        return false;
+    }
+
+    /**
+     * Clear all the snapshots for a given keyspace.
+     *
+     * @param snapshotName the user supplied snapshot name. It empty or null,
+     *                     all the snapshots will be cleaned
+     */
+    public void clearSnapshot(String snapshotName)
+    {
+        for (ColumnFamilyStore cfStore : columnFamilyStores.values())
+        {
+            cfStore.clearSnapshot(snapshotName);
+        }
+    }
+
+    /**
+     * @return A list of open SSTableReaders
+     */
+    public List<SSTableReader> getAllSSTables()
+    {
+        List<SSTableReader> list = new ArrayList<SSTableReader>(columnFamilyStores.size());
+        for (ColumnFamilyStore cfStore : columnFamilyStores.values())
+            list.addAll(cfStore.getSSTables());
+        return list;
+    }
+
+    private Keyspace(String keyspaceName, boolean loadSSTables)
+    {
+        metadata = Schema.instance.getKSMetaData(keyspaceName);
+        assert metadata != null : "Unknown keyspace " + keyspaceName;
+        createReplicationStrategy(metadata);
+
+        for (CFMetaData cfm : new ArrayList<CFMetaData>(metadata.cfMetaData().values()))
+        {
+            logger.debug("Initializing {}.{}", getName(), cfm.cfName);
+            initCf(cfm.cfId, cfm.cfName, loadSSTables);
+        }
+    }
+
+    public void createReplicationStrategy(KSMetaData ksm)
+    {
+        if (replicationStrategy != null)
+            StorageService.instance.getTokenMetadata().unregister(replicationStrategy);
+
+        replicationStrategy = AbstractReplicationStrategy.createReplicationStrategy(ksm.name,
+                                                                                    ksm.strategyClass,
+                                                                                    StorageService.instance.getTokenMetadata(),
+                                                                                    DatabaseDescriptor.getEndpointSnitch(),
+                                                                                    ksm.strategyOptions);
+    }
+
+    // best invoked on the compaction mananger.
+    public void dropCf(UUID cfId)
+    {
+        assert columnFamilyStores.containsKey(cfId);
+        ColumnFamilyStore cfs = columnFamilyStores.remove(cfId);
+        if (cfs == null)
+            return;
+
+        unloadCf(cfs);
+    }
+
+    // disassociate a cfs from this keyspace instance.
+    private void unloadCf(ColumnFamilyStore cfs)
+    {
+        cfs.forceBlockingFlush();
+        cfs.invalidate();
+    }
+
+    /**
+     * adds a cf to internal structures, ends up creating disk files).
+     */
+    public void initCf(UUID cfId, String cfName, boolean loadSSTables)
+    {
+        ColumnFamilyStore cfs = columnFamilyStores.get(cfId);
+
+        if (cfs == null)
+        {
+            // CFS being created for the first time, either on server startup or new CF being added.
+            // We don't worry about races here; startup is safe, and adding multiple idential CFs
+            // simultaneously is a "don't do that" scenario.
+            ColumnFamilyStore oldCfs = columnFamilyStores.putIfAbsent(cfId, ColumnFamilyStore.createColumnFamilyStore(this, cfName, loadSSTables));
+            // CFS mbean instantiation will error out before we hit this, but in case that changes...
+            if (oldCfs != null)
+                throw new IllegalStateException("added multiple mappings for cf id " + cfId);
+        }
+        else
+        {
+            // re-initializing an existing CF.  This will happen if you cleared the schema
+            // on this node and it's getting repopulated from the rest of the cluster.
+            assert cfs.name.equals(cfName);
+            cfs.metadata.reload();
+            cfs.reload();
+        }
+    }
+
+    public Row getRow(QueryFilter filter)
+    {
+        ColumnFamilyStore cfStore = getColumnFamilyStore(filter.getColumnFamilyName());
+        ColumnFamily columnFamily = cfStore.getColumnFamily(filter);
+        return new Row(filter.key, columnFamily);
+    }
+
+    public void apply(RowMutation mutation, boolean writeCommitLog)
+    {
+        apply(mutation, writeCommitLog, true);
+    }
+
+    /**
+     * This method appends a row to the global CommitLog, then updates memtables and indexes.
+     *
+     * @param mutation       the row to write.  Must not be modified after calling apply, since commitlog append
+     *                       may happen concurrently, depending on the CL Executor type.
+     * @param writeCommitLog false to disable commitlog append entirely
+     * @param updateIndexes  false to disable index updates (used by CollationController "defragmenting")
+     */
+    public void apply(RowMutation mutation, boolean writeCommitLog, boolean updateIndexes)
+    {
+        // write the mutation to the commitlog and memtables
+        Tracing.trace("Acquiring switchLock read lock");
+        switchLock.readLock().lock();
+        try
+        {
+            if (writeCommitLog)
+            {
+                Tracing.trace("Appending to commitlog");
+                CommitLog.instance.add(mutation);
+            }
+
+            DecoratedKey key = StorageService.getPartitioner().decorateKey(mutation.key());
+            for (ColumnFamily cf : mutation.getColumnFamilies())
+            {
+                ColumnFamilyStore cfs = columnFamilyStores.get(cf.id());
+                if (cfs == null)
+                {
+                    logger.error("Attempting to mutate non-existant column family " + cf.id());
+                    continue;
+                }
+
+                Tracing.trace("Adding to {} memtable", cf.metadata().cfName);
+                cfs.apply(key, cf, updateIndexes ? cfs.indexManager.updaterFor(key) : SecondaryIndexManager.nullUpdater);
+            }
+        }
+        finally
+        {
+            switchLock.readLock().unlock();
+        }
+    }
+
+    public AbstractReplicationStrategy getReplicationStrategy()
+    {
+        return replicationStrategy;
+    }
+
+    /**
+     * @param key row to index
+     * @param cfs ColumnFamily to index row in
+     * @param idxNames columns to index, in comparator order
+     */
+    public static void indexRow(DecoratedKey key, ColumnFamilyStore cfs, Set<String> idxNames)
+    {
+        if (logger.isDebugEnabled())
+            logger.debug("Indexing row {} ", cfs.metadata.getKeyValidator().getString(key.key));
+
+        Collection<SecondaryIndex> indexes = cfs.indexManager.getIndexesByNames(idxNames);
+
+        switchLock.readLock().lock();
+        try
+        {
+            Iterator<ColumnFamily> pager = QueryPagers.pageRowLocally(cfs, key.key, DEFAULT_PAGE_SIZE);
+            while (pager.hasNext())
+            {
+                ColumnFamily cf = pager.next();
+                ColumnFamily cf2 = cf.cloneMeShallow();
+                for (Column column : cf)
+                {
+                    if (cfs.indexManager.indexes(column.name(), indexes))
+                        cf2.addColumn(column);
+                }
+                cfs.indexManager.indexRow(key.key, cf2);
+            }
+        }
+        finally
+        {
+            switchLock.readLock().unlock();
+        }
+    }
+
+    public List<Future<?>> flush()
+    {
+        List<Future<?>> futures = new ArrayList<Future<?>>(columnFamilyStores.size());
+        for (UUID cfId : columnFamilyStores.keySet())
+            futures.add(columnFamilyStores.get(cfId).forceFlush());
+        return futures;
+    }
+
+    public static Iterable<Keyspace> all()
+    {
+        return Iterables.transform(Schema.instance.getKeyspaces(), keyspaceTransformer);
+    }
+
+    public static Iterable<Keyspace> nonSystem()
+    {
+        return Iterables.transform(Schema.instance.getNonSystemKeyspaces(), keyspaceTransformer);
+    }
+
+    public static Iterable<Keyspace> system()
+    {
+        return Iterables.transform(Schema.systemKeyspaceNames, keyspaceTransformer);
+    }
+
+    @Override
+    public String toString()
+    {
+        return getClass().getSimpleName() + "(name='" + getName() + "')";
+    }
+
+    public String getName()
+    {
+        return metadata.name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 c31c882..e323b69 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -410,7 +410,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(SystemTable.BATCHLOG_CF) && cfs.table.getName().equals(Table.SYSTEM_KS) && !(cf.getColumnCount() == 0))
+                        if (cfs.name.equals(SystemKeyspace.BATCHLOG_CF) && cfs.keyspace.getName().equals(Keyspace.SYSTEM_KS) && !(cf.getColumnCount() == 0))
                             continue;
 
                         // Pedantically, you could purge column level tombstones that are past GcGRace when writing to the SSTable.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
index 8986d5d..e593a98 100644
--- a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
@@ -40,7 +40,7 @@ public class MigrationRequestVerbHandler implements IVerbHandler
     {
         logger.debug("Received migration request from {}.", message.from);
 
-        Collection<RowMutation> schema = SystemTable.serializeSchema();
+        Collection<RowMutation> schema = SystemKeyspace.serializeSchema();
 
         MessageOut<Collection<RowMutation>> response = new MessageOut<Collection<RowMutation>>(MessagingService.Verb.INTERNAL_RESPONSE,
                                                                                                schema,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/PagedRangeCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/PagedRangeCommand.java b/src/java/org/apache/cassandra/db/PagedRangeCommand.java
index 0e1fa4f..265e9f7 100644
--- a/src/java/org/apache/cassandra/db/PagedRangeCommand.java
+++ b/src/java/org/apache/cassandra/db/PagedRangeCommand.java
@@ -104,7 +104,7 @@ public class PagedRangeCommand extends AbstractRangeCommand
 
     public List<Row> executeLocally()
     {
-        ColumnFamilyStore cfs = Table.open(keyspace).getColumnFamilyStore(columnFamily);
+        ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily);
 
         ExtendedFilter exFilter = cfs.makeExtendedFilter(keyRange, (SliceQueryFilter)predicate, start, stop, rowFilter, limit, timestamp);
         if (cfs.indexManager.hasIndexFor(rowFilter))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/RangeSliceCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RangeSliceCommand.java b/src/java/org/apache/cassandra/db/RangeSliceCommand.java
index c037518..b4afbcd 100644
--- a/src/java/org/apache/cassandra/db/RangeSliceCommand.java
+++ b/src/java/org/apache/cassandra/db/RangeSliceCommand.java
@@ -27,6 +27,7 @@ import java.util.concurrent.ExecutionException;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.filter.ExtendedFilter;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -128,7 +129,7 @@ public class RangeSliceCommand extends AbstractRangeCommand implements Pageable
 
     public List<Row> executeLocally()
     {
-        ColumnFamilyStore cfs = Table.open(keyspace).getColumnFamilyStore(columnFamily);
+        ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily);
 
         ExtendedFilter exFilter = cfs.makeExtendedFilter(keyRange, predicate, rowFilter, maxResults, countCQL3Rows, isPaging, timestamp);
         if (cfs.indexManager.hasIndexFor(rowFilter))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 3031da8..cadcd7d 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -63,28 +63,28 @@ public abstract class ReadCommand implements IReadCommand, Pageable
         return new MessageOut<ReadCommand>(MessagingService.Verb.READ, this, serializer);
     }
 
-    public final String table;
+    public final String ksName;
     public final String cfName;
     public final ByteBuffer key;
     public final long timestamp;
     private boolean isDigestQuery = false;
     protected final Type commandType;
 
-    protected ReadCommand(String table, ByteBuffer key, String cfName, long timestamp, Type cmdType)
+    protected ReadCommand(String ksName, ByteBuffer key, String cfName, long timestamp, Type cmdType)
     {
-        this.table = table;
+        this.ksName = ksName;
         this.key = key;
         this.cfName = cfName;
         this.timestamp = timestamp;
         this.commandType = cmdType;
     }
 
-    public static ReadCommand create(String table, ByteBuffer key, String cfName, long timestamp, IDiskAtomFilter filter)
+    public static ReadCommand create(String ksName, ByteBuffer key, String cfName, long timestamp, IDiskAtomFilter filter)
     {
         if (filter instanceof SliceQueryFilter)
-            return new SliceFromReadCommand(table, key, cfName, timestamp, (SliceQueryFilter)filter);
+            return new SliceFromReadCommand(ksName, key, cfName, timestamp, (SliceQueryFilter)filter);
         else
-            return new SliceByNamesReadCommand(table, key, cfName, timestamp, (NamesQueryFilter)filter);
+            return new SliceByNamesReadCommand(ksName, key, cfName, timestamp, (NamesQueryFilter)filter);
     }
 
     public boolean isDigestQuery()
@@ -104,13 +104,13 @@ public abstract class ReadCommand implements IReadCommand, Pageable
 
     public abstract ReadCommand copy();
 
-    public abstract Row getRow(Table table);
+    public abstract Row getRow(Keyspace keyspace);
 
     public abstract IDiskAtomFilter filter();
 
     public String getKeyspace()
     {
-        return table;
+        return ksName;
     }
 
     // maybeGenerateRetryCommand is used to generate a retry for short reads
@@ -142,11 +142,11 @@ class ReadCommandSerializer implements IVersionedSerializer<ReadCommand>
         ByteBuffer superColumn = null;
         if (version < MessagingService.VERSION_20)
         {
-            CFMetaData metadata = Schema.instance.getCFMetaData(command.table, command.cfName);
+            CFMetaData metadata = Schema.instance.getCFMetaData(command.ksName, command.cfName);
             if (metadata.cfType == ColumnFamilyType.Super)
             {
                 SuperColumns.SCFilter scFilter = SuperColumns.filterToSC((CompositeType)metadata.comparator, command.filter());
-                newCommand = ReadCommand.create(command.table, command.key, command.cfName, command.timestamp, scFilter.updatedFilter);
+                newCommand = ReadCommand.create(command.ksName, command.key, command.cfName, command.timestamp, scFilter.updatedFilter);
                 newCommand.setDigestQuery(command.isDigestQuery());
                 superColumn = scFilter.scName;
             }
@@ -186,11 +186,11 @@ class ReadCommandSerializer implements IVersionedSerializer<ReadCommand>
         ByteBuffer superColumn = null;
         if (version < MessagingService.VERSION_20)
         {
-            CFMetaData metadata = Schema.instance.getCFMetaData(command.table, command.cfName);
+            CFMetaData metadata = Schema.instance.getCFMetaData(command.ksName, command.cfName);
             if (metadata.cfType == ColumnFamilyType.Super)
             {
                 SuperColumns.SCFilter scFilter = SuperColumns.filterToSC((CompositeType)metadata.comparator, command.filter());
-                newCommand = ReadCommand.create(command.table, command.key, command.cfName, command.timestamp, scFilter.updatedFilter);
+                newCommand = ReadCommand.create(command.ksName, command.key, command.cfName, command.timestamp, scFilter.updatedFilter);
                 newCommand.setDigestQuery(command.isDigestQuery());
                 superColumn = scFilter.scName;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/ReadResponse.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadResponse.java b/src/java/org/apache/cassandra/db/ReadResponse.java
index fcab136..3fe6ec4 100644
--- a/src/java/org/apache/cassandra/db/ReadResponse.java
+++ b/src/java/org/apache/cassandra/db/ReadResponse.java
@@ -25,8 +25,8 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 
 /*
  * The read response message is sent by the server when reading data
- * this encapsulates the tablename and the row that has been read.
- * The table name is needed so that we can use it to create repairs.
+ * this encapsulates the keyspacename and the row that has been read.
+ * The keyspace name is needed so that we can use it to create repairs.
  */
 public class ReadResponse
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/ReadVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadVerbHandler.java b/src/java/org/apache/cassandra/db/ReadVerbHandler.java
index 2cb534a..24e02eb 100644
--- a/src/java/org/apache/cassandra/db/ReadVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/ReadVerbHandler.java
@@ -40,8 +40,8 @@ public class ReadVerbHandler implements IVerbHandler<ReadCommand>
         }
 
         ReadCommand command = message.payload;
-        Table table = Table.open(command.table);
-        Row row = command.getRow(table);
+        Keyspace keyspace = Keyspace.open(command.ksName);
+        Row row = command.getRow(keyspace);
 
         MessageOut<ReadResponse> reply = new MessageOut<ReadResponse>(MessagingService.Verb.REQUEST_RESPONSE,
                                                                       getResponse(command, row),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/RetriedSliceFromReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RetriedSliceFromReadCommand.java b/src/java/org/apache/cassandra/db/RetriedSliceFromReadCommand.java
index 7ca57a8..fe54917 100644
--- a/src/java/org/apache/cassandra/db/RetriedSliceFromReadCommand.java
+++ b/src/java/org/apache/cassandra/db/RetriedSliceFromReadCommand.java
@@ -29,16 +29,16 @@ public class RetriedSliceFromReadCommand extends SliceFromReadCommand
     static final Logger logger = LoggerFactory.getLogger(RetriedSliceFromReadCommand.class);
     public final int originalCount;
 
-    public RetriedSliceFromReadCommand(String table, ByteBuffer key, String cfName, long timestamp, SliceQueryFilter filter, int originalCount)
+    public RetriedSliceFromReadCommand(String keyspaceName, ByteBuffer key, String cfName, long timestamp, SliceQueryFilter filter, int originalCount)
     {
-        super(table, key, cfName, timestamp, filter);
+        super(keyspaceName, key, cfName, timestamp, filter);
         this.originalCount = originalCount;
     }
 
     @Override
     public ReadCommand copy()
     {
-        ReadCommand readCommand = new RetriedSliceFromReadCommand(table, key, cfName, timestamp, filter, originalCount);
+        ReadCommand readCommand = new RetriedSliceFromReadCommand(ksName, key, cfName, timestamp, filter, originalCount);
         readCommand.setDigestQuery(isDigestQuery());
         return readCommand;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/RowMutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RowMutation.java b/src/java/org/apache/cassandra/db/RowMutation.java
index d78247b..2d4188e 100644
--- a/src/java/org/apache/cassandra/db/RowMutation.java
+++ b/src/java/org/apache/cassandra/db/RowMutation.java
@@ -42,30 +42,30 @@ public class RowMutation implements IMutation
 
     // todo this is redundant
     // when we remove it, also restore SerializationsTest.testRowMutationRead to not regenerate new RowMutations each test
-    private final String table;
+    private final String keyspaceName;
 
     private final ByteBuffer key;
     // map of column family id to mutations for that column family.
     private final Map<UUID, ColumnFamily> modifications;
 
-    public RowMutation(String table, ByteBuffer key)
+    public RowMutation(String keyspaceName, ByteBuffer key)
     {
-        this(table, key, new HashMap<UUID, ColumnFamily>());
+        this(keyspaceName, key, new HashMap<UUID, ColumnFamily>());
     }
 
-    public RowMutation(String table, ByteBuffer key, ColumnFamily cf)
+    public RowMutation(String keyspaceName, ByteBuffer key, ColumnFamily cf)
     {
-        this(table, key, Collections.singletonMap(cf.id(), cf));
+        this(keyspaceName, key, Collections.singletonMap(cf.id(), cf));
     }
 
-    public RowMutation(String table, Row row)
+    public RowMutation(String keyspaceName, Row row)
     {
-        this(table, row.key.key, row.cf);
+        this(keyspaceName, row.key.key, row.cf);
     }
 
-    protected RowMutation(String table, ByteBuffer key, Map<UUID, ColumnFamily> modifications)
+    protected RowMutation(String keyspaceName, ByteBuffer key, Map<UUID, ColumnFamily> modifications)
     {
-        this.table = table;
+        this.keyspaceName = keyspaceName;
         this.key = key;
         this.modifications = modifications;
     }
@@ -75,9 +75,9 @@ public class RowMutation implements IMutation
         this(cf.metadata().ksName, key, cf);
     }
 
-    public String getTable()
+    public String getKeyspaceName()
     {
-        return table;
+        return keyspaceName;
     }
 
     public Collection<UUID> getColumnFamilyIds()
@@ -120,7 +120,7 @@ public class RowMutation implements IMutation
      */
     public ColumnFamily addOrGet(String cfName)
     {
-        CFMetaData cfm = Schema.instance.getCFMetaData(table, cfName);
+        CFMetaData cfm = Schema.instance.getCFMetaData(keyspaceName, cfName);
         ColumnFamily cf = modifications.get(cfm.cfId);
         if (cf == null)
         {
@@ -174,7 +174,7 @@ public class RowMutation implements IMutation
             throw new IllegalArgumentException();
 
         RowMutation rm = (RowMutation)m;
-        if (!table.equals(rm.table) || !key.equals(rm.key))
+        if (!keyspaceName.equals(rm.keyspaceName) || !key.equals(rm.key))
             throw new IllegalArgumentException();
 
         for (Map.Entry<UUID, ColumnFamily> entry : rm.modifications.entrySet())
@@ -189,17 +189,17 @@ public class RowMutation implements IMutation
 
     /*
      * This is equivalent to calling commit. Applies the changes to
-     * to the table that is obtained by calling Table.open().
+     * to the keyspace that is obtained by calling Keyspace.open().
      */
     public void apply()
     {
-        Table ks = Table.open(table);
+        Keyspace ks = Keyspace.open(keyspaceName);
         ks.apply(this, ks.metadata.durableWrites);
     }
 
     public void applyUnsafe()
     {
-        Table.open(table).apply(this, false);
+        Keyspace.open(keyspaceName).apply(this, false);
     }
 
     public MessageOut<RowMutation> createMessage()
@@ -220,7 +220,7 @@ public class RowMutation implements IMutation
     public String toString(boolean shallow)
     {
         StringBuilder buff = new StringBuilder("RowMutation(");
-        buff.append("keyspace='").append(table).append('\'');
+        buff.append("keyspace='").append(keyspaceName).append('\'');
         buff.append(", key='").append(ByteBufferUtil.bytesToHex(key)).append('\'');
         buff.append(", modifications=[");
         if (shallow)
@@ -240,7 +240,7 @@ public class RowMutation implements IMutation
 
     public RowMutation without(UUID cfId)
     {
-        RowMutation rm = new RowMutation(table, key);
+        RowMutation rm = new RowMutation(keyspaceName, key);
         for (Map.Entry<UUID, ColumnFamily> entry : modifications.entrySet())
             if (!entry.getKey().equals(cfId))
                 rm.add(entry.getValue());
@@ -252,7 +252,7 @@ public class RowMutation implements IMutation
         public void serialize(RowMutation rm, DataOutput out, int version) throws IOException
         {
             if (version < MessagingService.VERSION_20)
-                out.writeUTF(rm.getTable());
+                out.writeUTF(rm.getKeyspaceName());
 
             ByteBufferUtil.writeWithShortLength(rm.key(), out);
 
@@ -266,9 +266,9 @@ public class RowMutation implements IMutation
 
         public RowMutation deserialize(DataInput in, int version, ColumnSerializer.Flag flag) throws IOException
         {
-            String table = null; // will always be set from cf.metadata but javac isn't smart enough to see that
+            String keyspaceName = null; // will always be set from cf.metadata but javac isn't smart enough to see that
             if (version < MessagingService.VERSION_20)
-                table = in.readUTF();
+                keyspaceName = in.readUTF();
 
             ByteBuffer key = ByteBufferUtil.readWithShortLength(in);
             int size = in.readInt();
@@ -279,7 +279,7 @@ public class RowMutation implements IMutation
             {
                 ColumnFamily cf = deserializeOneCf(in, version, flag);
                 modifications = Collections.singletonMap(cf.id(), cf);
-                table = cf.metadata().ksName;
+                keyspaceName = cf.metadata().ksName;
             }
             else
             {
@@ -288,11 +288,11 @@ public class RowMutation implements IMutation
                 {
                     ColumnFamily cf = deserializeOneCf(in, version, flag);
                     modifications.put(cf.id(), cf);
-                    table = cf.metadata().ksName;
+                    keyspaceName = cf.metadata().ksName;
                 }
             }
 
-            return new RowMutation(table, key, modifications);
+            return new RowMutation(keyspaceName, key, modifications);
         }
 
         private ColumnFamily deserializeOneCf(DataInput in, int version, ColumnSerializer.Flag flag) throws IOException
@@ -314,7 +314,7 @@ public class RowMutation implements IMutation
             int size = 0;
 
             if (version < MessagingService.VERSION_20)
-                size += sizes.sizeof(rm.getTable());
+                size += sizes.sizeof(rm.getKeyspaceName());
 
             int keySize = rm.key().remaining();
             size += sizes.sizeof((short) keySize) + keySize;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java b/src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java
index 2942249..ae3db78 100644
--- a/src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java
@@ -36,30 +36,30 @@ public class SliceByNamesReadCommand extends ReadCommand
 
     public final NamesQueryFilter filter;
 
-    public SliceByNamesReadCommand(String table, ByteBuffer key, String cfName, long timestamp, NamesQueryFilter filter)
+    public SliceByNamesReadCommand(String keyspaceName, ByteBuffer key, String cfName, long timestamp, NamesQueryFilter filter)
     {
-        super(table, key, cfName, timestamp, Type.GET_BY_NAMES);
+        super(keyspaceName, key, cfName, timestamp, Type.GET_BY_NAMES);
         this.filter = filter;
     }
 
     public ReadCommand copy()
     {
-        ReadCommand readCommand= new SliceByNamesReadCommand(table, key, cfName, timestamp, filter);
+        ReadCommand readCommand= new SliceByNamesReadCommand(ksName, key, cfName, timestamp, filter);
         readCommand.setDigestQuery(isDigestQuery());
         return readCommand;
     }
 
-    public Row getRow(Table table)
+    public Row getRow(Keyspace keyspace)
     {
         DecoratedKey dk = StorageService.getPartitioner().decorateKey(key);
-        return table.getRow(new QueryFilter(dk, cfName, filter, timestamp));
+        return keyspace.getRow(new QueryFilter(dk, cfName, filter, timestamp));
     }
 
     @Override
     public String toString()
     {
         return "SliceByNamesReadCommand(" +
-               "table='" + table + '\'' +
+               "keyspace='" + ksName + '\'' +
                ", key=" + ByteBufferUtil.bytesToHex(key) +
                ", cfName='" + cfName + '\'' +
                ", timestamp='" + timestamp + '\'' +
@@ -84,7 +84,7 @@ class SliceByNamesReadCommandSerializer implements IVersionedSerializer<ReadComm
     {
         SliceByNamesReadCommand command = (SliceByNamesReadCommand) cmd;
         out.writeBoolean(command.isDigestQuery());
-        out.writeUTF(command.table);
+        out.writeUTF(command.ksName);
         ByteBufferUtil.writeWithShortLength(command.key, out);
 
         if (version < MessagingService.VERSION_20)
@@ -101,7 +101,7 @@ class SliceByNamesReadCommandSerializer implements IVersionedSerializer<ReadComm
     public ReadCommand deserialize(DataInput in, int version) throws IOException
     {
         boolean isDigest = in.readBoolean();
-        String table = in.readUTF();
+        String keyspaceName = in.readUTF();
         ByteBuffer key = ByteBufferUtil.readWithShortLength(in);
 
         String cfName;
@@ -119,7 +119,7 @@ class SliceByNamesReadCommandSerializer implements IVersionedSerializer<ReadComm
 
         long timestamp = version < MessagingService.VERSION_20 ? System.currentTimeMillis() : in.readLong();
 
-        CFMetaData metadata = Schema.instance.getCFMetaData(table, cfName);
+        CFMetaData metadata = Schema.instance.getCFMetaData(keyspaceName, cfName);
         ReadCommand command;
         if (version < MessagingService.VERSION_20)
         {
@@ -141,14 +141,14 @@ class SliceByNamesReadCommandSerializer implements IVersionedSerializer<ReadComm
 
             // Due to SC compat, it's possible we get back a slice filter at this point
             if (filter instanceof NamesQueryFilter)
-                command = new SliceByNamesReadCommand(table, key, cfName, timestamp, (NamesQueryFilter)filter);
+                command = new SliceByNamesReadCommand(keyspaceName, key, cfName, timestamp, (NamesQueryFilter)filter);
             else
-                command = new SliceFromReadCommand(table, key, cfName, timestamp, (SliceQueryFilter)filter);
+                command = new SliceFromReadCommand(keyspaceName, key, cfName, timestamp, (SliceQueryFilter)filter);
         }
         else
         {
             NamesQueryFilter filter = NamesQueryFilter.serializer.deserialize(in, version, metadata.comparator);
-            command = new SliceByNamesReadCommand(table, key, cfName, timestamp, filter);
+            command = new SliceByNamesReadCommand(keyspaceName, key, cfName, timestamp, filter);
         }
 
         command.setDigestQuery(isDigest);
@@ -167,7 +167,7 @@ class SliceByNamesReadCommandSerializer implements IVersionedSerializer<ReadComm
         int size = sizes.sizeof(command.isDigestQuery());
         int keySize = command.key.remaining();
 
-        size += sizes.sizeof(command.table);
+        size += sizes.sizeof(command.ksName);
         size += sizes.sizeof((short)keySize) + keySize;
 
         if (version < MessagingService.VERSION_20)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SliceFromReadCommand.java b/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
index 508d1d2..7526796 100644
--- a/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
@@ -46,23 +46,23 @@ public class SliceFromReadCommand extends ReadCommand
 
     public final SliceQueryFilter filter;
 
-    public SliceFromReadCommand(String table, ByteBuffer key, String cfName, long timestamp, SliceQueryFilter filter)
+    public SliceFromReadCommand(String keyspaceName, ByteBuffer key, String cfName, long timestamp, SliceQueryFilter filter)
     {
-        super(table, key, cfName, timestamp, Type.GET_SLICES);
+        super(keyspaceName, key, cfName, timestamp, Type.GET_SLICES);
         this.filter = filter;
     }
 
     public ReadCommand copy()
     {
-        ReadCommand readCommand = new SliceFromReadCommand(table, key, cfName, timestamp, filter);
+        ReadCommand readCommand = new SliceFromReadCommand(ksName, key, cfName, timestamp, filter);
         readCommand.setDigestQuery(isDigestQuery());
         return readCommand;
     }
 
-    public Row getRow(Table table)
+    public Row getRow(Keyspace keyspace)
     {
         DecoratedKey dk = StorageService.getPartitioner().decorateKey(key);
-        return table.getRow(new QueryFilter(dk, cfName, filter, timestamp));
+        return keyspace.getRow(new QueryFilter(dk, cfName, filter, timestamp));
     }
 
     @Override
@@ -86,7 +86,7 @@ public class SliceFromReadCommand extends ReadCommand
             // round we want to ask x column so that x * (l/t) == t, i.e. x = t^2/l.
             int retryCount = liveCountInRow == 0 ? count + 1 : ((count * count) / liveCountInRow) + 1;
             SliceQueryFilter newFilter = filter.withUpdatedCount(retryCount);
-            return new RetriedSliceFromReadCommand(table, key, cfName, timestamp, newFilter, getOriginalRequestedCount());
+            return new RetriedSliceFromReadCommand(ksName, key, cfName, timestamp, newFilter, getOriginalRequestedCount());
         }
 
         return null;
@@ -108,7 +108,7 @@ public class SliceFromReadCommand extends ReadCommand
 
     public SliceFromReadCommand withUpdatedFilter(SliceQueryFilter newFilter)
     {
-        return new SliceFromReadCommand(table, key, cfName, timestamp, newFilter);
+        return new SliceFromReadCommand(ksName, key, cfName, timestamp, newFilter);
     }
 
     /**
@@ -125,7 +125,7 @@ public class SliceFromReadCommand extends ReadCommand
     public String toString()
     {
         return "SliceFromReadCommand(" +
-               "table='" + table + '\'' +
+               "keyspace='" + ksName + '\'' +
                ", key='" + ByteBufferUtil.bytesToHex(key) + '\'' +
                ", cfName='" + cfName + '\'' +
                ", timestamp='" + timestamp + '\'' +
@@ -145,7 +145,7 @@ class SliceFromReadCommandSerializer implements IVersionedSerializer<ReadCommand
     {
         SliceFromReadCommand realRM = (SliceFromReadCommand)rm;
         out.writeBoolean(realRM.isDigestQuery());
-        out.writeUTF(realRM.table);
+        out.writeUTF(realRM.ksName);
         ByteBufferUtil.writeWithShortLength(realRM.key, out);
 
         if (version < MessagingService.VERSION_20)
@@ -162,7 +162,7 @@ class SliceFromReadCommandSerializer implements IVersionedSerializer<ReadCommand
     public ReadCommand deserialize(DataInput in, int version) throws IOException
     {
         boolean isDigest = in.readBoolean();
-        String table = in.readUTF();
+        String keyspaceName = in.readUTF();
         ByteBuffer key = ByteBufferUtil.readWithShortLength(in);
 
         String cfName;
@@ -180,7 +180,7 @@ class SliceFromReadCommandSerializer implements IVersionedSerializer<ReadCommand
 
         long timestamp = version < MessagingService.VERSION_20 ? System.currentTimeMillis() : in.readLong();
 
-        CFMetaData metadata = Schema.instance.getCFMetaData(table, cfName);
+        CFMetaData metadata = Schema.instance.getCFMetaData(keyspaceName, cfName);
         SliceQueryFilter filter;
         if (version < MessagingService.VERSION_20)
         {
@@ -194,7 +194,7 @@ class SliceFromReadCommandSerializer implements IVersionedSerializer<ReadCommand
             filter = SliceQueryFilter.serializer.deserialize(in, version);
         }
 
-        ReadCommand command = new SliceFromReadCommand(table, key, cfName, timestamp, filter);
+        ReadCommand command = new SliceFromReadCommand(keyspaceName, key, cfName, timestamp, filter);
         command.setDigestQuery(isDigest);
         return command;
     }
@@ -211,7 +211,7 @@ class SliceFromReadCommandSerializer implements IVersionedSerializer<ReadCommand
         int keySize = command.key.remaining();
 
         int size = sizes.sizeof(cmd.isDigestQuery()); // boolean
-        size += sizes.sizeof(command.table);
+        size += sizes.sizeof(command.ksName);
         size += sizes.sizeof((short) keySize) + keySize;
 
         if (version < MessagingService.VERSION_20)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
new file mode 100644
index 0000000..d518468
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -0,0 +1,814 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Function;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.SetMultimap;
+import com.google.common.collect.Sets;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.paxos.Commit;
+import org.apache.cassandra.service.paxos.PaxosState;
+import org.apache.cassandra.thrift.cassandraConstants;
+import org.apache.cassandra.utils.*;
+
+import static org.apache.cassandra.cql3.QueryProcessor.processInternal;
+
+public class SystemKeyspace
+{
+    private static final Logger logger = LoggerFactory.getLogger(SystemKeyspace.class);
+
+    // see CFMetaData for schema definitions
+    public static final String PEERS_CF = "peers";
+    public static final String PEER_EVENTS_CF = "peer_events";
+    public static final String LOCAL_CF = "local";
+    public static final String INDEX_CF = "IndexInfo";
+    public static final String COUNTER_ID_CF = "NodeIdInfo";
+    public static final String HINTS_CF = "hints";
+    public static final String RANGE_XFERS_CF = "range_xfers";
+    public static final String BATCHLOG_CF = "batchlog";
+    // see layout description in the DefsTables class header
+    public static final String SCHEMA_KEYSPACES_CF = "schema_keyspaces";
+    public static final String SCHEMA_COLUMNFAMILIES_CF = "schema_columnfamilies";
+    public static final String SCHEMA_COLUMNS_CF = "schema_columns";
+    public static final String SCHEMA_TRIGGERS_CF = "schema_triggers";
+    public static final String COMPACTION_LOG = "compactions_in_progress";
+    public static final String PAXOS_CF = "paxos";
+
+    private static final String LOCAL_KEY = "local";
+    private static final ByteBuffer ALL_LOCAL_NODE_ID_KEY = ByteBufferUtil.bytes("Local");
+
+    public enum BootstrapState
+    {
+        NEEDS_BOOTSTRAP,
+        COMPLETED,
+        IN_PROGRESS
+    }
+
+    private static DecoratedKey decorate(ByteBuffer key)
+    {
+        return StorageService.getPartitioner().decorateKey(key);
+    }
+
+    public static void finishStartup()
+    {
+        setupVersion();
+
+        // add entries to system schema columnfamilies for the hardcoded system definitions
+        for (String ksname : Schema.systemKeyspaceNames)
+        {
+            KSMetaData ksmd = Schema.instance.getKSMetaData(ksname);
+
+            // delete old, possibly obsolete entries in schema columnfamilies
+            for (String cfname : Arrays.asList(SystemKeyspace.SCHEMA_KEYSPACES_CF, SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, SystemKeyspace.SCHEMA_COLUMNS_CF))
+            {
+                String req = String.format("DELETE FROM system.%s WHERE keyspace_name = '%s'", cfname, ksmd.name);
+                processInternal(req);
+            }
+
+            // (+1 to timestamp to make sure we don't get shadowed by the tombstones we just added)
+            ksmd.toSchema(FBUtilities.timestampMicros() + 1).apply();
+        }
+    }
+
+    private static void setupVersion()
+    {
+        String req = "INSERT INTO system.%s (key, release_version, cql_version, thrift_version, data_center, rack, partitioner) VALUES ('%s', '%s', '%s', '%s', '%s', '%s', '%s')";
+        IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
+        processInternal(String.format(req, LOCAL_CF,
+                                         LOCAL_KEY,
+                                         FBUtilities.getReleaseVersionString(),
+                                         QueryProcessor.CQL_VERSION.toString(),
+                                         cassandraConstants.VERSION,
+                                         snitch.getDatacenter(FBUtilities.getBroadcastAddress()),
+                                         snitch.getRack(FBUtilities.getBroadcastAddress()),
+                                         DatabaseDescriptor.getPartitioner().getClass().getName()));
+    }
+
+    /**
+     * Write compaction log, except columfamilies under system keyspace.
+     *
+     * @param cfs
+     * @param toCompact sstables to compact
+     * @return compaction task id or null if cfs is under system keyspace
+     */
+    public static UUID startCompaction(ColumnFamilyStore cfs, Iterable<SSTableReader> toCompact)
+    {
+        if (Keyspace.SYSTEM_KS.equals(cfs.keyspace.getName()))
+            return null;
+
+        UUID compactionId = UUIDGen.getTimeUUID();
+        String req = "INSERT INTO system.%s (id, keyspace_name, columnfamily_name, inputs) VALUES (%s, '%s', '%s', {%s})";
+        Iterable<Integer> generations = Iterables.transform(toCompact, new Function<SSTableReader, Integer>()
+        {
+            public Integer apply(SSTableReader sstable)
+            {
+                return sstable.descriptor.generation;
+            }
+        });
+        processInternal(String.format(req, COMPACTION_LOG, compactionId, cfs.keyspace.getName(), cfs.name, StringUtils.join(Sets.newHashSet(generations), ',')));
+        forceBlockingFlush(COMPACTION_LOG);
+        return compactionId;
+    }
+
+    public static void finishCompaction(UUID taskId)
+    {
+        assert taskId != null;
+
+        String req = "DELETE FROM system.%s WHERE id = %s";
+        processInternal(String.format(req, COMPACTION_LOG, taskId));
+        forceBlockingFlush(COMPACTION_LOG);
+    }
+
+    /**
+     * @return unfinished compactions, grouped by keyspace/columnfamily pair.
+     */
+    public static SetMultimap<Pair<String, String>, Integer> getUnfinishedCompactions()
+    {
+        String req = "SELECT * FROM system.%s";
+        UntypedResultSet resultSet = processInternal(String.format(req, COMPACTION_LOG));
+
+        SetMultimap<Pair<String, String>, Integer> unfinishedCompactions = HashMultimap.create();
+        for (UntypedResultSet.Row row : resultSet)
+        {
+            String keyspace = row.getString("keyspace_name");
+            String columnfamily = row.getString("columnfamily_name");
+            Set<Integer> inputs = row.getSet("inputs", Int32Type.instance);
+
+            unfinishedCompactions.putAll(Pair.create(keyspace, columnfamily), inputs);
+        }
+        return unfinishedCompactions;
+    }
+
+    public static void discardCompactionsInProgress()
+    {
+        ColumnFamilyStore compactionLog = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(COMPACTION_LOG);
+        compactionLog.truncateBlocking();
+    }
+
+    public static void saveTruncationRecord(ColumnFamilyStore cfs, long truncatedAt, ReplayPosition position)
+    {
+        String req = "UPDATE system.%s SET truncated_at = truncated_at + %s WHERE key = '%s'";
+        processInternal(String.format(req, LOCAL_CF, truncationAsMapEntry(cfs, truncatedAt, position), LOCAL_KEY));
+        forceBlockingFlush(LOCAL_CF);
+    }
+
+    /**
+     * This method is used to remove information about truncation time for specified column family
+     */
+    public static void removeTruncationRecord(UUID cfId)
+    {
+        String req = "DELETE truncated_at[%s] from system.%s WHERE key = '%s'";
+        processInternal(String.format(req, cfId, LOCAL_CF, LOCAL_KEY));
+        forceBlockingFlush(LOCAL_CF);
+    }
+
+    private static String truncationAsMapEntry(ColumnFamilyStore cfs, long truncatedAt, ReplayPosition position)
+    {
+        DataOutputBuffer out = new DataOutputBuffer();
+        try
+        {
+            ReplayPosition.serializer.serialize(position, out);
+            out.writeLong(truncatedAt);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+        return String.format("{%s: 0x%s}",
+                             cfs.metadata.cfId,
+                             ByteBufferUtil.bytesToHex(ByteBuffer.wrap(out.getData(), 0, out.getLength())));
+    }
+
+    public static Map<UUID, Pair<ReplayPosition, Long>> getTruncationRecords()
+    {
+        String req = "SELECT truncated_at FROM system.%s WHERE key = '%s'";
+        UntypedResultSet rows = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+        if (rows.isEmpty())
+            return Collections.emptyMap();
+
+        UntypedResultSet.Row row = rows.one();
+        Map<UUID, ByteBuffer> rawMap = row.getMap("truncated_at", UUIDType.instance, BytesType.instance);
+        if (rawMap == null)
+            return Collections.emptyMap();
+
+        Map<UUID, Pair<ReplayPosition, Long>> positions = new HashMap<UUID, Pair<ReplayPosition, Long>>();
+        for (Map.Entry<UUID, ByteBuffer> entry : rawMap.entrySet())
+            positions.put(entry.getKey(), truncationRecordFromBlob(entry.getValue()));
+        return positions;
+    }
+
+    private static Pair<ReplayPosition, Long> truncationRecordFromBlob(ByteBuffer bytes)
+    {
+        try
+        {
+            DataInputStream in = new DataInputStream(ByteBufferUtil.inputStream(bytes));
+            return Pair.create(ReplayPosition.serializer.deserialize(in), in.available() > 0 ? in.readLong() : Long.MIN_VALUE);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Record tokens being used by another node
+     */
+    public static synchronized void updateTokens(InetAddress ep, Collection<Token> tokens)
+    {
+        if (ep.equals(FBUtilities.getBroadcastAddress()))
+        {
+            removeEndpoint(ep);
+            return;
+        }
+
+        String req = "INSERT INTO system.%s (peer, tokens) VALUES ('%s', %s)";
+        processInternal(String.format(req, PEERS_CF, ep.getHostAddress(), tokensAsSet(tokens)));
+        forceBlockingFlush(PEERS_CF);
+    }
+
+    public static synchronized void updatePeerInfo(InetAddress ep, String columnName, String value)
+    {
+        if (ep.equals(FBUtilities.getBroadcastAddress()))
+            return;
+
+        String req = "INSERT INTO system.%s (peer, %s) VALUES ('%s', %s)";
+        processInternal(String.format(req, PEERS_CF, columnName, ep.getHostAddress(), value));
+    }
+
+    public static synchronized void updateHintsDropped(InetAddress ep, UUID timePeriod, int value)
+    {
+        // with 30 day TTL
+        String req = "UPDATE system.%s USING TTL 2592000 SET hints_dropped[ %s ] = %s WHERE peer = '%s'";
+        processInternal(String.format(req, PEER_EVENTS_CF, timePeriod.toString(), value, ep.getHostAddress()));
+    }
+
+    public static synchronized void updateSchemaVersion(UUID version)
+    {
+        String req = "INSERT INTO system.%s (key, schema_version) VALUES ('%s', %s)";
+        processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, version.toString()));
+    }
+
+    private static String tokensAsSet(Collection<Token> tokens)
+    {
+        Token.TokenFactory factory = StorageService.getPartitioner().getTokenFactory();
+        StringBuilder sb = new StringBuilder();
+        sb.append("{");
+        Iterator<Token> iter = tokens.iterator();
+        while (iter.hasNext())
+        {
+            sb.append("'").append(factory.toString(iter.next())).append("'");
+            if (iter.hasNext())
+                sb.append(",");
+        }
+        sb.append("}");
+        return sb.toString();
+    }
+
+    private static Collection<Token> deserializeTokens(Collection<String> tokensStrings)
+    {
+        Token.TokenFactory factory = StorageService.getPartitioner().getTokenFactory();
+        List<Token> tokens = new ArrayList<Token>(tokensStrings.size());
+        for (String tk : tokensStrings)
+            tokens.add(factory.fromString(tk));
+        return tokens;
+    }
+
+    /**
+     * Remove stored tokens being used by another node
+     */
+    public static synchronized void removeEndpoint(InetAddress ep)
+    {
+        String req = "DELETE FROM system.%s WHERE peer = '%s'";
+        processInternal(String.format(req, PEERS_CF, ep.getHostAddress()));
+        forceBlockingFlush(PEERS_CF);
+    }
+
+    /**
+     * This method is used to update the System Keyspace with the new tokens for this node
+    */
+    public static synchronized void updateTokens(Collection<Token> tokens)
+    {
+        assert !tokens.isEmpty() : "removeEndpoint should be used instead";
+        String req = "INSERT INTO system.%s (key, tokens) VALUES ('%s', %s)";
+        processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, tokensAsSet(tokens)));
+        forceBlockingFlush(LOCAL_CF);
+    }
+
+    /**
+     * Convenience method to update the list of tokens in the local system keyspace.
+     *
+     * @param addTokens tokens to add
+     * @param rmTokens tokens to remove
+     * @return the collection of persisted tokens
+     */
+    public static synchronized Collection<Token> updateLocalTokens(Collection<Token> addTokens, Collection<Token> rmTokens)
+    {
+        Collection<Token> tokens = getSavedTokens();
+        tokens.removeAll(rmTokens);
+        tokens.addAll(addTokens);
+        updateTokens(tokens);
+        return tokens;
+    }
+
+    private static void forceBlockingFlush(String cfname)
+    {
+        Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(cfname).forceBlockingFlush();
+    }
+
+    /**
+     * Return a map of stored tokens to IP addresses
+     *
+     */
+    public static SetMultimap<InetAddress, Token> loadTokens()
+    {
+        SetMultimap<InetAddress, Token> tokenMap = HashMultimap.create();
+        for (UntypedResultSet.Row row : processInternal("SELECT peer, tokens FROM system." + PEERS_CF))
+        {
+            InetAddress peer = row.getInetAddress("peer");
+            if (row.has("tokens"))
+                tokenMap.putAll(peer, deserializeTokens(row.getSet("tokens", UTF8Type.instance)));
+        }
+
+        return tokenMap;
+    }
+
+    /**
+     * Return a map of store host_ids to IP addresses
+     *
+     */
+    public static Map<InetAddress, UUID> loadHostIds()
+    {
+        Map<InetAddress, UUID> hostIdMap = new HashMap<InetAddress, UUID>();
+        for (UntypedResultSet.Row row : processInternal("SELECT peer, host_id FROM system." + PEERS_CF))
+        {
+            InetAddress peer = row.getInetAddress("peer");
+            if (row.has("host_id"))
+            {
+                hostIdMap.put(peer, row.getUUID("host_id"));
+            }
+        }
+        return hostIdMap;
+    }
+
+    /**
+     * Return a map of IP addresses containing a map of dc and rack info
+     */
+    public static Map<InetAddress, Map<String,String>> loadDcRackInfo()
+    {
+        Map<InetAddress, Map<String, String>> result = new HashMap<InetAddress, Map<String, String>>();
+        for (UntypedResultSet.Row row : processInternal("SELECT peer, data_center, rack from system." + PEERS_CF))
+        {
+            InetAddress peer = row.getInetAddress("peer");
+            if (row.has("data_center") && row.has("rack"))
+            {
+                Map<String, String> dcRack = new HashMap<String, String>();
+                dcRack.put("data_center", row.getString("data_center"));
+                dcRack.put("rack", row.getString("rack"));
+                result.put(peer, dcRack);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * One of three things will happen if you try to read the system keyspace:
+     * 1. files are present and you can read them: great
+     * 2. no files are there: great (new node is assumed)
+     * 3. files are present but you can't read them: bad
+     * @throws ConfigurationException
+     */
+    public static void checkHealth() throws ConfigurationException
+    {
+        Keyspace keyspace;
+        try
+        {
+            keyspace = Keyspace.open(Keyspace.SYSTEM_KS);
+        }
+        catch (AssertionError err)
+        {
+            // this happens when a user switches from OPP to RP.
+            ConfigurationException ex = new ConfigurationException("Could not read system keyspace!");
+            ex.initCause(err);
+            throw ex;
+        }
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(LOCAL_CF);
+
+        String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
+        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+
+        if (result.isEmpty() || !result.one().has("cluster_name"))
+        {
+            // this is a brand new node
+            if (!cfs.getSSTables().isEmpty())
+                throw new ConfigurationException("Found system keyspace files, but they couldn't be loaded!");
+
+            // no system files.  this is a new node.
+            req = "INSERT INTO system.%s (key, cluster_name) VALUES ('%s', '%s')";
+            processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, DatabaseDescriptor.getClusterName()));
+            return;
+        }
+
+        String savedClusterName = result.one().getString("cluster_name");
+        if (!DatabaseDescriptor.getClusterName().equals(savedClusterName))
+            throw new ConfigurationException("Saved cluster name " + savedClusterName + " != configured name " + DatabaseDescriptor.getClusterName());
+    }
+
+    public static Collection<Token> getSavedTokens()
+    {
+        String req = "SELECT tokens FROM system.%s WHERE key='%s'";
+        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+        return result.isEmpty() || !result.one().has("tokens")
+             ? Collections.<Token>emptyList()
+             : deserializeTokens(result.one().<String>getSet("tokens", UTF8Type.instance));
+    }
+
+    public static int incrementAndGetGeneration()
+    {
+        String req = "SELECT gossip_generation FROM system.%s WHERE key='%s'";
+        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+
+        int generation;
+        if (result.isEmpty() || !result.one().has("gossip_generation"))
+        {
+            // seconds-since-epoch isn't a foolproof new generation
+            // (where foolproof is "guaranteed to be larger than the last one seen at this ip address"),
+            // but it's as close as sanely possible
+            generation = (int) (System.currentTimeMillis() / 1000);
+        }
+        else
+        {
+            // Other nodes will ignore gossip messages about a node that have a lower generation than previously seen.
+            final int storedGeneration = result.one().getInt("gossip_generation") + 1;
+            final int now = (int) (System.currentTimeMillis() / 1000);
+            if (storedGeneration >= now)
+            {
+                logger.warn("Using stored Gossip Generation {} as it is greater than current system time {}.  See CASSANDRA-3654 if you experience problems",
+                            storedGeneration, now);
+                generation = storedGeneration;
+            }
+            else
+            {
+                generation = now;
+            }
+        }
+
+        req = "INSERT INTO system.%s (key, gossip_generation) VALUES ('%s', %d)";
+        processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, generation));
+        forceBlockingFlush(LOCAL_CF);
+
+        return generation;
+    }
+
+    public static BootstrapState getBootstrapState()
+    {
+        String req = "SELECT bootstrapped FROM system.%s WHERE key='%s'";
+        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+
+        if (result.isEmpty() || !result.one().has("bootstrapped"))
+            return BootstrapState.NEEDS_BOOTSTRAP;
+
+        return BootstrapState.valueOf(result.one().getString("bootstrapped"));
+    }
+
+    public static boolean bootstrapComplete()
+    {
+        return getBootstrapState() == BootstrapState.COMPLETED;
+    }
+
+    public static boolean bootstrapInProgress()
+    {
+        return getBootstrapState() == BootstrapState.IN_PROGRESS;
+    }
+
+    public static void setBootstrapState(BootstrapState state)
+    {
+        String req = "INSERT INTO system.%s (key, bootstrapped) VALUES ('%s', '%s')";
+        processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, state.name()));
+        forceBlockingFlush(LOCAL_CF);
+    }
+
+    public static boolean isIndexBuilt(String keyspaceName, String indexName)
+    {
+        ColumnFamilyStore cfs = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(INDEX_CF);
+        QueryFilter filter = QueryFilter.getNamesFilter(decorate(ByteBufferUtil.bytes(keyspaceName)),
+                                                        INDEX_CF,
+                                                        ByteBufferUtil.bytes(indexName),
+                                                        System.currentTimeMillis());
+        return ColumnFamilyStore.removeDeleted(cfs.getColumnFamily(filter), Integer.MAX_VALUE) != null;
+    }
+
+    public static void setIndexBuilt(String keyspaceName, String indexName)
+    {
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Keyspace.SYSTEM_KS, INDEX_CF);
+        cf.addColumn(new Column(ByteBufferUtil.bytes(indexName), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()));
+        RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, ByteBufferUtil.bytes(keyspaceName), cf);
+        rm.apply();
+        forceBlockingFlush(INDEX_CF);
+    }
+
+    public static void setIndexRemoved(String keyspaceName, String indexName)
+    {
+        RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, ByteBufferUtil.bytes(keyspaceName));
+        rm.delete(INDEX_CF, ByteBufferUtil.bytes(indexName), FBUtilities.timestampMicros());
+        rm.apply();
+        forceBlockingFlush(INDEX_CF);
+    }
+
+    /**
+     * Read the host ID from the system keyspace, creating (and storing) one if
+     * none exists.
+     */
+    public static UUID getLocalHostId()
+    {
+        UUID hostId = null;
+
+        String req = "SELECT host_id FROM system.%s WHERE key='%s'";
+        UntypedResultSet result = processInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+
+        // Look up the Host UUID (return it if found)
+        if (!result.isEmpty() && result.one().has("host_id"))
+        {
+            return result.one().getUUID("host_id");
+        }
+
+        // ID not found, generate a new one, persist, and then return it.
+        hostId = UUID.randomUUID();
+        logger.warn("No host ID found, created {} (Note: This should happen exactly once per node).", hostId);
+
+        req = "INSERT INTO system.%s (key, host_id) VALUES ('%s', %s)";
+        processInternal(String.format(req, LOCAL_CF, LOCAL_KEY, hostId));
+        return hostId;
+    }
+
+    /**
+     * Read the current local node id from the system keyspace or null if no
+     * such node id is recorded.
+     */
+    public static CounterId getCurrentLocalCounterId()
+    {
+        Keyspace keyspace = Keyspace.open(Keyspace.SYSTEM_KS);
+
+        // Get the last CounterId (since CounterId are timeuuid is thus ordered from the older to the newer one)
+        QueryFilter filter = QueryFilter.getSliceFilter(decorate(ALL_LOCAL_NODE_ID_KEY),
+                                                        COUNTER_ID_CF,
+                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                        true,
+                                                        1,
+                                                        System.currentTimeMillis());
+        ColumnFamily cf = keyspace.getColumnFamilyStore(COUNTER_ID_CF).getColumnFamily(filter);
+        if (cf != null && cf.getColumnCount() != 0)
+            return CounterId.wrap(cf.iterator().next().name());
+        else
+            return null;
+    }
+
+    /**
+     * Write a new current local node id to the system keyspace.
+     *
+     * @param oldCounterId the previous local node id (that {@code newCounterId}
+     * replace) or null if no such node id exists (new node or removed system
+     * keyspace)
+     * @param newCounterId the new current local node id to record
+     * @param now microsecond time stamp.
+     */
+    public static void writeCurrentLocalCounterId(CounterId oldCounterId, CounterId newCounterId, long now)
+    {
+        ByteBuffer ip = ByteBuffer.wrap(FBUtilities.getBroadcastAddress().getAddress());
+
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Keyspace.SYSTEM_KS, COUNTER_ID_CF);
+        cf.addColumn(new Column(newCounterId.bytes(), ip, now));
+        RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, ALL_LOCAL_NODE_ID_KEY, cf);
+        rm.apply();
+        forceBlockingFlush(COUNTER_ID_CF);
+    }
+
+    public static List<CounterId.CounterIdRecord> getOldLocalCounterIds()
+    {
+        List<CounterId.CounterIdRecord> l = new ArrayList<CounterId.CounterIdRecord>();
+
+        Keyspace keyspace = Keyspace.open(Keyspace.SYSTEM_KS);
+        QueryFilter filter = QueryFilter.getIdentityFilter(decorate(ALL_LOCAL_NODE_ID_KEY), COUNTER_ID_CF, System.currentTimeMillis());
+        ColumnFamily cf = keyspace.getColumnFamilyStore(COUNTER_ID_CF).getColumnFamily(filter);
+
+        CounterId previous = null;
+        for (Column c : cf)
+        {
+            if (previous != null)
+                l.add(new CounterId.CounterIdRecord(previous, c.timestamp()));
+
+            // this will ignore the last column on purpose since it is the
+            // current local node id
+            previous = CounterId.wrap(c.name());
+        }
+        return l;
+    }
+
+    /**
+     * @param cfName The name of the ColumnFamily responsible for part of the schema (keyspace, ColumnFamily, columns)
+     * @return CFS responsible to hold low-level serialized schema
+     */
+    public static ColumnFamilyStore schemaCFS(String cfName)
+    {
+        return Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(cfName);
+    }
+
+    public static List<Row> serializedSchema()
+    {
+        List<Row> schema = new ArrayList<Row>(3);
+
+        schema.addAll(serializedSchema(SCHEMA_KEYSPACES_CF));
+        schema.addAll(serializedSchema(SCHEMA_COLUMNFAMILIES_CF));
+        schema.addAll(serializedSchema(SCHEMA_COLUMNS_CF));
+
+        return schema;
+    }
+
+    /**
+     * @param schemaCfName The name of the ColumnFamily responsible for part of the schema (keyspace, ColumnFamily, columns)
+     * @return low-level schema representation (each row represents individual Keyspace or ColumnFamily)
+     */
+    public static List<Row> serializedSchema(String schemaCfName)
+    {
+        Token minToken = StorageService.getPartitioner().getMinimumToken();
+
+        return schemaCFS(schemaCfName).getRangeSlice(new Range<RowPosition>(minToken.minKeyBound(), minToken.maxKeyBound()),
+                                                     null,
+                                                     new IdentityQueryFilter(),
+                                                     Integer.MAX_VALUE,
+                                                     System.currentTimeMillis());
+    }
+
+    public static Collection<RowMutation> serializeSchema()
+    {
+        Map<DecoratedKey, RowMutation> mutationMap = new HashMap<DecoratedKey, RowMutation>();
+
+        serializeSchema(mutationMap, SCHEMA_KEYSPACES_CF);
+        serializeSchema(mutationMap, SCHEMA_COLUMNFAMILIES_CF);
+        serializeSchema(mutationMap, SCHEMA_COLUMNS_CF);
+
+        return mutationMap.values();
+    }
+
+    private static void serializeSchema(Map<DecoratedKey, RowMutation> mutationMap, String schemaCfName)
+    {
+        for (Row schemaRow : serializedSchema(schemaCfName))
+        {
+            if (Schema.ignoredSchemaRow(schemaRow))
+                continue;
+
+            RowMutation mutation = mutationMap.get(schemaRow.key);
+            if (mutation == null)
+            {
+                mutation = new RowMutation(Keyspace.SYSTEM_KS, schemaRow.key.key);
+                mutationMap.put(schemaRow.key, mutation);
+            }
+
+            mutation.add(schemaRow.cf);
+        }
+    }
+
+    public static Map<DecoratedKey, ColumnFamily> getSchema(String cfName)
+    {
+        Map<DecoratedKey, ColumnFamily> schema = new HashMap<DecoratedKey, ColumnFamily>();
+
+        for (Row schemaEntity : SystemKeyspace.serializedSchema(cfName))
+            schema.put(schemaEntity.key, schemaEntity.cf);
+
+        return schema;
+    }
+
+    public static ByteBuffer getSchemaKSKey(String ksName)
+    {
+        return AsciiType.instance.fromString(ksName);
+    }
+
+    public static Row readSchemaRow(String ksName)
+    {
+        DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
+
+        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(SCHEMA_KEYSPACES_CF);
+        ColumnFamily result = schemaCFS.getColumnFamily(QueryFilter.getIdentityFilter(key, SCHEMA_KEYSPACES_CF, System.currentTimeMillis()));
+
+        return new Row(key, result);
+    }
+
+    public static Row readSchemaRow(String ksName, String cfName)
+    {
+        DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
+
+        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(SCHEMA_COLUMNFAMILIES_CF);
+        ColumnFamily result = schemaCFS.getColumnFamily(key,
+                                                        DefsTables.searchComposite(cfName, true),
+                                                        DefsTables.searchComposite(cfName, false),
+                                                        false,
+                                                        Integer.MAX_VALUE,
+                                                        System.currentTimeMillis());
+
+        return new Row(key, result);
+    }
+
+    public static PaxosState loadPaxosState(ByteBuffer key, CFMetaData metadata)
+    {
+        String req = "SELECT * FROM system.%s WHERE row_key = 0x%s AND cf_id = %s";
+        UntypedResultSet results = processInternal(String.format(req, PAXOS_CF, ByteBufferUtil.bytesToHex(key), metadata.cfId));
+        if (results.isEmpty())
+            return new PaxosState(key, metadata);
+        UntypedResultSet.Row row = results.one();
+        Commit inProgress = new Commit(key,
+                                       row.getUUID("in_progress_ballot"),
+                                       row.has("proposal") ? ColumnFamily.fromBytes(row.getBytes("proposal")) : EmptyColumns.factory.create(metadata));
+        // either most_recent_commit and most_recent_commit_at will both be set, or neither
+        Commit mostRecent = row.has("most_recent_commit")
+                          ? new Commit(key, row.getUUID("most_recent_commit_at"), ColumnFamily.fromBytes(row.getBytes("most_recent_commit")))
+                          : Commit.emptyCommit(key, metadata);
+        return new PaxosState(inProgress, mostRecent);
+    }
+
+    public static void savePaxosPromise(Commit promise)
+    {
+        String req = "UPDATE %s USING TIMESTAMP %d AND TTL %d SET in_progress_ballot = %s WHERE row_key = 0x%s AND cf_id = %s";
+        processInternal(String.format(req,
+                                      PAXOS_CF,
+                                      UUIDGen.microsTimestamp(promise.ballot),
+                                      paxosTtl(promise.update.metadata),
+                                      promise.ballot,
+                                      ByteBufferUtil.bytesToHex(promise.key),
+                                      promise.update.id()));
+    }
+
+    public static void savePaxosProposal(Commit commit)
+    {
+        processInternal(String.format("UPDATE %s USING TIMESTAMP %d AND TTL %d SET proposal = 0x%s WHERE row_key = 0x%s AND cf_id = %s",
+                                      PAXOS_CF,
+                                      UUIDGen.microsTimestamp(commit.ballot),
+                                      paxosTtl(commit.update.metadata),
+                                      ByteBufferUtil.bytesToHex(commit.update.toBytes()),
+                                      ByteBufferUtil.bytesToHex(commit.key),
+                                      commit.update.id()));
+    }
+
+    private static int paxosTtl(CFMetaData metadata)
+    {
+        // keep paxos state around for at least 3h
+        return Math.max(3 * 3600, metadata.getGcGraceSeconds());
+    }
+
+    public static void savePaxosCommit(Commit commit, boolean eraseInProgressProposal)
+    {
+        String preserveCql = "UPDATE %s USING TIMESTAMP %d AND TTL %d SET most_recent_commit_at = %s, most_recent_commit = 0x%s WHERE row_key = 0x%s AND cf_id = %s";
+        // identical except adds proposal = null
+        String eraseCql = "UPDATE %s USING TIMESTAMP %d AND TTL %d SET proposal = null, most_recent_commit_at = %s, most_recent_commit = 0x%s WHERE row_key = 0x%s AND cf_id = %s";
+        processInternal(String.format(eraseInProgressProposal ? eraseCql : preserveCql,
+                                      PAXOS_CF,
+                                      UUIDGen.microsTimestamp(commit.ballot),
+                                      paxosTtl(commit.update.metadata),
+                                      commit.ballot,
+                                      ByteBufferUtil.bytesToHex(commit.update.toBytes()),
+                                      ByteBufferUtil.bytesToHex(commit.key),
+                                      commit.update.id()));
+    }
+}


[06/11] Rename Table to Keyspace

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index 0286bd3..c57d01e 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -42,7 +42,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.ColumnNameBuilder;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
@@ -189,7 +189,7 @@ public class StorageProxy implements StorageProxyMBean
      *  values) between the prepare and accept phases.  This gives us a slightly longer window for another
      *  coordinator to come along and trump our own promise with a newer one but is otherwise safe.
      *
-     * @param table the table for the CAS
+     * @param keyspaceName the keyspace for the CAS
      * @param cfName the column family for the CAS
      * @param key the row key for the row to CAS
      * @param prefix a column name prefix that selects the CQL3 row to check if {@code expected} is null. If {@code expected}
@@ -203,19 +203,19 @@ public class StorageProxy implements StorageProxyMBean
      * expected (since, if the CAS doesn't succeed, it means the current value do not match the one in expected). If
      * expected == null and the CAS is unsuccessfull, the first live column of the CF is returned.
      */
-    public static ColumnFamily cas(String table, String cfName, ByteBuffer key, ColumnNameBuilder prefix, ColumnFamily expected, ColumnFamily updates, ConsistencyLevel consistencyLevel)
+    public static ColumnFamily cas(String keyspaceName, String cfName, ByteBuffer key, ColumnNameBuilder prefix, ColumnFamily expected, ColumnFamily updates, ConsistencyLevel consistencyLevel)
     throws UnavailableException, IsBootstrappingException, ReadTimeoutException, WriteTimeoutException, InvalidRequestException
     {
-        consistencyLevel.validateForCas(table);
+        consistencyLevel.validateForCas(keyspaceName);
 
-        CFMetaData metadata = Schema.instance.getCFMetaData(table, cfName);
+        CFMetaData metadata = Schema.instance.getCFMetaData(keyspaceName, cfName);
 
         long start = System.nanoTime();
         long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout());
         while (System.nanoTime() - start < timeout)
         {
             // for simplicity, we'll do a single liveness check at the start of each attempt
-            Pair<List<InetAddress>, Integer> p = getPaxosParticipants(table, key);
+            Pair<List<InetAddress>, Integer> p = getPaxosParticipants(keyspaceName, key);
             List<InetAddress> liveEndpoints = p.left;
             int requiredParticipants = p.right;
 
@@ -232,11 +232,11 @@ public class StorageProxy implements StorageProxyMBean
                 SliceQueryFilter filter = prefix == null
                                         ? new SliceQueryFilter(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1)
                                         : new SliceQueryFilter(prefix.build(), prefix.buildAsEndOfRange(), false, 1, prefix.componentCount());
-                readCommand = new SliceFromReadCommand(table, key, cfName, timestamp, filter);
+                readCommand = new SliceFromReadCommand(keyspaceName, key, cfName, timestamp, filter);
             }
             else
             {
-                readCommand = new SliceByNamesReadCommand(table, key, cfName, timestamp, new NamesQueryFilter(ImmutableSortedSet.copyOf(expected.getColumnNames())));
+                readCommand = new SliceByNamesReadCommand(keyspaceName, key, cfName, timestamp, new NamesQueryFilter(ImmutableSortedSet.copyOf(expected.getColumnNames())));
             }
             List<Row> rows = read(Arrays.asList(readCommand), ConsistencyLevel.QUORUM);
             ColumnFamily current = rows.get(0).cf;
@@ -303,11 +303,11 @@ public class StorageProxy implements StorageProxyMBean
         return true;
     }
 
-    private static Pair<List<InetAddress>, Integer> getPaxosParticipants(String table, ByteBuffer key) throws UnavailableException
+    private static Pair<List<InetAddress>, Integer> getPaxosParticipants(String keyspaceName, ByteBuffer key) throws UnavailableException
     {
         Token tk = StorageService.getPartitioner().getToken(key);
-        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(table, tk);
-        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, table);
+        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
+        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
         int requiredParticipants = pendingEndpoints.size() + 1 + naturalEndpoints.size() / 2; // See CASSANDRA-833
         List<InetAddress> liveEndpoints = ImmutableList.copyOf(Iterables.filter(Iterables.concat(naturalEndpoints, pendingEndpoints), IAsyncCallback.isAlive));
         if (liveEndpoints.size() < requiredParticipants)
@@ -415,13 +415,13 @@ public class StorageProxy implements StorageProxyMBean
 
     private static void commitPaxos(Commit proposal, ConsistencyLevel consistencyLevel) throws WriteTimeoutException
     {
-        Table table = Table.open(proposal.update.metadata().ksName);
+        Keyspace keyspace = Keyspace.open(proposal.update.metadata().ksName);
 
         Token tk = StorageService.getPartitioner().getToken(proposal.key);
-        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(table.getName(), tk);
-        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, table.getName());
+        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspace.getName(), tk);
+        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspace.getName());
 
-        AbstractReplicationStrategy rs = table.getReplicationStrategy();
+        AbstractReplicationStrategy rs = keyspace.getReplicationStrategy();
         AbstractWriteResponseHandler responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistencyLevel, null, WriteType.SIMPLE);
 
         MessageOut<Commit> message = new MessageOut<Commit>(MessagingService.Verb.PAXOS_COMMIT, proposal, Commit.serializer);
@@ -584,7 +584,7 @@ public class StorageProxy implements StorageProxyMBean
         AbstractWriteResponseHandler handler = new WriteResponseHandler(endpoints,
                                                                         Collections.<InetAddress>emptyList(),
                                                                         ConsistencyLevel.ONE,
-                                                                        Table.open(Table.SYSTEM_KS),
+                                                                        Keyspace.open(Keyspace.SYSTEM_KS),
                                                                         null,
                                                                         WriteType.BATCH_LOG);
         updateBatchlog(rm, endpoints, handler);
@@ -593,15 +593,15 @@ public class StorageProxy implements StorageProxyMBean
 
     private static void asyncRemoveFromBatchlog(Collection<InetAddress> endpoints, UUID uuid)
     {
-        ColumnFamily cf = EmptyColumns.factory.create(Schema.instance.getCFMetaData(Table.SYSTEM_KS, SystemTable.BATCHLOG_CF));
+        ColumnFamily cf = EmptyColumns.factory.create(Schema.instance.getCFMetaData(Keyspace.SYSTEM_KS, SystemKeyspace.BATCHLOG_CF));
         cf.delete(new DeletionInfo(FBUtilities.timestampMicros(), (int) (System.currentTimeMillis() / 1000)));
         AbstractWriteResponseHandler handler = new WriteResponseHandler(endpoints,
                                                                         Collections.<InetAddress>emptyList(),
                                                                         ConsistencyLevel.ANY,
-                                                                        Table.open(Table.SYSTEM_KS),
+                                                                        Keyspace.open(Keyspace.SYSTEM_KS),
                                                                         null,
                                                                         WriteType.SIMPLE);
-        RowMutation rm = new RowMutation(Table.SYSTEM_KS, UUIDType.instance.decompose(uuid), cf);
+        RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, UUIDType.instance.decompose(uuid), cf);
         updateBatchlog(rm, endpoints, handler);
     }
 
@@ -659,12 +659,12 @@ public class StorageProxy implements StorageProxyMBean
                                                             WriteType writeType)
     throws UnavailableException, OverloadedException
     {
-        String table = mutation.getTable();
-        AbstractReplicationStrategy rs = Table.open(table).getReplicationStrategy();
+        String keyspaceName = mutation.getKeyspaceName();
+        AbstractReplicationStrategy rs = Keyspace.open(keyspaceName).getReplicationStrategy();
 
         Token tk = StorageService.getPartitioner().getToken(mutation.key());
-        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(table, tk);
-        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, table);
+        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
+        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
 
         AbstractWriteResponseHandler responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, callback, writeType);
 
@@ -678,11 +678,11 @@ public class StorageProxy implements StorageProxyMBean
     // same as above except does not initiate writes (but does perfrom availability checks).
     private static WriteResponseHandlerWrapper wrapResponseHandler(RowMutation mutation, ConsistencyLevel consistency_level, WriteType writeType)
     {
-        AbstractReplicationStrategy rs = Table.open(mutation.getTable()).getReplicationStrategy();
-        String table = mutation.getTable();
+        AbstractReplicationStrategy rs = Keyspace.open(mutation.getKeyspaceName()).getReplicationStrategy();
+        String keyspaceName = mutation.getKeyspaceName();
         Token tk = StorageService.getPartitioner().getToken(mutation.key());
-        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(table, tk);
-        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, table);
+        List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
+        Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
         AbstractWriteResponseHandler responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, null, writeType);
         return new WriteResponseHandlerWrapper(responseHandler, mutation);
     }
@@ -939,7 +939,7 @@ public class StorageProxy implements StorageProxyMBean
      */
     public static AbstractWriteResponseHandler mutateCounter(CounterMutation cm, String localDataCenter) throws UnavailableException, OverloadedException
     {
-        InetAddress endpoint = findSuitableEndpoint(cm.getTable(), cm.key(), localDataCenter, cm.consistency());
+        InetAddress endpoint = findSuitableEndpoint(cm.getKeyspaceName(), cm.key(), localDataCenter, cm.consistency());
 
         if (endpoint.equals(FBUtilities.getBroadcastAddress()))
         {
@@ -948,11 +948,11 @@ public class StorageProxy implements StorageProxyMBean
         else
         {
             // Exit now if we can't fulfill the CL here instead of forwarding to the leader replica
-            String table = cm.getTable();
-            AbstractReplicationStrategy rs = Table.open(table).getReplicationStrategy();
+            String keyspaceName = cm.getKeyspaceName();
+            AbstractReplicationStrategy rs = Keyspace.open(keyspaceName).getReplicationStrategy();
             Token tk = StorageService.getPartitioner().getToken(cm.key());
-            List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(table, tk);
-            Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, table);
+            List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
+            Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
 
             rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, cm.consistency(), null, WriteType.COUNTER).assureSufficientLiveNodes();
 
@@ -975,14 +975,14 @@ public class StorageProxy implements StorageProxyMBean
      * is unclear we want to mix those latencies with read latencies, so this
      * may be a bit involved.
      */
-    private static InetAddress findSuitableEndpoint(String tableName, ByteBuffer key, String localDataCenter, ConsistencyLevel cl) throws UnavailableException
+    private static InetAddress findSuitableEndpoint(String keyspaceName, ByteBuffer key, String localDataCenter, ConsistencyLevel cl) throws UnavailableException
     {
-        Table table = Table.open(tableName);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        List<InetAddress> endpoints = StorageService.instance.getLiveNaturalEndpoints(table, key);
+        List<InetAddress> endpoints = StorageService.instance.getLiveNaturalEndpoints(keyspace, key);
         if (endpoints.isEmpty())
             // TODO have a way to compute the consistency level
-            throw new UnavailableException(cl, cl.blockFor(table), 0);
+            throw new UnavailableException(cl, cl.blockFor(keyspace), 0);
 
         List<InetAddress> localEndpoints = new ArrayList<InetAddress>();
         for (InetAddress endpoint : endpoints)
@@ -1054,10 +1054,10 @@ public class StorageProxy implements StorageProxyMBean
         };
     }
 
-    private static boolean systemTableQuery(List<ReadCommand> cmds)
+    private static boolean systemKeyspaceQuery(List<ReadCommand> cmds)
     {
         for (ReadCommand cmd : cmds)
-            if (!cmd.table.equals(Table.SYSTEM_KS))
+            if (!cmd.ksName.equals(Keyspace.SYSTEM_KS))
                 return false;
         return true;
     }
@@ -1069,7 +1069,7 @@ public class StorageProxy implements StorageProxyMBean
     public static List<Row> read(List<ReadCommand> commands, ConsistencyLevel consistency_level)
     throws UnavailableException, IsBootstrappingException, ReadTimeoutException, InvalidRequestException, WriteTimeoutException
     {
-        if (StorageService.instance.isBootstrapMode() && !systemTableQuery(commands))
+        if (StorageService.instance.isBootstrapMode() && !systemKeyspaceQuery(commands))
         {
             readMetrics.unavailables.mark();
             ClientRequestMetrics.readUnavailables.inc();
@@ -1087,13 +1087,13 @@ public class StorageProxy implements StorageProxyMBean
                     throw new InvalidRequestException("SERIAL consistency may only be requested for one row at a time");
 
                 ReadCommand command = commands.get(0);
-                CFMetaData metadata = Schema.instance.getCFMetaData(command.table, command.cfName);
+                CFMetaData metadata = Schema.instance.getCFMetaData(command.ksName, command.cfName);
 
                 long start = System.nanoTime();
                 long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout());
                 while (true)
                 {
-                    Pair<List<InetAddress>, Integer> p = getPaxosParticipants(command.table, command.key);
+                    Pair<List<InetAddress>, Integer> p = getPaxosParticipants(command.ksName, command.key);
                     List<InetAddress> liveEndpoints = p.left;
                     int requiredParticipants = p.right;
 
@@ -1192,7 +1192,7 @@ public class StorageProxy implements StorageProxyMBean
                     ReadRepairMetrics.repairedBlocking.mark();
 
                     // Do a full data read to resolve the correct response (and repair node that need be)
-                    RowDataResolver resolver = new RowDataResolver(exec.command.table, exec.command.key, exec.command.filter(), exec.command.timestamp);
+                    RowDataResolver resolver = new RowDataResolver(exec.command.ksName, exec.command.key, exec.command.filter(), exec.command.timestamp);
                     ReadCallback<ReadResponse, Row> repairHandler = exec.handler.withNewResolver(resolver);
 
                     if (repairCommands == null)
@@ -1242,7 +1242,7 @@ public class StorageProxy implements StorageProxyMBean
                     }
                     catch (TimeoutException e)
                     {
-                        int blockFor = consistency_level.blockFor(Table.open(command.getKeyspace()));
+                        int blockFor = consistency_level.blockFor(Keyspace.open(command.getKeyspace()));
                         throw new ReadTimeoutException(consistency_level, blockFor, blockFor, true);
                     }
 
@@ -1284,8 +1284,8 @@ public class StorageProxy implements StorageProxyMBean
 
         protected void runMayThrow()
         {
-            Table table = Table.open(command.table);
-            Row r = command.getRow(table);
+            Keyspace keyspace = Keyspace.open(command.ksName);
+            Row r = command.getRow(keyspace);
             ReadResponse result = ReadVerbHandler.getResponse(command, r);
             MessagingService.instance().addLatency(FBUtilities.getBroadcastAddress(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
             handler.response(result);
@@ -1313,14 +1313,14 @@ public class StorageProxy implements StorageProxyMBean
         }
     }
 
-    public static List<InetAddress> getLiveSortedEndpoints(Table table, ByteBuffer key)
+    public static List<InetAddress> getLiveSortedEndpoints(Keyspace keyspace, ByteBuffer key)
     {
-        return getLiveSortedEndpoints(table, StorageService.instance.getPartitioner().decorateKey(key));
+        return getLiveSortedEndpoints(keyspace, StorageService.instance.getPartitioner().decorateKey(key));
     }
 
-    private static List<InetAddress> getLiveSortedEndpoints(Table table, RingPosition pos)
+    private static List<InetAddress> getLiveSortedEndpoints(Keyspace keyspace, RingPosition pos)
     {
-        List<InetAddress> liveEndpoints = StorageService.instance.getLiveNaturalEndpoints(table, pos);
+        List<InetAddress> liveEndpoints = StorageService.instance.getLiveNaturalEndpoints(keyspace, pos);
         DatabaseDescriptor.getEndpointSnitch().sortByProximity(FBUtilities.getBroadcastAddress(), liveEndpoints);
         return liveEndpoints;
     }
@@ -1343,7 +1343,7 @@ public class StorageProxy implements StorageProxyMBean
         Tracing.trace("Determining replicas to query");
         long startTime = System.nanoTime();
 
-        Table table = Table.open(command.keyspace);
+        Keyspace keyspace = Keyspace.open(command.keyspace);
         List<Row> rows;
         // now scan until we have enough results
         try
@@ -1361,10 +1361,10 @@ public class StorageProxy implements StorageProxyMBean
                                                   ? ranges.get(i)
                                                   : nextRange;
                 List<InetAddress> liveEndpoints = nextEndpoints == null
-                                                ? getLiveSortedEndpoints(table, range.right)
+                                                ? getLiveSortedEndpoints(keyspace, range.right)
                                                 : nextEndpoints;
                 List<InetAddress> filteredEndpoints = nextFilteredEndpoints == null
-                                                    ? consistency_level.filterForQuery(table, liveEndpoints)
+                                                    ? consistency_level.filterForQuery(keyspace, liveEndpoints)
                                                     : nextFilteredEndpoints;
                 ++i;
 
@@ -1374,8 +1374,8 @@ public class StorageProxy implements StorageProxyMBean
                 while (i < ranges.size())
                 {
                     nextRange = ranges.get(i);
-                    nextEndpoints = getLiveSortedEndpoints(table, nextRange.right);
-                    nextFilteredEndpoints = consistency_level.filterForQuery(table, nextEndpoints);
+                    nextEndpoints = getLiveSortedEndpoints(keyspace, nextRange.right);
+                    nextFilteredEndpoints = consistency_level.filterForQuery(keyspace, nextEndpoints);
 
                     /*
                      * If the current range right is the min token, we should stop merging because CFS.getRangeSlice
@@ -1390,10 +1390,10 @@ public class StorageProxy implements StorageProxyMBean
                     List<InetAddress> merged = intersection(liveEndpoints, nextEndpoints);
 
                     // Check if there is enough endpoint for the merge to be possible.
-                    if (!consistency_level.isSufficientLiveNodes(table, merged))
+                    if (!consistency_level.isSufficientLiveNodes(keyspace, merged))
                         break;
 
-                    List<InetAddress> filteredMerged = consistency_level.filterForQuery(table, merged);
+                    List<InetAddress> filteredMerged = consistency_level.filterForQuery(keyspace, merged);
 
                     // Estimate whether merging will be a win or not
                     if (!DatabaseDescriptor.getEndpointSnitch().isWorthMergingForRangeQuery(filteredMerged, filteredEndpoints, nextFilteredEndpoints))
@@ -1443,7 +1443,7 @@ public class StorageProxy implements StorageProxyMBean
                 {
                     logger.debug("Range slice timeout: {}", ex.toString());
                     // We actually got all response at that point
-                    int blockFor = consistency_level.blockFor(table);
+                    int blockFor = consistency_level.blockFor(keyspace);
                     throw new ReadTimeoutException(consistency_level, blockFor, blockFor, true);
                 }
                 catch (DigestMismatchException e)


[11/11] git commit: Rename Table to Keyspace

Posted by al...@apache.org.
Rename Table to Keyspace

patch by Jeremiah Jordan; reviewed by Aleksey Yeschenko for
CASSANDRA-5613


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

Branch: refs/heads/trunk
Commit: 0e96e585837c277073779a5f004fbb1e245964af
Parents: e22de43
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Thu Jun 27 21:35:18 2013 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Thu Jun 27 21:35:18 2013 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/cassandra/auth/DataResource.java |   2 +-
 .../apache/cassandra/cache/AutoSavingCache.java |   8 +-
 .../org/apache/cassandra/cli/CliClient.java     |  18 +-
 .../org/apache/cassandra/cli/CliCompiler.java   |   4 +-
 .../org/apache/cassandra/config/CFMetaData.java |  52 +-
 .../cassandra/config/ColumnDefinition.java      |  12 +-
 .../cassandra/config/DatabaseDescriptor.java    |  20 +-
 .../org/apache/cassandra/config/KSMetaData.java |  20 +-
 .../org/apache/cassandra/config/Schema.java     | 138 ++--
 .../apache/cassandra/config/TriggerOptions.java |   8 +-
 .../apache/cassandra/cql/QueryProcessor.java    |   4 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |   2 +-
 .../cassandra/cql3/functions/Function.java      |   2 +-
 .../cql3/statements/AlterKeyspaceStatement.java |   4 +-
 .../cql3/statements/AlterTableStatement.java    |   1 -
 .../cql3/statements/BatchStatement.java         |   2 +-
 .../statements/CreateColumnFamilyStatement.java |   2 +-
 .../cql3/statements/SelectStatement.java        |   6 +-
 .../apache/cassandra/db/BatchlogManager.java    |  14 +-
 .../cassandra/db/CollationController.java       |   4 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |  92 +--
 .../apache/cassandra/db/ConsistencyLevel.java   |  64 +-
 .../apache/cassandra/db/CounterMutation.java    |  22 +-
 .../org/apache/cassandra/db/DataTracker.java    |   4 +-
 .../db/DefinitionsUpdateVerbHandler.java        |   2 +-
 src/java/org/apache/cassandra/db/DefsTable.java | 470 -----------
 .../org/apache/cassandra/db/DefsTables.java     | 470 +++++++++++
 .../org/apache/cassandra/db/Directories.java    |  16 +-
 .../cassandra/db/HintedHandOffManager.java      |  24 +-
 src/java/org/apache/cassandra/db/IMutation.java |   2 +-
 src/java/org/apache/cassandra/db/Keyspace.java  | 454 +++++++++++
 src/java/org/apache/cassandra/db/Memtable.java  |   2 +-
 .../db/MigrationRequestVerbHandler.java         |   2 +-
 .../apache/cassandra/db/PagedRangeCommand.java  |   2 +-
 .../apache/cassandra/db/RangeSliceCommand.java  |   3 +-
 .../org/apache/cassandra/db/ReadCommand.java    |  24 +-
 .../org/apache/cassandra/db/ReadResponse.java   |   4 +-
 .../apache/cassandra/db/ReadVerbHandler.java    |   4 +-
 .../db/RetriedSliceFromReadCommand.java         |   6 +-
 .../org/apache/cassandra/db/RowMutation.java    |  50 +-
 .../cassandra/db/SliceByNamesReadCommand.java   |  26 +-
 .../cassandra/db/SliceFromReadCommand.java      |  26 +-
 .../org/apache/cassandra/db/SystemKeyspace.java | 814 +++++++++++++++++++
 .../org/apache/cassandra/db/SystemTable.java    | 814 -------------------
 src/java/org/apache/cassandra/db/Table.java     | 455 -----------
 .../cassandra/db/TruncateVerbHandler.java       |   2 +-
 .../db/columniterator/SSTableSliceIterator.java |   2 +-
 .../db/commitlog/CommitLogAllocator.java        |  10 +-
 .../db/commitlog/CommitLogReplayer.java         |  26 +-
 .../db/compaction/CompactionController.java     |   4 +-
 .../db/compaction/CompactionManager.java        |  28 +-
 .../cassandra/db/compaction/CompactionTask.java |   4 +-
 .../db/compaction/LegacyLeveledManifest.java    |   2 +-
 .../db/compaction/LeveledManifest.java          |   4 +-
 .../SizeTieredCompactionStrategy.java           |   8 +-
 .../AbstractSimplePerColumnSecondaryIndex.java  |   4 +-
 .../db/index/PerColumnSecondaryIndex.java       |   2 +-
 .../db/index/PerRowSecondaryIndex.java          |   2 +-
 .../cassandra/db/index/SecondaryIndex.java      |  14 +-
 .../db/index/SecondaryIndexBuilder.java         |   4 +-
 .../db/index/SecondaryIndexManager.java         |   2 +-
 .../dht/AbstractByteOrderedPartitioner.java     |   2 +-
 .../org/apache/cassandra/dht/BootStrapper.java  |   8 +-
 .../dht/OrderPreservingPartitioner.java         |   2 +-
 .../org/apache/cassandra/dht/RangeStreamer.java |  18 +-
 .../hadoop/ColumnFamilyRecordReader.java        |   2 +-
 .../cassandra/io/compress/SnappyCompressor.java |   2 +-
 .../apache/cassandra/io/sstable/SSTable.java    |   2 +-
 .../cassandra/io/sstable/SSTableReader.java     |   4 +-
 .../cassandra/io/sstable/SSTableWriter.java     |   2 +-
 .../org/apache/cassandra/io/util/FileUtils.java |   4 +-
 .../locator/AbstractReplicationStrategy.java    |  42 +-
 .../locator/GossipingPropertyFileSnitch.java    |   6 +-
 .../apache/cassandra/locator/LocalStrategy.java |   4 +-
 .../locator/NetworkTopologyStrategy.java        |   4 +-
 .../locator/OldNetworkTopologyStrategy.java     |   4 +-
 .../cassandra/locator/SimpleStrategy.java       |   4 +-
 .../apache/cassandra/locator/TokenMetadata.java |  26 +-
 .../cassandra/metrics/ColumnFamilyMetrics.java  |   6 +-
 .../cassandra/metrics/CompactionMetrics.java    |   6 +-
 .../cassandra/metrics/HintedHandoffMetrics.java |   4 +-
 .../org/apache/cassandra/repair/RepairJob.java  |   4 +-
 .../repair/RepairMessageVerbHandler.java        |   4 +-
 .../cassandra/repair/StreamingRepairTask.java   |   1 -
 .../cassandra/service/AbstractReadExecutor.java |  14 +-
 .../cassandra/service/AbstractRowResolver.java  |   6 +-
 .../service/AbstractWriteResponseHandler.java   |  12 +-
 .../cassandra/service/ActiveRepairService.java  |   8 +-
 .../cassandra/service/CassandraDaemon.java      |  44 +-
 .../apache/cassandra/service/ClientState.java   |  16 +-
 .../DatacenterSyncWriteResponseHandler.java     |   8 +-
 .../service/DatacenterWriteResponseHandler.java |   6 +-
 .../cassandra/service/MigrationManager.java     |   8 +-
 .../apache/cassandra/service/MigrationTask.java |   4 +-
 .../service/RangeSliceResponseResolver.java     |   8 +-
 .../apache/cassandra/service/ReadCallback.java  |  16 +-
 .../cassandra/service/RowDataResolver.java      |  10 +-
 .../cassandra/service/RowDigestResolver.java    |   4 +-
 .../ScheduledRangeTransferExecutorService.java  |   6 +-
 .../cassandra/service/SnapshotVerbHandler.java  |   6 +-
 .../apache/cassandra/service/StorageProxy.java  | 114 +--
 .../cassandra/service/StorageService.java       | 436 +++++-----
 .../cassandra/service/StorageServiceMBean.java  |  48 +-
 .../cassandra/service/WriteResponseHandler.java |   6 +-
 .../service/pager/NamesQueryPager.java          |   2 +-
 .../service/pager/SliceQueryPager.java          |   4 +-
 .../cassandra/service/paxos/PaxosState.java     |  18 +-
 .../cassandra/streaming/StreamReader.java       |   4 +-
 .../cassandra/streaming/StreamReceiveTask.java  |   4 +-
 .../cassandra/streaming/StreamSession.java      |   6 +-
 .../compress/CompressedStreamReader.java        |   4 +-
 .../cassandra/thrift/CassandraServer.java       |  14 +-
 .../cassandra/thrift/ThriftValidation.java      |  26 +-
 .../org/apache/cassandra/tools/BulkLoader.java  |   8 +-
 .../org/apache/cassandra/tools/NodeCmd.java     |  62 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  56 +-
 .../apache/cassandra/tools/SSTableExport.java   |  10 +-
 .../apache/cassandra/tools/SSTableImport.java   |   4 +-
 .../cassandra/tools/StandaloneScrubber.java     |  20 +-
 .../cassandra/tools/StandaloneUpgrader.java     |   6 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   2 +-
 .../cassandra/utils/BloomCalculations.java      |   2 +-
 .../org/apache/cassandra/utils/CounterId.java   |  12 +-
 .../apache/cassandra/utils/StatusLogger.java    |   2 +-
 .../org/apache/cassandra/utils/obs/BitUtil.java |   2 +-
 .../apache/cassandra/db/LongKeyspaceTest.java   |  71 ++
 .../org/apache/cassandra/db/LongTableTest.java  |  71 --
 .../apache/cassandra/db/MeteredFlusherTest.java |   4 +-
 .../db/compaction/LongCompactionsTest.java      |  14 +-
 .../LongLeveledCompactionStrategyTest.java      |   6 +-
 .../unit/org/apache/cassandra/SchemaLoader.java |   4 +-
 test/unit/org/apache/cassandra/Util.java        |  10 +-
 .../cassandra/cache/CacheProviderTest.java      |   4 +-
 .../apache/cassandra/config/CFMetaDataTest.java |  16 +-
 .../config/DatabaseDescriptorTest.java          |  12 +-
 .../org/apache/cassandra/config/DefsTest.java   |  26 +-
 .../org/apache/cassandra/db/CleanupTest.java    |  14 +-
 .../cassandra/db/CollationControllerTest.java   |   4 +-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 168 ++--
 .../apache/cassandra/db/CounterColumnTest.java  |   2 +-
 .../cassandra/db/CounterMutationTest.java       |   8 +-
 .../apache/cassandra/db/HintedHandOffTest.java  |  10 +-
 .../org/apache/cassandra/db/KeyCacheTest.java   |  16 +-
 .../apache/cassandra/db/KeyCollisionTest.java   |   6 +-
 .../org/apache/cassandra/db/KeyspaceTest.java   | 649 +++++++++++++++
 .../org/apache/cassandra/db/MultitableTest.java |  14 +-
 .../org/apache/cassandra/db/NameSortTest.java   |  14 +-
 .../apache/cassandra/db/RangeTombstoneTest.java |   8 +-
 .../apache/cassandra/db/ReadMessageTest.java    |   4 +-
 .../cassandra/db/RecoveryManager2Test.java      |   6 +-
 .../cassandra/db/RecoveryManager3Test.java      |  14 +-
 .../cassandra/db/RecoveryManagerTest.java       |  20 +-
 .../db/RecoveryManagerTruncateTest.java         |  12 +-
 .../cassandra/db/RemoveColumnFamilyTest.java    |   4 +-
 .../db/RemoveColumnFamilyWithFlush1Test.java    |   4 +-
 .../db/RemoveColumnFamilyWithFlush2Test.java    |   4 +-
 .../apache/cassandra/db/RemoveColumnTest.java   |   4 +-
 .../cassandra/db/RemoveSubColumnTest.java       |   8 +-
 .../org/apache/cassandra/db/RowCacheTest.java   |   6 +-
 .../apache/cassandra/db/RowIterationTest.java   |  22 +-
 .../apache/cassandra/db/SerializationsTest.java |   4 +-
 .../apache/cassandra/db/SystemKeyspaceTest.java |  80 ++
 .../apache/cassandra/db/SystemTableTest.java    |  81 --
 .../unit/org/apache/cassandra/db/TableTest.java | 653 ---------------
 .../org/apache/cassandra/db/TimeSortTest.java   |  26 +-
 .../compaction/BlacklistingCompactionsTest.java |   6 +-
 .../db/compaction/CompactionsPurgeTest.java     |  72 +-
 .../db/compaction/CompactionsTest.java          |  56 +-
 .../LeveledCompactionStrategyTest.java          |  16 +-
 .../db/compaction/OneCompactionTest.java        |   6 +-
 .../cassandra/db/compaction/TTLExpiryTest.java  |   7 +-
 .../cassandra/db/marshal/CompositeTypeTest.java |   4 +-
 .../db/marshal/DynamicCompositeTypeTest.java    |   4 +-
 .../apache/cassandra/dht/BootStrapperTest.java  |  14 +-
 .../cassandra/io/LazilyCompactedRowTest.java    |  24 +-
 .../io/sstable/SSTableMetadataTest.java         |  18 +-
 .../cassandra/io/sstable/SSTableReaderTest.java |  32 +-
 .../io/sstable/SSTableSimpleWriterTest.java     |   8 +-
 .../cassandra/io/sstable/SSTableUtils.java      |  24 +-
 .../locator/NetworkTopologyStrategyTest.java    |   8 +-
 .../ReplicationStrategyEndpointCacheTest.java   |  18 +-
 .../cassandra/locator/SimpleStrategyTest.java   |  24 +-
 .../apache/cassandra/repair/ValidatorTest.java  |   4 +-
 .../service/AntiEntropyServiceCounterTest.java  |   4 +-
 .../service/AntiEntropyServiceStandardTest.java |   4 +-
 .../service/AntiEntropyServiceTestAbstract.java |  40 +-
 .../service/LeaveAndBootstrapTest.java          |  74 +-
 .../org/apache/cassandra/service/MoveTest.java  |  50 +-
 .../cassandra/service/QueryPagerTest.java       |   2 +-
 .../apache/cassandra/service/RelocateTest.java  |  16 +-
 .../cassandra/service/RowResolverTest.java      |   2 +-
 .../service/StorageServiceServerTest.java       |  24 +-
 .../streaming/StreamingTransferTest.java        |  30 +-
 .../cassandra/utils/EncodedStreamsTest.java     |   6 +-
 195 files changed, 4095 insertions(+), 4108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 59c9dab..af824bc 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -66,6 +66,7 @@
  * Redesign repair messages (CASSANDRA-5426)
  * Fix ALTER RENAME post-5125 (CASSANDRA-5702)
  * Disallow renaming a 2ndary indexed column (CASSANDRA-5705)
+ * Rename Table to Keyspace (CASSANDRA-5613)
 
 
 1.2.7

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/auth/DataResource.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/auth/DataResource.java b/src/java/org/apache/cassandra/auth/DataResource.java
index 3ee40a7..9c6fd31 100644
--- a/src/java/org/apache/cassandra/auth/DataResource.java
+++ b/src/java/org/apache/cassandra/auth/DataResource.java
@@ -202,7 +202,7 @@ public class DataResource implements IResource
             case ROOT:
                 return true;
             case KEYSPACE:
-                return Schema.instance.getTables().contains(keyspace);
+                return Schema.instance.getKeyspaces().contains(keyspace);
             case COLUMN_FAMILY:
                 return Schema.instance.getCFMetaData(keyspace, columnFamily) != null;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 bcb1b7f..61ac3a5 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -31,12 +31,12 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.db.Table;
 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.FileUtils;
 import org.apache.cassandra.io.util.LengthAvailableInputStream;
@@ -105,7 +105,7 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
         long start = System.nanoTime();
 
         // old cache format that only saves keys
-        File path = getCachePath(cfs.table.getName(), cfs.name, null);
+        File path = getCachePath(cfs.keyspace.getName(), cfs.name, null);
         if (path.exists())
         {
             DataInputStream in = null;
@@ -133,7 +133,7 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
         }
 
         // modern format, allows both key and value (so key cache load can be purely sequential)
-        path = getCachePath(cfs.table.getName(), cfs.name, CURRENT_VERSION);
+        path = getCachePath(cfs.keyspace.getName(), cfs.name, CURRENT_VERSION);
         if (path.exists())
         {
             DataInputStream in = null;
@@ -199,7 +199,7 @@ public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K
             else
                 type = OperationType.UNKNOWN;
 
-            info = new CompactionInfo(new CFMetaData(Table.SYSTEM_KS, cacheType.toString(), ColumnFamilyType.Standard, BytesType.instance, null),
+            info = new CompactionInfo(new CFMetaData(Keyspace.SYSTEM_KS, cacheType.toString(), ColumnFamilyType.Standard, BytesType.instance, null),
                                       type,
                                       0,
                                       keys.size(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/cli/CliClient.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliClient.java b/src/java/org/apache/cassandra/cli/CliClient.java
index 7d5212f..066c1cc 100644
--- a/src/java/org/apache/cassandra/cli/CliClient.java
+++ b/src/java/org/apache/cassandra/cli/CliClient.java
@@ -502,7 +502,7 @@ public class CliClient
             assert columnTree != null;
             assert subColumnTree != null;
 
-            // table.cf['key']['column']['column']
+            // keyspace.cf['key']['column']['column']
             superColumnName = (columnTree.getType() == CliParser.FUNCTION_CALL)
                                       ? convertValueByFunction(columnTree, null, null).array()
                                       : columnNameAsByteArray(CliCompiler.getColumn(columnFamilySpec, 0), cfDef);
@@ -679,13 +679,13 @@ public class CliClient
             }
         }
 
-        // table.cf['key'] -- row slice
+        // keyspace.cf['key'] -- row slice
         if (columnSpecCnt == 0)
         {
             doSlice(keySpace, key, columnFamily, superColumnName, limit);
             return;
         }
-        // table.cf['key']['column'] -- slice of a super, or get of a standard
+        // keyspace.cf['key']['column'] -- slice of a super, or get of a standard
         else if (columnSpecCnt == 1)
         {
             columnName = getColumnName(columnFamily, columnFamilySpec.getChild(2));
@@ -697,7 +697,7 @@ public class CliClient
                 return;
             }
         }
-        // table.cf['key']['column']['column'] -- get of a sub-column
+        // keyspace.cf['key']['column']['column'] -- get of a sub-column
         else if (columnSpecCnt == 2)
         {
             superColumnName = getColumnName(columnFamily, columnFamilySpec.getChild(2)).array();
@@ -916,13 +916,13 @@ public class CliClient
         byte[] superColumnName = null;
         ByteBuffer columnName;
 
-        // table.cf['key']
+        // keyspace.cf['key']
         if (columnSpecCnt == 0)
         {
             sessionState.err.println("No cell name specified, (type 'help;' or '?' for help on syntax).");
             return;
         }
-        // table.cf['key']['column'] = 'value'
+        // keyspace.cf['key']['column'] = 'value'
         else if (columnSpecCnt == 1)
         {
             // get the column name
@@ -933,7 +933,7 @@ public class CliClient
             }
             columnName = getColumnName(columnFamily, columnFamilySpec.getChild(2));
         }
-        // table.cf['key']['super_column']['column'] = 'value'
+        // keyspace.cf['key']['super_column']['column'] = 'value'
         else
         {
             assert (columnSpecCnt == 2) : "serious parsing error (this is a bug).";
@@ -1001,12 +1001,12 @@ public class CliClient
         byte[] superColumnName = null;
         ByteBuffer columnName;
 
-        // table.cf['key']['column'] -- incr standard
+        // keyspace.cf['key']['column'] -- incr standard
         if (columnSpecCnt == 1)
         {
             columnName = getColumnName(columnFamily, columnFamilySpec.getChild(2));
         }
-        // table.cf['key']['column']['column'] -- incr super
+        // keyspace.cf['key']['column']['column'] -- incr super
         else if (columnSpecCnt == 2)
         {
             superColumnName = getColumnName(columnFamily, columnFamilySpec.getChild(2)).array();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/cli/CliCompiler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cli/CliCompiler.java b/src/java/org/apache/cassandra/cli/CliCompiler.java
index ad12821..6b84be1 100644
--- a/src/java/org/apache/cassandra/cli/CliCompiler.java
+++ b/src/java/org/apache/cassandra/cli/CliCompiler.java
@@ -158,14 +158,14 @@ public class CliCompiler
 
     public static int numColumnSpecifiers(Tree astNode)
     {
-        // Skip over table, column family and rowKey
+        // Skip over keyspace, column family and rowKey
         return astNode.getChildCount() - 2;
     }
 
     // Returns the pos'th (0-based index) column specifier in the astNode
     public static String getColumn(Tree astNode, int pos)
     {
-        // Skip over table, column family and rowKey
+        // Skip over keyspace, column family and rowKey
         return CliUtils.unescapeSQLString(astNode.getChild(pos + 2).getText());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 6594d75..38692b8 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -90,27 +90,27 @@ public final class CFMetaData
     // Note that this is the default only for user created tables
     public final static String DEFAULT_COMPRESSOR = LZ4Compressor.class.getCanonicalName();
 
-    public static final CFMetaData IndexCf = compile(5, "CREATE TABLE \"" + SystemTable.INDEX_CF + "\" ("
+    public static final CFMetaData IndexCf = compile(5, "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 CounterIdCf = compile(6, "CREATE TABLE \"" + SystemTable.COUNTER_ID_CF + "\" ("
+    public static final CFMetaData CounterIdCf = compile(6, "CREATE TABLE \"" + SystemKeyspace.COUNTER_ID_CF + "\" ("
                                                             + "key text,"
                                                             + "id timeuuid,"
                                                             + "PRIMARY KEY (key, id)"
                                                             + ") WITH COMPACT STORAGE AND COMMENT='counter node IDs'");
 
     // new-style schema
-    public static final CFMetaData SchemaKeyspacesCf = compile(8, "CREATE TABLE " + SystemTable.SCHEMA_KEYSPACES_CF + "("
+    public static final CFMetaData SchemaKeyspacesCf = compile(8, "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=8640");
 
-    public static final CFMetaData SchemaColumnFamiliesCf = compile(9, "CREATE TABLE " + SystemTable.SCHEMA_COLUMNFAMILIES_CF + "("
+    public static final CFMetaData SchemaColumnFamiliesCf = compile(9, "CREATE TABLE " + SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF + "("
                                                                        + "keyspace_name text,"
                                                                        + "columnfamily_name text,"
                                                                        + "type text,"
@@ -145,7 +145,7 @@ public final class CFMetaData
                                                                        + "PRIMARY KEY (keyspace_name, columnfamily_name)"
                                                                        + ") WITH COMMENT='ColumnFamily definitions' AND gc_grace_seconds=8640");
 
-    public static final CFMetaData SchemaColumnsCf = compile(10, "CREATE TABLE " + SystemTable.SCHEMA_COLUMNS_CF + "("
+    public static final CFMetaData SchemaColumnsCf = compile(10, "CREATE TABLE " + SystemKeyspace.SCHEMA_COLUMNS_CF + "("
                                                                  + "keyspace_name text,"
                                                                  + "columnfamily_name text,"
                                                                  + "column_name text,"
@@ -158,7 +158,7 @@ public final class CFMetaData
                                                                  + "PRIMARY KEY(keyspace_name, columnfamily_name, column_name)"
                                                                  + ") WITH COMMENT='ColumnFamily column attributes' AND gc_grace_seconds=8640");
 
-    public static final CFMetaData SchemaTriggerCf = compile("CREATE TABLE \"" + SystemTable.SCHEMA_TRIGGERS_CF + "\" ("
+    public static final CFMetaData SchemaTriggerCf = compile("CREATE TABLE \"" + SystemKeyspace.SCHEMA_TRIGGERS_CF + "\" ("
                                                                 + "keyspace_name text,"
                                                                 + "column_family text,"
                                                                 + "trigger_name text,"
@@ -166,7 +166,7 @@ public final class CFMetaData
                                                                 + "PRIMARY KEY (keyspace_name, column_family, trigger_name)"
                                                                 + ") WITH COMMENT='triggers metadata table'");
 
-    public static final CFMetaData HintsCf = compile("CREATE TABLE " + SystemTable.HINTS_CF + " ("
+    public static final CFMetaData HintsCf = compile("CREATE TABLE " + SystemKeyspace.HINTS_CF + " ("
                                                      + "target_id uuid,"
                                                      + "hint_id timeuuid,"
                                                      + "message_version int,"
@@ -177,7 +177,7 @@ public final class CFMetaData
                                                      + "AND COMMENT='hints awaiting delivery'"
                                                      + "AND gc_grace_seconds=0");
 
-    public static final CFMetaData PeersCf = compile("CREATE TABLE " + SystemTable.PEERS_CF + " ("
+    public static final CFMetaData PeersCf = compile("CREATE TABLE " + SystemKeyspace.PEERS_CF + " ("
                                                      + "peer inet PRIMARY KEY,"
                                                      + "host_id uuid,"
                                                      + "tokens set<varchar>,"
@@ -188,12 +188,12 @@ public final class CFMetaData
                                                      + "rack text"
                                                      + ") WITH COMMENT='known peers in the cluster'");
 
-    public static final CFMetaData PeerEventsCf = compile("CREATE TABLE " + SystemTable.PEER_EVENTS_CF + " ("
+    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 " + SystemTable.LOCAL_CF + " ("
+    public static final CFMetaData LocalCf = compile("CREATE TABLE " + SystemKeyspace.LOCAL_CF + " ("
                                                      + "key text PRIMARY KEY,"
                                                      + "tokens set<varchar>,"
                                                      + "cluster_name text,"
@@ -231,26 +231,26 @@ public final class CFMetaData
                                                            + ")",
                                                            Tracing.TRACE_KS);
 
-    public static final CFMetaData BatchlogCf = compile("CREATE TABLE " + SystemTable.BATCHLOG_CF + " ("
+    public static final CFMetaData BatchlogCf = compile("CREATE TABLE " + SystemKeyspace.BATCHLOG_CF + " ("
                                                         + "id uuid PRIMARY KEY,"
                                                         + "written_at timestamp,"
                                                         + "data blob"
                                                         + ") WITH COMMENT='uncommited batches' AND gc_grace_seconds=0 "
                                                         + "AND COMPACTION={'class' : 'SizeTieredCompactionStrategy', 'min_threshold' : 2}");
 
-    public static final CFMetaData RangeXfersCf = compile("CREATE TABLE " + SystemTable.RANGE_XFERS_CF + " ("
+    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 " + SystemTable.COMPACTION_LOG + " ("
+    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 " + SystemTable.PAXOS_CF + " ("
+    public static final CFMetaData PaxosCf = compile("CREATE TABLE " + SystemKeyspace.PAXOS_CF + " ("
                                                      + "row_key blob,"
                                                      + "cf_id UUID,"
                                                      + "in_progress_ballot timeuuid,"
@@ -472,12 +472,12 @@ public final class CFMetaData
 
     private static CFMetaData compile(String cql)
     {
-        return compile(null, cql, Table.SYSTEM_KS);
+        return compile(null, cql, Keyspace.SYSTEM_KS);
     }
 
     private static CFMetaData compile(int id, String cql)
     {
-        return compile(id, cql, Table.SYSTEM_KS);
+        return compile(id, cql, Keyspace.SYSTEM_KS);
     }
 
     private static AbstractType<?> makeComparator(ColumnFamilyType cftype, AbstractType<?> comp, AbstractType<?> subcc)
@@ -948,10 +948,10 @@ public final class CFMetaData
 
     public void reload()
     {
-        Row cfDefRow = SystemTable.readSchemaRow(ksName, cfName);
+        Row cfDefRow = SystemKeyspace.readSchemaRow(ksName, cfName);
 
         if (cfDefRow.cf == null || cfDefRow.cf.getColumnCount() == 0)
-            throw new RuntimeException(String.format("%s not found in the schema definitions table.", ksName + ":" + cfName));
+            throw new RuntimeException(String.format("%s not found in the schema definitions keyspace.", ksName + ":" + cfName));
 
         try
         {
@@ -1425,7 +1425,7 @@ public final class CFMetaData
      */
     public RowMutation toSchemaUpdate(CFMetaData newState, long modificationTimestamp, boolean fromThrift)
     {
-        RowMutation rm = new RowMutation(Table.SYSTEM_KS, SystemTable.getSchemaKSKey(ksName));
+        RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(ksName));
 
         newState.toSchemaNoColumns(rm, modificationTimestamp);
 
@@ -1471,15 +1471,15 @@ public final class CFMetaData
      */
     public RowMutation dropFromSchema(long timestamp)
     {
-        RowMutation rm = new RowMutation(Table.SYSTEM_KS, SystemTable.getSchemaKSKey(ksName));
-        ColumnFamily cf = rm.addOrGet(SystemTable.SCHEMA_COLUMNFAMILIES_CF);
+        RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(ksName));
+        ColumnFamily cf = rm.addOrGet(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF);
         int ldt = (int) (System.currentTimeMillis() / 1000);
 
         ColumnNameBuilder builder = SchemaColumnFamiliesCf.getCfDef().getColumnNameBuilder();
         builder.add(ByteBufferUtil.bytes(cfName));
         cf.addAtom(new RangeTombstone(builder.build(), builder.buildAsEndOfRange(), timestamp, ldt));
 
-        ColumnFamily tcf = rm.addOrGet(SystemTable.SCHEMA_TRIGGERS_CF);
+        ColumnFamily tcf = rm.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_CF);
         ColumnNameBuilder tbuilder = SchemaTriggerCf.getCfDef().getColumnNameBuilder();
         tbuilder.add(ByteBufferUtil.bytes(cfName));
         tcf.addAtom(new RangeTombstone(tbuilder.build(), tbuilder.buildAsEndOfRange(), timestamp, ldt));
@@ -1502,7 +1502,7 @@ 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 = rm.addOrGet(SystemTable.SCHEMA_COLUMNFAMILIES_CF);
+        ColumnFamily cf = rm.addOrGet(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF);
         int ldt = (int) (System.currentTimeMillis() / 1000);
 
         cf.addColumn(Column.create("", timestamp, cfName, ""));
@@ -1711,7 +1711,7 @@ public final class CFMetaData
      */
     public RowMutation toSchema(long timestamp) throws ConfigurationException
     {
-        RowMutation rm = new RowMutation(Table.SYSTEM_KS, SystemTable.getSchemaKSKey(ksName));
+        RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(ksName));
         toSchema(rm, timestamp);
         return rm;
     }
@@ -1788,10 +1788,10 @@ public final class CFMetaData
     {
         ColumnDefinition def = column_metadata.get(from);
         if (def == null)
-            throw new InvalidRequestException(String.format("Cannot rename unknown column %s in table %s", strFrom, cfName));
+            throw new InvalidRequestException(String.format("Cannot rename unknown column %s in keyspace %s", strFrom, cfName));
 
         if (column_metadata.get(to) != null)
-            throw new InvalidRequestException(String.format("Cannot rename column %s to %s in table %s; another column of that name already exist", strFrom, strTo, cfName));
+            throw new InvalidRequestException(String.format("Cannot rename column %s to %s in keyspace %s; another column of that name already exist", strFrom, strTo, cfName));
 
         if (def.type == ColumnDefinition.Type.REGULAR)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 d63e2ce..00a784f 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -197,7 +197,7 @@ public class ColumnDefinition
      */
     public void deleteFromSchema(RowMutation rm, String cfName, AbstractType<?> comparator, long timestamp)
     {
-        ColumnFamily cf = rm.addOrGet(SystemTable.SCHEMA_COLUMNS_CF);
+        ColumnFamily cf = rm.addOrGet(SystemKeyspace.SCHEMA_COLUMNS_CF);
         int ldt = (int) (System.currentTimeMillis() / 1000);
 
         ColumnNameBuilder builder = CFMetaData.SchemaColumnsCf.getCfDef().getColumnNameBuilder();
@@ -207,7 +207,7 @@ public class ColumnDefinition
 
     public void toSchema(RowMutation rm, String cfName, AbstractType<?> comparator, long timestamp)
     {
-        ColumnFamily cf = rm.addOrGet(SystemTable.SCHEMA_COLUMNS_CF);
+        ColumnFamily cf = rm.addOrGet(SystemKeyspace.SCHEMA_COLUMNS_CF);
         int ldt = (int) (System.currentTimeMillis() / 1000);
 
         cf.addColumn(Column.create("", timestamp, cfName, comparator.getString(name), ""));
@@ -300,11 +300,11 @@ public class ColumnDefinition
 
     public static Row readSchema(String ksName, String cfName)
     {
-        DecoratedKey key = StorageService.getPartitioner().decorateKey(SystemTable.getSchemaKSKey(ksName));
-        ColumnFamilyStore columnsStore = SystemTable.schemaCFS(SystemTable.SCHEMA_COLUMNS_CF);
+        DecoratedKey key = StorageService.getPartitioner().decorateKey(SystemKeyspace.getSchemaKSKey(ksName));
+        ColumnFamilyStore columnsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_COLUMNS_CF);
         ColumnFamily cf = columnsStore.getColumnFamily(key,
-                                                       DefsTable.searchComposite(cfName, true),
-                                                       DefsTable.searchComposite(cfName, false),
+                                                       DefsTables.searchComposite(cfName, true),
+                                                       DefsTables.searchComposite(cfName, false),
                                                        false,
                                                        Integer.MAX_VALUE,
                                                        System.currentTimeMillis());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 a015774..b59fbf8 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -34,8 +34,8 @@ import org.apache.cassandra.config.Config.RequestSchedulerId;
 import org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DefsTable;
-import org.apache.cassandra.db.SystemTable;
+import org.apache.cassandra.db.DefsTables;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.FSWriteError;
@@ -437,7 +437,7 @@ public class DatabaseDescriptor
             conf.server_encryption_options = conf.encryption_options;
         }
 
-        // Hardcoded system tables
+        // Hardcoded system keyspaces
         List<KSMetaData> systemKeyspaces = Arrays.asList(KSMetaData.systemKeyspace(), KSMetaData.traceKeyspace());
         assert systemKeyspaces.size() == Schema.systemKeyspaceNames.size();
         for (KSMetaData ksmd : systemKeyspaces)
@@ -445,7 +445,7 @@ public class DatabaseDescriptor
             // install the definition
             for (CFMetaData cfm : ksmd.cfMetaData().values())
                 Schema.instance.load(cfm);
-            Schema.instance.setTableDefinition(ksmd);
+            Schema.instance.setKeyspaceDefinition(ksmd);
         }
 
         /* Load the seeds for node contact points */
@@ -477,24 +477,24 @@ public class DatabaseDescriptor
         return conf.dynamic_snitch ? new DynamicEndpointSnitch(snitch) : snitch;
     }
 
-    /** load keyspace (table) definitions, but do not initialize the table instances. */
+    /** load keyspace (keyspace) definitions, but do not initialize the keyspace instances. */
     public static void loadSchemas()
     {
-        ColumnFamilyStore schemaCFS = SystemTable.schemaCFS(SystemTable.SCHEMA_KEYSPACES_CF);
+        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_KEYSPACES_CF);
 
-        // if table with definitions is empty try loading the old way
+        // if keyspace with definitions is empty try loading the old way
         if (schemaCFS.estimateKeys() == 0)
         {
             logger.info("Couldn't detect any schema definitions in local storage.");
             // peek around the data directories to see if anything is there.
             if (hasExistingNoSystemTables())
-                logger.info("Found table data in data directories. Consider using cqlsh to define your schema.");
+                logger.info("Found keyspace data in data directories. Consider using cqlsh to define your schema.");
             else
-                logger.info("To create keyspaces and column families, see 'help create table' in cqlsh.");
+                logger.info("To create keyspaces and column families, see 'help create keyspace' in cqlsh.");
         }
         else
         {
-            Schema.instance.load(DefsTable.loadFromTable());
+            Schema.instance.load(DefsTables.loadFromKeyspace());
         }
 
         Schema.instance.updateVersion();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 1988a9f..798a688 100644
--- a/src/java/org/apache/cassandra/config/KSMetaData.java
+++ b/src/java/org/apache/cassandra/config/KSMetaData.java
@@ -91,7 +91,7 @@ public final class KSMetaData
                                                 CFMetaData.SchemaColumnsCf,
                                                 CFMetaData.CompactionLogCf,
                                                 CFMetaData.PaxosCf);
-        return new KSMetaData(Table.SYSTEM_KS, LocalStrategy.class, Collections.<String, String>emptyMap(), true, cfDefs);
+        return new KSMetaData(Keyspace.SYSTEM_KS, LocalStrategy.class, Collections.<String, String>emptyMap(), true, cfDefs);
     }
 
     public static KSMetaData traceKeyspace()
@@ -215,29 +215,29 @@ public final class KSMetaData
 
     public KSMetaData reloadAttributes()
     {
-        Row ksDefRow = SystemTable.readSchemaRow(name);
+        Row ksDefRow = SystemKeyspace.readSchemaRow(name);
 
         if (ksDefRow.cf == null)
-            throw new RuntimeException(String.format("%s not found in the schema definitions table (%s).", name, SystemTable.SCHEMA_KEYSPACES_CF));
+            throw new RuntimeException(String.format("%s not found in the schema definitions keyspaceName (%s).", name, SystemKeyspace.SCHEMA_KEYSPACES_CF));
 
         return fromSchema(ksDefRow, Collections.<CFMetaData>emptyList());
     }
 
     public RowMutation dropFromSchema(long timestamp)
     {
-        RowMutation rm = new RowMutation(Table.SYSTEM_KS, SystemTable.getSchemaKSKey(name));
-        rm.delete(SystemTable.SCHEMA_KEYSPACES_CF, timestamp);
-        rm.delete(SystemTable.SCHEMA_COLUMNFAMILIES_CF, timestamp);
-        rm.delete(SystemTable.SCHEMA_COLUMNS_CF, timestamp);
-        rm.delete(SystemTable.SCHEMA_TRIGGERS_CF, timestamp);
+        RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(name));
+        rm.delete(SystemKeyspace.SCHEMA_KEYSPACES_CF, timestamp);
+        rm.delete(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, timestamp);
+        rm.delete(SystemKeyspace.SCHEMA_COLUMNS_CF, timestamp);
+        rm.delete(SystemKeyspace.SCHEMA_TRIGGERS_CF, timestamp);
 
         return rm;
     }
 
     public RowMutation toSchema(long timestamp)
     {
-        RowMutation rm = new RowMutation(Table.SYSTEM_KS, SystemTable.getSchemaKSKey(name));
-        ColumnFamily cf = rm.addOrGet(SystemTable.SCHEMA_KEYSPACES_CF);
+        RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(name));
+        ColumnFamily cf = rm.addOrGet(SystemKeyspace.SCHEMA_KEYSPACES_CF);
 
         cf.addColumn(Column.create(durableWrites, timestamp, "durable_writes"));
         cf.addColumn(Column.create(strategyClass.getName(), timestamp, "strategy_class"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 ae31269..2a9a731 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -28,7 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.service.MigrationManager;
@@ -51,11 +51,11 @@ public class Schema
      */
     public static final int NAME_LENGTH = 48;
 
-    /* metadata map for faster table lookup */
-    private final Map<String, KSMetaData> tables = new NonBlockingHashMap<String, KSMetaData>();
+    /* metadata map for faster keyspace lookup */
+    private final Map<String, KSMetaData> keyspaces = new NonBlockingHashMap<String, KSMetaData>();
 
-    /* Table objects, one per keyspace. Only one instance should ever exist for any given keyspace. */
-    private final Map<String, Table> tableInstances = new NonBlockingHashMap<String, Table>();
+    /* 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>();
 
     /* metadata map for faster ColumnFamily lookup */
     private final BiMap<Pair<String, String>, UUID> cfIdMap = HashBiMap.create();
@@ -64,7 +64,7 @@ public class Schema
 
     // 59adb24e-f3cd-3e02-97f0-5b395827453f
     public static final UUID emptyVersion;
-    public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Table.SYSTEM_KS, Tracing.TRACE_KS);
+    public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Keyspace.SYSTEM_KS, Tracing.TRACE_KS);
 
     static
     {
@@ -85,15 +85,15 @@ public class Schema
     {}
 
     /**
-     * Load up non-system tables
+     * Load up non-system keyspaces
      *
-     * @param tableDefs The non-system table definitions
+     * @param keyspaceDefs The non-system keyspace definitions
      *
      * @return self to support chaining calls
      */
-    public Schema load(Collection<KSMetaData> tableDefs)
+    public Schema load(Collection<KSMetaData> keyspaceDefs)
     {
-        for (KSMetaData def : tableDefs)
+        for (KSMetaData def : keyspaceDefs)
             load(def);
 
         return this;
@@ -111,74 +111,74 @@ public class Schema
         for (CFMetaData cfm : keyspaceDef.cfMetaData().values())
             load(cfm);
 
-        setTableDefinition(keyspaceDef);
+        setKeyspaceDefinition(keyspaceDef);
 
         return this;
     }
 
     /**
-     * Get table instance by name
+     * Get keyspace instance by name
      *
-     * @param tableName The name of the table
+     * @param keyspaceName The name of the keyspace
      *
-     * @return Table object or null if table was not found
+     * @return Keyspace object or null if keyspace was not found
      */
-    public Table getTableInstance(String tableName)
+    public Keyspace getKeyspaceInstance(String keyspaceName)
     {
-        return tableInstances.get(tableName);
+        return keyspaceInstances.get(keyspaceName);
     }
 
     /**
-     * Store given Table instance to the schema
+     * Store given Keyspace instance to the schema
      *
-     * @param table The Table instance to store
+     * @param keyspace The Keyspace instance to store
      *
-     * @throws IllegalArgumentException if Table is already stored
+     * @throws IllegalArgumentException if Keyspace is already stored
      */
-    public void storeTableInstance(Table table)
+    public void storeKeyspaceInstance(Keyspace keyspace)
     {
-        if (tableInstances.containsKey(table.getName()))
-            throw new IllegalArgumentException(String.format("Table %s was already initialized.", table.getName()));
+        if (keyspaceInstances.containsKey(keyspace.getName()))
+            throw new IllegalArgumentException(String.format("Keyspace %s was already initialized.", keyspace.getName()));
 
-        tableInstances.put(table.getName(), table);
+        keyspaceInstances.put(keyspace.getName(), keyspace);
     }
 
     /**
-     * Remove table from schema
+     * Remove keyspace from schema
      *
-     * @param tableName The name of the table to remove
+     * @param keyspaceName The name of the keyspace to remove
      *
-     * @return removed table instance or null if it wasn't found
+     * @return removed keyspace instance or null if it wasn't found
      */
-    public Table removeTableInstance(String tableName)
+    public Keyspace removeKeyspaceInstance(String keyspaceName)
     {
-        return tableInstances.remove(tableName);
+        return keyspaceInstances.remove(keyspaceName);
     }
 
     /**
-     * Remove table definition from system
+     * Remove keyspace definition from system
      *
-     * @param ksm The table definition to remove
+     * @param ksm The keyspace definition to remove
      */
-    public void clearTableDefinition(KSMetaData ksm)
+    public void clearKeyspaceDefinition(KSMetaData ksm)
     {
-        tables.remove(ksm.name);
+        keyspaces.remove(ksm.name);
     }
 
     /**
-     * Given a table name & column family name, get the column family
-     * meta data. If the table name or column family name is not valid
+     * Given a keyspace name & column family name, get the column family
+     * meta data. If the keyspace name or column family name is not valid
      * this function returns null.
      *
-     * @param tableName The table name
+     * @param keyspaceName The keyspace name
      * @param cfName The ColumnFamily name
      *
      * @return ColumnFamily Metadata object or null if it wasn't found
      */
-    public CFMetaData getCFMetaData(String tableName, String cfName)
+    public CFMetaData getCFMetaData(String keyspaceName, String cfName)
     {
-        assert tableName != null;
-        KSMetaData ksm = tables.get(tableName);
+        assert keyspaceName != null;
+        KSMetaData ksm = keyspaces.get(keyspaceName);
         return (ksm == null) ? null : ksm.cfMetaData().get(cfName);
     }
 
@@ -247,66 +247,66 @@ public class Schema
     }
 
     /**
-     * Get metadata about table by its name
+     * Get metadata about keyspace by its name
      *
-     * @param table The name of the table
+     * @param keyspaceName The name of the keyspace
      *
-     * @return The table metadata or null if it wasn't found
+     * @return The keyspace metadata or null if it wasn't found
      */
-    public KSMetaData getKSMetaData(String table)
+    public KSMetaData getKSMetaData(String keyspaceName)
     {
-        assert table != null;
-        return tables.get(table);
+        assert keyspaceName != null;
+        return keyspaces.get(keyspaceName);
     }
 
     /**
-     * @return collection of the non-system tables
+     * @return collection of the non-system keyspaces
      */
-    public List<String> getNonSystemTables()
+    public List<String> getNonSystemKeyspaces()
     {
-        return ImmutableList.copyOf(Sets.difference(tables.keySet(), systemKeyspaceNames));
+        return ImmutableList.copyOf(Sets.difference(keyspaces.keySet(), systemKeyspaceNames));
     }
 
     /**
-     * Get metadata about table inner ColumnFamilies
+     * Get metadata about keyspace inner ColumnFamilies
      *
-     * @param tableName The name of the table
+     * @param keyspaceName The name of the keyspace
      *
-     * @return metadata about ColumnFamilies the belong to the given table
+     * @return metadata about ColumnFamilies the belong to the given keyspace
      */
-    public Map<String, CFMetaData> getTableMetaData(String tableName)
+    public Map<String, CFMetaData> getKeyspaceMetaData(String keyspaceName)
     {
-        assert tableName != null;
-        KSMetaData ksm = tables.get(tableName);
+        assert keyspaceName != null;
+        KSMetaData ksm = keyspaces.get(keyspaceName);
         assert ksm != null;
         return ksm.cfMetaData();
     }
 
     /**
-     * @return collection of the all table names registered in the system (system and non-system)
+     * @return collection of the all keyspace names registered in the system (system and non-system)
      */
-    public Set<String> getTables()
+    public Set<String> getKeyspaces()
     {
-        return tables.keySet();
+        return keyspaces.keySet();
     }
 
     /**
-     * @return collection of the metadata about all tables registered in the system (system and non-system)
+     * @return collection of the metadata about all keyspaces registered in the system (system and non-system)
      */
-    public Collection<KSMetaData> getTableDefinitions()
+    public Collection<KSMetaData> getKeyspaceDefinitions()
     {
-        return tables.values();
+        return keyspaces.values();
     }
 
     /**
-     * Update (or insert) new table definition
+     * Update (or insert) new keyspace definition
      *
-     * @param ksm The metadata about table
+     * @param ksm The metadata about keyspace
      */
-    public void setTableDefinition(KSMetaData ksm)
+    public void setKeyspaceDefinition(KSMetaData ksm)
     {
         assert ksm != null;
-        tables.put(ksm.name, ksm);
+        keyspaces.put(ksm.name, ksm);
     }
 
     /* ColumnFamily query/control methods */
@@ -371,7 +371,7 @@ public class Schema
     }
 
     /**
-     * Read schema from system table and calculate MD5 digest of every row, resulting digest
+     * Read schema from system keyspace and calculate MD5 digest of every row, resulting digest
      * will be converted into UUID which would act as content-based version of the schema.
      */
     public void updateVersion()
@@ -380,7 +380,7 @@ public class Schema
         {
             MessageDigest versionDigest = MessageDigest.getInstance("MD5");
 
-            for (Row row : SystemTable.serializedSchema())
+            for (Row row : SystemKeyspace.serializedSchema())
             {
                 if (invalidSchemaRow(row) || ignoredSchemaRow(row))
                     continue;
@@ -389,7 +389,7 @@ public class Schema
             }
 
             version = UUID.nameUUIDFromBytes(versionDigest.digest());
-            SystemTable.updateSchemaVersion(version);
+            SystemKeyspace.updateSchemaVersion(version);
         }
         catch (Exception e)
         {
@@ -411,12 +411,12 @@ public class Schema
      */
     public synchronized void clear()
     {
-        for (String table : getNonSystemTables())
+        for (String keyspaceName : getNonSystemKeyspaces())
         {
-            KSMetaData ksm = getKSMetaData(table);
+            KSMetaData ksm = getKSMetaData(keyspaceName);
             for (CFMetaData cfm : ksm.cfMetaData().values())
                 purge(cfm);
-            clearTableDefinition(ksm);
+            clearKeyspaceDefinition(ksm);
         }
 
         updateVersionAndAnnounce();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/config/TriggerOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/TriggerOptions.java b/src/java/org/apache/cassandra/config/TriggerOptions.java
index 3ab5f86..3423bf3 100644
--- a/src/java/org/apache/cassandra/config/TriggerOptions.java
+++ b/src/java/org/apache/cassandra/config/TriggerOptions.java
@@ -15,7 +15,7 @@ import org.apache.cassandra.cql3.UntypedResultSet.Row;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.RangeTombstone;
 import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.db.SystemTable;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -27,7 +27,7 @@ public class TriggerOptions
     public static Map<String, Map<String, String>> getAllTriggers(String ksName, String cfName)
     {
         String req = "SELECT * FROM system.%s WHERE keyspace_name='%s' AND column_family='%s'";
-        UntypedResultSet result = processInternal(String.format(req, SystemTable.SCHEMA_TRIGGERS_CF, ksName, cfName));
+        UntypedResultSet result = processInternal(String.format(req, SystemKeyspace.SCHEMA_TRIGGERS_CF, ksName, cfName));
         Map<String, Map<String, String>> triggers = new HashMap<>();
         if (result.isEmpty())
             return triggers;
@@ -38,7 +38,7 @@ public class TriggerOptions
 
     public static void addColumns(RowMutation rm, String cfName, Entry<String, Map<String, String>> tentry, long modificationTimestamp)
     {
-        ColumnFamily cf = rm.addOrGet(SystemTable.SCHEMA_TRIGGERS_CF);
+        ColumnFamily cf = rm.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_CF);
         assert tentry.getValue().get(CLASS_KEY) != null;
         ColumnNameBuilder builder = CFMetaData.SchemaTriggerCf.getCfDef().getColumnNameBuilder();
         builder.add(ByteBufferUtil.bytes(cfName)).add(ByteBufferUtil.bytes(tentry.getKey())).add(ByteBufferUtil.bytes(OPTIONS_KEY));
@@ -52,7 +52,7 @@ public class TriggerOptions
 
     public static void deleteColumns(RowMutation rm, String cfName, Entry<String, Map<String, String>> tentry, long modificationTimestamp)
     {
-        ColumnFamily cf = rm.addOrGet(SystemTable.SCHEMA_TRIGGERS_CF);
+        ColumnFamily cf = rm.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_CF);
         int ldt = (int) (System.currentTimeMillis() / 1000);
         ColumnNameBuilder builder = CFMetaData.SchemaTriggerCf.getCfDef().getColumnNameBuilder();
         builder.add(ByteBufferUtil.bytes(cfName)).add(ByteBufferUtil.bytes(tentry.getKey()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/cql/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/QueryProcessor.java b/src/java/org/apache/cassandra/cql/QueryProcessor.java
index ea179b4..9e437b1 100644
--- a/src/java/org/apache/cassandra/cql/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql/QueryProcessor.java
@@ -241,7 +241,7 @@ public class QueryProcessor
         if (select.getColumnRelations().size() > 0)
         {
             AbstractType<?> comparator = select.getComparator(keyspace);
-            SecondaryIndexManager idxManager = Table.open(keyspace).getColumnFamilyStore(select.getColumnFamily()).indexManager;
+            SecondaryIndexManager idxManager = Keyspace.open(keyspace).getColumnFamilyStore(select.getColumnFamily()).indexManager;
             for (Relation relation : select.getColumnRelations())
             {
                 ByteBuffer name = relation.getEntity().getByteBuffer(comparator, variables);
@@ -361,7 +361,7 @@ public class QueryProcessor
                 if (select.isSetKeyspace())
                 {
                     keyspace = CliUtils.unescapeSQLString(select.getKeyspace());
-                    ThriftValidation.validateTable(keyspace);
+                    ThriftValidation.validateKeyspace(keyspace);
                 }
                 else if (oldKeyspace == null)
                     throw new InvalidRequestException("no keyspace has been specified");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 1de985b..b2da29d 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -160,7 +160,7 @@ public class QueryProcessor
         {
             ClientState state = new ClientState(true);
             QueryState qState = new QueryState(state);
-            state.setKeyspace(Table.SYSTEM_KS);
+            state.setKeyspace(Keyspace.SYSTEM_KS);
             CQLStatement statement = getStatement(query, state).statement;
             statement.validate(state);
             ResultMessage result = statement.executeInternal(qState);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/cql3/functions/Function.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/Function.java b/src/java/org/apache/cassandra/cql3/functions/Function.java
index 7adc7ca..ba5ae20 100644
--- a/src/java/org/apache/cassandra/cql3/functions/Function.java
+++ b/src/java/org/apache/cassandra/cql3/functions/Function.java
@@ -36,7 +36,7 @@ public interface Function
 
     public interface Factory
     {
-        // We allow the function to be parametered by the table it is part of because the
+        // We allow the function to be parametered by the keyspace it is part of because the
         // "token" function needs it (the argument depends on the keyValidator). However,
         // for most function, the factory will just always the same function object (see
         // AbstractFunction).

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 48b2f99..2fc0ce6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
@@ -22,7 +22,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.KSPropDefs;
-import org.apache.cassandra.db.Table;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.service.ClientState;
@@ -61,7 +61,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(Table.SYSTEM_KS))
+        if (ksm.name.equalsIgnoreCase(Keyspace.SYSTEM_KS))
             throw new InvalidRequestException("Cannot alter system keyspace");
 
         attrs.validate();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index 80835a6..945d202 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.CFMetaData;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index 6fbab72..dd15270 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -119,7 +119,7 @@ public class BatchStatement implements CQLStatement
         // Group mutation together, otherwise they won't get applied atomically
         for (IMutation m : statement.getMutations(variables, local, cl, attrs.getTimestamp(now, variables), true))
         {
-            Pair<String, ByteBuffer> key = Pair.create(m.getTable(), m.key());
+            Pair<String, ByteBuffer> key = Pair.create(m.getKeyspaceName(), m.key());
             IMutation existing = mutations.get(key);
 
             if (existing == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
index d5855be..996f221 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateColumnFamilyStatement.java
@@ -300,7 +300,7 @@ public class CreateColumnFamilyStatement extends SchemaAlteringStatement
                     stmt.defaultValidator = BytesType.instance;
                     // We need to distinguish between
                     //   * I'm upgrading from thrift so the valueAlias is null
-                    //   * I've define my table with only a PK (and the column value will be empty)
+                    //   * I've defined my table with only a PK (and the column value will be empty)
                     // So, we use an empty valueAlias (rather than null) for the second case
                     stmt.valueAlias = ByteBufferUtil.EMPTY_BYTE_BUFFER;
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index f481879..b323a33 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -214,12 +214,12 @@ public class SelectStatement implements CQLStatement
         return new ResultMessage.Rows(rset);
     }
 
-    static List<Row> readLocally(String keyspace, List<ReadCommand> cmds)
+    static List<Row> readLocally(String keyspaceName, List<ReadCommand> cmds)
     {
-        Table table = Table.open(keyspace);
+        Keyspace keyspace = Keyspace.open(keyspaceName);
         List<Row> rows = new ArrayList<Row>(cmds.size());
         for (ReadCommand cmd : cmds)
-            rows.add(cmd.getRow(table));
+            rows.add(cmd.getRow(keyspace));
         return rows;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/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 2e1b1a7..558a498 100644
--- a/src/java/org/apache/cassandra/db/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/db/BatchlogManager.java
@@ -94,7 +94,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
     public int countAllBatches()
     {
-        return (int) process("SELECT count(*) FROM %s.%s", Table.SYSTEM_KS, SystemTable.BATCHLOG_CF).one().getLong("count");
+        return (int) process("SELECT count(*) FROM %s.%s", Keyspace.SYSTEM_KS, SystemKeyspace.BATCHLOG_CF).one().getLong("count");
     }
 
     public long getTotalBatchesReplayed()
@@ -125,7 +125,7 @@ public class BatchlogManager implements BatchlogManagerMBean
         cf.addColumn(new Column(columnName("data"), data, timestamp));
         cf.addColumn(new Column(columnName("written_at"), writtenAt, timestamp));
 
-        return new RowMutation(Table.SYSTEM_KS, UUIDType.instance.decompose(uuid), cf);
+        return new RowMutation(Keyspace.SYSTEM_KS, UUIDType.instance.decompose(uuid), cf);
     }
 
     private static ByteBuffer serializeRowMutations(Collection<RowMutation> mutations)
@@ -156,7 +156,7 @@ public class BatchlogManager implements BatchlogManagerMBean
 
         try
         {
-            for (UntypedResultSet.Row row : process("SELECT id, written_at FROM %s.%s", Table.SYSTEM_KS, SystemTable.BATCHLOG_CF))
+            for (UntypedResultSet.Row row : process("SELECT id, written_at FROM %s.%s", Keyspace.SYSTEM_KS, SystemKeyspace.BATCHLOG_CF))
                 if (System.currentTimeMillis() > row.getLong("written_at") + TIMEOUT)
                     replayBatch(row.getUUID("id"));
             cleanup();
@@ -173,7 +173,7 @@ public class BatchlogManager implements BatchlogManagerMBean
     {
         logger.debug("Replaying batch {}", id);
 
-        UntypedResultSet result = process("SELECT written_at, data FROM %s.%s WHERE id = %s", Table.SYSTEM_KS, SystemTable.BATCHLOG_CF, id);
+        UntypedResultSet result = process("SELECT written_at, data FROM %s.%s WHERE id = %s", Keyspace.SYSTEM_KS, SystemKeyspace.BATCHLOG_CF, id);
         if (result.isEmpty())
             return;
 
@@ -187,7 +187,7 @@ public class BatchlogManager implements BatchlogManagerMBean
             logger.warn("Skipped batch replay of {} due to {}", id, e);
         }
 
-        process("DELETE FROM %s.%s WHERE id = %s", Table.SYSTEM_KS, SystemTable.BATCHLOG_CF, id);
+        process("DELETE FROM %s.%s WHERE id = %s", Keyspace.SYSTEM_KS, SystemKeyspace.BATCHLOG_CF, id);
 
         totalBatchesReplayed.incrementAndGet();
     }
@@ -211,7 +211,7 @@ public class BatchlogManager implements BatchlogManagerMBean
             return; // the mutation isn't safe to replay.
 
         Set<InetAddress> liveEndpoints = new HashSet<InetAddress>();
-        String ks = mutation.getTable();
+        String ks = mutation.getKeyspaceName();
         Token<?> tk = StorageService.getPartitioner().getToken(mutation.key());
         for (InetAddress endpoint : Iterables.concat(StorageService.instance.getNaturalEndpoints(ks, tk),
                                                      StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, ks)))
@@ -283,7 +283,7 @@ public class BatchlogManager implements BatchlogManagerMBean
     // force flush + compaction to reclaim space from the replayed batches
     private void cleanup() throws ExecutionException, InterruptedException
     {
-        ColumnFamilyStore cfs = Table.open(Table.SYSTEM_KS).getColumnFamilyStore(SystemTable.BATCHLOG_CF);
+        ColumnFamilyStore cfs = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(SystemKeyspace.BATCHLOG_CF);
         cfs.forceBlockingFlush();
         Collection<Descriptor> descriptors = new ArrayList<Descriptor>();
         for (SSTableReader sstr : cfs.getSSTables())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0e96e585/src/java/org/apache/cassandra/db/CollationController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CollationController.java b/src/java/org/apache/cassandra/db/CollationController.java
index d0d22c5..4ae27a2 100644
--- a/src/java/org/apache/cassandra/db/CollationController.java
+++ b/src/java/org/apache/cassandra/db/CollationController.java
@@ -170,9 +170,9 @@ public class CollationController
                 && cfs.getCompactionStrategy() instanceof SizeTieredCompactionStrategy)
             {
                 Tracing.trace("Defragmenting requested data");
-                RowMutation rm = new RowMutation(cfs.table.getName(), filter.key.key, returnCF.cloneMe());
+                RowMutation rm = new RowMutation(cfs.keyspace.getName(), filter.key.key, returnCF.cloneMe());
                 // skipping commitlog and index updates is fine since we're just de-fragmenting existing data
-                Table.open(rm.getTable()).apply(rm, false, false);
+                Keyspace.open(rm.getKeyspaceName()).apply(rm, false, false);
             }
 
             // Caller is responsible for final removeDeletedCF.  This is important for cacheRow to work correctly: