mirror of https://github.com/apache/lucene.git
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
This commit is contained in:
parent
fb1884358d
commit
52dc6a5908
|
@ -34,19 +34,24 @@ public class DocumentsWriterFlushQueue {
|
||||||
private final AtomicInteger ticketCount = new AtomicInteger();
|
private final AtomicInteger ticketCount = new AtomicInteger();
|
||||||
private final ReentrantLock purgeLock = new ReentrantLock();
|
private final ReentrantLock purgeLock = new ReentrantLock();
|
||||||
|
|
||||||
synchronized void addDeletesAndPurge(DocumentsWriter writer,
|
void addDeletesAndPurge(DocumentsWriter writer,
|
||||||
DocumentsWriterDeleteQueue deleteQueue) throws IOException {
|
DocumentsWriterDeleteQueue deleteQueue) throws IOException {
|
||||||
incTickets();// first inc the ticket count - freeze opens
|
synchronized (this) {
|
||||||
// a window for #anyChanges to fail
|
incTickets();// first inc the ticket count - freeze opens
|
||||||
boolean success = false;
|
// a window for #anyChanges to fail
|
||||||
try {
|
boolean success = false;
|
||||||
queue.add(new GlobalDeletesTicket(deleteQueue.freezeGlobalBuffer(null)));
|
try {
|
||||||
success = true;
|
queue
|
||||||
} finally {
|
.add(new GlobalDeletesTicket(deleteQueue.freezeGlobalBuffer(null)));
|
||||||
if (!success) {
|
success = true;
|
||||||
decTickets();
|
} 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);
|
forcePurge(writer);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -127,6 +132,7 @@ public class DocumentsWriterFlushQueue {
|
||||||
}
|
}
|
||||||
|
|
||||||
void forcePurge(DocumentsWriter writer) throws IOException {
|
void forcePurge(DocumentsWriter writer) throws IOException {
|
||||||
|
assert !Thread.holdsLock(this);
|
||||||
purgeLock.lock();
|
purgeLock.lock();
|
||||||
try {
|
try {
|
||||||
innerPurge(writer);
|
innerPurge(writer);
|
||||||
|
@ -136,6 +142,7 @@ public class DocumentsWriterFlushQueue {
|
||||||
}
|
}
|
||||||
|
|
||||||
void tryPurge(DocumentsWriter writer) throws IOException {
|
void tryPurge(DocumentsWriter writer) throws IOException {
|
||||||
|
assert !Thread.holdsLock(this);
|
||||||
if (purgeLock.tryLock()) {
|
if (purgeLock.tryLock()) {
|
||||||
try {
|
try {
|
||||||
innerPurge(writer);
|
innerPurge(writer);
|
||||||
|
|
Loading…
Reference in New Issue