You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2012/01/19 09:54:33 UTC
svn commit: r1233248 -
/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java
Author: simonw
Date: Thu Jan 19 08:54:33 2012
New Revision: 1233248
URL: http://svn.apache.org/viewvc?rev=1233248&view=rev
Log:
LUCENE-3705: IWFlushQueue deadlocks if deletes are flushed and a segment is published at the same time
Modified:
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java?rev=1233248&r1=1233247&r2=1233248&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java Thu Jan 19 08:54:33 2012
@@ -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);