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

[02/11] Rename Table to Keyspace

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