Tyler Patterson created CASSANDRA-4221:
------------------------------------------

             Summary: Error while deleting a columnfamily that is being 
compacted.
                 Key: CASSANDRA-4221
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4221
             Project: Cassandra
          Issue Type: Bug
          Components: Core
         Environment: ccm, dtest, cassandra-1.1. The error does not happen in 
cassandra-1.0.
            Reporter: Tyler Patterson


The following dtest command produces an error:
{code}export CASSANDRA_VERSION=git:cassandra-1.1; nosetests --nocapture 
--nologcapture 
concurrent_schema_changes_test.py:TestConcurrentSchemaChanges.load_test{code}

Here is the error:
{code}
Error occured during compaction
java.util.concurrent.ExecutionException: java.io.IOError: 
java.io.FileNotFoundException: 
/tmp/dtest-6ECMgy/test/node1/data/Keyspace1/Standard1/Keyspace1-Standard1-hc-47-Data.db
 (No such file or directory)
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:252)
        at java.util.concurrent.FutureTask.get(FutureTask.java:111)
        at 
org.apache.cassandra.db.compaction.CompactionManager.performMaximal(CompactionManager.java:239)
        at 
org.apache.cassandra.db.ColumnFamilyStore.forceMajorCompaction(ColumnFamilyStore.java:1580)
        at 
org.apache.cassandra.service.StorageService.forceTableCompaction(StorageService.java:1770)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:616)
        at 
com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:111)
        at 
com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:45)
        at 
com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:226)
        at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138)
        at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:251)
        at 
com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:857)
        at 
com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:795)
        at 
javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1450)
        at 
javax.management.remote.rmi.RMIConnectionImpl.access$200(RMIConnectionImpl.java:90)
        at 
javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1285)
        at 
javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1383)
        at 
javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:807)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:616)
        at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322)
        at sun.rmi.transport.Transport$1.run(Transport.java:177)
        at java.security.AccessController.doPrivileged(Native Method)
        at sun.rmi.transport.Transport.serviceCall(Transport.java:173)
        at 
sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:553)
        at 
sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:808)
        at 
sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:667)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:679)
Caused by: java.io.IOError: java.io.FileNotFoundException: 
/tmp/dtest-6ECMgy/test/node1/data/Keyspace1/Standard1/Keyspace1-Standard1-hc-47-Data.db
 (No such file or directory)
        at 
org.apache.cassandra.io.sstable.SSTableScanner.<init>(SSTableScanner.java:61)
        at 
org.apache.cassandra.io.sstable.SSTableReader.getDirectScanner(SSTableReader.java:839)
        at 
org.apache.cassandra.io.sstable.SSTableReader.getDirectScanner(SSTableReader.java:851)
        at 
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:142)
        at 
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:148)
        at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:121)
        at 
org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:264)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
        at java.util.concurrent.FutureTask.run(FutureTask.java:166)
        ... 3 more
Caused by: java.io.FileNotFoundException: 
/tmp/dtest-6ECMgy/test/node1/data/Keyspace1/Standard1/Keyspace1-Standard1-hc-47-Data.db
 (No such file or directory)
        at java.io.RandomAccessFile.open(Native Method)
        at java.io.RandomAccessFile.<init>(RandomAccessFile.java:233)
        at 
org.apache.cassandra.io.util.RandomAccessReader.<init>(RandomAccessReader.java:67)
        at 
org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:102)
        at 
org.apache.cassandra.io.util.RandomAccessReader.open(RandomAccessReader.java:87)
        at 
org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:985)
        at 
org.apache.cassandra.io.sstable.SSTableScanner.<init>(SSTableScanner.java:56)
        ... 13 more
{code}

For reference, here is the dtest function that causes the failure:
{code}
    def load_test(self):                                                        
        """                                                                     
        apply schema changes while the cluster is under load.                   
        """                                                                     
        debug("load_test()")                                                    
                                                                                
        cluster = self.cluster                                                  
        cluster.populate(1).start()                                             
        node1 = cluster.nodelist()[0]                                           
        wait(2)                                                                 
        cursor = self.cql_connection(node1).cursor()                            
                                                                                
        def stress(args=[]):                                                    
            debug("Stressing")                                                  
            node1.stress(args)                                                  
            debug("Done Stressing")                                             
                                                                                
        def compact():                                                          
            debug("Compacting...")                                              
            node1.nodetool('compact')                                           
            debug("Done Compacting.")                                           
                                                                                
        # put some data into the cluster                                        
        stress(['--num-keys=1000000'])                                          
                                                                                
        # now start compacting...                   
        tcompact = Thread(target=compact)                                       
        tcompact.start()                                                        
        wait(1)                                                                 
                                                                                
        # now the cluster is under a lot of load. Make some schema changes.     
        cursor.execute("USE Keyspace1")                                         
        wait(1)                                                                 
        cursor.execute("DROP COLUMNFAMILY Standard1")                           
                                                                                
        wait(3)                                                                 
                                                                                
        cursor.execute("CREATE COLUMNFAMILY Standard1 (KEY text PRIMARY KEY)")  
                                                                                
        tcompact.join()                                                         
 
{code}
Again, the error happens on cassandra-1.1, but not on cassandra-1.0.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to