You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Tyler Hobbs (JIRA)" <ji...@apache.org> on 2013/10/10 00:10:42 UTC

[jira] [Updated] (CASSANDRA-5957) Cannot drop keyspace Keyspace1 after running cassandra-stress

     [ https://issues.apache.org/jira/browse/CASSANDRA-5957?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Tyler Hobbs updated CASSANDRA-5957:
-----------------------------------

    Attachment: 5957-1.2-v1.patch

Quick summary of the problem: when the table is dropped, there are multiple ongoing compactions.  When handling the table drop, DataTracker first builds a view of non-compacting sstables.  Meanwhile, the compaction tasks get cancelled, and as part of the cleanup, it marks the sstables as compacted (because the column family is no longer valid).  Then the DataTracker finally tries to mark the sstables in its view as compacted, hitting the failing assertion.  Additionally, when a compaction task is cancelled, it doesn't unreference the sstables involved, and since the DataTracker ignored compacting sstables, they never get unreferenced (even if the failing assertion is disabled).

5957-1.2-v1.patch (and [branch|https://github.com/thobbs/cassandra/tree/CASSANDRA-5957]) uses the simplest fix I could think of, which is to block the table drop operation until all compactions on that table have stopped (either normally, because of an error, or because they saw the stop signal).  This way, when the DataTracker cleans up sstables as part of the drop operation, there are no compacting sstables.

> Cannot drop keyspace Keyspace1 after running cassandra-stress
> -------------------------------------------------------------
>
>                 Key: CASSANDRA-5957
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-5957
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Cassandra 1.2.9 freshly built from cassandra-1.2 branch (f5b224cf9aa0f319d51078ef4b78d55e36613963)
>            Reporter: Piotr Kołaczkowski
>            Assignee: Tyler Hobbs
>            Priority: Minor
>             Fix For: 1.2.11
>
>         Attachments: 5957-1.2-v1.patch, system.log
>
>
> Steps to reproduce:
> # Set MAX_HEAP="2G", HEAP_NEWSIZE="400M"
> # Run ./cassandra-stress -n 50000 -c 400 -S 256
> # The test should complete despite several warnings about low heap memory.
> # Try to drop keyspace:
> {noformat}
> cqlsh> drop keyspace Keyspace1;
> TSocket read 0 bytes
> {noformat}
> system.log:
> {noformat}
>  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, }
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.1#6144)