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/04/29 03:12:39 UTC

svn commit: r769608 - in /incubator/cassandra/trunk: ./ src/org/apache/cassandra/config/ src/org/apache/cassandra/db/ src/org/apache/cassandra/utils/ test/conf/ test/system/

Author: jbellis
Date: Wed Apr 29 01:12:38 2009
New Revision: 769608

URL: http://svn.apache.org/viewvc?rev=769608&view=rev
Log:
add system tests.  each method in test/system/test_server.py exercises part of the thrift API, and manages cleaning out data files and starting a fresh server for each test.
patch by jbellis; reviewed by Eric Evans for CASSANDRA-102

Added:
    incubator/cassandra/trunk/test/system/
    incubator/cassandra/trunk/test/system/__init__.py
    incubator/cassandra/trunk/test/system/stress.py
    incubator/cassandra/trunk/test/system/test_server.py
Modified:
    incubator/cassandra/trunk/build.xml
    incubator/cassandra/trunk/src/org/apache/cassandra/config/DatabaseDescriptor.java
    incubator/cassandra/trunk/src/org/apache/cassandra/db/CalloutManager.java
    incubator/cassandra/trunk/src/org/apache/cassandra/utils/FileUtils.java
    incubator/cassandra/trunk/test/conf/storage-conf.xml

Modified: incubator/cassandra/trunk/build.xml
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/build.xml?rev=769608&r1=769607&r2=769608&view=diff
==============================================================================
--- incubator/cassandra/trunk/build.xml (original)
+++ incubator/cassandra/trunk/build.xml Wed Apr 29 01:12:38 2009
@@ -200,6 +200,7 @@
             verbose="2">
       <sysproperty key="net.sourceforge.cobertura.datafile" file="${cobertura.datafile}"/>
       <jvmarg value="-Dstorage-config=${test.conf}"/>
+      <jvmarg value="-ea"/>
       <classpath>
         <pathelement location="${cobertura.classes.dir}"/>
         <path refid="cassandra.classpath" />

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/config/DatabaseDescriptor.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/config/DatabaseDescriptor.java?rev=769608&r1=769607&r2=769608&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/config/DatabaseDescriptor.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/config/DatabaseDescriptor.java Wed Apr 29 01:12:38 2009
@@ -249,7 +249,7 @@
                 FileUtils.createDirectory(snapshotDirectory_);
             else
             {
-                    snapshotDirectory_ = metadataDirectory_ + System.getProperty("file.separator") + "snapshot";
+                snapshotDirectory_ = metadataDirectory_ + System.getProperty("file.separator") + "snapshot";
             }
 
             /* data file directory */
@@ -265,7 +265,7 @@
             bootstrapFileDirectory_ = xmlUtils.getNodeValue("/Storage/BootstrapFileDirectory");
             if (bootstrapFileDirectory_ == null)
             {
-                throw new ConfigurationException("MetadataDirectory must be specified");
+                throw new ConfigurationException("BootstrapFileDirectory must be specified");
             }
             FileUtils.createDirectory(bootstrapFileDirectory_);
 
@@ -273,7 +273,7 @@
             logFileDirectory_ = xmlUtils.getNodeValue("/Storage/CommitLogDirectory");
             if (logFileDirectory_ == null)
             {
-                throw new ConfigurationException("MetadataDirectory must be specified");
+                throw new ConfigurationException("CommitLogDirectory must be specified");
             }
             FileUtils.createDirectory(logFileDirectory_);
 

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/CalloutManager.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/CalloutManager.java?rev=769608&r1=769607&r2=769608&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/CalloutManager.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/CalloutManager.java Wed Apr 29 01:12:38 2009
@@ -39,6 +39,8 @@
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.procedures.GroovyScriptRunner;
 import org.apache.cassandra.utils.LogUtil;
+import org.apache.cassandra.utils.FileUtils;
+
 import org.apache.log4j.Logger;
 
 public class CalloutManager
@@ -111,12 +113,9 @@
     	if ( location == null )
     		return;
     	
-        File directory = new File(location);        
-        
-        if ( !directory.exists() )
-            directory.mkdir();
+        FileUtils.createDirectory(location);
         
-        File[] files = directory.listFiles();
+        File[] files = new File(location).listFiles();
         
         for ( File file : files )
         {

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/utils/FileUtils.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/utils/FileUtils.java?rev=769608&r1=769607&r2=769608&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/utils/FileUtils.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/utils/FileUtils.java Wed Apr 29 01:12:38 2009
@@ -97,8 +97,13 @@
     public static void createDirectory(String directory) throws IOException
     {
         File file = new File(directory);
-        if ( !file.exists() )
-            file.mkdir();
+        if (!file.exists())
+        {
+            if (!file.mkdirs())
+            {
+                throw new IOException("unable to mkdirs " + directory);
+            }
+        }
     }
 
     public static void createFile(String directory) throws IOException

Modified: incubator/cassandra/trunk/test/conf/storage-conf.xml
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/conf/storage-conf.xml?rev=769608&r1=769607&r2=769608&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/conf/storage-conf.xml (original)
+++ incubator/cassandra/trunk/test/conf/storage-conf.xml Wed Apr 29 01:12:38 2009
@@ -17,7 +17,7 @@
    <ListenAddress>127.0.0.1</ListenAddress>
    <StoragePort>7000</StoragePort>
    <ControlPort>7001</ControlPort>
-   <ThriftPort>7001</ThriftPort>
+   <ThriftPort>9160</ThriftPort>
    <ColumnIndexSizeInKB>256</ColumnIndexSizeInKB>
    <HttpPort>7002</HttpPort>
    <MetadataDirectory>build/test/cassandra/system</MetadataDirectory>

Added: incubator/cassandra/trunk/test/system/__init__.py
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/system/__init__.py?rev=769608&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/system/__init__.py (added)
+++ incubator/cassandra/trunk/test/system/__init__.py Wed Apr 29 01:12:38 2009
@@ -0,0 +1,81 @@
+import os, sys, time, signal
+
+__all__ = ['root', 'client']
+
+from thrift.transport import TTransport
+from thrift.transport import TSocket
+from thrift.transport import THttpClient
+from thrift.protocol import TBinaryProtocol
+
+# add cassandra directory to sys.path
+L = os.path.abspath(__file__).split(os.path.sep)[:-3]
+root = os.path.sep.join(L)
+_ipath = os.path.join(root, 'interface', 'gen-py')
+sys.path.append(os.path.join(_ipath, 'org', 'apache', 'cassandra'))
+import Cassandra
+
+host, port = '127.0.0.1', 9160
+def get_client():
+    socket = TSocket.TSocket(host, port)
+    transport = TTransport.TBufferedTransport(socket)
+    protocol = TBinaryProtocol.TBinaryProtocol(transport)
+    client = Cassandra.Client(protocol)
+    client.transport = transport
+    return client
+
+client = get_client()
+
+
+import tempfile
+_, pid_fname = tempfile.mkstemp()
+def pid():
+    return int(open(pid_fname).read())
+
+
+class CassandraTester(object):
+    # leave this True unless you are manually starting a server and then
+    # running only a single test against it; tests assume they start against an empty db.
+    runserver = True
+
+    def setUp(self):
+        if self.runserver:
+            # clean out old stuff
+            import shutil
+            # todo get directories from conf/storage-conf.xml
+            for dirname in ['system', 'data', 'commitlog']:
+                try:
+                    shutil.rmtree('build/test/cassandra/' + dirname)
+                except OSError:
+                    pass
+            # start the server
+            import subprocess as sp
+            os.chdir(root)
+            os.environ['CASSANDRA_INCLUDE'] = 'test/cassandra.in.sh'
+            args = ['bin/cassandra', '-p', pid_fname]
+            sp.Popen(args, stderr=sp.PIPE, stdout=sp.PIPE)
+            time.sleep(0.1)
+
+            # connect to it, with a timeout in case something went wrong
+            start = time.time()
+            while time.time() < start + 20:
+                try:
+                    client.transport.open()
+                except:
+                    time.sleep(0.1)
+                else:
+                    break
+            else:
+                os.kill(pid(), signal.SIGKILL) # just in case
+                print "Couldn't connect to server; aborting regression test"
+                sys.exit()
+        else:
+            client.transport.open()
+
+    def tearDown(self):
+        if self.runserver:
+            client.transport.close()
+            open('/tmp/kill', 'w').write('killing %s\n' % pid())
+            os.kill(pid(), signal.SIGTERM)
+            # TODO kill server with SIGKILL if it's still alive
+            time.sleep(0.5)
+            # TODO assert server is Truly Dead

Added: incubator/cassandra/trunk/test/system/stress.py
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/system/stress.py?rev=769608&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/system/stress.py (added)
+++ incubator/cassandra/trunk/test/system/stress.py Wed Apr 29 01:12:38 2009
@@ -0,0 +1,41 @@
+# nosetests --tests=test.stress:Stress.ten_million_inserts
+
+from hashlib import md5
+from threading import Thread
+from thread import get_ident
+import time
+
+from . import get_client, root, CassandraTester
+
+class Inserter(Thread):
+    def run(self):
+        id = get_ident()
+        self.count = 0
+        client = get_client()
+        client.transport.open()
+        for i in xrange(0, 1000):
+            data = md5(str(i)).hexdigest()
+            for j in xrange(0, 1000):
+                key = '%s.%s.%s' % (time.time(), id, j)
+                client.insert('Table1', key, 'Standard1:A', data, i)
+                client.insert('Table1', key, 'Standard1:B', data, i)
+            self.count += 1000
+
+class Stress(CassandraTester):
+    runserver = False
+
+    def ten_million_inserts(self):
+        threads = []
+        for i in xrange(0, 10):
+            th = Inserter()
+            threads.append(th)
+            th.start()
+
+        total = 0
+        while True:
+            time.sleep(1)
+            total = sum(th.count for th in threads)
+            file('/tmp/progress', 'w').write('%s\n' % str(total))
+            if not [th for th in threads if th.isAlive()]:
+                file('/tmp/progress', 'w').write('done -- %s\n' % str(total))
+                break

Added: incubator/cassandra/trunk/test/system/test_server.py
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/system/test_server.py?rev=769608&view=auto
==============================================================================
--- incubator/cassandra/trunk/test/system/test_server.py (added)
+++ incubator/cassandra/trunk/test/system/test_server.py Wed Apr 29 01:12:38 2009
@@ -0,0 +1,223 @@
+import os, sys, time
+
+from . import client, root, CassandraTester
+
+from thrift.Thrift import TApplicationException
+from ttypes import batch_mutation_t, batch_mutation_super_t, superColumn_t, column_t, NotFoundException
+
+_SIMPLE_COLUMNS = [column_t(columnName='c1', value='value1', timestamp=0),
+                   column_t(columnName='c2', value='value2', timestamp=0)]
+_SUPER_COLUMNS = [superColumn_t(name='sc1', 
+                                columns=[column_t(columnName='c4', value='value4', timestamp=0)]),
+                  superColumn_t(name='sc2', 
+                                columns=[column_t(columnName='c5', value='value5', timestamp=0),
+                                         column_t(columnName='c6', value='value6', timestamp=0)])]
+
+def _insert_simple(method=client.insert_blocking):
+    method('Table1', 'key1', 'Standard1:c1', 'value1', 0)
+    method('Table1', 'key1', 'Standard1:c2', 'value2', 0)
+
+def _insert_batch(method):
+    cfmap = {'Standard1': _SIMPLE_COLUMNS,
+             'Standard2': _SIMPLE_COLUMNS}
+    method(batch_mutation_t(table='Table1', key='key1', cfmap=cfmap))
+
+def _verify_batch():
+    _verify_simple()
+    L = client.get_slice('Table1', 'key1', 'Standard2', -1, -1)
+    assert L == _SIMPLE_COLUMNS, L
+
+def _verify_simple():
+    assert client.get_column('Table1', 'key1', 'Standard1:c1') == \
+        column_t(columnName='c1', value='value1', timestamp=0)
+    L = client.get_slice('Table1', 'key1', 'Standard1', -1, -1)
+    assert L == _SIMPLE_COLUMNS, L
+
+def _insert_super():
+    client.insert('Table1', 'key1', 'Super1:sc1:c4', 'value4', 0)
+    client.insert('Table1', 'key1', 'Super1:sc2:c5', 'value5', 0)
+    client.insert('Table1', 'key1', 'Super1:sc2:c6', 'value6', 0)
+    time.sleep(0.1)
+
+def _verify_super():
+    assert client.get_column('Table1', 'key1', 'Super1:sc1:c4') == \
+        column_t(columnName='c4', value='value4', timestamp=0)
+    slice = client.get_slice_super('Table1', 'key1', 'Super1', -1, -1)
+    assert slice == _SUPER_COLUMNS, slice
+
+def _expect_missing(fn):
+    try:
+        r = fn()
+    except NotFoundException:
+        pass
+    else:
+        raise Exception('expected missing result; got %s' % r)
+
+
+class TestMutations(CassandraTester):
+    def test_empty_slice(self):
+        assert client.get_slice('Table1', 'key1', 'Standard2', -1, -1) == []
+
+    def test_empty_slice_super(self):
+        assert client.get_slice('Table1', 'key1', 'Super1', -1, -1) == []
+
+    def test_missing_super(self):
+        _expect_missing(lambda: client.get_column('Table1', 'key1', 'Super1:sc1'))
+
+    def test_count(self):
+        assert client.get_column_count('Table1', 'key1', 'Standard2') == 0
+
+    def test_insert(self):
+        _insert_simple(client.insert)
+        time.sleep(0.1)
+        _verify_simple()
+
+    def test_insert_blocking(self):
+        _insert_simple(client.insert_blocking)
+        _verify_simple()
+
+    def test_super_insert(self):
+        _insert_super()
+        _verify_super()
+
+    def test_batch_insert(self):
+        _insert_batch(client.batch_insert)
+        time.sleep(0.1)
+        _verify_batch()
+
+    def test_batch_insert_blocking(self):
+        _insert_batch(client.batch_insert_blocking)
+        _verify_batch()
+
+    def test_cf_remove_column(self):
+        _insert_simple()
+        client.remove('Table1', 'key1', 'Standard1:c1', 1, True)
+        time.sleep(0.1)
+        _expect_missing(lambda: client.get_column('Table1', 'key1', 'Standard1:c1'))
+        assert client.get_column('Table1', 'key1', 'Standard1:c2') == \
+            column_t(columnName='c2', value='value2', timestamp=0)
+        assert client.get_slice('Table1', 'key1', 'Standard1', -1, -1) == \
+            [column_t(columnName='c2', value='value2', timestamp=0)]
+
+        # New insert, make sure it shows up post-remove:
+        client.insert('Table1', 'key1', 'Standard1:c3', 'value3', 0)
+        time.sleep(0.1)
+        assert client.get_slice('Table1', 'key1', 'Standard1', -1, -1) == \
+            [column_t(columnName='c2', value='value2', timestamp=0), 
+             column_t(columnName='c3', value='value3', timestamp=0)]
+
+        # Test resurrection.  First, re-insert the value w/ older timestamp, 
+        # and make sure it stays removed:
+        client.insert('Table1', 'key1', 'Standard1:c1', 'value1', 0)
+        time.sleep(0.1)
+        assert client.get_slice('Table1', 'key1', 'Standard1', -1, -1) == \
+            [column_t(columnName='c2', value='value2', timestamp=0), 
+             column_t(columnName='c3', value='value3', timestamp=0)]
+        # Next, w/ a newer timestamp; it should come back:
+        client.insert('Table1', 'key1', 'Standard1:c1', 'value1', 2)
+        time.sleep(0.1)
+        assert client.get_slice('Table1', 'key1', 'Standard1', -1, -1) == \
+            [column_t(columnName='c1', value='value1', timestamp=2),
+             column_t(columnName='c2', value='value2', timestamp=0), 
+             column_t(columnName='c3', value='value3', timestamp=0)]
+
+
+    def test_cf_remove(self):
+        _insert_simple()
+        _insert_super()
+
+        # Remove the key1:Standard1 cf:
+        client.remove('Table1', 'key1', 'Standard1', 3, True)
+        time.sleep(0.1)
+        assert client.get_slice('Table1', 'key1', 'Standard1', -1, -1) == []
+        _verify_super()
+
+        # Test resurrection.  First, re-insert a value w/ older timestamp, 
+        # and make sure it stays removed:
+        client.insert('Table1', 'key1', 'Standard1:c1', 'value1', 0)
+        time.sleep(0.1)
+        assert client.get_slice('Table1', 'key1', 'Standard1', -1, -1) == []
+        # Next, w/ a newer timestamp; it should come back:
+        client.insert('Table1', 'key1', 'Standard1:c1', 'value1', 4)
+        # time.sleep(0.1)
+        assert client.get_slice('Table1', 'key1', 'Standard1', -1, -1) == \
+            [column_t(columnName='c1', value='value1', timestamp=4)]
+
+
+    def test_super_cf_remove_column(self):
+        _insert_simple()
+        _insert_super()
+
+        # Make sure remove clears out what it's supposed to, and _only_ that:
+        client.remove('Table1', 'key1', 'Super1:sc2:c5', 5, True)
+        time.sleep(0.1)
+        _expect_missing(lambda: client.get_column('Table1', 'key1', 'Super1:sc2:c5'))
+        assert client.get_slice_super('Table1', 'key1', 'Super1', -1, -1) == \
+            [superColumn_t(name='sc1', 
+                           columns=[column_t(columnName='c4', value='value4', timestamp=0)]),
+             superColumn_t(name='sc2', 
+                           columns=[column_t(columnName='c6', value='value6', timestamp=0)])]
+        _verify_simple()
+
+        # New insert, make sure it shows up post-remove:
+        client.insert('Table1', 'key1', 'Super1:sc2:c7', 'value7', 0)
+        time.sleep(0.1)
+        scs = [superColumn_t(name='sc1', 
+                             columns=[column_t(columnName='c4', value='value4', timestamp=0)]),
+               superColumn_t(name='sc2', 
+                             columns=[column_t(columnName='c6', value='value6', timestamp=0),
+                                      column_t(columnName='c7', value='value7', timestamp=0)])]
+
+        assert client.get_slice_super('Table1', 'key1', 'Super1', -1, -1) == scs
+
+        # Test resurrection.  First, re-insert the value w/ older timestamp, 
+        # and make sure it stays removed:
+        client.insert('Table1', 'key1', 'Super1:sc2:c5', 'value5', 0)
+        time.sleep(0.1)
+        actual = client.get_slice_super('Table1', 'key1', 'Super1', -1, -1)
+        assert actual == scs, actual
+
+        # Next, w/ a newer timestamp; it should come back
+        client.insert('Table1', 'key1', 'Super1:sc2:c5', 'value5', 6)
+        time.sleep(0.1)
+        actual = client.get_slice_super('Table1', 'key1', 'Super1', -1, -1)
+        assert actual == \
+            [superColumn_t(name='sc1', 
+                           columns=[column_t(columnName='c4', value='value4', timestamp=0)]), 
+             superColumn_t(name='sc2', 
+                           columns=[column_t(columnName='c5', value='value5', timestamp=6), 
+                                    column_t(columnName='c6', value='value6', timestamp=0), 
+                                    column_t(columnName='c7', value='value7', timestamp=0)])], actual
+
+    def test_super_cf_remove_supercolumn(self):
+        _insert_simple()
+        _insert_super()
+
+        # Make sure remove clears out what it's supposed to, and _only_ that:
+        client.remove('Table1', 'key1', 'Super1:sc2', 5, True)
+        time.sleep(0.1)
+        _expect_missing(lambda: client.get_column('Table1', 'key1', 'Super1:sc2:c5'))
+        actual = client.get_slice_super('Table1', 'key1', 'Super1:sc2', -1, -1)
+        assert actual == [], actual
+        scs = [superColumn_t(name='sc1', 
+                             columns=[column_t(columnName='c4', value='value4', timestamp=0)])]
+        actual = client.get_slice_super('Table1', 'key1', 'Super1', -1, -1)
+        assert actual == scs, actual
+        _verify_simple()
+
+        # Test resurrection.  First, re-insert the value w/ older timestamp, 
+        # and make sure it stays removed:
+        client.insert('Table1', 'key1', 'Super1:sc2:c5', 'value5', 0)
+        time.sleep(0.1)
+        actual = client.get_slice_super('Table1', 'key1', 'Super1', -1, -1)
+        assert actual == scs, actual
+
+        # Next, w/ a newer timestamp; it should come back
+        client.insert('Table1', 'key1', 'Super1:sc2:c5', 'value5', 6)
+        time.sleep(0.1)
+        actual = client.get_slice_super('Table1', 'key1', 'Super1', -1, -1)
+        assert actual == \
+            [superColumn_t(name='sc1', 
+                           columns=[column_t(columnName='c4', value='value4', timestamp=0)]),
+             superColumn_t(name='sc2', 
+                           columns=[column_t(columnName='c5', value='value5', timestamp=6)])], actual