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

[04/11] Rename Table to Keyspace

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");