Index: lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java	(revision 1233244)
+++ lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java	(working copy)
@@ -34,19 +34,24 @@
   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 @@
   }
 
   void forcePurge(DocumentsWriter writer) throws IOException {
+    assert !Thread.holdsLock(this);
     purgeLock.lock();
     try {
       innerPurge(writer);
@@ -136,6 +142,7 @@
   }
 
   void tryPurge(DocumentsWriter writer) throws IOException {
+    assert !Thread.holdsLock(this);
     if (purgeLock.tryLock()) {
       try {
         innerPurge(writer);
