From 52dc6a5908fc301b72e0268371e26a9e66aa6d61 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 19 Jan 2012 08:54:33 +0000 Subject: [PATCH] LUCENE-3705: IWFlushQueue deadlocks if deletes are flushed and a segment is published at the same time git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1233248 13f79535-47bb-0310-9956-ffa450edef68 --- .../index/DocumentsWriterFlushQueue.java | 27 ++++++++++++------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java b/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java index 73d62505327..c9a0db5be2a 100644 --- a/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java +++ b/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java @@ -34,19 +34,24 @@ public class DocumentsWriterFlushQueue { private final AtomicInteger ticketCount = new AtomicInteger(); private final ReentrantLock purgeLock = new ReentrantLock(); - synchronized void addDeletesAndPurge(DocumentsWriter writer, + void addDeletesAndPurge(DocumentsWriter writer, DocumentsWriterDeleteQueue deleteQueue) throws IOException { - incTickets();// first inc the ticket count - freeze opens - // a window for #anyChanges to fail - boolean success = false; - try { - queue.add(new GlobalDeletesTicket(deleteQueue.freezeGlobalBuffer(null))); - success = true; - } finally { - if (!success) { - decTickets(); + synchronized (this) { + incTickets();// first inc the ticket count - freeze opens + // a window for #anyChanges to fail + boolean success = false; + try { + queue + .add(new GlobalDeletesTicket(deleteQueue.freezeGlobalBuffer(null))); + success = true; + } finally { + if (!success) { + decTickets(); + } } } + // don't hold the lock on the FlushQueue when forcing the purge - this blocks and deadlocks + // if we hold the lock. forcePurge(writer); } @@ -127,6 +132,7 @@ public class DocumentsWriterFlushQueue { } void forcePurge(DocumentsWriter writer) throws IOException { + assert !Thread.holdsLock(this); purgeLock.lock(); try { innerPurge(writer); @@ -136,6 +142,7 @@ public class DocumentsWriterFlushQueue { } void tryPurge(DocumentsWriter writer) throws IOException { + assert !Thread.holdsLock(this); if (purgeLock.tryLock()) { try { innerPurge(writer);