i made a copy and also have the permission to upload sstables for that 
particular column_family

is it possible to track down which sstable of that cf is affected or should i 
upload all of them?


br,
roland


On Thu, 2017-04-13 at 13:57 +0200, benjamin roth wrote:
I think thats a good reproduction case for the issue - you should copy the 
sstable away for further testing. Are you allowed to upload the broken sstable 
to JIRA?

2017-04-13 13:15 GMT+02:00 Roland Otta 
<roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>:
sorry .. i have to correct myself .. the problem still persists.

tried nodetool scrub now for the table ... but scrub is also stuck at the same 
percentage

id                                   compaction type keyspace table    
completed total     unit  progress
380e4980-2037-11e7-a9a4-a5f3eec2d826 Validation      bds      ad_event 
805955242 841258085 bytes 95.80%
fb17b8b0-2039-11e7-a9a4-a5f3eec2d826 Scrub           bds      ad_event 
805961728 841258085 bytes 95.80%
Active compaction remaining time :   0h00m00s

according to the thread dump its the same issue

Stack trace:
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)


br,
roland


On Thu, 2017-04-13 at 10:04 +0000, Roland Otta wrote:
i did 2 restarts before which did not help

after that i have set for testing purposes file_cache_size_in_mb: 0 and 
buffer_pool_use_heap_if_exhausted: false and restarted again

after that it worked ... but it also could be that it just worked by accident 
after the last restart and is not related to my config changes

On Thu, 2017-04-13 at 11:58 +0200, benjamin roth wrote:
If you restart the server the same validation completes successfully?
If not, have you tries scrubbing the affected sstables?

2017-04-13 11:43 GMT+02:00 Roland Otta 
<roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>:
thank you guys ... i will

i just wanted to make sure that i am not doing something completely wrong 
before opening an issue

br,
roland


On Thu, 2017-04-13 at 21:35 +1200, Nate McCall wrote:
Not sure what is going on there either. Roland - can you open an issue with the 
information above:
https://issues.apache.org/jira/browse/CASSANDRA

On Thu, Apr 13, 2017 at 7:49 PM, benjamin roth 
<brs...@gmail.com<mailto:brs...@gmail.com>> wrote:
What I can tell you from that trace - given that this is the correct thread and 
it really hangs there:

The validation is stuck when reading from an SSTable.
Unfortunately I am no caffeine expert. It looks like the read is cached and 
after the read caffeine tries to drain the cache and this is stuck. I don't see 
the reason from that stack trace.
Someone had to dig deeper into caffeine to find the root cause.

2017-04-13 9:27 GMT+02:00 Roland Otta 
<roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>:
i had a closer look at the validation executor thread (i hope thats what you 
meant)

it seems the thread is always repeating stuff in
org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:235)

here is the full stack trace ...

i am sorry .. but i have no clue whats happening there ..

com.github.benmanes.caffeine.cache.BoundedLocalCache$$Lambda$64/2098345091<tel:(209)%20834-5091>.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<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.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.Va<http://org.apache.cassandra.repair.Va>lidator.rowHash(Validator.java:221)
org.apache.cassandra.repair.Va<http://org.apache.cassandra.repair.Va>lidator.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)

On Thu, 2017-04-13 at 08:47 +0200, benjamin roth wrote:
You should connect to the node with JConsole and see where the compaction 
thread is stuck

2017-04-13 8:34 GMT+02:00 Roland Otta 
<roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>:
hi,

we have the following issue on our 3.10 development cluster.

we are doing regular repairs with thelastpickle's fork of creaper.
sometimes the repair (it is a full repair in that case) hangs because
of a stuck validation compaction

nodetool compactionstats gives me
a1bb45c0-1fc6-11e7-81de-0fb0b3f5a345 Validation      bds      ad_event
805955242 841258085 bytes 95.80%
we have here no more progress for hours

nodetool tpstats shows
alidationExecutor                1         1          16186         0
               0

i checked the logs on the affected node and could not find any
suspicious errors.

anyone that already had this issue and knows how to cope with that?

a restart of the node helps to finish the repair ... but i am not sure
whether that somehow breaks the full repair

bg,
roland











Reply via email to