Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 4A0BE200C55 for ; Thu, 13 Apr 2017 09:54:48 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 43C94160B8B; Thu, 13 Apr 2017 07:54:48 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 15B9A160B98 for ; Thu, 13 Apr 2017 09:54:46 +0200 (CEST) Received: (qmail 4324 invoked by uid 500); 13 Apr 2017 07:54:46 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 4313 invoked by uid 99); 13 Apr 2017 07:54:46 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 13 Apr 2017 07:54:46 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id BF70DC047D for ; Thu, 13 Apr 2017 07:54:45 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.202 X-Spam-Level: X-Spam-Status: No, score=-99.202 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id tiIVkSG48tQ5 for ; Thu, 13 Apr 2017 07:54:42 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 892A45FB5C for ; Thu, 13 Apr 2017 07:54:42 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 11B43E0C15 for ; Thu, 13 Apr 2017 07:54:42 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id B36BC24069 for ; Thu, 13 Apr 2017 07:54:41 +0000 (UTC) Date: Thu, 13 Apr 2017 07:54:41 +0000 (UTC) From: "Corentin Chary (JIRA)" To: commits@cassandra.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Comment Edited] (CASSANDRA-13432) MemtableReclaimMemory can get stuck because of lack of timeout in getTopLevelColumns() MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 13 Apr 2017 07:54:48 -0000 [ https://issues.apache.org/jira/browse/CASSANDRA-13432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15967256#comment-15967256 ] Corentin Chary edited comment on CASSANDRA-13432 at 4/13/17 7:54 AM: --------------------------------------------------------------------- Tried the patch, setting the tombstone threshold to one: {code} ERROR [SharedPool-Worker-4] 2017-04-13 09:51:55,891 QueryFilter.java:201 - Scanned over 1 tombstones in system.size_estimates for key: unknown; query aborted (see tombstone_failure_threshold). 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} was (Author: iksaif): 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 Name Active 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 > CompactionExecutor 0 0 86835 0 0 > ValidationExecutor 0 0 0 0 0 > MigrationStage 0 0 0 0 0 > AntiEntropyStage 0 0 0 0 0 > PendingRangeCalculator 0 0 92 0 0 > Sampler 0 0 0 0 0 > MemtableFlushWriter 0 0 563 0 0 > MemtablePostFlush 0 0 1500 0 0 > MemtableReclaimMemory 1 29 534 0 0 > Native-Transport-Requests 41 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 org.apache.cassandra.db.filter.QueryFilter$2.hasNext(QueryFilter.java:156) > at org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:146) > at org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:125) > at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:99) > at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) > at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) > at org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:263) > at org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:108) > at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:82) > at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:69) > at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:316) > at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:62) > at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:2015) > at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1858) > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:353) > at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:85) > at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:47) > at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:64) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164) > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) > at java.lang.Thread.run(Thread.java:745) > Locked ownable synchronizers: > - None > "SharedPool-Worker-206" - Thread t@1014 > 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 org.apache.cassandra.db.filter.QueryFilter$2.hasNext(QueryFilter.java:156) > at org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:146) > at org.apache.cassandra.utils.MergeIterator$ManyToOne.(MergeIterator.java:89) > at org.apache.cassandra.utils.MergeIterator.get(MergeIterator.java:48) > at org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:105) > at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:82) > at org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:69) > at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:316) > at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:62) > at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:2015) > at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1858) > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:353) > at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:85) > at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:47) > at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:64) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164) > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) > at java.lang.Thread.run(Thread.java:745) > Locked ownable synchronizers: > - None > {code} > As you can see MemtableReclaimMemory is waiting on the read barrier to be released, but there are two queries currently being executed which are locking this. > Since most of the time is spent pretty low in the stack, these read operations will never timeout (they are reading rows with tons of tombstones). > We also can easily detect or purge the offending line because there is no easy way to find out which partition is currently being read. > The TombstoneFailureThreshold should also protect us, but it is probably being checked too high in the call stack. > Looks like RangeTombstoneList or DeletionInfo should also check for DatabaseDescriptor.getTombstoneFailureThreshold() -- This message was sent by Atlassian JIRA (v6.3.15#6346)