Return-Path: X-Original-To: apmail-lucene-dev-archive@www.apache.org Delivered-To: apmail-lucene-dev-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id BBB501734E for ; Wed, 25 Feb 2015 09:16:10 +0000 (UTC) Received: (qmail 92340 invoked by uid 500); 25 Feb 2015 09:16:04 -0000 Delivered-To: apmail-lucene-dev-archive@lucene.apache.org Received: (qmail 92275 invoked by uid 500); 25 Feb 2015 09:16:04 -0000 Mailing-List: contact dev-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list dev@lucene.apache.org Received: (qmail 92264 invoked by uid 99); 25 Feb 2015 09:16:04 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 25 Feb 2015 09:16:04 +0000 Date: Wed, 25 Feb 2015 09:16:04 +0000 (UTC) From: "Michael McCandless (JIRA)" To: dev@lucene.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Reopened] (LUCENE-6214) IW deadlocks if commit and reopen happens concurrently while exception is hit MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/LUCENE-6214?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael McCandless reopened LUCENE-6214: ---------------------------------------- Reopen for 4.10.4 > IW deadlocks if commit and reopen happens concurrently while exception is hit > ----------------------------------------------------------------------------- > > Key: LUCENE-6214 > URL: https://issues.apache.org/jira/browse/LUCENE-6214 > Project: Lucene - Core > Issue Type: Bug > Affects Versions: 5.0 > Reporter: Simon Willnauer > Assignee: Michael McCandless > Priority: Blocker > Fix For: 4.10.4, 5.0, Trunk, 5.1 > > Attachments: LUCENE-6214.patch, LUCENE-6214.patch > > > I just hit this while working on an elasticseach test using a lucene 5.1 snapshot (5.1.0-snapshot-1654549). The test throws random exceptions via MockDirWrapper and deadlocks, jstack says: > {noformat} > Found one Java-level deadlock: > ============================= > "elasticsearch[node_2][refresh][T#2]": > waiting to lock monitor 0x00007fe51314c098 (object 0x00000007018ee8d8, a java.lang.Object), > which is held by "elasticsearch[node_2][generic][T#1]" > "elasticsearch[node_2][generic][T#1]": > waiting to lock monitor 0x00007fe512d74b68 (object 0x00000007018ee8e8, a java.lang.Object), > which is held by "elasticsearch[node_2][refresh][T#2]" > Java stack information for the threads listed above: > =================================================== > "elasticsearch[node_2][refresh][T#2]": > at org.apache.lucene.index.IndexWriter.tragicEvent(IndexWriter.java:4441) > - waiting to lock <0x00000007018ee8d8> (a java.lang.Object) > at org.apache.lucene.index.IndexWriter.getReader(IndexWriter.java:436) > - locked <0x00000007018ee8e8> (a java.lang.Object) > at org.apache.lucene.index.StandardDirectoryReader.doOpenFromWriter(StandardDirectoryReader.java:281) > at org.apache.lucene.index.StandardDirectoryReader.doOpenIfChanged(StandardDirectoryReader.java:256) > at org.apache.lucene.index.StandardDirectoryReader.doOpenIfChanged(StandardDirectoryReader.java:246) > at org.apache.lucene.index.FilterDirectoryReader.doOpenIfChanged(FilterDirectoryReader.java:104) > at org.apache.lucene.index.DirectoryReader.openIfChanged(DirectoryReader.java:123) > at org.apache.lucene.search.SearcherManager.refreshIfNeeded(SearcherManager.java:137) > at org.apache.lucene.search.SearcherManager.refreshIfNeeded(SearcherManager.java:58) > at org.apache.lucene.search.ReferenceManager.doMaybeRefresh(ReferenceManager.java:176) > at org.apache.lucene.search.ReferenceManager.maybeRefreshBlocking(ReferenceManager.java:253) > at org.elasticsearch.index.engine.internal.InternalEngine.refresh(InternalEngine.java:703) > at org.elasticsearch.index.shard.IndexShard.refresh(IndexShard.java:500) > at org.elasticsearch.index.shard.IndexShard$EngineRefresher$1.run(IndexShard.java:954) > at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > "elasticsearch[node_2][generic][T#1]": > at org.apache.lucene.index.IndexWriter.prepareCommitInternal(IndexWriter.java:2730) > - waiting to lock <0x00000007018ee8e8> (a java.lang.Object) > - locked <0x00000007018ee8d8> (a java.lang.Object) > at org.apache.lucene.index.IndexWriter.commitInternal(IndexWriter.java:2888) > - locked <0x00000007018ee8d8> (a java.lang.Object) > at org.apache.lucene.index.IndexWriter.commit(IndexWriter.java:2855) > at org.elasticsearch.index.engine.internal.InternalEngine.commitIndexWriter(InternalEngine.java:722) > at org.elasticsearch.index.engine.internal.InternalEngine.flush(InternalEngine.java:800) > at org.elasticsearch.index.engine.internal.InternalEngine$RecoveryCounter.endRecovery(InternalEngine.java:1520) > at org.elasticsearch.index.engine.internal.InternalEngine$RecoveryCounter.close(InternalEngine.java:1533) > at org.elasticsearch.common.lease.Releasables.close(Releasables.java:45) > at org.elasticsearch.common.lease.Releasables.closeWhileHandlingException(Releasables.java:70) > at org.elasticsearch.common.lease.Releasables.closeWhileHandlingException(Releasables.java:75) > at org.elasticsearch.index.engine.internal.InternalEngine.recover(InternalEngine.java:1048) > at org.elasticsearch.index.shard.IndexShard.recover(IndexShard.java:635) > at org.elasticsearch.indices.recovery.RecoverySource.recover(RecoverySource.java:120) > at org.elasticsearch.indices.recovery.RecoverySource.access$200(RecoverySource.java:48) > at org.elasticsearch.indices.recovery.RecoverySource$StartRecoveryTransportRequestHandler.messageReceived(RecoverySource.java:141) > at org.elasticsearch.indices.recovery.RecoverySource$StartRecoveryTransportRequestHandler.messageReceived(RecoverySource.java:127) > at org.elasticsearch.transport.local.LocalTransport$2.doRun(LocalTransport.java:287) > at org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:36) > at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > Found 1 deadlock. > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org For additional commands, e-mail: dev-help@lucene.apache.org