[ 
https://issues.apache.org/jira/browse/CASSANDRA-4687?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13701709#comment-13701709
 ] 

Andrew Bialecki commented on CASSANDRA-4687:
--------------------------------------------

Just hit this while doing some load testing. I can provide a bit more 
information about what we were doing and how we created it:

Running Cassandra 1.2.5 on Ubuntu 12.04 with the default config file. 3 node 
cluster with the following Keyspace/CF:

    CREATE KEYSPACE TestKeyspace1 WITH placement_strategy='SimpleStrategy'
        AND strategy_options = { replication_factor: 3};

    USE TestKeyspace1;

    CREATE COLUMN FAMILY TestCounters1 WITH
        default_validation_class=CounterColumnType AND 
key_validation_class=UTF8Type
        AND comparator=UTF8Type AND replicate_on_write=true
        AND compaction_strategy=LeveledCompactionStrategy AND 
compaction_strategy_options={sstable_size_in_mb: 5};

Used our equivalent of the load utility to use 40 threads to increment counters 
across 1M keys with up to 100 columns randomly. Peak load with this 
configuration was about ~9.5k writes/sec, no reads. Although reads were being 
done as a read is part of a counter write.

We had been playing with different setting to benchmark and to clear the 
keyspace/CF, we were running:

    DROP KEYSPACE TestKeyspace1;

and then re-creating it with the script above. After a few drops/creates I 
noticed the SSTable size didn't appear to adhere to the sstable size 
(potentially another bug?). No errors in logs, but after a few minutes of 
writes, we saw no sstables written, so we forced a flush with "nodetool flush." 
Upon doing so on one node, we ended up with a single sstable that was a 60M 
file.

Right after the flush, we got a lot of errors in the log like:

java.lang.AssertionError: DecoratedKey(151132845099071093774085323081542634448, 
636f6c756d6e34) != DecoratedKey(52766578155366297747310347893720908288, 
726f77353035333134) in 
/var/lib/cassandra/data/TestKeyspace1/TestCounters1/TestKeyspace1-TestCounters1-ic-1-Data.db
        at 
org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:119)
        at 
org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init>(SSTableNamesIterator.java:60)
        at 
org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:81)
        at 
org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:68)
        at 
org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:274)
        at 
org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:65)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1357)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1214)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1126)
        at org.apache.cassandra.db.Table.getRow(Table.java:347)
        at 
org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.java:64)
        at 
org.apache.cassandra.db.CounterMutation.makeReplicationMutation(CounterMutation.java:90)
        at 
org.apache.cassandra.service.StorageProxy$7$1.runMayThrow(StorageProxy.java:796)
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1578)
        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)

By the way, we changed how we dropped/recreated the keyspace from dropping the 
keyspace and running:

    nodetool clearsnapshot
    nodetool cleanup

to dropping the keyspace and then running:

    rm -rf /var/lib/cassandra/data/TestKeyspace1
    nodetool invalidatekeycache

and now we're seeing the sstable size respected. Not sure if that's a bug, but 
I can take the relevant bits from above and open a ticket if that's useful. Or 
if it's better, when we have some free time we can try to simplify the steps to 
reproduce.

                
> Exception: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk)
> -----------------------------------------------------------
>
>                 Key: CASSANDRA-4687
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4687
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: CentOS 6.3 64-bit, Oracle JRE 1.6.0.33 64-bit, single 
> node cluster
>            Reporter: Leonid Shalupov
>            Priority: Minor
>         Attachments: 4687-debugging.txt
>
>
> Under heavy write load sometimes cassandra fails with assertion error.
> git bisect leads to commit 295aedb278e7a495213241b66bc46d763fd4ce66.
> works fine if global key/row caches disabled in code.
> {quote}
> java.lang.AssertionError: DecoratedKey(xxx, yyy) != DecoratedKey(zzz, kkk) in 
> /var/lib/cassandra/data/...-he-1-Data.db
>       at 
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:60)
>       at 
> org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67)
>       at 
> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79)
>       at 
> org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256)
>       at 
> org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1345)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1207)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1142)
>       at org.apache.cassandra.db.Table.getRow(Table.java:378)
>       at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69)
>       at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:819)
>       at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1253)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>       at java.lang.Thread.run(Thread.java:662)
> {quote}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to