You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2009/05/12 17:26:53 UTC

svn commit: r773941 - in /incubator/cassandra/trunk: ./ src/java/org/apache/cassandra/db/ test/unit/org/apache/cassandra/ test/unit/org/apache/cassandra/db/

Author: jbellis
Date: Tue May 12 15:26:52 2009
New Revision: 773941

URL: http://svn.apache.org/viewvc?rev=773941&view=rev
Log:
extract tests that could conflict with each other to a separate test class (= gets own jvm)
patch by jbellis; reviewed by Sandeep Tata for CASSANDRA-162

Added:
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/NameSortTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TimeSortTest.java
Modified:
    incubator/cassandra/trunk/build.xml
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CommitLog.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/CleanupHelper.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java

Modified: incubator/cassandra/trunk/build.xml
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/build.xml?rev=773941&r1=773940&r2=773941&view=diff
==============================================================================
--- incubator/cassandra/trunk/build.xml (original)
+++ incubator/cassandra/trunk/build.xml Tue May 12 15:26:52 2009
@@ -189,7 +189,7 @@
    <target name="test" depends="build-test">
     <echo message="running tests"/>
     <mkdir dir="${build.test.dir}/cassandra"/>
-    <junit fork="yes">
+    <junit fork="on">
       <formatter type="brief" usefile="false"/>
       <jvmarg value="-Dstorage-config=${test.conf}"/>
       <jvmarg value="-ea"/>

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=773941&r1=773940&r2=773941&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Tue May 12 15:26:52 2009
@@ -1468,20 +1468,6 @@
         return memtableSwitchCount;
     }
 
-    /**
-     * clears out all data associated with this ColumnFamily.
-     * For use in testing.
-     */
-    public void reset() throws IOException, ExecutionException, InterruptedException
-    {
-        forceBlockingFlush();
-        for (String fName : ssTables_)
-        {
-            new File(fName).delete();
-        }
-        ssTables_.clear();
-    }
-
     public Object getMemtable()
     {
         return memtable_.get();

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CommitLog.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CommitLog.java?rev=773941&r1=773940&r2=773941&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CommitLog.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CommitLog.java Tue May 12 15:26:52 2009
@@ -627,11 +627,6 @@
     	forcedRollOver_ = true;
     }
 
-    public static void reset()
-    {
-        CommitLog.instances_.clear();
-    }
-
     public static void main(String[] args) throws Throwable
     {
         LogUtil.init();

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/CleanupHelper.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/CleanupHelper.java?rev=773941&r1=773940&r2=773941&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/CleanupHelper.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/CleanupHelper.java Tue May 12 15:26:52 2009
@@ -16,29 +16,6 @@
     {
         // we clean the fs twice, once to start with (so old data files don't get stored by anything static if this is the first run)
         // and once after flushing stuff (to try to clean things out if it is not.)  part #2 seems to be less than perfect.
-        cleanUpFilesystem();
-
-        Table table = Table.open("Table1");
-        for (String cfName : table.getColumnFamilies())
-        {
-            ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
-            try
-            {
-                cfs.reset();
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-
-        CommitLog.reset();
-
-        cleanUpFilesystem();
-    }
-
-    private static void cleanUpFilesystem()
-    {
         String[] directoryNames = {
                 DatabaseDescriptor.getBootstrapFileLocation(),
                 DatabaseDescriptor.getLogFileLocation(),

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java?rev=773941&r1=773940&r2=773941&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java Tue May 12 15:26:52 2009
@@ -5,9 +5,7 @@
 import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 
-import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.StringUtils;
 import static org.junit.Assert.assertNull;
 import org.junit.Test;
@@ -32,328 +30,6 @@
     }
 
     @Test
-    public void testNameSort1() throws IOException, ExecutionException, InterruptedException
-    {
-        // single key
-        testNameSort(1);
-    }
-
-    @Test
-    public void testNameSort10() throws IOException, ExecutionException, InterruptedException
-    {
-        // multiple keys, flushing concurrently w/ inserts
-        testNameSort(10);
-    }
-
-    @Test
-    public void testNameSort100() throws IOException, ExecutionException, InterruptedException
-    {
-        // enough keys to force compaction concurrently w/ inserts
-        testNameSort(100);
-    }
-
-
-    private void testNameSort(int N) throws IOException, ExecutionException, InterruptedException
-    {
-        Table table = Table.open("Table1");
-
-        for (int i = 0; i < N; ++i)
-        {
-            String key = Integer.toString(i);
-            RowMutation rm;
-
-            // standard
-            for (int j = 0; j < 8; ++j)
-            {
-                byte[] bytes = j % 2 == 0 ? bytes1 : bytes2;
-                rm = new RowMutation("Table1", key);
-                rm.add("Standard1:" + "Column-" + j, bytes, j);
-                rm.apply();
-            }
-
-            // super
-            for (int j = 0; j < 8; ++j)
-            {
-                for (int k = 0; k < 4; ++k)
-                {
-                    byte[] bytes = (j + k) % 2 == 0 ? bytes1 : bytes2;
-                    rm = new RowMutation("Table1", key);
-                    rm.add("Super1:" + "SuperColumn-" + j + ":Column-" + k, bytes, k);
-                    rm.apply();
-                }
-            }
-        }
-
-        validateNameSort(table, N);
-
-        table.getColumnFamilyStore("Standard1").forceBlockingFlush();
-        table.getColumnFamilyStore("Super1").forceBlockingFlush();
-        validateNameSort(table, N);
-    }
-
-    @Test
-    public void testTimeSort() throws IOException, ExecutionException, InterruptedException
-    {
-        Table table = Table.open("Table1");
-
-        for (int i = 900; i < 1000; ++i)
-        {
-            String key = Integer.toString(i);
-            RowMutation rm;
-            for (int j = 0; j < 8; ++j)
-            {
-                byte[] bytes = j % 2 == 0 ? bytes1 : bytes2;
-                rm = new RowMutation("Table1", key);
-                rm.add("StandardByTime1:" + "Column-" + j, bytes, j * 2);
-                rm.apply();
-            }
-        }
-
-        validateTimeSort(table);
-
-        table.getColumnFamilyStore("StandardByTime1").forceBlockingFlush();
-        validateTimeSort(table);
-
-        // interleave some new data to test memtable + sstable
-        String key = "900";
-        RowMutation rm;
-        for (int j = 0; j < 4; ++j)
-        {
-            rm = new RowMutation("Table1", key);
-            rm.add("StandardByTime1:" + "Column+" + j, ArrayUtils.EMPTY_BYTE_ARRAY, j * 2 + 1);
-            rm.apply();
-        }
-        // and some overwrites
-        for (int j = 4; j < 8; ++j)
-        {
-            rm = new RowMutation("Table1", key);
-            rm.add("StandardByTime1:" + "Column-" + j, ArrayUtils.EMPTY_BYTE_ARRAY, j * 3);
-            rm.apply();
-        }
-        // verify
-        ColumnFamily cf = table.getRow(key, "StandardByTime1", 0).getColumnFamilies().iterator().next();
-        SortedSet<IColumn> columns = cf.getAllColumns();
-        assert columns.size() == 12;
-        Iterator<IColumn> iter = columns.iterator();
-        IColumn column;
-        for (int j = 7; j >= 4; j--)
-        {
-            column = iter.next();
-            assert column.name().equals("Column-" + j);
-            assert column.timestamp() == j * 3;
-            assert column.value().length == 0;
-        }
-        for (int j = 3; j >= 0; j--)
-        {
-            column = iter.next();
-            assert column.name().equals("Column+" + j);
-            column = iter.next();
-            assert column.name().equals("Column-" + j);
-        }
-    }
-
-    private void validateTimeSort(Table table) throws IOException
-    {
-        for (int i = 900; i < 1000; ++i)
-        {
-            String key = Integer.toString(i);
-            for (int j = 0; j < 8; j += 3)
-            {
-                ColumnFamily cf = table.getRow(key, "StandardByTime1", j * 2).getColumnFamilies().iterator().next();
-                SortedSet<IColumn> columns = cf.getAllColumns();
-                assert columns.size() == 8 - j;
-                int k = 7;
-                for (IColumn c : columns)
-                {
-                    assert c.timestamp() == (k--) * 2;
-                }
-            }
-        }
-    }
-
-    private void validateNameSort(Table table, int N) throws IOException
-    {
-        for (int i = 0; i < N; ++i)
-        {
-            String key = Integer.toString(i);
-            ColumnFamily cf;
-
-            cf = table.get(key, "Standard1");
-            Collection<IColumn> columns = cf.getAllColumns();
-            for (IColumn column : columns)
-            {
-                int j = Integer.valueOf(column.name().split("-")[1]);
-                byte[] bytes = j % 2 == 0 ? bytes1 : bytes2;
-                assert Arrays.equals(bytes, column.value());
-            }
-
-            cf = table.get(key, "Super1");
-            assert cf != null;
-            Collection<IColumn> superColumns = cf.getAllColumns();
-            assert superColumns.size() == 8;
-            for (IColumn superColumn : superColumns)
-            {
-                int j = Integer.valueOf(superColumn.name().split("-")[1]);
-                Collection<IColumn> subColumns = superColumn.getSubColumns();
-                assert subColumns.size() == 4;
-                for (IColumn subColumn : subColumns)
-                {
-                    int k = Integer.valueOf(subColumn.name().split("-")[1]);
-                    byte[] bytes = (j + k) % 2 == 0 ? bytes1 : bytes2;
-                    assert Arrays.equals(bytes, subColumn.value());
-                }
-            }
-        }
-    }
-
-    @Test
-    public void testRemoveColumn() throws IOException, ExecutionException, InterruptedException
-    {
-        Table table = Table.open("Table1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
-        RowMutation rm;
-
-        // add data
-        rm = new RowMutation("Table1", "key1");
-        rm.add("Standard1:Column1", "asdf".getBytes(), 0);
-        rm.apply();
-        store.forceBlockingFlush();
-
-        // remove
-        rm = new RowMutation("Table1", "key1");
-        rm.delete("Standard1:Column1", 1);
-        rm.apply();
-
-        ColumnFamily retrieved = store.getColumnFamily("key1", "Standard1", new IdentityFilter());
-        assert retrieved.getColumn("Column1").isMarkedForDelete();
-        assertNull(ColumnFamilyStore.removeDeleted(retrieved, Integer.MAX_VALUE));
-    }
-
-    @Test
-    public void testRemoveSubColumn() throws IOException, ExecutionException, InterruptedException
-    {
-        Table table = Table.open("Table1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Super1");
-        RowMutation rm;
-
-        // add data
-        rm = new RowMutation("Table1", "key1");
-        rm.add("Super1:SC1:Column1", "asdf".getBytes(), 0);
-        rm.apply();
-        store.forceBlockingFlush();
-
-        // remove
-        rm = new RowMutation("Table1", "key1");
-        rm.delete("Super1:SC1:Column1", 1);
-        rm.apply();
-
-        ColumnFamily retrieved = store.getColumnFamily("key1", "Super1:SC1", new IdentityFilter());
-        assert retrieved.getColumn("SC1").getSubColumn("Column1").isMarkedForDelete();
-        assertNull(ColumnFamilyStore.removeDeleted(retrieved, Integer.MAX_VALUE));
-    }
-
-    @Test
-    public void testRemoveSuperColumn() throws IOException, ExecutionException, InterruptedException
-    {
-        Table table = Table.open("Table1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Super1");
-        RowMutation rm;
-
-        // add data
-        rm = new RowMutation("Table1", "key1");
-        rm.add("Super1:SC1:Column1", "asdf".getBytes(), 0);
-        rm.apply();
-        store.forceBlockingFlush();
-
-        // remove
-        rm = new RowMutation("Table1", "key1");
-        rm.delete("Super1:SC1", 1);
-        rm.apply();
-
-        List<ColumnFamily> families = store.getColumnFamilies("key1", "Super1", new IdentityFilter());
-        assert families.size() == 2 : StringUtils.join(families, ", ");
-        assert families.get(0).getAllColumns().first().getMarkedForDeleteAt() == 1; // delete marker, just added
-        assert !families.get(1).getAllColumns().first().isMarkedForDelete(); // flushed old version
-        ColumnFamily resolved = ColumnFamily.resolve(families);
-        assert resolved.getAllColumns().first().getMarkedForDeleteAt() == 1;
-        Collection<IColumn> subColumns = resolved.getAllColumns().first().getSubColumns();
-        assert subColumns.size() == 1;
-        assert subColumns.iterator().next().timestamp() == 0;
-        assertNull(ColumnFamilyStore.removeDeleted(resolved, Integer.MAX_VALUE));
-    }
-
-    @Test
-    public void testRemoveColumnFamily() throws IOException, ExecutionException, InterruptedException
-    {
-        Table table = Table.open("Table1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
-        RowMutation rm;
-
-        // add data
-        rm = new RowMutation("Table1", "key1");
-        rm.add("Standard1:Column1", "asdf".getBytes(), 0);
-        rm.apply();
-
-        // remove
-        rm = new RowMutation("Table1", "key1");
-        rm.delete("Standard1", 1);
-        rm.apply();
-
-        ColumnFamily retrieved = store.getColumnFamily("key1", "Standard1:Column1", new IdentityFilter());
-        assert retrieved.isMarkedForDelete();
-        assertNull(retrieved.getColumn("Column1"));
-        assertNull(ColumnFamilyStore.removeDeleted(retrieved, Integer.MAX_VALUE));
-    }
-
-    @Test
-    public void testRemoveColumnFamilyWithFlush1() throws IOException, ExecutionException, InterruptedException
-    {
-        Table table = Table.open("Table1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
-        RowMutation rm;
-
-        // add data
-        rm = new RowMutation("Table1", "key1");
-        rm.add("Standard1:Column1", "asdf".getBytes(), 0);
-        rm.add("Standard1:Column2", "asdf".getBytes(), 0);
-        rm.apply();
-        store.forceBlockingFlush();
-
-        // remove
-        rm = new RowMutation("Table1", "key1");
-        rm.delete("Standard1", 1);
-        rm.apply();
-
-        ColumnFamily retrieved = store.getColumnFamily("key1", "Standard1", new IdentityFilter());
-        assert retrieved.isMarkedForDelete();
-        assertNull(retrieved.getColumn("Column1"));
-        assertNull(ColumnFamilyStore.removeDeleted(retrieved, Integer.MAX_VALUE));
-    }
-
-    @Test
-    public void testRemoveColumnFamilyWithFlush2() throws IOException, ExecutionException, InterruptedException
-    {
-        Table table = Table.open("Table1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
-        RowMutation rm;
-
-        // add data
-        rm = new RowMutation("Table1", "key1");
-        rm.add("Standard1:Column1", "asdf".getBytes(), 0);
-        rm.apply();
-        // remove
-        rm = new RowMutation("Table1", "key1");
-        rm.delete("Standard1", 1);
-        rm.apply();
-        store.forceBlockingFlush();
-
-        ColumnFamily retrieved = store.getColumnFamily("key1", "Standard1:Column1", new IdentityFilter());
-        assert retrieved.isMarkedForDelete();
-        assertNull(retrieved.getColumn("Column1"));
-        assertNull(ColumnFamilyStore.removeDeleted(retrieved, Integer.MAX_VALUE));
-    }
-
-    @Test
     public void testGetCompactionBuckets() throws IOException
     {
         // create files 20 40 60 ... 180
@@ -407,59 +83,6 @@
     }
 
     @Test
-    public void testOneCompaction() throws IOException, ExecutionException, InterruptedException
-    {
-        Table table = Table.open("Table1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
-
-        Set<String> inserted = new HashSet<String>();
-        for (int j = 0; j < 2; j++) {
-            String key = "0";
-            RowMutation rm = new RowMutation("Table1", key);
-            rm.add("Standard1:0", new byte[0], j);
-            rm.apply();
-            inserted.add(key);
-            store.forceBlockingFlush();
-            assertEquals(table.getKeyRange("", "", 10000).size(), inserted.size());
-        }
-        store.doCompaction(2);
-        assertEquals(table.getKeyRange("", "", 10000).size(), inserted.size());
-    }
-
-    @Test
-    public void testCompactions() throws IOException, ExecutionException, InterruptedException
-    {
-        // this test does enough rows to force multiple block indexes to be used
-        Table table = Table.open("Table1");
-        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
-
-        final int ROWS_PER_SSTABLE = 10;
-        Set<String> inserted = new HashSet<String>();
-        for (int j = 0; j < (SSTable.indexInterval() * 3) / ROWS_PER_SSTABLE; j++) {
-            for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
-                String key = String.valueOf(i % 2);
-                RowMutation rm = new RowMutation("Table1", key);
-                rm.add("Standard1:" + (i / 2), new byte[0], j * ROWS_PER_SSTABLE + i);
-                rm.apply();
-                inserted.add(key);
-            }
-            store.forceBlockingFlush();
-            assertEquals(table.getKeyRange("", "", 10000).size(), inserted.size());
-        }
-        while (true)
-        {
-            Future<Integer> ft = MinorCompactionManager.instance().submit(store);
-            if (ft.get() == 0)
-                break;
-        }
-        if (store.getSSTableFilenames().size() > 1)
-        {
-            store.doCompaction(store.getSSTableFilenames().size());
-        }
-        assertEquals(table.getKeyRange("", "", 10000).size(), inserted.size());
-    }
-    
-    @Test
     public void testGetColumnWithWrongBF() throws IOException, ExecutionException, InterruptedException
     {
         Table table = Table.open("Table1");

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java?rev=773941&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java Tue May 12 15:26:52 2009
@@ -0,0 +1,48 @@
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.Set;
+import java.util.HashSet;
+
+import org.junit.Test;
+
+import org.apache.cassandra.io.SSTable;
+import static junit.framework.Assert.assertEquals;
+
+public class CompactionsTest
+{
+    @Test
+    public void testCompactions() throws IOException, ExecutionException, InterruptedException
+    {
+        // this test does enough rows to force multiple block indexes to be used
+        Table table = Table.open("Table1");
+        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+
+        final int ROWS_PER_SSTABLE = 10;
+        Set<String> inserted = new HashSet<String>();
+        for (int j = 0; j < (SSTable.indexInterval() * 3) / ROWS_PER_SSTABLE; j++) {
+            for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
+                String key = String.valueOf(i % 2);
+                RowMutation rm = new RowMutation("Table1", key);
+                rm.add("Standard1:" + (i / 2), new byte[0], j * ROWS_PER_SSTABLE + i);
+                rm.apply();
+                inserted.add(key);
+            }
+            store.forceBlockingFlush();
+            assertEquals(table.getKeyRange("", "", 10000).size(), inserted.size());
+        }
+        while (true)
+        {
+            Future<Integer> ft = MinorCompactionManager.instance().submit(store);
+            if (ft.get() == 0)
+                break;
+        }
+        if (store.getSSTableFilenames().size() > 1)
+        {
+            store.doCompaction(store.getSSTableFilenames().size());
+        }
+        assertEquals(table.getKeyRange("", "", 10000).size(), inserted.size());
+    }
+}

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/NameSortTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/NameSortTest.java?rev=773941&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/NameSortTest.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/NameSortTest.java Tue May 12 15:26:52 2009
@@ -0,0 +1,110 @@
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.SortedSet;
+import java.util.Iterator;
+import java.util.Collection;
+import java.util.Arrays;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.junit.Test;
+
+public class NameSortTest extends ColumnFamilyStoreTest
+{
+    @Test
+    public void testNameSort1() throws IOException, ExecutionException, InterruptedException
+    {
+        // single key
+        testNameSort(1);
+    }
+
+    @Test
+    public void testNameSort10() throws IOException, ExecutionException, InterruptedException
+    {
+        // multiple keys, flushing concurrently w/ inserts
+        testNameSort(10);
+    }
+
+    @Test
+    public void testNameSort100() throws IOException, ExecutionException, InterruptedException
+    {
+        // enough keys to force compaction concurrently w/ inserts
+        testNameSort(100);
+    }
+
+
+    private void testNameSort(int N) throws IOException, ExecutionException, InterruptedException
+    {
+        Table table = Table.open("Table1");
+
+        for (int i = 0; i < N; ++i)
+        {
+            String key = Integer.toString(i);
+            RowMutation rm;
+
+            // standard
+            for (int j = 0; j < 8; ++j)
+            {
+                byte[] bytes = j % 2 == 0 ? bytes1 : bytes2;
+                rm = new RowMutation("Table1", key);
+                rm.add("Standard1:" + "Column-" + j, bytes, j);
+                rm.apply();
+            }
+
+            // super
+            for (int j = 0; j < 8; ++j)
+            {
+                for (int k = 0; k < 4; ++k)
+                {
+                    byte[] bytes = (j + k) % 2 == 0 ? bytes1 : bytes2;
+                    rm = new RowMutation("Table1", key);
+                    rm.add("Super1:" + "SuperColumn-" + j + ":Column-" + k, bytes, k);
+                    rm.apply();
+                }
+            }
+        }
+
+        validateNameSort(table, N);
+
+        table.getColumnFamilyStore("Standard1").forceBlockingFlush();
+        table.getColumnFamilyStore("Super1").forceBlockingFlush();
+        validateNameSort(table, N);
+    }
+
+    private void validateNameSort(Table table, int N) throws IOException
+    {
+        for (int i = 0; i < N; ++i)
+        {
+            String key = Integer.toString(i);
+            ColumnFamily cf;
+
+            cf = table.get(key, "Standard1");
+            Collection<IColumn> columns = cf.getAllColumns();
+            for (IColumn column : columns)
+            {
+                int j = Integer.valueOf(column.name().split("-")[1]);
+                byte[] bytes = j % 2 == 0 ? bytes1 : bytes2;
+                assert Arrays.equals(bytes, column.value());
+            }
+
+            cf = table.get(key, "Super1");
+            assert cf != null;
+            Collection<IColumn> superColumns = cf.getAllColumns();
+            assert superColumns.size() == 8;
+            for (IColumn superColumn : superColumns)
+            {
+                int j = Integer.valueOf(superColumn.name().split("-")[1]);
+                Collection<IColumn> subColumns = superColumn.getSubColumns();
+                assert subColumns.size() == 4;
+                for (IColumn subColumn : subColumns)
+                {
+                    int k = Integer.valueOf(subColumn.name().split("-")[1]);
+                    byte[] bytes = (j + k) % 2 == 0 ? bytes1 : bytes2;
+                    assert Arrays.equals(bytes, subColumn.value());
+                }
+            }
+        }
+    }
+
+}

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java?rev=773941&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java Tue May 12 15:26:52 2009
@@ -0,0 +1,33 @@
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.Set;
+import java.util.HashSet;
+
+import org.junit.Test;
+
+import static junit.framework.Assert.assertEquals;
+
+public class OneCompactionTest
+{
+    @Test
+    public void testOneCompaction() throws IOException, ExecutionException, InterruptedException
+    {
+        Table table = Table.open("Table1");
+        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+
+        Set<String> inserted = new HashSet<String>();
+        for (int j = 0; j < 2; j++) {
+            String key = "0";
+            RowMutation rm = new RowMutation("Table1", key);
+            rm.add("Standard1:0", new byte[0], j);
+            rm.apply();
+            inserted.add(key);
+            store.forceBlockingFlush();
+            assertEquals(table.getKeyRange("", "", 10000).size(), inserted.size());
+        }
+        store.doCompaction(2);
+        assertEquals(table.getKeyRange("", "", 10000).size(), inserted.size());
+    }
+}

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java?rev=773941&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java Tue May 12 15:26:52 2009
@@ -0,0 +1,34 @@
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNull;
+
+public class RemoveColumnFamilyTest
+{
+    @Test
+    public void testRemoveColumnFamily() throws IOException, ExecutionException, InterruptedException
+    {
+        Table table = Table.open("Table1");
+        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        RowMutation rm;
+
+        // add data
+        rm = new RowMutation("Table1", "key1");
+        rm.add("Standard1:Column1", "asdf".getBytes(), 0);
+        rm.apply();
+
+        // remove
+        rm = new RowMutation("Table1", "key1");
+        rm.delete("Standard1", 1);
+        rm.apply();
+
+        ColumnFamily retrieved = store.getColumnFamily("key1", "Standard1:Column1", new IdentityFilter());
+        assert retrieved.isMarkedForDelete();
+        assertNull(retrieved.getColumn("Column1"));
+        assertNull(ColumnFamilyStore.removeDeleted(retrieved, Integer.MAX_VALUE));
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java?rev=773941&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java Tue May 12 15:26:52 2009
@@ -0,0 +1,36 @@
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNull;
+
+public class RemoveColumnFamilyWithFlush1Test
+{
+    @Test
+    public void testRemoveColumnFamilyWithFlush1() throws IOException, ExecutionException, InterruptedException
+    {
+        Table table = Table.open("Table1");
+        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        RowMutation rm;
+
+        // add data
+        rm = new RowMutation("Table1", "key1");
+        rm.add("Standard1:Column1", "asdf".getBytes(), 0);
+        rm.add("Standard1:Column2", "asdf".getBytes(), 0);
+        rm.apply();
+        store.forceBlockingFlush();
+
+        // remove
+        rm = new RowMutation("Table1", "key1");
+        rm.delete("Standard1", 1);
+        rm.apply();
+
+        ColumnFamily retrieved = store.getColumnFamily("key1", "Standard1", new IdentityFilter());
+        assert retrieved.isMarkedForDelete();
+        assertNull(retrieved.getColumn("Column1"));
+        assertNull(ColumnFamilyStore.removeDeleted(retrieved, Integer.MAX_VALUE));
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java?rev=773941&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java Tue May 12 15:26:52 2009
@@ -0,0 +1,34 @@
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNull;
+
+public class RemoveColumnFamilyWithFlush2Test
+{
+    @Test
+    public void testRemoveColumnFamilyWithFlush2() throws IOException, ExecutionException, InterruptedException
+    {
+        Table table = Table.open("Table1");
+        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        RowMutation rm;
+
+        // add data
+        rm = new RowMutation("Table1", "key1");
+        rm.add("Standard1:Column1", "asdf".getBytes(), 0);
+        rm.apply();
+        // remove
+        rm = new RowMutation("Table1", "key1");
+        rm.delete("Standard1", 1);
+        rm.apply();
+        store.forceBlockingFlush();
+
+        ColumnFamily retrieved = store.getColumnFamily("key1", "Standard1:Column1", new IdentityFilter());
+        assert retrieved.isMarkedForDelete();
+        assertNull(retrieved.getColumn("Column1"));
+        assertNull(ColumnFamilyStore.removeDeleted(retrieved, Integer.MAX_VALUE));
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnTest.java?rev=773941&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnTest.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveColumnTest.java Tue May 12 15:26:52 2009
@@ -0,0 +1,34 @@
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNull;
+
+public class RemoveColumnTest
+{
+    @Test
+    public void testRemoveColumn() throws IOException, ExecutionException, InterruptedException
+    {
+        Table table = Table.open("Table1");
+        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        RowMutation rm;
+
+        // add data
+        rm = new RowMutation("Table1", "key1");
+        rm.add("Standard1:Column1", "asdf".getBytes(), 0);
+        rm.apply();
+        store.forceBlockingFlush();
+
+        // remove
+        rm = new RowMutation("Table1", "key1");
+        rm.delete("Standard1:Column1", 1);
+        rm.apply();
+
+        ColumnFamily retrieved = store.getColumnFamily("key1", "Standard1", new IdentityFilter());
+        assert retrieved.getColumn("Column1").isMarkedForDelete();
+        assertNull(ColumnFamilyStore.removeDeleted(retrieved, Integer.MAX_VALUE));
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java?rev=773941&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java Tue May 12 15:26:52 2009
@@ -0,0 +1,34 @@
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Test;
+
+import static junit.framework.Assert.assertNull;
+
+public class RemoveSubColumnTest
+{
+    @Test
+    public void testRemoveSubColumn() throws IOException, ExecutionException, InterruptedException
+    {
+        Table table = Table.open("Table1");
+        ColumnFamilyStore store = table.getColumnFamilyStore("Super1");
+        RowMutation rm;
+
+        // add data
+        rm = new RowMutation("Table1", "key1");
+        rm.add("Super1:SC1:Column1", "asdf".getBytes(), 0);
+        rm.apply();
+        store.forceBlockingFlush();
+
+        // remove
+        rm = new RowMutation("Table1", "key1");
+        rm.delete("Super1:SC1:Column1", 1);
+        rm.apply();
+
+        ColumnFamily retrieved = store.getColumnFamily("key1", "Super1:SC1", new IdentityFilter());
+        assert retrieved.getColumn("SC1").getSubColumn("Column1").isMarkedForDelete();
+        assertNull(ColumnFamilyStore.removeDeleted(retrieved, Integer.MAX_VALUE));
+    }
+}
\ No newline at end of file

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java?rev=773941&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java Tue May 12 15:26:52 2009
@@ -0,0 +1,43 @@
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.List;
+import java.util.Collection;
+
+import org.apache.commons.lang.StringUtils;
+import org.junit.Test;
+import static org.junit.Assert.assertNull;
+
+public class RemoveSuperColumnTest
+{
+    @Test
+    public void testRemoveSuperColumn() throws IOException, ExecutionException, InterruptedException
+    {
+        Table table = Table.open("Table1");
+        ColumnFamilyStore store = table.getColumnFamilyStore("Super1");
+        RowMutation rm;
+
+        // add data
+        rm = new RowMutation("Table1", "key1");
+        rm.add("Super1:SC1:Column1", "asdf".getBytes(), 0);
+        rm.apply();
+        store.forceBlockingFlush();
+
+        // remove
+        rm = new RowMutation("Table1", "key1");
+        rm.delete("Super1:SC1", 1);
+        rm.apply();
+
+        List<ColumnFamily> families = store.getColumnFamilies("key1", "Super1", new IdentityFilter());
+        assert families.size() == 2 : StringUtils.join(families, ", ");
+        assert families.get(0).getAllColumns().first().getMarkedForDeleteAt() == 1; // delete marker, just added
+        assert !families.get(1).getAllColumns().first().isMarkedForDelete(); // flushed old version
+        ColumnFamily resolved = ColumnFamily.resolve(families);
+        assert resolved.getAllColumns().first().getMarkedForDeleteAt() == 1;
+        Collection<IColumn> subColumns = resolved.getAllColumns().first().getSubColumns();
+        assert subColumns.size() == 1;
+        assert subColumns.iterator().next().timestamp() == 0;
+        assertNull(ColumnFamilyStore.removeDeleted(resolved, Integer.MAX_VALUE));
+    }
+}

Added: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TimeSortTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TimeSortTest.java?rev=773941&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TimeSortTest.java (added)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TimeSortTest.java Tue May 12 15:26:52 2009
@@ -0,0 +1,92 @@
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+import java.util.SortedSet;
+import java.util.Iterator;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.junit.Test;
+
+public class TimeSortTest extends ColumnFamilyStoreTest
+{
+    @Test
+    public void testTimeSort() throws IOException, ExecutionException, InterruptedException
+    {
+        Table table = Table.open("Table1");
+
+        for (int i = 900; i < 1000; ++i)
+        {
+            String key = Integer.toString(i);
+            RowMutation rm;
+            for (int j = 0; j < 8; ++j)
+            {
+                byte[] bytes = j % 2 == 0 ? bytes1 : bytes2;
+                rm = new RowMutation("Table1", key);
+                rm.add("StandardByTime1:" + "Column-" + j, bytes, j * 2);
+                rm.apply();
+            }
+        }
+
+        validateTimeSort(table);
+
+        table.getColumnFamilyStore("StandardByTime1").forceBlockingFlush();
+        validateTimeSort(table);
+
+        // interleave some new data to test memtable + sstable
+        String key = "900";
+        RowMutation rm;
+        for (int j = 0; j < 4; ++j)
+        {
+            rm = new RowMutation("Table1", key);
+            rm.add("StandardByTime1:" + "Column+" + j, ArrayUtils.EMPTY_BYTE_ARRAY, j * 2 + 1);
+            rm.apply();
+        }
+        // and some overwrites
+        for (int j = 4; j < 8; ++j)
+        {
+            rm = new RowMutation("Table1", key);
+            rm.add("StandardByTime1:" + "Column-" + j, ArrayUtils.EMPTY_BYTE_ARRAY, j * 3);
+            rm.apply();
+        }
+        // verify
+        ColumnFamily cf = table.getRow(key, "StandardByTime1", 0).getColumnFamilies().iterator().next();
+        SortedSet<IColumn> columns = cf.getAllColumns();
+        assert columns.size() == 12;
+        Iterator<IColumn> iter = columns.iterator();
+        IColumn column;
+        for (int j = 7; j >= 4; j--)
+        {
+            column = iter.next();
+            assert column.name().equals("Column-" + j);
+            assert column.timestamp() == j * 3;
+            assert column.value().length == 0;
+        }
+        for (int j = 3; j >= 0; j--)
+        {
+            column = iter.next();
+            assert column.name().equals("Column+" + j);
+            column = iter.next();
+            assert column.name().equals("Column-" + j);
+        }
+    }
+
+    private void validateTimeSort(Table table) throws IOException
+    {
+        for (int i = 900; i < 1000; ++i)
+        {
+            String key = Integer.toString(i);
+            for (int j = 0; j < 8; j += 3)
+            {
+                ColumnFamily cf = table.getRow(key, "StandardByTime1", j * 2).getColumnFamilies().iterator().next();
+                SortedSet<IColumn> columns = cf.getAllColumns();
+                assert columns.size() == 8 - j;
+                int k = 7;
+                for (IColumn c : columns)
+                {
+                    assert c.timestamp() == (k--) * 2;
+                }
+            }
+        }
+    }
+}