Uploaded image for project: 'Apache Cassandra'
  1. Apache Cassandra
  2. CASSANDRA-5957

Cannot drop keyspace Keyspace1 after running cassandra-stress

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Low
    • Resolution: Fixed
    • 1.2.11, 2.0.2
    • None
    • None
    • Cassandra 1.2.9 freshly built from cassandra-1.2 branch (f5b224cf9aa0f319d51078ef4b78d55e36613963)

    • Low

    Description

      Steps to reproduce:

      1. Set MAX_HEAP="2G", HEAP_NEWSIZE="400M"
      2. Run ./cassandra-stress -n 50000 -c 400 -S 256
      3. The test should complete despite several warnings about low heap memory.
      4. Try to drop keyspace:
        cqlsh> drop keyspace Keyspace1;
        TSocket read 0 bytes
        

      system.log:

       INFO 15:10:46,516 Enqueuing flush of Memtable-schema_columnfamilies@2127258371(0/0 serialized/live bytes, 1 ops)
       INFO 15:10:46,516 Writing Memtable-schema_columnfamilies@2127258371(0/0 serialized/live bytes, 1 ops)
       INFO 15:10:46,690 Completed flushing /var/lib/cassandra/data/system/schema_columnfamilies/system-schema_columnfamilies-ic-6-Data.db (38 bytes) for commitlog position ReplayPosition(segmentId=1377867520699, position=19794574)
       INFO 15:10:46,692 Enqueuing flush of Memtable-schema_columns@1997964959(0/0 serialized/live bytes, 1 ops)
       INFO 15:10:46,693 Writing Memtable-schema_columns@1997964959(0/0 serialized/live bytes, 1 ops)
       INFO 15:10:46,857 Completed flushing /var/lib/cassandra/data/system/schema_columns/system-schema_columns-ic-6-Data.db (38 bytes) for commitlog position ReplayPosition(segmentId=1377867520699, position=19794574)
       INFO 15:10:46,897 Enqueuing flush of Memtable-local@1366216652(98/98 serialized/live bytes, 3 ops)
       INFO 15:10:46,898 Writing Memtable-local@1366216652(98/98 serialized/live bytes, 3 ops)
       INFO 15:10:47,064 Completed flushing /var/lib/cassandra/data/system/local/system-local-ic-12-Data.db (139 bytes) for commitlog position ReplayPosition(segmentId=1377867520699, position=19794845)
       INFO 15:10:48,956 Enqueuing flush of Memtable-local@432522279(46/46 serialized/live bytes, 1 ops)
       INFO 15:10:48,957 Writing Memtable-local@432522279(46/46 serialized/live bytes, 1 ops)
       INFO 15:10:49,132 Compaction interrupted: Compaction@4d331c44-f018-302b-91c2-2dcf94c4bfad(Keyspace1, Standard1, 400882073/1094043713)bytes
       INFO 15:10:49,175 Compaction interrupted: Compaction@4d331c44-f018-302b-91c2-2dcf94c4bfad(Keyspace1, Standard1, 147514075/645675954)bytes
       INFO 15:10:49,185 Compaction interrupted: Compaction@4d331c44-f018-302b-91c2-2dcf94c4bfad(Keyspace1, Standard1, 223249644/609072261)bytes
       INFO 15:10:49,202 Compaction interrupted: Compaction@4d331c44-f018-302b-91c2-2dcf94c4bfad(Keyspace1, Standard1, 346471085/990388210)bytes
       INFO 15:10:49,215 Compaction interrupted: Compaction@4d331c44-f018-302b-91c2-2dcf94c4bfad(Keyspace1, Standard1, 294748503/2092376617)bytes
       INFO 15:10:49,257 Compaction interrupted: Compaction@4d331c44-f018-302b-91c2-2dcf94c4bfad(Keyspace1, Standard1, 692722235/739328646)bytes
       INFO 15:10:49,285 Completed flushing /var/lib/cassandra/data/system/local/system-local-ic-13-Data.db (82 bytes) for commitlog position ReplayPosition(segmentId=1377867520699, position=19794974)
       INFO 15:10:49,286 Compacting [SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-ic-10-Data.db'), SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-ic-13-Data.db'), SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-ic-12-Data.db'), SSTableReader(path='/var/lib/cassandra/data/system/local/system-local-ic-11-Data.db')]
      ERROR 15:10:49,287 Error occurred during processing of message.
      java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.lang.AssertionError: SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-ic-78-Data.db') was already marked compacted
      	at org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:378)
      	at org.apache.cassandra.service.MigrationManager.announce(MigrationManager.java:281)
      	at org.apache.cassandra.service.MigrationManager.announceKeyspaceDrop(MigrationManager.java:262)
      	at org.apache.cassandra.cql.QueryProcessor.processStatement(QueryProcessor.java:718)
      	at org.apache.cassandra.cql.QueryProcessor.process(QueryProcessor.java:775)
      	at org.apache.cassandra.thrift.CassandraServer.execute_cql_query(CassandraServer.java:1668)
      	at org.apache.cassandra.thrift.Cassandra$Processor$execute_cql_query.getResult(Cassandra.java:4048)
      	at org.apache.cassandra.thrift.Cassandra$Processor$execute_cql_query.getResult(Cassandra.java:4036)
      	at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:32)
      	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:34)
      	at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:199)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
      	at java.lang.Thread.run(Thread.java:662)
      Caused by: java.util.concurrent.ExecutionException: java.lang.AssertionError: SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-ic-78-Data.db') was already marked compacted
      	at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
      	at java.util.concurrent.FutureTask.get(FutureTask.java:83)
      	at org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:374)
      	... 13 more
      Caused by: java.lang.AssertionError: SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-ic-78-Data.db') was already marked compacted
      	at org.apache.cassandra.db.DataTracker.removeOldSSTablesSize(DataTracker.java:354)
      	at org.apache.cassandra.db.DataTracker.postReplace(DataTracker.java:325)
      	at org.apache.cassandra.db.DataTracker.unreferenceSSTables(DataTracker.java:264)
      	at org.apache.cassandra.db.ColumnFamilyStore.invalidate(ColumnFamilyStore.java:302)
      	at org.apache.cassandra.db.Table.unloadCf(Table.java:314)
      	at org.apache.cassandra.db.Table.dropCf(Table.java:296)
      	at org.apache.cassandra.db.DefsTable.dropColumnFamily(DefsTable.java:607)
      	at org.apache.cassandra.db.DefsTable.mergeColumnFamilies(DefsTable.java:469)
      	at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:355)
      	at org.apache.cassandra.service.MigrationManager$2.runMayThrow(MigrationManager.java:299)
      	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)
      	at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:138)
      	... 3 more
      ERROR 15:10:49,287 Exception in thread Thread[MigrationStage:1,5,main]
      java.lang.AssertionError: SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-ic-78-Data.db') was already marked compacted
      	at org.apache.cassandra.db.DataTracker.removeOldSSTablesSize(DataTracker.java:354)
      	at org.apache.cassandra.db.DataTracker.postReplace(DataTracker.java:325)
      	at org.apache.cassandra.db.DataTracker.unreferenceSSTables(DataTracker.java:264)
      	at org.apache.cassandra.db.ColumnFamilyStore.invalidate(ColumnFamilyStore.java:302)
      	at org.apache.cassandra.db.Table.unloadCf(Table.java:314)
      	at org.apache.cassandra.db.Table.dropCf(Table.java:296)
      	at org.apache.cassandra.db.DefsTable.dropColumnFamily(DefsTable.java:607)
      	at org.apache.cassandra.db.DefsTable.mergeColumnFamilies(DefsTable.java:469)
      	at org.apache.cassandra.db.DefsTable.mergeSchema(DefsTable.java:355)
      	at org.apache.cassandra.service.MigrationManager$2.runMayThrow(MigrationManager.java:299)
      	at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
      	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)
      	at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:138)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
      	at java.lang.Thread.run(Thread.java:662)
       INFO 15:10:49,471 Compacted 4 sstables to [/var/lib/cassandra/data/system/local/system-local-ic-14,].  829 bytes to 501 (~60% of original) in 184ms = 0,002597MB/s.  4 total rows, 1 unique.  Row merge counts were {1:0, 2:0, 3:0, 4:1, }
      

      Attachments

        1. system.log
          690 kB
          Piotr Kolaczkowski
        2. 5957-1.2-v1.patch
          7 kB
          Tom Hobbs
        3. 5957-1.2-v2.patch
          4 kB
          Tom Hobbs

        Activity

          People

            thobbs Tom Hobbs
            pkolaczk Piotr Kolaczkowski
            Tom Hobbs
            Jonathan Ellis
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: