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

Roland Otta edited comment on CASSANDRA-13445 at 4/13/17 9:36 PM:
------------------------------------------------------------------

btw: the issue also occurs when trying to scrub the sstable via nodetool scrub

after that i have 2 hanging compactions at the same stage

{noformat}
id                                   compaction type keyspace table    
completed total     unit  progress
2505fe00-207d-11e7-ad57-c9e86a8710f5 Validation      bds      ad_event 
805955242 841258085 bytes 95.80%  
6c6654a0-208e-11e7-ad57-c9e86a8710f5 Scrub           bds      ad_event 
805961728 841258085 bytes 95.80% 
{noformat}

also the stack trace looks quite similar

{noformat}
com.github.benmanes.caffeine.cache.BoundedLocalCache$$Lambda$65/60401277.accept(Unknown
 Source)
com.github.benmanes.caffeine.cache.BoundedBuffer$RingBuffer.drainTo(BoundedBuffer.java:104)
com.github.benmanes.caffeine.cache.StripedBuffer.drainTo(StripedBuffer.java:160)
com.github.benmanes.caffeine.cache.BoundedLocalCache.drainReadBuffer(BoundedLocalCache.java:964)
com.github.benmanes.caffeine.cache.BoundedLocalCache.maintenance(BoundedLocalCache.java:918)
com.github.benmanes.caffeine.cache.BoundedLocalCache.performCleanUp(BoundedLocalCache.java:903)
com.github.benmanes.caffeine.cache.BoundedLocalCache$PerformCleanupTask.run(BoundedLocalCache.java:2680)
com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:457)
com.github.benmanes.caffeine.cache.BoundedLocalCache.scheduleDrainBuffers(BoundedLocalCache.java:875)
com.github.benmanes.caffeine.cache.BoundedLocalCache.afterRead(BoundedLocalCache.java:748)
com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:1783)
com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:97)
com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:66)
org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:235)
org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:213)
org.apache.cassandra.io.util.LimitingRebufferer.rebuffer(LimitingRebufferer.java:54)
org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:65)
org.apache.cassandra.io.util.RandomAccessReader.reBuffer(RandomAccessReader.java:59)
org.apache.cassandra.io.util.RebufferingInputStream.read(RebufferingInputStream.java:88)
org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:66)
org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402)
org.apache.cassandra.db.marshal.AbstractType.readValue(AbstractType.java:420)
org.apache.cassandra.db.rows.Cell$Serializer.deserialize(Cell.java:245)
org.apache.cassandra.db.rows.UnfilteredSerializer.readSimpleColumn(UnfilteredSerializer.java:610)
org.apache.cassandra.db.rows.UnfilteredSerializer.lambda$deserializeRowBody$1(UnfilteredSerializer.java:575)
org.apache.cassandra.db.rows.UnfilteredSerializer$$Lambda$85/168219100.accept(Unknown
 Source)
org.apache.cassandra.utils.btree.BTree.applyForwards(BTree.java:1222)
org.apache.cassandra.utils.btree.BTree.apply(BTree.java:1177)
org.apache.cassandra.db.Columns.apply(Columns.java:377)
org.apache.cassandra.db.rows.UnfilteredSerializer.deserializeRowBody(UnfilteredSerializer.java:571)
org.apache.cassandra.db.rows.UnfilteredSerializer.deserialize(UnfilteredSerializer.java:440)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:95)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:73)
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:122)
org.apache.cassandra.db.compaction.Scrubber$RowMergingSSTableIterator.next(Scrubber.java:503)
org.apache.cassandra.db.compaction.Scrubber$RowMergingSSTableIterator.next(Scrubber.java:481)
org.apache.cassandra.db.compaction.Scrubber$OrderCheckerIterator.computeNext(Scrubber.java:609)
org.apache.cassandra.db.compaction.Scrubber$OrderCheckerIterator.computeNext(Scrubber.java:526)
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:133)
org.apache.cassandra.db.ColumnIndex.buildRowIndex(ColumnIndex.java:110)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.format.big.BigTableWriter.append(BigTableWriter.java:173)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.SSTableRewriter.append(SSTableRewriter.java:135)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.SSTableRewriter.tryAppend(SSTableRewriter.java:156)
org.apache.cassandra.db.compaction.Scrubber.tryAppend(Scrubber.java:319)
org.apache.cassandra.db.compaction.Scrubber.scrub(Scrubber.java:214)
org.apache.cassandra.db.compaction.CompactionManager.scrubOne(CompactionManager.java:966)
org.apache.cassandra.db.compaction.CompactionManager.access$300(CompactionManager.java:85)
org.apache.cassandra.db.compaction.CompactionManager$3.execute(CompactionManager.java:368)
org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:311)
java.util.concurrent.FutureTask.run(FutureTask.java:266)
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
java.util.concurrent.FutureTask.run(FutureTask.java:266)
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$5/899929247.run(Unknown
 Source)
java.lang.Thread.run(Thread.java:745)
{noformat}

if it would be helpful i could provide the sstable, that leads to that hanging 
validation/scrub

from what i have seen when creating several thread dumps over some minutes
com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:66)
calls
org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:235)
in an infinite loop


was (Author: whrootta):
btw: the issue also occurs when trying to scrub the sstable via nodetool scrub

after that i have 2 hanging compactions at the same stage

{noformat}
id                                   compaction type keyspace table    
completed total     unit  progress
2505fe00-207d-11e7-ad57-c9e86a8710f5 Validation      bds      ad_event 
805955242 841258085 bytes 95.80%  
6c6654a0-208e-11e7-ad57-c9e86a8710f5 Scrub           bds      ad_event 
805961728 841258085 bytes 95.80% 
{noformat}

also the stack trace looks quite similar

{noformat}
com.github.benmanes.caffeine.cache.BoundedLocalCache$$Lambda$65/60401277.accept(Unknown
 Source)
com.github.benmanes.caffeine.cache.BoundedBuffer$RingBuffer.drainTo(BoundedBuffer.java:104)
com.github.benmanes.caffeine.cache.StripedBuffer.drainTo(StripedBuffer.java:160)
com.github.benmanes.caffeine.cache.BoundedLocalCache.drainReadBuffer(BoundedLocalCache.java:964)
com.github.benmanes.caffeine.cache.BoundedLocalCache.maintenance(BoundedLocalCache.java:918)
com.github.benmanes.caffeine.cache.BoundedLocalCache.performCleanUp(BoundedLocalCache.java:903)
com.github.benmanes.caffeine.cache.BoundedLocalCache$PerformCleanupTask.run(BoundedLocalCache.java:2680)
com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:457)
com.github.benmanes.caffeine.cache.BoundedLocalCache.scheduleDrainBuffers(BoundedLocalCache.java:875)
com.github.benmanes.caffeine.cache.BoundedLocalCache.afterRead(BoundedLocalCache.java:748)
com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:1783)
com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:97)
com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:66)
org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:235)
org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:213)
org.apache.cassandra.io.util.LimitingRebufferer.rebuffer(LimitingRebufferer.java:54)
org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:65)
org.apache.cassandra.io.util.RandomAccessReader.reBuffer(RandomAccessReader.java:59)
org.apache.cassandra.io.util.RebufferingInputStream.read(RebufferingInputStream.java:88)
org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:66)
org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402)
org.apache.cassandra.db.marshal.AbstractType.readValue(AbstractType.java:420)
org.apache.cassandra.db.rows.Cell$Serializer.deserialize(Cell.java:245)
org.apache.cassandra.db.rows.UnfilteredSerializer.readSimpleColumn(UnfilteredSerializer.java:610)
org.apache.cassandra.db.rows.UnfilteredSerializer.lambda$deserializeRowBody$1(UnfilteredSerializer.java:575)
org.apache.cassandra.db.rows.UnfilteredSerializer$$Lambda$85/168219100.accept(Unknown
 Source)
org.apache.cassandra.utils.btree.BTree.applyForwards(BTree.java:1222)
org.apache.cassandra.utils.btree.BTree.apply(BTree.java:1177)
org.apache.cassandra.db.Columns.apply(Columns.java:377)
org.apache.cassandra.db.rows.UnfilteredSerializer.deserializeRowBody(UnfilteredSerializer.java:571)
org.apache.cassandra.db.rows.UnfilteredSerializer.deserialize(UnfilteredSerializer.java:440)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:95)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:73)
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:122)
org.apache.cassandra.db.compaction.Scrubber$RowMergingSSTableIterator.next(Scrubber.java:503)
org.apache.cassandra.db.compaction.Scrubber$RowMergingSSTableIterator.next(Scrubber.java:481)
org.apache.cassandra.db.compaction.Scrubber$OrderCheckerIterator.computeNext(Scrubber.java:609)
org.apache.cassandra.db.compaction.Scrubber$OrderCheckerIterator.computeNext(Scrubber.java:526)
org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:133)
org.apache.cassandra.db.ColumnIndex.buildRowIndex(ColumnIndex.java:110)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.format.big.BigTableWriter.append(BigTableWriter.java:173)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.SSTableRewriter.append(SSTableRewriter.java:135)
org.apache.cassandra.io<http://org.apache.cassandra.io>.sstable.SSTableRewriter.tryAppend(SSTableRewriter.java:156)
org.apache.cassandra.db.compaction.Scrubber.tryAppend(Scrubber.java:319)
org.apache.cassandra.db.compaction.Scrubber.scrub(Scrubber.java:214)
org.apache.cassandra.db.compaction.CompactionManager.scrubOne(CompactionManager.java:966)
org.apache.cassandra.db.compaction.CompactionManager.access$300(CompactionManager.java:85)
org.apache.cassandra.db.compaction.CompactionManager$3.execute(CompactionManager.java:368)
org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:311)
java.util.concurrent.FutureTask.run(FutureTask.java:266)
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
java.util.concurrent.FutureTask.run(FutureTask.java:266)
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$5/899929247.run(Unknown
 Source)
java.lang.Thread.run(Thread.java:745)
{noformat}

if it would be helpful i could provide the sstable, that leads to that hanging 
validation/scrub

> validation executor thread is stuck
> -----------------------------------
>
>                 Key: CASSANDRA-13445
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13445
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Compaction
>         Environment: cassandra 3.10
>            Reporter: Roland Otta
>
> we have the following issue on our 3.10 development cluster.
> sometimes the repairs (it is a full repair in that case) hang because
> of a stuck validation compaction.
> nodetool compactionstats says 
> a1bb45c0-1fc6-11e7-81de-0fb0b3f5a345 Validation      bds      ad_event
> 805955242 841258085 bytes 95.80% 
> and there is no more progress at this percentage.
> i checked the logs on the affected node and could not find any
> suspicious errors.
> a thread dump shows that the validation executor threads is always repeating 
> stuff in 
> org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:235)
> here is the full stack trace
> {noformat}
> com.github.benmanes.caffeine.cache.BoundedLocalCache$$Lambda$64/2098345091.accept(Unknown
>  Source)
> com.github.benmanes.caffeine.cache.BoundedBuffer$RingBuffer.drainTo(BoundedBuffer.java:104)
> com.github.benmanes.caffeine.cache.StripedBuffer.drainTo(StripedBuffer.java:160)
> com.github.benmanes.caffeine.cache.BoundedLocalCache.drainReadBuffer(BoundedLocalCache.java:964)
> com.github.benmanes.caffeine.cache.BoundedLocalCache.maintenance(BoundedLocalCache.java:918)
> com.github.benmanes.caffeine.cache.BoundedLocalCache.performCleanUp(BoundedLocalCache.java:903)
> com.github.benmanes.caffeine.cache.BoundedLocalCache$PerformCleanupTask.run(BoundedLocalCache.java:2680)
> com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:457)
> com.github.benmanes.caffeine.cache.BoundedLocalCache.scheduleDrainBuffers(BoundedLocalCache.java:875)
> com.github.benmanes.caffeine.cache.BoundedLocalCache.afterRead(BoundedLocalCache.java:748)
> com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:1783)
> com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:97)
> com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:66)
> org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:235)
> org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:213)
> org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:65)
> org.apache.cassandra.io.util.RandomAccessReader.reBuffer(RandomAccessReader.java:59)
> org.apache.cassandra.io.util.RebufferingInputStream.read(RebufferingInputStream.java:88)
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:66)
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402)
> org.apache.cassandra.db.marshal.AbstractType.readValue(AbstractType.java:420)
> org.apache.cassandra.db.rows.Cell$Serializer.deserialize(Cell.java:245)
> org.apache.cassandra.db.rows.UnfilteredSerializer.readSimpleColumn(UnfilteredSerializer.java:610)
> org.apache.cassandra.db.rows.UnfilteredSerializer.lambda$deserializeRowBody$1(UnfilteredSerializer.java:575)
> org.apache.cassandra.db.rows.UnfilteredSerializer$$Lambda$84/898489541.accept(Unknown
>  Source)
> org.apache.cassandra.utils.btree.BTree.applyForwards(BTree.java:1222)
> org.apache.cassandra.utils.btree.BTree.apply(BTree.java:1177)
> org.apache.cassandra.db.Columns.apply(Columns.java:377)
> org.apache.cassandra.db.rows.UnfilteredSerializer.deserializeRowBody(UnfilteredSerializer.java:571)
> org.apache.cassandra.db.rows.UnfilteredSerializer.deserialize(UnfilteredSerializer.java:440)
> org.apache.cassandra.io.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:95)
> org.apache.cassandra.io.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:73)
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> org.apache.cassandra.io.sstable.SSTableIdentityIterator.hasNext(SSTableIdentityIterator.java:122)
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:100)
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:374)
> org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:186)
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:155)
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:500)
> org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:360)
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47)
> org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:133)
> org.apache.cassandra.db.rows.UnfilteredRowIterators.digest(UnfilteredRowIterators.java:178)
> org.apache.cassandra.repair.Validator.rowHash(Validator.java:221)
> org.apache.cassandra.repair.Validator.add(Validator.java:160)
> org.apache.cassandra.db.compaction.CompactionManager.doValidationCompaction(CompactionManager.java:1364)
> org.apache.cassandra.db.compaction.CompactionManager.access$700(CompactionManager.java:85)
> org.apache.cassandra.db.compaction.CompactionManager$13.call(CompactionManager.java:933)
> java.util.concurrent.FutureTask.run(FutureTask.java:266)
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> java.util.concurrent.FutureTask.run(FutureTask.java:266)
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$5/1371495133.run(Unknown
>  Source)
> java.lang.Thread.run(Thread.java:745)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to