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

Vijay edited comment on CASSANDRA-4934 at 11/9/12 6:43 PM:
-----------------------------------------------------------

+1 (ignore my first comment getOverlappingSSTables will return a diffrent set).
Honestly, i did think about this somehow missed the overlapping SST code path 
earlier :)
                
      was (Author: vijay2...@yahoo.com):
    Hi Jonathan, If the above senario happens again then the code

{code}
        while (true)
        {
            Set<SSTableReader> overlapped = getOverlappingSSTables(sstables);
            if (SSTableReader.acquireReferences(overlapped))
                return overlapped;
        }
{code}

Wont we run into a spin lock because the aquireReference code will always 
return false always? 
Will it be better to just cancel the compaction and let the next catchup?

Honestly, i did think about this somehow missed the overlapping SST code path 
earlier :)
                  
> assertion error in offheap bloom filter
> ---------------------------------------
>
>                 Key: CASSANDRA-4934
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-4934
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.2.0 beta 2
>            Reporter: Brandon Williams
>            Assignee: Jonathan Ellis
>             Fix For: 1.2.0 rc1
>
>         Attachments: log.txt
>
>
> Saw this while running the dtests:
> {noformat}
>  INFO [CompactionExecutor:2] 2012-11-08 09:35:18,206 CompactionTask.java 
> (line 116) Compacting 
> [SSTableReader(path='/tmp/dtest-n2D_fM/test/node2/data/system/local/system-local-ia-9-Data.db'),
>  
> SSTableReader(path='/tmp/dtest-n2D_fM/test/node2/data/system/local/system-local-ia-11-Data.db'),
>  
> SSTableReader(path='/tmp/dtest-n2D_fM/test/node2/data/system/local/system-local-ia-10-Data.db'),
>  
> SSTableReader(path='/tmp/dtest-n2D_fM/test/node2/data/system/local/system-local-ia-13-Data.db')]
> ERROR [CompactionExecutor:2] 2012-11-08 09:35:18,257 CassandraDaemon.java 
> (line 132) Exception in thread Thread[CompactionExecutor:2,1,main]
> java.lang.AssertionError: Memory was freed
>     at org.apache.cassandra.io.util.Memory.checkPosition(Memory.java:134)
>     at org.apache.cassandra.io.util.Memory.getByte(Memory.java:104)
>     at org.apache.cassandra.utils.obs.OffHeapBitSet.get(OffHeapBitSet.java:60)
>     at org.apache.cassandra.utils.BloomFilter.isPresent(BloomFilter.java:71)
>     at 
> org.apache.cassandra.db.compaction.CompactionController.shouldPurge(CompactionController.java:106)
>     at 
> org.apache.cassandra.db.compaction.PrecompactedRow.removeDeletedAndOldShards(PrecompactedRow.java:64)
>     at 
> org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:95)
>     at 
> org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:151)
>     at 
> org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:72)
>     at 
> org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:57)
>     at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:114)
>     at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:97)
>     at 
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>     at 
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>     at com.google.common.collect.Iterators$8.computeNext(Iterators.java:734)
>     at 
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>     at 
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>     at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:146)
>     at 
> org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
>     at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>     at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:69)
>     at 
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:179)
>     at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
>     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)
> {noformat}

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