Index: src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java	(revision 1059624)
+++ src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java	(working copy)
@@ -6,27 +6,19 @@
 import org.apache.lucene.document.Document;
 
 public abstract class DocumentsWriterPerThreadPool {
-  final static class ThreadState extends ReentrantLock {
-    final DocumentsWriterPerThread perThread;
-
-    ThreadState(DocumentsWriterPerThread perThread) {
-      this.perThread = perThread;
-    }
-  }
-
-  private final ThreadState[] perThreads;
+  private final DocumentsWriterPerThread[] perThreads;
   private volatile int numThreadStatesActive;
 
   public DocumentsWriterPerThreadPool(int maxNumPerThreads) {
     maxNumPerThreads = (maxNumPerThreads < 1) ? IndexWriterConfig.DEFAULT_MAX_THREAD_STATES : maxNumPerThreads;
-    this.perThreads = new ThreadState[maxNumPerThreads];
+    this.perThreads = new DocumentsWriterPerThread[maxNumPerThreads];
 
     numThreadStatesActive = 0;
   }
 
   public void initialize(DocumentsWriter documentsWriter) {
     for (int i = 0; i < perThreads.length; i++) {
-      perThreads[i] = new ThreadState(new DocumentsWriterPerThread(documentsWriter.directory, documentsWriter, documentsWriter.chain));
+      perThreads[i] = new DocumentsWriterPerThread(documentsWriter.directory, documentsWriter, documentsWriter.chain);
     }
   }
 
@@ -34,9 +26,9 @@
     return perThreads.length;
   }
 
-  public synchronized ThreadState newThreadState() {
+  public synchronized DocumentsWriterPerThread newThreadState() {
     if (numThreadStatesActive < perThreads.length) {
-      ThreadState state = perThreads[numThreadStatesActive];
+      DocumentsWriterPerThread state = perThreads[numThreadStatesActive];
       numThreadStatesActive++;
       return state;
     }
@@ -44,22 +36,22 @@
     return null;
   }
 
-  public abstract ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter, Document doc);
+  public abstract DocumentsWriterPerThread getAndLock(Thread requestingThread, DocumentsWriter documentsWriter, Document doc);
 
-  public abstract void clearThreadBindings(ThreadState perThread);
+  public abstract void clearThreadBindings(DocumentsWriterPerThread perThread);
 
   public abstract void clearAllThreadBindings();
 
-  public Iterator<ThreadState> getAllPerThreadsIterator() {
+  public Iterator<DocumentsWriterPerThread> getAllPerThreadsIterator() {
     return getPerThreadsIterator(this.perThreads.length);
   }
 
-  public Iterator<ThreadState> getActivePerThreadsIterator() {
+  public Iterator<DocumentsWriterPerThread> getActivePerThreadsIterator() {
     return getPerThreadsIterator(this.numThreadStatesActive);
   }
 
-  private Iterator<ThreadState> getPerThreadsIterator(final int upto) {
-    return new Iterator<ThreadState>() {
+  private Iterator<DocumentsWriterPerThread> getPerThreadsIterator(final int upto) {
+    return new Iterator<DocumentsWriterPerThread>() {
       int i = 0;
 
       @Override
@@ -68,7 +60,7 @@
       }
 
       @Override
-      public ThreadState next() {
+      public DocumentsWriterPerThread next() {
         return perThreads[i++];
       }
 
Index: src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriterPerThread.java	(revision 1059624)
+++ src/java/org/apache/lucene/index/DocumentsWriterPerThread.java	(working copy)
@@ -24,6 +24,7 @@
 import java.io.PrintStream;
 import java.text.NumberFormat;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.Document;
@@ -33,7 +34,7 @@
 import org.apache.lucene.util.ByteBlockPool.DirectAllocator;
 import org.apache.lucene.util.RamUsageEstimator;
 
-public class DocumentsWriterPerThread {
+public class DocumentsWriterPerThread extends ReentrantLock {
 
   /**
    * The IndexingChain must define the {@link #getChain(DocumentsWriter)} method
@@ -145,12 +146,15 @@
   private int numDocsInRAM;
   private int flushedDocCount;
   SegmentWriteState flushState;
+  boolean isFlushing = false;
 
   final AtomicLong bytesUsed = new AtomicLong(0);
 
   FieldInfos fieldInfos = new FieldInfos();
 
-  public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent, IndexingChain indexingChain) {
+  public DocumentsWriterPerThread(Directory directory, 
+      DocumentsWriter parent, 
+      IndexingChain indexingChain) {
     this.directory = directory;
     this.parent = parent;
     this.writer = parent.indexWriter;
@@ -170,6 +174,9 @@
   }
 
   public void addDocument(Document doc, Analyzer analyzer) throws IOException {
+    
+    assert isHeldByCurrentThread();
+    
     assert writer.testPoint("DocumentsWriterPerThread addDocument start");
     docState.doc = doc;
     docState.analyzer = analyzer;
@@ -223,22 +230,31 @@
   }
 
   void deleteQueries(Query... queries) {
+    assert isHeldByCurrentThread();
+    assert numDocsInRAM > 0;
+    
     for (Query query : queries) {
       pendingDeletes.addQuery(query, numDocsInRAM);
     }
   }
 
   void deleteQuery(Query query) {
+    assert isHeldByCurrentThread();
+    assert numDocsInRAM > 0;
     pendingDeletes.addQuery(query, numDocsInRAM);
   }
 
   synchronized void deleteTerms(Term... terms) {
+    assert isHeldByCurrentThread();
+    assert numDocsInRAM > 0;
     for (Term term : terms) {
       pendingDeletes.addTerm(term, numDocsInRAM);
     }
   }
 
   void deleteTerm(Term term) {
+    assert isHeldByCurrentThread();
+    assert numDocsInRAM > 0;
     pendingDeletes.addTerm(term, numDocsInRAM);
   }
 
@@ -270,7 +286,12 @@
   /** Flush all pending docs to a new segment */
   SegmentInfo flush() throws IOException {
     assert numDocsInRAM > 0;
-
+    assert isHeldByCurrentThread();
+    isFlushing = true;
+    
+    int numDeleteTerms = pendingDeletes.numTermDeletes.get();
+    int numDeleteQueries = pendingDeletes.queries.size();
+    
     flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos,
         numDocsInRAM, writer.getConfig().getTermIndexInterval(),
         SegmentCodecs.build(fieldInfos, writer.codecs));
@@ -320,8 +341,12 @@
       IndexWriter.setDiagnostics(newSegment, "flush");
       success = true;
 
+      assert pendingDeletes.numTermDeletes.get() == numDeleteTerms;
+      assert numDeleteQueries == pendingDeletes.queries.size();
+      
       return newSegment;
     } finally {
+      isFlushing = false;
       if (!success) {
         abort();
       }
Index: src/java/org/apache/lucene/index/DocumentsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriter.java	(revision 1059624)
+++ src/java/org/apache/lucene/index/DocumentsWriter.java	(working copy)
@@ -28,7 +28,6 @@
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
-import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.store.AlreadyClosedException;
@@ -150,17 +149,20 @@
   }
 
   boolean deleteQueries(final Query... queries) throws IOException {
-    Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
+    Iterator<DocumentsWriterPerThread> threadsIterator = perThreadPool.getActivePerThreadsIterator();
 
     boolean deleted = false;
     while (threadsIterator.hasNext()) {
-      ThreadState state = threadsIterator.next();
-      state.lock();
+      DocumentsWriterPerThread perThread = threadsIterator.next();
+      perThread.lock();
       try {
-        state.perThread.deleteQueries(queries);
+        if (!perThread.aborting ||
+            !perThread.isFlushing) {
+          perThread.deleteQueries(queries);
+        }
         deleted = true;
       } finally {
-        state.unlock();
+        perThread.unlock();
       }
     }
 
@@ -180,17 +182,20 @@
   }
 
   boolean deleteTerms(final Term... terms) throws IOException {
-    Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
+    Iterator<DocumentsWriterPerThread> threadsIterator = perThreadPool.getActivePerThreadsIterator();
 
     boolean deleted = false;
     while (threadsIterator.hasNext()) {
-      ThreadState state = threadsIterator.next();
+      DocumentsWriterPerThread perThread = threadsIterator.next();
       deleted = true;
-      state.lock();
+      perThread.lock();
       try {
-        state.perThread.deleteTerms(terms);
+        if (!perThread.aborting ||
+            !perThread.isFlushing) {
+          perThread.deleteTerms(terms);
+        }
       } finally {
-        state.unlock();
+        perThread.unlock();
       }
     }
 
@@ -209,17 +214,17 @@
     return deleteTerms(term);
   }
 
-  boolean deleteTerm(final Term term, ThreadState exclude) {
-    Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
+  boolean deleteTerm(final Term term, DocumentsWriterPerThread exclude) {
+    Iterator<DocumentsWriterPerThread> threadsIterator = perThreadPool.getActivePerThreadsIterator();
 
     boolean deleted = false;
     while (threadsIterator.hasNext()) {
       deleted = true;
-      ThreadState state = threadsIterator.next();
+      DocumentsWriterPerThread state = threadsIterator.next();
       if (state != exclude) {
         state.lock();
         try {
-          state.perThread.deleteTerm(term);
+          state.deleteTerm(term);
         } finally {
           state.unlock();
         }
@@ -257,9 +262,9 @@
   }
 
   private final void pushConfigChange() {
-    Iterator<ThreadState> it = perThreadPool.getAllPerThreadsIterator();
+    Iterator<DocumentsWriterPerThread> it = perThreadPool.getAllPerThreadsIterator();
     while (it.hasNext()) {
-      DocumentsWriterPerThread perThread = it.next().perThread;
+      DocumentsWriterPerThread perThread = it.next();
       perThread.docState.infoStream = this.infoStream;
       perThread.docState.maxFieldLength = this.maxFieldLength;
       perThread.docState.similarity = this.similarity;
@@ -326,13 +331,13 @@
         message("docWriter: abort");
       }
 
-      Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
+      Iterator<DocumentsWriterPerThread> threadsIterator = perThreadPool.getActivePerThreadsIterator();
 
       while (threadsIterator.hasNext()) {
-        ThreadState perThread = threadsIterator.next();
+        DocumentsWriterPerThread perThread = threadsIterator.next();
         perThread.lock();
         try {
-          perThread.perThread.abort();
+          perThread.abort();
         } finally {
           perThread.unlock();
         }
@@ -373,9 +378,9 @@
 
     SegmentInfo newSegment = null;
 
-    ThreadState perThread = perThreadPool.getAndLock(Thread.currentThread(), this, doc);
+    DocumentsWriterPerThread perThread = perThreadPool.getAndLock(Thread.currentThread(), this, doc);
     try {
-      DocumentsWriterPerThread dwpt = perThread.perThread;
+      DocumentsWriterPerThread dwpt = perThread;
       long perThreadRAMUsedBeforeAdd = dwpt.bytesUsed();
       dwpt.addDocument(doc, analyzer);
 
@@ -460,40 +465,39 @@
       pushToLastSegment(pendingDeletes);
     }
 
-    Iterator<ThreadState> threadsIterator = perThreadPool.getActivePerThreadsIterator();
+    Iterator<DocumentsWriterPerThread> threadsIterator = perThreadPool.getActivePerThreadsIterator();
     boolean anythingFlushed = false;
 
     while (threadsIterator.hasNext()) {
       SegmentInfo newSegment = null;
 
-      ThreadState perThread = threadsIterator.next();
+      DocumentsWriterPerThread perThread = threadsIterator.next();
       perThread.lock();
       try {
-        DocumentsWriterPerThread dwpt = perThread.perThread;
-        final int numDocs = dwpt.getNumDocsInRAM();
+        final int numDocs = perThread.getNumDocsInRAM();
 
         // Always flush docs if there are any
         boolean flushDocs = numDocs > 0;
 
-        String segment = dwpt.getSegment();
+        String segment = perThread.getSegment();
 
         // If we are flushing docs, segment must not be null:
         assert segment != null || !flushDocs;
 
         if (flushDocs) {
-          newSegment = dwpt.flush();
+          newSegment = perThread.flush();
 
           if (newSegment != null) {
             anythingFlushed = true;
             perThreadPool.clearThreadBindings(perThread);
-            if (dwpt.pendingDeletes.any()) {
-              bufferedDeletes.pushDeletes(dwpt.pendingDeletes, newSegment);
-              dwpt.pendingDeletes = new SegmentDeletes();
+            if (perThread.pendingDeletes.any()) {
+              bufferedDeletes.pushDeletes(perThread.pendingDeletes, newSegment);
+              perThread.pendingDeletes = new SegmentDeletes();
             }
           }
         }
-        else if (flushDeletes && dwpt.pendingDeletes.any()) {
-          pushToLastSegment(dwpt.pendingDeletes);
+        else if (flushDeletes && perThread.pendingDeletes.any()) {
+          pushToLastSegment(perThread.pendingDeletes);
         }
       } finally {
         perThread.unlock();
@@ -538,7 +542,7 @@
         if (!success) {
           if (infoStream != null) {
             message("hit exception " +
-                "reating compound file for newly flushed segment " + newSegment.name);
+                "creating compound file for newly flushed segment " + newSegment.name);
           }
 
           indexWriter.deleter.refresh(newSegment.name);
Index: src/java/org/apache/lucene/index/ThreadAffinityDocumentsWriterThreadPool.java
===================================================================
--- src/java/org/apache/lucene/index/ThreadAffinityDocumentsWriterThreadPool.java	(revision 1059624)
+++ src/java/org/apache/lucene/index/ThreadAffinityDocumentsWriterThreadPool.java	(working copy)
@@ -7,15 +7,15 @@
 import org.apache.lucene.document.Document;
 
 public class ThreadAffinityDocumentsWriterThreadPool extends DocumentsWriterPerThreadPool {
-  private Map<Thread, ThreadState> threadBindings = new ConcurrentHashMap<Thread, ThreadState>();
+  private Map<Thread, DocumentsWriterPerThread> threadBindings = new ConcurrentHashMap<Thread, DocumentsWriterPerThread>();
 
   public ThreadAffinityDocumentsWriterThreadPool(int maxNumPerThreads) {
     super(maxNumPerThreads);
   }
 
   @Override
-  public ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter, Document doc) {
-    ThreadState threadState = threadBindings.get(requestingThread);
+  public DocumentsWriterPerThread getAndLock(Thread requestingThread, DocumentsWriter documentsWriter, Document doc) {
+    DocumentsWriterPerThread threadState = threadBindings.get(requestingThread);
     if (threadState != null) {
       if (threadState.tryLock()) {
         return threadState;
@@ -23,17 +23,17 @@
     }
 
     // find the state that has minimum amount of threads waiting
-    Iterator<ThreadState> it = getActivePerThreadsIterator();
-    ThreadState minThreadState = null;
+    Iterator<DocumentsWriterPerThread> it = getActivePerThreadsIterator();
+    DocumentsWriterPerThread minThreadState = null;
     while (it.hasNext()) {
-      ThreadState state = it.next();
+      DocumentsWriterPerThread state = it.next();
       if (minThreadState == null || state.getQueueLength() < minThreadState.getQueueLength()) {
         minThreadState = state;
       }
     }
 
     if (minThreadState == null || minThreadState.hasQueuedThreads()) {
-      ThreadState newState = newThreadState();
+      DocumentsWriterPerThread newState = newThreadState();
       if (newState != null) {
         minThreadState = newState;
         threadBindings.put(requestingThread, newState);
@@ -45,7 +45,7 @@
   }
 
   @Override
-  public void clearThreadBindings(ThreadState perThread) {
+  public void clearThreadBindings(DocumentsWriterPerThread perThread) {
     threadBindings.clear();
   }
 
