[jira] [Commented] (CASSANDRA-13432) MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns()
[ https://issues.apache.org/jira/browse/CASSANDRA-13432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16103168#comment-16103168 ] Corentin Chary commented on CASSANDRA-13432: [~rgerard] this is already part of 3.x, this only applies to 2.x. > MemtableReclaimMemory can get stuck because of lack of timeout in > getTopLevelColumns() > -- > > Key: CASSANDRA-13432 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13432 > Project: Cassandra > Issue Type: Bug > Environment: cassandra 2.1.15 >Reporter: Corentin Chary > Fix For: 2.1.x > > > This might affect 3.x too, I'm not sure. > {code} > $ nodetool tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 32135875 0 > 0 > ReadStage 114 0 29492940 0 > 0 > RequestResponseStage 0 0 86090931 0 > 0 > ReadRepairStage 0 0 166645 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > MiscStage 0 0 0 0 > 0 > HintedHandoff 0 0 47 0 > 0 > GossipStage 0 0 188769 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > CommitLogArchiver 0 0 0 0 > 0 > CompactionExecutor0 0 86835 0 > 0 > ValidationExecutor0 0 0 0 > 0 > MigrationStage0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > PendingRangeCalculator0 0 92 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0563 0 > 0 > MemtablePostFlush 0 0 1500 0 > 0 > MemtableReclaimMemory 129534 0 > 0 > Native-Transport-Requests41 0 54819182 0 > 1896 > {code} > {code} > "MemtableReclaimMemory:195" - Thread t@6268 >java.lang.Thread.State: WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:283) > at > org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:417) > at > org.apache.cassandra.db.ColumnFamilyStore$Flush$1.runMayThrow(ColumnFamilyStore.java:1151) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) >Locked ownable synchronizers: > - locked <6e7b1160> (a java.util.concurrent.ThreadPoolExecutor$Worker) > "SharedPool-Worker-195" - Thread t@989 >java.lang.Thread.State: RUNNABLE > at > org.apache.cassandra.db.RangeTombstoneList.addInternal(RangeTombstoneList.java:690) > at > org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:650) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:171) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:143) > at org.apache.cassandra.db.DeletionInfo.add(DeletionInfo.java:240) > at > org.apache.cassandra.db.ArrayBackedSortedColumns.delete(ArrayBackedSortedColumns.java:483) > at org.apache.cassandra.db.ColumnFamily.addAtom(ColumnFamily.java:153) > at > org.apache.cassandra.db.filter.QueryFilter$2.getNext(QueryFilter.java:184) >
[jira] [Commented] (CASSANDRA-13432) MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns()
[ https://issues.apache.org/jira/browse/CASSANDRA-13432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16074387#comment-16074387 ] Romain GERARD commented on CASSANDRA-13432: --- I am seconding this patch for the 3.x branch as it helps detect bad data model before it is too late and without impacting the integrity of the whole system. This kind of error messages create a positive feedback loop where we can improve things upon. > MemtableReclaimMemory can get stuck because of lack of timeout in > getTopLevelColumns() > -- > > Key: CASSANDRA-13432 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13432 > Project: Cassandra > Issue Type: Bug > Environment: cassandra 2.1.15 >Reporter: Corentin Chary > Fix For: 2.1.x > > > This might affect 3.x too, I'm not sure. > {code} > $ nodetool tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 32135875 0 > 0 > ReadStage 114 0 29492940 0 > 0 > RequestResponseStage 0 0 86090931 0 > 0 > ReadRepairStage 0 0 166645 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > MiscStage 0 0 0 0 > 0 > HintedHandoff 0 0 47 0 > 0 > GossipStage 0 0 188769 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > CommitLogArchiver 0 0 0 0 > 0 > CompactionExecutor0 0 86835 0 > 0 > ValidationExecutor0 0 0 0 > 0 > MigrationStage0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > PendingRangeCalculator0 0 92 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0563 0 > 0 > MemtablePostFlush 0 0 1500 0 > 0 > MemtableReclaimMemory 129534 0 > 0 > Native-Transport-Requests41 0 54819182 0 > 1896 > {code} > {code} > "MemtableReclaimMemory:195" - Thread t@6268 >java.lang.Thread.State: WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:283) > at > org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:417) > at > org.apache.cassandra.db.ColumnFamilyStore$Flush$1.runMayThrow(ColumnFamilyStore.java:1151) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) >Locked ownable synchronizers: > - locked <6e7b1160> (a java.util.concurrent.ThreadPoolExecutor$Worker) > "SharedPool-Worker-195" - Thread t@989 >java.lang.Thread.State: RUNNABLE > at > org.apache.cassandra.db.RangeTombstoneList.addInternal(RangeTombstoneList.java:690) > at > org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:650) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:171) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:143) > at org.apache.cassandra.db.DeletionInfo.add(DeletionInfo.java:240) > at >
[jira] [Commented] (CASSANDRA-13432) MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns()
[ https://issues.apache.org/jira/browse/CASSANDRA-13432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16044493#comment-16044493 ] Corentin Chary commented on CASSANDRA-13432: We have a case internally were upgrading to 3.0 or changing the data model won't happen, and we know that we *need* this patch for another year. We're currently keeping a forked version, so that's not so much of an issue. I don't believe this patch really changes the behavior as it simply aborts earlier what would anyway would have been aborted later (later may currently be minutes~hours later). > MemtableReclaimMemory can get stuck because of lack of timeout in > getTopLevelColumns() > -- > > Key: CASSANDRA-13432 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13432 > Project: Cassandra > Issue Type: Bug > Environment: cassandra 2.1.15 >Reporter: Corentin Chary > Fix For: 2.1.x > > > This might affect 3.x too, I'm not sure. > {code} > $ nodetool tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 32135875 0 > 0 > ReadStage 114 0 29492940 0 > 0 > RequestResponseStage 0 0 86090931 0 > 0 > ReadRepairStage 0 0 166645 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > MiscStage 0 0 0 0 > 0 > HintedHandoff 0 0 47 0 > 0 > GossipStage 0 0 188769 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > CommitLogArchiver 0 0 0 0 > 0 > CompactionExecutor0 0 86835 0 > 0 > ValidationExecutor0 0 0 0 > 0 > MigrationStage0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > PendingRangeCalculator0 0 92 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0563 0 > 0 > MemtablePostFlush 0 0 1500 0 > 0 > MemtableReclaimMemory 129534 0 > 0 > Native-Transport-Requests41 0 54819182 0 > 1896 > {code} > {code} > "MemtableReclaimMemory:195" - Thread t@6268 >java.lang.Thread.State: WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:283) > at > org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:417) > at > org.apache.cassandra.db.ColumnFamilyStore$Flush$1.runMayThrow(ColumnFamilyStore.java:1151) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) >Locked ownable synchronizers: > - locked <6e7b1160> (a java.util.concurrent.ThreadPoolExecutor$Worker) > "SharedPool-Worker-195" - Thread t@989 >java.lang.Thread.State: RUNNABLE > at > org.apache.cassandra.db.RangeTombstoneList.addInternal(RangeTombstoneList.java:690) > at > org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:650) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:171) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:143) > at
[jira] [Commented] (CASSANDRA-13432) MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns()
[ https://issues.apache.org/jira/browse/CASSANDRA-13432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16044470#comment-16044470 ] Stefan Podkowinski commented on CASSANDRA-13432: I'm not really sure if this qualifies as a bug or improvement tbh. The changes seem to be reasonable, but keep in mind that if this goes in it would become part of the potentially last release before 2.x goes EOL. Is the case for the patch really strong enough to introducing more restrictive behaviour during query execution? Shouldn't we rather suggest people to upgrade to 3.0 when hitting this issue (or even better changing their data model)? > MemtableReclaimMemory can get stuck because of lack of timeout in > getTopLevelColumns() > -- > > Key: CASSANDRA-13432 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13432 > Project: Cassandra > Issue Type: Bug > Environment: cassandra 2.1.15 >Reporter: Corentin Chary > Fix For: 2.1.x > > > This might affect 3.x too, I'm not sure. > {code} > $ nodetool tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 32135875 0 > 0 > ReadStage 114 0 29492940 0 > 0 > RequestResponseStage 0 0 86090931 0 > 0 > ReadRepairStage 0 0 166645 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > MiscStage 0 0 0 0 > 0 > HintedHandoff 0 0 47 0 > 0 > GossipStage 0 0 188769 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > CommitLogArchiver 0 0 0 0 > 0 > CompactionExecutor0 0 86835 0 > 0 > ValidationExecutor0 0 0 0 > 0 > MigrationStage0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > PendingRangeCalculator0 0 92 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0563 0 > 0 > MemtablePostFlush 0 0 1500 0 > 0 > MemtableReclaimMemory 129534 0 > 0 > Native-Transport-Requests41 0 54819182 0 > 1896 > {code} > {code} > "MemtableReclaimMemory:195" - Thread t@6268 >java.lang.Thread.State: WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:283) > at > org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:417) > at > org.apache.cassandra.db.ColumnFamilyStore$Flush$1.runMayThrow(ColumnFamilyStore.java:1151) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) >Locked ownable synchronizers: > - locked <6e7b1160> (a java.util.concurrent.ThreadPoolExecutor$Worker) > "SharedPool-Worker-195" - Thread t@989 >java.lang.Thread.State: RUNNABLE > at > org.apache.cassandra.db.RangeTombstoneList.addInternal(RangeTombstoneList.java:690) > at > org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:650) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:171) > at >
[jira] [Commented] (CASSANDRA-13432) MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns()
[ https://issues.apache.org/jira/browse/CASSANDRA-13432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15972825#comment-15972825 ] Corentin Chary commented on CASSANDRA-13432: Latest patch https://github.com/iksaif/cassandra/commits/CASSANDRA-13432-2.x > MemtableReclaimMemory can get stuck because of lack of timeout in > getTopLevelColumns() > -- > > Key: CASSANDRA-13432 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13432 > Project: Cassandra > Issue Type: Bug > Environment: cassandra 2.1.15 >Reporter: Corentin Chary > Fix For: 2.1.x > > Attachments: CASSANDRA-13432.patch > > > This might affect 3.x too, I'm not sure. > {code} > $ nodetool tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 32135875 0 > 0 > ReadStage 114 0 29492940 0 > 0 > RequestResponseStage 0 0 86090931 0 > 0 > ReadRepairStage 0 0 166645 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > MiscStage 0 0 0 0 > 0 > HintedHandoff 0 0 47 0 > 0 > GossipStage 0 0 188769 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > CommitLogArchiver 0 0 0 0 > 0 > CompactionExecutor0 0 86835 0 > 0 > ValidationExecutor0 0 0 0 > 0 > MigrationStage0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > PendingRangeCalculator0 0 92 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0563 0 > 0 > MemtablePostFlush 0 0 1500 0 > 0 > MemtableReclaimMemory 129534 0 > 0 > Native-Transport-Requests41 0 54819182 0 > 1896 > {code} > {code} > "MemtableReclaimMemory:195" - Thread t@6268 >java.lang.Thread.State: WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:283) > at > org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:417) > at > org.apache.cassandra.db.ColumnFamilyStore$Flush$1.runMayThrow(ColumnFamilyStore.java:1151) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) >Locked ownable synchronizers: > - locked <6e7b1160> (a java.util.concurrent.ThreadPoolExecutor$Worker) > "SharedPool-Worker-195" - Thread t@989 >java.lang.Thread.State: RUNNABLE > at > org.apache.cassandra.db.RangeTombstoneList.addInternal(RangeTombstoneList.java:690) > at > org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:650) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:171) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:143) > at org.apache.cassandra.db.DeletionInfo.add(DeletionInfo.java:240) > at > org.apache.cassandra.db.ArrayBackedSortedColumns.delete(ArrayBackedSortedColumns.java:483) > at org.apache.cassandra.db.ColumnFamily.addAtom(ColumnFamily.java:153) > at >
[jira] [Commented] (CASSANDRA-13432) MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns()
[ https://issues.apache.org/jira/browse/CASSANDRA-13432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15967256#comment-15967256 ] Corentin Chary commented on CASSANDRA-13432: Tried the patch, setting the tombstone threshold to one: {code} WARN [SharedPool-Worker-4] 2017-04-13 09:51:55,894 AbstractTracingAwareExecutorService.java:169 - Uncaught exception on thread Thread[SharedPool-Worker-4,10,main]: {} java.lang.RuntimeException: org.apache.cassandra.db.filter.TombstoneOverwhelmingException at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2249) ~[main/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_121] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164) ~[main/:na] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$TraceSessionFutureTask.run(AbstractTracingAwareExecutorService.java:136) [main/:na] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [main/:na] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_121] Caused by: org.apache.cassandra.db.filter.TombstoneOverwhelmingException: null at org.apache.cassandra.db.filter.QueryFilter$2.getNext(QueryFilter.java:202) ~[main/:na] at org.apache.cassandra.db.filter.QueryFilter$2.hasNext(QueryFilter.java:163) ~[main/:na] at org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:146) ~[main/:na] at org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:125) ~[main/:na] at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:99) ~[main/:na] at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na] at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na] at org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:263) ~[main/:na] at org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:114) ~[main/:na] at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:88) ~[main/:na] at org.apache.cassandra.db.RowIteratorFactory$2.getReduced(RowIteratorFactory.java:99) ~[main/:na] at org.apache.cassandra.db.RowIteratorFactory$2.getReduced(RowIteratorFactory.java:71) ~[main/:na] at org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:117) ~[main/:na] at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:100) ~[main/:na] at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na] at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na] at org.apache.cassandra.db.ColumnFamilyStore$9.computeNext(ColumnFamilyStore.java:2115) ~[main/:na] at org.apache.cassandra.db.ColumnFamilyStore$9.computeNext(ColumnFamilyStore.java:2111) ~[main/:na] at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) ~[guava-16.0.jar:na] at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) ~[guava-16.0.jar:na] at org.apache.cassandra.db.ColumnFamilyStore.filter(ColumnFamilyStore.java:2266) ~[main/:na] at org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:2224) ~[main/:na] at org.apache.cassandra.db.PagedRangeCommand.executeLocally(PagedRangeCommand.java:115) ~[main/:na] at org.apache.cassandra.service.StorageProxy$LocalRangeSliceRunnable.runMayThrow(StorageProxy.java:1572) ~[main/:na] at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2246) ~[main/:na] ... 5 common frames omitted {code} > MemtableReclaimMemory can get stuck because of lack of timeout in > getTopLevelColumns() > -- > > Key: CASSANDRA-13432 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13432 > Project: Cassandra > Issue Type: Bug > Environment: cassandra 2.1.15 >Reporter: Corentin Chary > Fix For: 2.1.x > > > This might affect 3.x too, I'm not sure. > {code} > $ nodetool tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 32135875 0 > 0 > ReadStage 114 0 29492940 0 > 0 >
[jira] [Commented] (CASSANDRA-13432) MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns()
[ https://issues.apache.org/jira/browse/CASSANDRA-13432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15965534#comment-15965534 ] Corentin Chary commented on CASSANDRA-13432: It's 2.1.15 but I don't believe it has been fixed. I believe that it's stuck in org.apache.cassandra.db.filter.QueryFilter$2.hasNext(QueryFilter.java:156) which doesn't count tombstones. A simple patch could be something like: {code} diff --git a/src/java/org/apache/cassandra/db/filter/QueryFilter.java b/src/java/org/apache/cassandra/db/filter/QueryFilter.java index db531a5..8b718db 100644 --- a/src/java/org/apache/cassandra/db/filter/QueryFilter.java +++ b/src/java/org/apache/cassandra/db/filter/QueryFilter.java @@ -23,6 +23,10 @@ import java.util.Iterator; import java.util.List; import java.util.SortedSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.Cell; import org.apache.cassandra.db.ColumnFamily; import org.apache.cassandra.db.DecoratedKey; @@ -34,10 +38,12 @@ import org.apache.cassandra.db.columniterator.OnDiskAtomIterator; import org.apache.cassandra.db.composites.CellName; import org.apache.cassandra.db.composites.Composite; import org.apache.cassandra.io.sstable.SSTableReader; +import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.utils.MergeIterator; public class QueryFilter { +private static final Logger logger = LoggerFactory.getLogger(QueryFilter.class); public final DecoratedKey key; public final String cfName; public final IDiskAtomFilter filter; @@ -147,6 +153,7 @@ public class QueryFilter return new Iterator() { private Cell next; +private int tombstoneCount = 0; public boolean hasNext() { @@ -181,6 +188,19 @@ public class QueryFilter } else { +tombstoneCount++; +if (tombstoneCount > DatabaseDescriptor.getTombstoneFailureThreshold()) +{ +Tracing.trace("Scanned over {} tombstones; query aborted (see tombstone_failure_threshold)", + DatabaseDescriptor.getTombstoneFailureThreshold());
[jira] [Commented] (CASSANDRA-13432) MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns()
[ https://issues.apache.org/jira/browse/CASSANDRA-13432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15965529#comment-15965529 ] ZhaoYang commented on CASSANDRA-13432: -- [~iksaif] can you share which version are you using? as far as I remember in early 2.1.X, there is a bug related to over-counting tombstones that shouldn't be counted.. fixed in 2.1.6 (https://issues.apache.org/jira/browse/CASSANDRA-9299) > MemtableReclaimMemory can get stuck because of lack of timeout in > getTopLevelColumns() > -- > > Key: CASSANDRA-13432 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13432 > Project: Cassandra > Issue Type: Bug >Reporter: Corentin Chary > Fix For: 2.1.x > > > This might affect 3.x too, I'm not sure. > {code} > $ nodetool tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 32135875 0 > 0 > ReadStage 114 0 29492940 0 > 0 > RequestResponseStage 0 0 86090931 0 > 0 > ReadRepairStage 0 0 166645 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > MiscStage 0 0 0 0 > 0 > HintedHandoff 0 0 47 0 > 0 > GossipStage 0 0 188769 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > CommitLogArchiver 0 0 0 0 > 0 > CompactionExecutor0 0 86835 0 > 0 > ValidationExecutor0 0 0 0 > 0 > MigrationStage0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > PendingRangeCalculator0 0 92 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0563 0 > 0 > MemtablePostFlush 0 0 1500 0 > 0 > MemtableReclaimMemory 129534 0 > 0 > Native-Transport-Requests41 0 54819182 0 > 1896 > {code} > {code} > "MemtableReclaimMemory:195" - Thread t@6268 >java.lang.Thread.State: WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:283) > at > org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:417) > at > org.apache.cassandra.db.ColumnFamilyStore$Flush$1.runMayThrow(ColumnFamilyStore.java:1151) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) >Locked ownable synchronizers: > - locked <6e7b1160> (a java.util.concurrent.ThreadPoolExecutor$Worker) > "SharedPool-Worker-195" - Thread t@989 >java.lang.Thread.State: RUNNABLE > at > org.apache.cassandra.db.RangeTombstoneList.addInternal(RangeTombstoneList.java:690) > at > org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:650) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:171) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:143) > at org.apache.cassandra.db.DeletionInfo.add(DeletionInfo.java:240) > at > org.apache.cassandra.db.ArrayBackedSortedColumns.delete(ArrayBackedSortedColumns.java:483) > at
[jira] [Commented] (CASSANDRA-13432) MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns()
[ https://issues.apache.org/jira/browse/CASSANDRA-13432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15964465#comment-15964465 ] Corentin Chary commented on CASSANDRA-13432: I checked, 3.x has a different code to count tombstones so it's likely not affected > MemtableReclaimMemory can get stuck because of lack of timeout in > getTopLevelColumns() > -- > > Key: CASSANDRA-13432 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13432 > Project: Cassandra > Issue Type: Bug >Reporter: Corentin Chary > Fix For: 2.1.x > > > This might affect 3.x too, I'm not sure. > {code} > $ nodetool tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 32135875 0 > 0 > ReadStage 114 0 29492940 0 > 0 > RequestResponseStage 0 0 86090931 0 > 0 > ReadRepairStage 0 0 166645 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > MiscStage 0 0 0 0 > 0 > HintedHandoff 0 0 47 0 > 0 > GossipStage 0 0 188769 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > CommitLogArchiver 0 0 0 0 > 0 > CompactionExecutor0 0 86835 0 > 0 > ValidationExecutor0 0 0 0 > 0 > MigrationStage0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > PendingRangeCalculator0 0 92 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0563 0 > 0 > MemtablePostFlush 0 0 1500 0 > 0 > MemtableReclaimMemory 129534 0 > 0 > Native-Transport-Requests41 0 54819182 0 > 1896 > {code} > {code} > "MemtableReclaimMemory:195" - Thread t@6268 >java.lang.Thread.State: WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:283) > at > org.apache.cassandra.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:417) > at > org.apache.cassandra.db.ColumnFamilyStore$Flush$1.runMayThrow(ColumnFamilyStore.java:1151) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) >Locked ownable synchronizers: > - locked <6e7b1160> (a java.util.concurrent.ThreadPoolExecutor$Worker) > "SharedPool-Worker-195" - Thread t@989 >java.lang.Thread.State: RUNNABLE > at > org.apache.cassandra.db.RangeTombstoneList.addInternal(RangeTombstoneList.java:690) > at > org.apache.cassandra.db.RangeTombstoneList.insertFrom(RangeTombstoneList.java:650) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:171) > at > org.apache.cassandra.db.RangeTombstoneList.add(RangeTombstoneList.java:143) > at org.apache.cassandra.db.DeletionInfo.add(DeletionInfo.java:240) > at > org.apache.cassandra.db.ArrayBackedSortedColumns.delete(ArrayBackedSortedColumns.java:483) > at org.apache.cassandra.db.ColumnFamily.addAtom(ColumnFamily.java:153) > at > org.apache.cassandra.db.filter.QueryFilter$2.getNext(QueryFilter.java:184) > at >