Re: hanging validation compaction
oh yes .. you are absolutely right thank you! i will provide all the necessary info in the created jira issue On Thu, 2017-04-13 at 15:09 +0200, benjamin roth wrote: you should be able to find that out by scrubbing the corresponding table(s) and see wich one hangs? i guess the debuglog tells you which sstable is being scrubbed. 2017-04-13 15:07 GMT+02:00 Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>: 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$RowMer
Re: hanging validation compaction
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.Base
Re: hanging validation compaction
) 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 +, 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$dese
Re: hanging validation compaction
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:7
Re: hanging validation compaction
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.MergeI
Re: force processing of pending hinted handoffs
unfortunately it does not. i guess this is intended for resuming hinted handoff handling in case it hase been paused with the pausehandoff before. i have tested it (resuming .. pausing & resuming) but it has no effect on those old hints On Thu, 2017-04-13 at 10:27 +0200, benjamin roth wrote: There is a nodetool command to resume hints. Maybe that helps? Am 13.04.2017 09:42 schrieb "Roland Otta" <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>: oh ... the operation is deprecated according to the docs ... On Thu, 2017-04-13 at 07:40 +, Roland Otta wrote: > i figured out that there is an mbean > org.apache.cassandra.db.type=HintedHandoffManager with the operation > scheduleHintDelivery > > i guess thats what i would need in that case. at least the docs let > me > think so http://javadox.com/org.apache.cassandra/cassandra-all/3.0.0/ > or > g/apache/cassandra/db/HintedHandOffManagerMBean.html > > but everytime i try invoking that operation i get an > UnsupportedOperationException (tried it with hostname, ip and host-id > as parameters - everytime the same exception) > > > > On Tue, 2017-04-11 at 07:40 +, Roland Otta wrote: > > hi, > > > > sometimes we have the problem that we have hinted handoffs (for > > example > > because auf network problems between 2 DCs) that do not get > > processed > > even if the connection problem between the dcs recovers. Some of > > the > > files stay in the hints directory until we restart the node that > > contains the hints. > > > > after the restart of cassandra we can see the proper messages for > > the > > hints handling > > > > Apr 11 09:28:56 bigd006 cassandra: INFO 07:28:56 Deleted hint file > > c429ad19-ee9f-4b5a-abcd-1da1516d1003-1491895717182-1.hints > > Apr 11 09:28:56 bigd006 cassandra: INFO 07:28:56 Finished hinted > > handoff of file c429ad19-ee9f-4b5a-abcd-1da1516d1003-1491895717182- > > 1.hints to endpoint c429ad19-ee9f-4b5a-abcd-1da1516d1003 > > > > is there a way (for example via jmx) to force a node to process > > outstanding hints instead of restarting the node? > > does anyone know whats the cause for not retrying to process those > > hints automatically? > > > > br, > > roland > >
Re: force processing of pending hinted handoffs
oh ... the operation is deprecated according to the docs ... On Thu, 2017-04-13 at 07:40 +, Roland Otta wrote: > i figured out that there is an mbean > org.apache.cassandra.db.type=HintedHandoffManager with the operation > scheduleHintDelivery > > i guess thats what i would need in that case. at least the docs let > me > think so http://javadox.com/org.apache.cassandra/cassandra-all/3.0.0/ > or > g/apache/cassandra/db/HintedHandOffManagerMBean.html > > but everytime i try invoking that operation i get an > UnsupportedOperationException (tried it with hostname, ip and host-id > as parameters - everytime the same exception) > > > > On Tue, 2017-04-11 at 07:40 +, Roland Otta wrote: > > hi, > > > > sometimes we have the problem that we have hinted handoffs (for > > example > > because auf network problems between 2 DCs) that do not get > > processed > > even if the connection problem between the dcs recovers. Some of > > the > > files stay in the hints directory until we restart the node that > > contains the hints. > > > > after the restart of cassandra we can see the proper messages for > > the > > hints handling > > > > Apr 11 09:28:56 bigd006 cassandra: INFO 07:28:56 Deleted hint file > > c429ad19-ee9f-4b5a-abcd-1da1516d1003-1491895717182-1.hints > > Apr 11 09:28:56 bigd006 cassandra: INFO 07:28:56 Finished hinted > > handoff of file c429ad19-ee9f-4b5a-abcd-1da1516d1003-1491895717182- > > 1.hints to endpoint c429ad19-ee9f-4b5a-abcd-1da1516d1003 > > > > is there a way (for example via jmx) to force a node to process > > outstanding hints instead of restarting the node? > > does anyone know whats the cause for not retrying to process those > > hints automatically? > > > > br, > > roland > >
Re: force processing of pending hinted handoffs
i figured out that there is an mbean org.apache.cassandra.db.type=HintedHandoffManager with the operation scheduleHintDelivery i guess thats what i would need in that case. at least the docs let me think so http://javadox.com/org.apache.cassandra/cassandra-all/3.0.0/or g/apache/cassandra/db/HintedHandOffManagerMBean.html but everytime i try invoking that operation i get an UnsupportedOperationException (tried it with hostname, ip and host-id as parameters - everytime the same exception) On Tue, 2017-04-11 at 07:40 +, Roland Otta wrote: > hi, > > sometimes we have the problem that we have hinted handoffs (for > example > because auf network problems between 2 DCs) that do not get processed > even if the connection problem between the dcs recovers. Some of the > files stay in the hints directory until we restart the node that > contains the hints. > > after the restart of cassandra we can see the proper messages for the > hints handling > > Apr 11 09:28:56 bigd006 cassandra: INFO 07:28:56 Deleted hint file > c429ad19-ee9f-4b5a-abcd-1da1516d1003-1491895717182-1.hints > Apr 11 09:28:56 bigd006 cassandra: INFO 07:28:56 Finished hinted > handoff of file c429ad19-ee9f-4b5a-abcd-1da1516d1003-1491895717182- > 1.hints to endpoint c429ad19-ee9f-4b5a-abcd-1da1516d1003 > > is there a way (for example via jmx) to force a node to process > outstanding hints instead of restarting the node? > does anyone know whats the cause for not retrying to process those > hints automatically? > > br, > roland > >
Re: hanging validation compaction
) 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 alidationExecutor1 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
hanging validation compaction
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 alidationExecutor1 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
Re: WriteTimeoutException with LWT after few milliseconds
sorry .. ignore my comment ... i missed your comment that the record is in the table ... On Wed, 2017-04-12 at 16:48 +0200, Roland Otta wrote: Hi Benjamin, its unlikely that i can assist you .. but nevertheless ... i give it a try ;-) whats your consistency level for the insert? what if one ore more nodes are marked down and proper consistency cant be achieved? of course the error message does not indicate that problem (as it says its a timeout)... but in that case you would get an instant error for inserts. wouldn't you? br, roland On Wed, 2017-04-12 at 15:09 +0200, benjamin roth wrote: Hi folks, Can someone explain why that occurs? Write timeout after 0.006s Query: 'INSERT INTO log_moment_import ("source", "reference", "user_id", "moment_id", "date", "finished") VALUES (3, '1305821272790495', 65675537, 0, '2017-04-12 13:00:51', NULL) IF NOT EXISTS Primary key and parition key is source + reference Message: Operation timed out - received only 1 responses. This appears every now and then in the log. When I check the for the record in the table, it is there. I could explain that, if the WTE occured after the configured write timeout but it happens withing a few milliseconds. Is this caused by lock contention? It is possible that there are concurrent inserts on the same PK - actually thats the reason why I use LWTs. Thanks!
Re: WriteTimeoutException with LWT after few milliseconds
Hi Benjamin, its unlikely that i can assist you .. but nevertheless ... i give it a try ;-) whats your consistency level for the insert? what if one ore more nodes are marked down and proper consistency cant be achieved? of course the error message does not indicate that problem (as it says its a timeout)... but in that case you would get an instant error for inserts. wouldn't you? br, roland On Wed, 2017-04-12 at 15:09 +0200, benjamin roth wrote: Hi folks, Can someone explain why that occurs? Write timeout after 0.006s Query: 'INSERT INTO log_moment_import ("source", "reference", "user_id", "moment_id", "date", "finished") VALUES (3, '1305821272790495', 65675537, 0, '2017-04-12 13:00:51', NULL) IF NOT EXISTS Primary key and parition key is source + reference Message: Operation timed out - received only 1 responses. This appears every now and then in the log. When I check the for the record in the table, it is there. I could explain that, if the WTE occured after the configured write timeout but it happens withing a few milliseconds. Is this caused by lock contention? It is possible that there are concurrent inserts on the same PK - actually thats the reason why I use LWTs. Thanks!
Re: Inconsistent data after adding a new DC and rebuilding
well .. thats pretty much the same we saw in our environment (cassandra 3.7). in our case a full repair fixed the issues. but no doubt .. it would be more satisfying to know the root cause for that issue br, roland On Mon, 2017-04-10 at 19:12 +0200, George Sigletos wrote: In 3 out of 5 nodes of our new DC the rebuild process finished successfully. In the other two nodes not (the process was hanging doing nothing) so we killed it, removed all data and started again. This time finished successfully. Here is the netstats output of one of the new newly added nodes: Mode: NORMAL Not sending any streams. Read Repair Statistics: Attempted: 269142 Mismatch (Blocking): 169866 Mismatch (Background): 4 Pool NameActive Pending Completed Dropped Commandsn/a 2 10031126 1935 Responses n/a97 22565129 n/a On Mon, Apr 10, 2017 at 5:28 PM, Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>> wrote: Hi, we have seen similar issues here. have you verified that your rebuilds have been finished successfully? we have seen rebuilds that stopped streaming and working but have not finished. what does nodetool netstats output for your newly built up nodes? br, roland On Mon, 2017-04-10 at 17:15 +0200, George Sigletos wrote: Hello, We recently added a new datacenter to our cluster and run "nodetool rebuild -- " in all 5 new nodes, one by one. After this process finished we noticed there is data missing from the new datacenter, although it exists on the current one. How would that be possible? Should I maybe have run repair in all nodes of the current DC before adding the new one? Running Cassandra 2.1.15 Kind regards, George
force processing of pending hinted handoffs
hi, sometimes we have the problem that we have hinted handoffs (for example because auf network problems between 2 DCs) that do not get processed even if the connection problem between the dcs recovers. Some of the files stay in the hints directory until we restart the node that contains the hints. after the restart of cassandra we can see the proper messages for the hints handling Apr 11 09:28:56 bigd006 cassandra: INFO 07:28:56 Deleted hint file c429ad19-ee9f-4b5a-abcd-1da1516d1003-1491895717182-1.hints Apr 11 09:28:56 bigd006 cassandra: INFO 07:28:56 Finished hinted handoff of file c429ad19-ee9f-4b5a-abcd-1da1516d1003-1491895717182- 1.hints to endpoint c429ad19-ee9f-4b5a-abcd-1da1516d1003 is there a way (for example via jmx) to force a node to process outstanding hints instead of restarting the node? does anyone know whats the cause for not retrying to process those hints automatically? br, roland
Re: Inconsistent data after adding a new DC and rebuilding
Hi, we have seen similar issues here. have you verified that your rebuilds have been finished successfully? we have seen rebuilds that stopped streaming and working but have not finished. what does nodetool netstats output for your newly built up nodes? br, roland On Mon, 2017-04-10 at 17:15 +0200, George Sigletos wrote: Hello, We recently added a new datacenter to our cluster and run "nodetool rebuild -- " in all 5 new nodes, one by one. After this process finished we noticed there is data missing from the new datacenter, although it exists on the current one. How would that be possible? Should I maybe have run repair in all nodes of the current DC before adding the new one? Running Cassandra 2.1.15 Kind regards, George
Re: cassandra node stops streaming data during nodetool rebuild
good point! on the source side i can see the following error ERROR [STREAM-OUT-/192.168.0.114:34094] 2017-04-06 17:18:56,532 StreamSession.java:529 - [Stream #41606030-1ad9-11e7-9f16-51230e2be4e9] Streaming error occurred on session with peer 10.192.116.1 through 192.168. 0.114 org.apache.cassandra.io.FSReadError: java.io.IOException: Broken pipe at org.apache.cassandra.io.util.ChannelProxy.transferTo(ChannelProxy.java:145) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.streaming.compress.CompressedStreamWriter.lambda$write$0(CompressedStreamWriter.java:90) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.io.util.BufferedDataOutputStreamPlus.applyToChannel(BufferedDataOutputStreamPlus.java:350) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.streaming.compress.CompressedStreamWriter.write(CompressedStreamWriter.java:90) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.streaming.messages.OutgoingFileMessage.serialize(OutgoingFileMessage.java:91) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:48) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:40) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:48) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:370) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:342) ~[apache-cassandra-3.7.jar:3.7] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_77] Caused by: java.io.IOException: Broken pipe at sun.nio.ch.FileChannelImpl.transferTo0(Native Method) ~[na:1.8.0_77] at sun.nio.ch.FileChannelImpl.transferToDirectlyInternal(FileChannelImpl.java:428) ~[na:1.8.0_77] at sun.nio.ch.FileChannelImpl.transferToDirectly(FileChannelImpl.java:493) ~[na:1.8.0_77] at sun.nio.ch.FileChannelImpl.transferTo(FileChannelImpl.java:608) ~[na:1.8.0_77] at org.apache.cassandra.io.util.ChannelProxy.transferTo(ChannelProxy.java:141) ~[apache-cassandra-3.7.jar:3.7] ... 10 common frames omitted DEBUG [STREAM-OUT-/192.168.0.114:34094] 2017-04-06 17:18:56,532 ConnectionHandler.java:110 - [Stream #41606030-1ad9-11e7-9f16-51230e2be4e9] Closing stream connection handler on /10.192.116.1 INFO [STREAM-OUT-/192.168.0.114:34094] 2017-04-06 17:18:56,532 StreamResultFuture.java:187 - [Stream #41606030-1ad9-11e7-9f16-51230e2be4e9] Session with /10.192.116.1 is complete WARN [STREAM-OUT-/192.168.0.114:34094] 2017-04-06 17:18:56,532 StreamResultFuture.java:214 - [Stream #41606030-1ad9-11e7-9f16-51230e2be4e9] Stream failed the dataset is approx 300GB / Node. does that mean that cassandra does not try to reconnect (for streaming) in case of short network dropouts? On Fri, 2017-04-07 at 08:53 -0400, Jacob Shadix wrote: Did you look at the logs on the source DC as well? How big is the dataset? -- Jacob Shadix On Fri, Apr 7, 2017 at 7:16 AM, Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>> wrote: Hi! we are on 3.7. we have some debug messages ... but i guess they are not related to that issue DEBUG [GossipStage:1] 2017-04-07 13:11:00,440 FailureDetector.java:456 - Ignoring interval time of 2002469610 for /192.168.0.27<http://192.168.0.27> DEBUG [GossipStage:1] 2017-04-07 13:11:00,441 FailureDetector.java:456 - Ignoring interval time of 2598593732 for /10.192.116.4<http://10.192.116.4> DEBUG [GossipStage:1] 2017-04-07 13:11:00,441 FailureDetector.java:456 - Ignoring interval time of 2002612298 for /10.192.116.5<http://10.192.116.5> DEBUG [GossipStage:1] 2017-04-07 13:11:00,441 FailureDetector.java:456 - Ignoring interval time of 2002660534 for /10.192.116.9<http://10.192.116.9> DEBUG [GossipStage:1] 2017-04-07 13:11:00,465 FailureDetector.java:456 - Ignoring interval time of 2027212880 for /10.192.116.3<http://10.192.116.3> DEBUG [GossipStage:1] 2017-04-07 13:11:00,465 FailureDetector.java:456 - Ignoring interval time of 2027279042 for /192.168.0.188<http://192.168.0.188> DEBUG [GossipStage:1] 2017-04-07 13:11:00,465 FailureDetector.java:456 - Ignoring interval time of 2027313992 for /10.192.116.10<http://10.192.116.10> beside that the debug.log is clean all the mentioned cassandra.yml parameters are the shipped defaults (streaming_socket_timeout_in_ms does not exist at all in my cassandra.yml) i also checked the pending compactions. there are no pending compactions at the moment. bg - roland otta On Fri, 2017-04-07 at 06:47 -0400, Jacob Shadix wrote: What version are you running? Do you see any
Re: cassandra node stops streaming data during nodetool rebuild
Hi! we are on 3.7. we have some debug messages ... but i guess they are not related to that issue DEBUG [GossipStage:1] 2017-04-07 13:11:00,440 FailureDetector.java:456 - Ignoring interval time of 2002469610 for /192.168.0.27 DEBUG [GossipStage:1] 2017-04-07 13:11:00,441 FailureDetector.java:456 - Ignoring interval time of 2598593732 for /10.192.116.4 DEBUG [GossipStage:1] 2017-04-07 13:11:00,441 FailureDetector.java:456 - Ignoring interval time of 2002612298 for /10.192.116.5 DEBUG [GossipStage:1] 2017-04-07 13:11:00,441 FailureDetector.java:456 - Ignoring interval time of 2002660534 for /10.192.116.9 DEBUG [GossipStage:1] 2017-04-07 13:11:00,465 FailureDetector.java:456 - Ignoring interval time of 2027212880 for /10.192.116.3 DEBUG [GossipStage:1] 2017-04-07 13:11:00,465 FailureDetector.java:456 - Ignoring interval time of 2027279042 for /192.168.0.188 DEBUG [GossipStage:1] 2017-04-07 13:11:00,465 FailureDetector.java:456 - Ignoring interval time of 2027313992 for /10.192.116.10 beside that the debug.log is clean all the mentioned cassandra.yml parameters are the shipped defaults (streaming_socket_timeout_in_ms does not exist at all in my cassandra.yml) i also checked the pending compactions. there are no pending compactions at the moment. bg - roland otta On Fri, 2017-04-07 at 06:47 -0400, Jacob Shadix wrote: What version are you running? Do you see any errors in the system.log (SocketTimeout, for instance)? And what values do you have for the following in cassandra.yaml: - - stream_throughput_outbound_megabits_per_sec - - compaction_throughput_mb_per_sec - - streaming_socket_timeout_in_ms -- Jacob Shadix On Fri, Apr 7, 2017 at 6:00 AM, Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>> wrote: hi, we are trying to setup a new datacenter and are initalizing the data with nodetool rebuild. after some hours it seems that the node stopped streaming (at least there is no more streaming traffic on the network interface). nodetool netstats shows that the streaming is still in progress Mode: NORMAL Bootstrap 6918dc90-1ad6-11e7-9f16-51230e2be4e9 Rebuild 41606030-1ad9-11e7-9f16-51230e2be4e9 /192.168.0.26<http://192.168.0.26> Receiving 257 files, 145444246572 bytes total. Already received 1 files, 1744027 bytes total bds/adcounter_total 76456/47310255 bytes(0%) received from idx:0/192.168.0.26<http://192.168.0.26> bds/upselling_event 1667571/1667571 bytes(100%) received from idx:0/192.168.0.26<http://192.168.0.26> /192.168.0.188<http://192.168.0.188> /192.168.0.27<http://192.168.0.27> Receiving 169 files, 79355302464 bytes total. Already received 1 files, 81585975 bytes total bds/ad_event_history 81585975/81585975 bytes(100%) received from idx:0/192.168.0.27<http://192.168.0.27> /192.168.0.189<http://192.168.0.189> Receiving 140 files, 19673034809 bytes total. Already received 1 files, 5996604 bytes total bds/adcounter_per_day 5956840/42259846 bytes(14%) received from idx:0/192.168.0.189<http://192.168.0.189> bds/user_event 39764/39764 bytes(100%) received from idx:0/192.168.0.189<http://192.168.0.189> Read Repair Statistics: Attempted: 0 Mismatch (Blocking): 0 Mismatch (Background): 0 Pool NameActive Pending Completed Dropped Large messages n/a 2 3 0 Small messages n/a 0 68632465 0 Gossip messages n/a 0 217661 0 it is in that state for approx 15 hours now does it make sense waiting for the streaming to finish or do i have to restart the node, discard data and restart the rebuild?
cassandra node stops streaming data during nodetool rebuild
hi, we are trying to setup a new datacenter and are initalizing the data with nodetool rebuild. after some hours it seems that the node stopped streaming (at least there is no more streaming traffic on the network interface). nodetool netstats shows that the streaming is still in progress Mode: NORMAL Bootstrap 6918dc90-1ad6-11e7-9f16-51230e2be4e9 Rebuild 41606030-1ad9-11e7-9f16-51230e2be4e9 /192.168.0.26 Receiving 257 files, 145444246572 bytes total. Already received 1 files, 1744027 bytes total bds/adcounter_total 76456/47310255 bytes(0%) received from idx:0/192.168.0.26 bds/upselling_event 1667571/1667571 bytes(100%) received from idx:0/192.168.0.26 /192.168.0.188 /192.168.0.27 Receiving 169 files, 79355302464 bytes total. Already received 1 files, 81585975 bytes total bds/ad_event_history 81585975/81585975 bytes(100%) received from idx:0/192.168.0.27 /192.168.0.189 Receiving 140 files, 19673034809 bytes total. Already received 1 files, 5996604 bytes total bds/adcounter_per_day 5956840/42259846 bytes(14%) received from idx:0/192.168.0.189 bds/user_event 39764/39764 bytes(100%) received from idx:0/192.168.0.189 Read Repair Statistics: Attempted: 0 Mismatch (Blocking): 0 Mismatch (Background): 0 Pool NameActive Pending Completed Dropped Large messages n/a 2 3 0 Small messages n/a 0 68632465 0 Gossip messages n/a 0 217661 0 it is in that state for approx 15 hours now does it make sense waiting for the streaming to finish or do i have to restart the node, discard data and restart the rebuild?
Re: nodes are always out of sync
thank you both chris and benjamin for taking time to clarify that. On Sat, 2017-04-01 at 21:17 +0200, benjamin roth wrote: Tl;Dr: there are race conditions in a repair and it is not trivial to fix them. So we rather stay with these race conditions. Actually they don't really hurt. The worst case is that ranges are repaired that don't really need a repair. Am 01.04.2017 21:14 schrieb "Chris Lohfink" <clohfin...@gmail.com<mailto:clohfin...@gmail.com>>: Repairs do not have an ability to instantly build a perfect view of its data between your 3 nodes at an exact time. When a piece of data is written there is a delay between when they applied between the nodes, even if its just 500ms. So if a request to read the data and build the merkle tree of the data occurs and it finishes on node1 at 12:01 while node2 finishes at 12:02 the 1 minute or so delta (even if a few seconds, or if using snapshot repairs) between the partition/range hashes in the merkle tree can be different. On a moving data set its almost impossible to have the clusters perfectly in sync for a repair. I wouldnt worry about that log message. If you are worried about consistency between your read/writes use each or local quorum for both. Chris On Thu, Mar 30, 2017 at 1:22 AM, Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>> wrote: hi, we see the following behaviour in our environment: cluster consists of 6 nodes (cassandra version 3.0.7). keyspace has a replication factor 3. clients are writing data to the keyspace with consistency one. we are doing parallel, incremental repairs with cassandra reaper. even if a repair just finished and we are starting a new one immediately, we can see the following entries in our logs: INFO [RepairJobTask:1] 2017-03-30 10:14:00,782 SyncTask.java:73 - [repair #d0f651f6-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.188<http://192.168.0.188> and /192.168.0.191<http://192.168.0.191> have 1 range(s) out of sync for ad_event_history INFO [RepairJobTask:2] 2017-03-30 10:14:00,782 SyncTask.java:73 - [repair #d0f651f6-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.188<http://192.168.0.188> and /192.168.0.189<http://192.168.0.189> have 1 range(s) out of sync for ad_event_history INFO [RepairJobTask:4] 2017-03-30 10:14:00,782 SyncTask.java:73 - [repair #d0f651f6-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.189<http://192.168.0.189> and /192.168.0.191<http://192.168.0.191> have 1 range(s) out of sync for ad_event_history INFO [RepairJobTask:2] 2017-03-30 10:14:03,997 SyncTask.java:73 - [repair #d0fa70a1-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.26<http://192.168.0.26> and /192.168.0.189<http://192.168.0.189> have 2 range(s) out of sync for ad_event_history INFO [RepairJobTask:1] 2017-03-30 10:14:03,997 SyncTask.java:73 - [repair #d0fa70a1-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.26<http://192.168.0.26> and /192.168.0.191<http://192.168.0.191> have 2 range(s) out of sync for ad_event_history INFO [RepairJobTask:4] 2017-03-30 10:14:03,997 SyncTask.java:73 - [repair #d0fa70a1-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.189<http://192.168.0.189> and /192.168.0.191<http://192.168.0.191> have 2 range(s) out of sync for ad_event_history INFO [RepairJobTask:1] 2017-03-30 10:14:05,375 SyncTask.java:73 - [repair #d0fbd033-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.189<http://192.168.0.189> and /192.168.0.191<http://192.168.0.191> have 1 range(s) out of sync for ad_event_history INFO [RepairJobTask:2] 2017-03-30 10:14:05,375 SyncTask.java:73 - [repair #d0fbd033-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.189<http://192.168.0.189> and /192.168.0.190<http://192.168.0.190> have 1 range(s) out of sync for ad_event_history INFO [RepairJobTask:4] 2017-03-30 10:14:05,375 SyncTask.java:73 - [repair #d0fbd033-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.190<http://192.168.0.190> and /192.168.0.191<http://192.168.0.191> have 1 range(s) out of sync for ad_event_history we cant see any hints on the systems ... so we thought everything is running smoothly with the writes. do we have to be concerned about the nodes always being out of sync or is this a normal behaviour in a write intensive table (as the tables will never be 100% in sync for the latest inserts)? bg, roland
nodes are always out of sync
hi, we see the following behaviour in our environment: cluster consists of 6 nodes (cassandra version 3.0.7). keyspace has a replication factor 3. clients are writing data to the keyspace with consistency one. we are doing parallel, incremental repairs with cassandra reaper. even if a repair just finished and we are starting a new one immediately, we can see the following entries in our logs: INFO [RepairJobTask:1] 2017-03-30 10:14:00,782 SyncTask.java:73 - [repair #d0f651f6-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.188 and /192.168.0.191 have 1 range(s) out of sync for ad_event_history INFO [RepairJobTask:2] 2017-03-30 10:14:00,782 SyncTask.java:73 - [repair #d0f651f6-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.188 and /192.168.0.189 have 1 range(s) out of sync for ad_event_history INFO [RepairJobTask:4] 2017-03-30 10:14:00,782 SyncTask.java:73 - [repair #d0f651f6-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.189 and /192.168.0.191 have 1 range(s) out of sync for ad_event_history INFO [RepairJobTask:2] 2017-03-30 10:14:03,997 SyncTask.java:73 - [repair #d0fa70a1-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.26 and /192.168.0.189 have 2 range(s) out of sync for ad_event_history INFO [RepairJobTask:1] 2017-03-30 10:14:03,997 SyncTask.java:73 - [repair #d0fa70a1-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.26 and /192.168.0.191 have 2 range(s) out of sync for ad_event_history INFO [RepairJobTask:4] 2017-03-30 10:14:03,997 SyncTask.java:73 - [repair #d0fa70a1-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.189 and /192.168.0.191 have 2 range(s) out of sync for ad_event_history INFO [RepairJobTask:1] 2017-03-30 10:14:05,375 SyncTask.java:73 - [repair #d0fbd033-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.189 and /192.168.0.191 have 1 range(s) out of sync for ad_event_history INFO [RepairJobTask:2] 2017-03-30 10:14:05,375 SyncTask.java:73 - [repair #d0fbd033-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.189 and /192.168.0.190 have 1 range(s) out of sync for ad_event_history INFO [RepairJobTask:4] 2017-03-30 10:14:05,375 SyncTask.java:73 - [repair #d0fbd033-1520-11e7-a443-d9f5b942818e] Endpoints /192.168.0.190 and /192.168.0.191 have 1 range(s) out of sync for ad_event_history we cant see any hints on the systems ... so we thought everything is running smoothly with the writes. do we have to be concerned about the nodes always being out of sync or is this a normal behaviour in a write intensive table (as the tables will never be 100% in sync for the latest inserts)? bg, roland
spikes in blocked native transport requests
we have a datacenter which is currently used exlusively for spark batch jobs. in case batch jobs are running against that environment we can see very high peaks in blocked native transport requests (up to 10k / minute). i am concerned because i guess that will slow other queries (in case other applications are going to use that dc as well). i already tried increasing native_transport_max_threads + concurrent_reads without success. during the jobs i cant find any resource limitiations on my hardware (iops, disk usage, cpu, ... is fine). am i missing something? any suggestions how to cope with that? br// roland
Re: repair performance
good point! i did not (so far) i will do that - especially because i often see all compaction threads being used during repair (according to compactionstats). thank you also for your link recommendations. i will go through them. On Sat, 2017-03-18 at 16:54 +, Thakrar, Jayesh wrote: You changed compaction_throughput_mb_per_sec, but did you also increase concurrent_compactors? In reference to the reaper and some other info I received on the user forum to my question on "nodetool repair", here are some useful links/slides - https://www.datastax.com/dev/blog/repair-in-cassandra https://www.pythian.com/blog/effective-anti-entropy-repair-cassandra/ http://www.slideshare.net/DataStax/real-world-tales-of-repair-alexander-dejanovski-the-last-pickle-cassandra-summit-2016 http://www.slideshare.net/DataStax/real-world-repairs-vinay-chella-netflix-cassandra-summit-2016 From: Roland Otta <roland.o...@willhaben.at> Date: Friday, March 17, 2017 at 5:47 PM To: "user@cassandra.apache.org" <user@cassandra.apache.org> Subject: Re: repair performance did not recognize that so far. thank you for the hint. i will definitely give it a try On Fri, 2017-03-17 at 22:32 +0100, benjamin roth wrote: The fork from thelastpickle is. I'd recommend to give it a try over pure nodetool. 2017-03-17 22:30 GMT+01:00 Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>: forgot to mention the version we are using: we are using 3.0.7 - so i guess we should have incremental repairs by default. it also prints out incremental:true when starting a repair INFO [Thread-7281] 2017-03-17 09:40:32,059 RepairRunnable.java:125 - Starting repair command #7, repairing keyspace xxx with repair options (parallelism: parallel, primary range: false, incremental: true, job threads: 1, ColumnFamilies: [], dataCenters: [ProdDC2], hosts: [], # of ranges: 1758) 3.0.7 is also the reason why we are not using reaper ... as far as i could figure out it's not compatible with 3.0+ On Fri, 2017-03-17 at 22:13 +0100, benjamin roth wrote: It depends a lot ... - Repairs can be very slow, yes! (And unreliable, due to timeouts, outages, whatever) - You can use incremental repairs to speed things up for regular repairs - You can use "reaper" to schedule repairs and run them sliced, automated, failsafe The time repairs actually may vary a lot depending on how much data has to be streamed or how inconsistent your cluster is. 50mbit/s is really a bit low! The actual performance depends on so many factors like your CPU, RAM, HD/SSD, concurrency settings, load of the "old nodes" of the cluster. This is a quite individual problem you have to track down individually. 2017-03-17 22:07 GMT+01:00 Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>: hello, we are quite inexperienced with cassandra at the moment and are playing around with a new cluster we built up for getting familiar with cassandra and its possibilites. while getting familiar with that topic we recognized that repairs in our cluster take a long time. To get an idea of our current setup here are some numbers: our cluster currently consists of 4 nodes (replication factor 3). these nodes are all on dedicated physical hardware in our own datacenter. all of the nodes have 32 cores @2,9Ghz 64 GB ram 2 ssds (raid0) 900 GB each for data 1 seperate hdd for OS + commitlogs current dataset: approx 530 GB per node 21 tables (biggest one has more than 200 GB / node) i already tried setting compactionthroughput + streamingthroughput to unlimited for testing purposes ... but that did not change anything. when checking system resources i cannot see any bottleneck (cpus are pretty idle and we have no iowaits). when issuing a repair via nodetool repair -local on a node the repair takes longer than a day. is this normal or could we normally expect a faster repair? i also recognized that initalizing of new nodes in the datacenter was really slow (approx 50 mbit/s). also here i expected a much better performance - could those 2 problems be somehow related? br// roland
Re: repair performance
did not recognize that so far. thank you for the hint. i will definitely give it a try On Fri, 2017-03-17 at 22:32 +0100, benjamin roth wrote: The fork from thelastpickle is. I'd recommend to give it a try over pure nodetool. 2017-03-17 22:30 GMT+01:00 Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>: forgot to mention the version we are using: we are using 3.0.7 - so i guess we should have incremental repairs by default. it also prints out incremental:true when starting a repair INFO [Thread-7281] 2017-03-17 09:40:32,059 RepairRunnable.java:125 - Starting repair command #7, repairing keyspace xxx with repair options (parallelism: parallel, primary range: false, incremental: true, job threads: 1, ColumnFamilies: [], dataCenters: [ProdDC2], hosts: [], # of ranges: 1758) 3.0.7 is also the reason why we are not using reaper ... as far as i could figure out it's not compatible with 3.0+ On Fri, 2017-03-17 at 22:13 +0100, benjamin roth wrote: It depends a lot ... - Repairs can be very slow, yes! (And unreliable, due to timeouts, outages, whatever) - You can use incremental repairs to speed things up for regular repairs - You can use "reaper" to schedule repairs and run them sliced, automated, failsafe The time repairs actually may vary a lot depending on how much data has to be streamed or how inconsistent your cluster is. 50mbit/s is really a bit low! The actual performance depends on so many factors like your CPU, RAM, HD/SSD, concurrency settings, load of the "old nodes" of the cluster. This is a quite individual problem you have to track down individually. 2017-03-17 22:07 GMT+01:00 Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>: hello, we are quite inexperienced with cassandra at the moment and are playing around with a new cluster we built up for getting familiar with cassandra and its possibilites. while getting familiar with that topic we recognized that repairs in our cluster take a long time. To get an idea of our current setup here are some numbers: our cluster currently consists of 4 nodes (replication factor 3). these nodes are all on dedicated physical hardware in our own datacenter. all of the nodes have 32 cores @2,9Ghz 64 GB ram 2 ssds (raid0) 900 GB each for data 1 seperate hdd for OS + commitlogs current dataset: approx 530 GB per node 21 tables (biggest one has more than 200 GB / node) i already tried setting compactionthroughput + streamingthroughput to unlimited for testing purposes ... but that did not change anything. when checking system resources i cannot see any bottleneck (cpus are pretty idle and we have no iowaits). when issuing a repair via nodetool repair -local on a node the repair takes longer than a day. is this normal or could we normally expect a faster repair? i also recognized that initalizing of new nodes in the datacenter was really slow (approx 50 mbit/s). also here i expected a much better performance - could those 2 problems be somehow related? br// roland
Re: repair performance
... maybe i should just try increasing the job threads with --job-threads shame on me On Fri, 2017-03-17 at 21:30 +, Roland Otta wrote: forgot to mention the version we are using: we are using 3.0.7 - so i guess we should have incremental repairs by default. it also prints out incremental:true when starting a repair INFO [Thread-7281] 2017-03-17 09:40:32,059 RepairRunnable.java:125 - Starting repair command #7, repairing keyspace xxx with repair options (parallelism: parallel, primary range: false, incremental: true, job threads: 1, ColumnFamilies: [], dataCenters: [ProdDC2], hosts: [], # of ranges: 1758) 3.0.7 is also the reason why we are not using reaper ... as far as i could figure out it's not compatible with 3.0+ On Fri, 2017-03-17 at 22:13 +0100, benjamin roth wrote: It depends a lot ... - Repairs can be very slow, yes! (And unreliable, due to timeouts, outages, whatever) - You can use incremental repairs to speed things up for regular repairs - You can use "reaper" to schedule repairs and run them sliced, automated, failsafe The time repairs actually may vary a lot depending on how much data has to be streamed or how inconsistent your cluster is. 50mbit/s is really a bit low! The actual performance depends on so many factors like your CPU, RAM, HD/SSD, concurrency settings, load of the "old nodes" of the cluster. This is a quite individual problem you have to track down individually. 2017-03-17 22:07 GMT+01:00 Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>: hello, we are quite inexperienced with cassandra at the moment and are playing around with a new cluster we built up for getting familiar with cassandra and its possibilites. while getting familiar with that topic we recognized that repairs in our cluster take a long time. To get an idea of our current setup here are some numbers: our cluster currently consists of 4 nodes (replication factor 3). these nodes are all on dedicated physical hardware in our own datacenter. all of the nodes have 32 cores @2,9Ghz 64 GB ram 2 ssds (raid0) 900 GB each for data 1 seperate hdd for OS + commitlogs current dataset: approx 530 GB per node 21 tables (biggest one has more than 200 GB / node) i already tried setting compactionthroughput + streamingthroughput to unlimited for testing purposes ... but that did not change anything. when checking system resources i cannot see any bottleneck (cpus are pretty idle and we have no iowaits). when issuing a repair via nodetool repair -local on a node the repair takes longer than a day. is this normal or could we normally expect a faster repair? i also recognized that initalizing of new nodes in the datacenter was really slow (approx 50 mbit/s). also here i expected a much better performance - could those 2 problems be somehow related? br// roland
Re: repair performance
forgot to mention the version we are using: we are using 3.0.7 - so i guess we should have incremental repairs by default. it also prints out incremental:true when starting a repair INFO [Thread-7281] 2017-03-17 09:40:32,059 RepairRunnable.java:125 - Starting repair command #7, repairing keyspace xxx with repair options (parallelism: parallel, primary range: false, incremental: true, job threads: 1, ColumnFamilies: [], dataCenters: [ProdDC2], hosts: [], # of ranges: 1758) 3.0.7 is also the reason why we are not using reaper ... as far as i could figure out it's not compatible with 3.0+ On Fri, 2017-03-17 at 22:13 +0100, benjamin roth wrote: It depends a lot ... - Repairs can be very slow, yes! (And unreliable, due to timeouts, outages, whatever) - You can use incremental repairs to speed things up for regular repairs - You can use "reaper" to schedule repairs and run them sliced, automated, failsafe The time repairs actually may vary a lot depending on how much data has to be streamed or how inconsistent your cluster is. 50mbit/s is really a bit low! The actual performance depends on so many factors like your CPU, RAM, HD/SSD, concurrency settings, load of the "old nodes" of the cluster. This is a quite individual problem you have to track down individually. 2017-03-17 22:07 GMT+01:00 Roland Otta <roland.o...@willhaben.at<mailto:roland.o...@willhaben.at>>: hello, we are quite inexperienced with cassandra at the moment and are playing around with a new cluster we built up for getting familiar with cassandra and its possibilites. while getting familiar with that topic we recognized that repairs in our cluster take a long time. To get an idea of our current setup here are some numbers: our cluster currently consists of 4 nodes (replication factor 3). these nodes are all on dedicated physical hardware in our own datacenter. all of the nodes have 32 cores @2,9Ghz 64 GB ram 2 ssds (raid0) 900 GB each for data 1 seperate hdd for OS + commitlogs current dataset: approx 530 GB per node 21 tables (biggest one has more than 200 GB / node) i already tried setting compactionthroughput + streamingthroughput to unlimited for testing purposes ... but that did not change anything. when checking system resources i cannot see any bottleneck (cpus are pretty idle and we have no iowaits). when issuing a repair via nodetool repair -local on a node the repair takes longer than a day. is this normal or could we normally expect a faster repair? i also recognized that initalizing of new nodes in the datacenter was really slow (approx 50 mbit/s). also here i expected a much better performance - could those 2 problems be somehow related? br// roland
repair performance
hello, we are quite inexperienced with cassandra at the moment and are playing around with a new cluster we built up for getting familiar with cassandra and its possibilites. while getting familiar with that topic we recognized that repairs in our cluster take a long time. To get an idea of our current setup here are some numbers: our cluster currently consists of 4 nodes (replication factor 3). these nodes are all on dedicated physical hardware in our own datacenter. all of the nodes have 32 cores @2,9Ghz 64 GB ram 2 ssds (raid0) 900 GB each for data 1 seperate hdd for OS + commitlogs current dataset: approx 530 GB per node 21 tables (biggest one has more than 200 GB / node) i already tried setting compactionthroughput + streamingthroughput to unlimited for testing purposes ... but that did not change anything. when checking system resources i cannot see any bottleneck (cpus are pretty idle and we have no iowaits). when issuing a repair via nodetool repair -local on a node the repair takes longer than a day. is this normal or could we normally expect a faster repair? i also recognized that initalizing of new nodes in the datacenter was really slow (approx 50 mbit/s). also here i expected a much better performance - could those 2 problems be somehow related? br// roland