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

Tobias Schlottke commented on CASSANDRA-8851:
---------------------------------------------

With 2.1 branch it looks like this:

{code}
INFO  [CompactionExecutor:3] 2015-02-24 09:24:20,187 ColumnFamilyStore.java:877 
- Enqueuing flush of compactions_in_progress: 228 (0%) on-heap, 0 (0%) off-heap
INFO  [MemtableFlushWriter:8] 2015-02-24 09:24:20,189 Memtable.java:339 - 
Writing Memtable-compactions_in_progress@981052240(0 serialized bytes, 1 ops, 
0%/0% of on/off-heap limit)
INFO  [MemtableFlushWriter:8] 2015-02-24 09:24:20,196 Memtable.java:378 - 
Completed flushing 
/data/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-ka-1699-Data.db
 (42 bytes) for commitlog position ReplayPosition(segmentId=1424765925698, 
position=66884939)
ERROR [CompactionExecutor:3] 2015-02-24 09:24:20,204 CassandraDaemon.java:167 - 
Exception in thread Thread[CompactionExecutor:3,1,main]
java.lang.AssertionError: null
        at 
org.apache.cassandra.io.compress.CompressionMetadata$Writer.open(CompressionMetadata.java:361)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.io.compress.CompressedSequentialWriter.open(CompressedSequentialWriter.java:155)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.io.util.CompressedSegmentedFile$Builder.metadata(CompressedSegmentedFile.java:81)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Builder.complete(CompressedPoolingSegmentedFile.java:72)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.io.util.SegmentedFile$Builder.complete(SegmentedFile.java:172)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.io.sstable.SSTableWriter.openEarly(SSTableWriter.java:402) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.io.sstable.SSTableRewriter.maybeReopenEarly(SSTableRewriter.java:165)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:110)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:192)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:72)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:240)
 ~[apache-cassandra-2.1.3.jar:2.1.3]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_45]
        at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_45]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
~[na:1.7.0_45]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_45]
        at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
INFO  [CompactionExecutor:3] 2015-02-24 09:24:20,293 CompactionTask.java:140 - 
Compacting 
[SSTableReader(path='/data/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-ka-1698-Data.db'),
 
SSTableReader(path='/data/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-ka-1697-Data.db'),
 
SSTableReader(path='/data/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-ka-1696-Data.db'),
 
SSTableReader(path='/data/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-ka-1699-Data.db')]
INFO  [CompactionExecutor:3] 2015-02-24 09:24:20,306 CompactionTask.java:267 - 
Compacted 4 sstables to 
[/data/cassandra/data/system/compactions_in_progress/system-compactions_in_progress-ka-1700,].
  971 bytes to 606 (~62% of original) in 11ms = 0,052539MB/s.  3 total 
partitions merged to 2.  Partition merge counts were {1:1, 2:1, }
ERROR [Reference-Reaper:1] 2015-02-24 09:24:21,517 Ref.java:177 - LEAK 
DETECTED: a reference 
(org.apache.cassandra.utils.concurrent.Ref$State@7d15adc3) to class 
org.apache.cassandra.io.util.MmappedSegmentedFile$Cleanup@440606045:/data/cassandra/data/metrigo_prod/new_user_data/metrigo_prod-new_user_data-tmplink-ka-257734-Index.db
 was not released before the reference was garbage collected
{code}



> Uncaught exception on thread Thread[SharedPool-Worker-16,5,main] after 
> upgrade to 2.1.3
> ---------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-8851
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8851
>             Project: Cassandra
>          Issue Type: Bug
>         Environment: ubuntu 
>            Reporter: Tobias Schlottke
>            Assignee: Benedict
>            Priority: Critical
>
> Hi there,
> after upgrading to 2.1.3 we've got the following error every few seconds:
> {code}
> WARN  [SharedPool-Worker-16] 2015-02-23 10:20:36,392 
> AbstractTracingAwareExecutorService.java:169 - Uncaught exception on thread 
> Thread[SharedPool-Worker-16,5,main]: {}
> java.lang.AssertionError: null
>       at org.apache.cassandra.io.util.Memory.size(Memory.java:307) 
> ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.utils.obs.OffHeapBitSet.capacity(OffHeapBitSet.java:61) 
> ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at org.apache.cassandra.utils.BloomFilter.indexes(BloomFilter.java:74) 
> ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.utils.BloomFilter.isPresent(BloomFilter.java:98) 
> ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:1366)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:1350)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:41)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:185)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:273)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:62)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1915)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1748)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:342) 
> ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:57)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:47) 
> ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:62) 
> ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
> ~[na:1.7.0_45]
>       at 
> org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164)
>  ~[apache-cassandra-2.1.3.jar:2.1.3]
>       at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
> [apache-cassandra-2.1.3.jar:2.1.3]
>       at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> {code}
> This seems to crash the compactions and pushes up server load and piles up 
> compactions.
> Any idea / possible workaround?
> Best,
> Tobias



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to