Index: src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java	(revision 1055743)
+++ src/java/org/apache/lucene/index/DocumentsWriterThreadPool.java	(working copy)
@@ -230,6 +230,10 @@
     return getPerThreadIterator(allThreadStates);
   }
 
+  int getNumThreads() {
+    return allThreadStates.length;
+  }
+  
   private static final Iterator<DocumentsWriterPerThread> getPerThreadIterator(final ThreadState[] localAllThreads) {
     return new Iterator<DocumentsWriterPerThread>() {
       int i = 0;
Index: src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriterPerThread.java	(revision 1055743)
+++ src/java/org/apache/lucene/index/DocumentsWriterPerThread.java	(working copy)
@@ -82,7 +82,7 @@
   };
 
   // Deletes for our still-in-RAM (to be flushed next) segment
-  private SegmentDeletes pendingDeletes = new SegmentDeletes();
+  SegmentDeletes pendingDeletes = new SegmentDeletes();
 
   static class DocState {
     final DocumentsWriterPerThread docWriter;
Index: src/java/org/apache/lucene/index/DocumentsWriter.java
===================================================================
--- src/java/org/apache/lucene/index/DocumentsWriter.java	(revision 1055743)
+++ src/java/org/apache/lucene/index/DocumentsWriter.java	(working copy)
@@ -162,6 +162,7 @@
   final BufferedDeletes bufferedDeletes;
   private final IndexWriter.FlushControl flushControl;
   private final IndexingChain chain;
+  SegmentDeletes defaultDeletes = new SegmentDeletes();
 
   DocumentsWriter(Directory directory, IndexWriter writer, IndexingChain chain, DocumentsWriterThreadPool indexerThreadPool, FieldInfos fieldInfos, BufferedDeletes bufferedDeletes) throws IOException {
     this.directory = directory;
@@ -173,40 +174,85 @@
     this.chain = chain;
     flushControl = writer.flushControl;
   }
-
+  
+  boolean hasThreads() {
+    return threadPool.getNumThreads() > 0;
+  }
+  
   boolean deleteQueries(Query... queries) {
     final boolean doFlush = flushControl.waitUpdate(0, queries.length);
-    Iterator<DocumentsWriterPerThread> it = threadPool.getPerThreadIterator();
-    while (it.hasNext()) {
-      it.next().deleteQueries(queries);
+    
+    if (hasThreads()) {
+      Iterator<DocumentsWriterPerThread> it = threadPool.getPerThreadIterator();
+      while (it.hasNext()) {
+        it.next().deleteQueries(queries);
+      }
+    } else {
+      synchronized (defaultDeletes) {
+        for (Query query : queries) {
+          defaultDeletes.addQuery(query, SegmentDeletes.MAX_INT);
+        }
+      }
     }
     return doFlush;
   }
 
   boolean deleteQuery(Query query) {
     final boolean doFlush = flushControl.waitUpdate(0, 1);
-    Iterator<DocumentsWriterPerThread> it = threadPool.getPerThreadIterator();
-    while (it.hasNext()) {
-      it.next().deleteQuery(query);
+    
+    if (hasThreads()) {
+      Iterator<DocumentsWriterPerThread> it = threadPool.getPerThreadIterator();
+      while (it.hasNext()) {
+        it.next().deleteQuery(query);
+      }
+    } else {
+      synchronized (defaultDeletes) {
+        defaultDeletes.addQuery(query, SegmentDeletes.MAX_INT);
+      }
     }
     return doFlush;
   }
 
   boolean deleteTerms(Term... terms) {
     final boolean doFlush = flushControl.waitUpdate(0, terms.length);
-    Iterator<DocumentsWriterPerThread> it = threadPool.getPerThreadIterator();
-    while (it.hasNext()) {
-      it.next().deleteTerms(terms);
+    
+    if (hasThreads()) {
+      Iterator<DocumentsWriterPerThread> it = threadPool.getPerThreadIterator();
+      while (it.hasNext()) {
+        it.next().deleteTerms(terms);
+      }
+    } else {
+      synchronized (defaultDeletes) {
+        for (Term term : terms) {
+          defaultDeletes.addTerm(term, SegmentDeletes.MAX_INT);
+        }
+      }
     }
     return doFlush;
   }
-
-  boolean deleteTerm(Term term, boolean skipWait) {
-    final boolean doFlush = flushControl.waitUpdate(0, 1, skipWait);
+  
+  void deleteTermNoWait(Term term) {
+    assert hasThreads();
+    
     Iterator<DocumentsWriterPerThread> it = threadPool.getPerThreadIterator();
     while (it.hasNext()) {
       it.next().deleteTerm(term);
     }
+  }
+  
+  boolean deleteTerm(Term term, boolean skipWait) {
+    final boolean doFlush = flushControl.waitUpdate(0, 1, skipWait);
+    
+    if (hasThreads()) {
+      Iterator<DocumentsWriterPerThread> it = threadPool.getPerThreadIterator();
+      while (it.hasNext()) {
+        it.next().deleteTerm(term);
+      }
+    } else {
+      synchronized (defaultDeletes) {
+        defaultDeletes.addTerm(term, SegmentDeletes.MAX_INT);
+      }
+    }
     return doFlush;
   }
 
@@ -322,14 +368,29 @@
     return numDocsInRAM.get() != 0;
     //return numDocsInRAM.get() != 0 || pendingDeletes.any();
   }
-
+  
   // for testing
-  public SegmentDeletes getPendingDeletes() {
-    return null;
-    // nocommit
-    //return pendingDeletes;
+  public int getBufferedDeleteTermsSize() {
+    int total = 0;
+    Iterator<DocumentsWriterPerThread> it = threadPool.getPerThreadIterator();
+    while (it.hasNext()) {
+      DocumentsWriterPerThread dwpt = it.next();
+      total += dwpt.pendingDeletes.terms.size();
+    }
+    return total;
   }
 
+  //for testing
+  public int getNumBufferedDeleteTerms() {
+    int total = 0;
+    Iterator<DocumentsWriterPerThread> it = threadPool.getPerThreadIterator();
+    while (it.hasNext()) {
+      DocumentsWriterPerThread dwpt = it.next();
+      total += dwpt.pendingDeletes.numTermDeletes.get();
+    }
+    return total;
+  }
+  
   public boolean anyDeletions() {
     // nocommit
     return true;
@@ -360,7 +421,7 @@
             synchronized(DocumentsWriter.this) {
               ensureOpen();
               if (delTerm != null) {
-                deleteTerm(delTerm, true);
+                deleteTermNoWait(delTerm);
               }
               perThread.commitDocument();
               numDocsInRAM.incrementAndGet();
@@ -427,7 +488,14 @@
 
   final boolean flushAllThreads(final boolean flushDeletes)
     throws IOException {
-
+    
+    if (flushDeletes) {
+      if (indexWriter.segmentInfos.size() > 0 && defaultDeletes.any()) {
+        bufferedDeletes.pushDeletes(defaultDeletes, indexWriter.segmentInfos.lastElement(), true);
+        defaultDeletes = new SegmentDeletes();
+      }
+    }
+    
     return threadPool.executeAllThreads(this, new DocumentsWriterThreadPool.AllThreadsTask<Boolean>() {
       @Override
       public Boolean process(Iterator<DocumentsWriterPerThread> threadsIterator) throws IOException {
Index: src/java/org/apache/lucene/index/IndexWriter.java
===================================================================
--- src/java/org/apache/lucene/index/IndexWriter.java	(revision 1055743)
+++ src/java/org/apache/lucene/index/IndexWriter.java	(working copy)
@@ -3209,12 +3209,12 @@
 
   // For test purposes.
   final int getBufferedDeleteTermsSize() {
-    return docWriter.getPendingDeletes().terms.size();
+    return docWriter.getBufferedDeleteTermsSize();
   }
 
   // For test purposes.
   final int getNumBufferedDeleteTerms() {
-    return docWriter.getPendingDeletes().numTermDeletes.get();
+    return docWriter.getNumBufferedDeleteTerms();
   }
 
   // utility routines for tests
