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

Yuki Morishita reopened CASSANDRA-3306:
---------------------------------------

      Assignee: Yuki Morishita

This error actually happens on 1.1. And I can easily reproduce with unit 
test(Test code attached).

{code}
    [junit] ERROR 17:34:46,696 Fatal exception in thread 
Thread[CompactionExecutor:3,1,main]
    [junit] java.lang.AssertionError: Expecting new size of 2, got 1 while 
replacing 
[SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-1-Data.db'),
 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-5-Data.db'),
 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-4-Data.db'),
 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-2-Data.db')]
 by 
[SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-6-Data.db')]
 in View(pending_count=0, 
sstables=[SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-1-Data.db'),
 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-2-Data.db'),
 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-4-Data.db'),
 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-4-Data.db'),
 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-5-Data.db')],
 
compacting=[SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-1-Data.db'),
 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-5-Data.db'),
 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-4-Data.db'),
 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-hf-2-Data.db')])
    [junit]     at 
org.apache.cassandra.db.DataTracker$View.newSSTables(DataTracker.java:651)
    [junit]     at 
org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:616)
    [junit]     at 
org.apache.cassandra.db.DataTracker.replace(DataTracker.java:320)
    [junit]     at 
org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:253)
    [junit]     at 
org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:994)
    [junit]     at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
    [junit]     at 
org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:154)
    [junit]     at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
    [junit]     at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
    [junit]     at 
java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
    [junit]     at java.util.concurrent.FutureTask.run(FutureTask.java:138)
    [junit]     at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
    [junit]     at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
    [junit]     at java.lang.Thread.run(Thread.java:680)
{code}

The cause is actually in streaming. StreamInSession can add duplicate reference 
to SSTable to DataTracker when it is left even after stream session finishes. 
This typically happens when source node is marked as dead by FailureDetector 
during streaming session(GC storm is the one I saw) and keep sending file in 
same session after the node comes back.
                
> Error in LeveledCompactionStrategy
> ----------------------------------
>
>                 Key: CASSANDRA-3306
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3306
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.0
>            Reporter: Radim Kolar
>            Assignee: Yuki Morishita
>
> during stress testing, i always get this error making leveledcompaction 
> strategy unusable. Should be easy to reproduce - just write fast.
> ERROR [CompactionExecutor:6] 2011-10-04 15:48:52,179 
> AbstractCassandraDaemon.java (line 133) Fatal exception in thread 
> Thread[CompactionExecutor:6,5,main]
> java.lang.AssertionError
>       at 
> org.apache.cassandra.db.DataTracker$View.newSSTables(DataTracker.java:580)
>       at 
> org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:546)
>       at org.apache.cassandra.db.DataTracker.replace(DataTracker.java:268)
>       at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:232)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:960)
>       at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:199)
>       at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:47)
>       at 
> org.apache.cassandra.db.compaction.CompactionManager$1.call(CompactionManager.java:131)
>       at 
> org.apache.cassandra.db.compaction.CompactionManager$1.call(CompactionManager.java:114)
>       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:886)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>       at java.lang.Thread.run(Thread.java:662)
> and this is in json data for table:
> {
>   "generations" : [ {
>     "generation" : 0,
>     "members" : [ 459, 460, 461, 462, 463, 464, 465, 466, 467, 468, 469, 470, 
> 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, 484 ]
>   }, {
>     "generation" : 1,
>     "members" : [ ]
>   }, {
>     "generation" : 2,
>     "members" : [ ]
>   }, {
>     "generation" : 3,
>     "members" : [ ]
>   }, {
>     "generation" : 4,
>     "members" : [ ]
>   }, {
>     "generation" : 5,
>     "members" : [ ]
>   }, {
>     "generation" : 6,
>     "members" : [ ]
>   }, {
>     "generation" : 7,
>     "members" : [ ]
>   } ]
> }

--
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