Index: lucene/CHANGES.txt
===================================================================
--- lucene/CHANGES.txt	(revision 1155278)
+++ lucene/CHANGES.txt	(working copy)
@@ -38,6 +38,9 @@
   this is fixed if you supply Version >= 3.4 If you supply a previous 
   lucene version, you get the old buggy behavior for backwards compatibility.  
   (Trejkaz, Robert Muir)
+  
+* LUCENE-3368: IndexWriter commits segments without applying their buffered
+  deletes when flushing concurrently. (Simon Willnauer, Mike McCandless)
 
 New Features
 
Index: lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
===================================================================
--- lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java	(revision 1155278)
+++ lucene/backwards/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java	(working copy)
@@ -486,7 +486,7 @@
       fail("fake disk full IOExceptions not hit");
     } catch (IOException ioe) {
       // expected
-      assertTrue(ftdm.didFail1);
+      assertTrue(ftdm.didFail1 || ftdm.didFail2);
     }
     _TestUtil.checkIndex(dir);
     ftdm.clearDoFail();
Index: lucene/src/java/org/apache/lucene/index/IndexWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/IndexWriter.java	(revision 1155278)
+++ lucene/src/java/org/apache/lucene/index/IndexWriter.java	(working copy)
@@ -3360,21 +3360,54 @@
    *  only "stick" if there are actually changes in the
    *  index to commit.
    */
-  public final void prepareCommit(Map<String,String> commitUserData) throws CorruptIndexException, IOException {
+  public final void prepareCommit(Map<String, String> commitUserData)
+      throws CorruptIndexException, IOException {
 
     if (hitOOM) {
-      throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot commit");
+      throw new IllegalStateException(
+          "this writer hit an OutOfMemoryError; cannot commit");
     }
 
     if (pendingCommit != null)
-      throw new IllegalStateException("prepareCommit was already called with no corresponding call to commit");
+      throw new IllegalStateException(
+          "prepareCommit was already called with no corresponding call to commit");
 
     if (infoStream != null)
       message("prepareCommit: flush");
 
-    flush(true, true);
+    ensureOpen(false);
+    final boolean anySegmentsFlushed;
+    final SegmentInfos toCommit;
+    synchronized (this) {
+      anySegmentsFlushed = doFlush(true);
+      readerPool.commit(segmentInfos);
+      // Must clone the segmentInfos while we still
+      // hold fullFlushLock and while sync'd so that
+      // no partial changes (eg a delete w/o
+      // corresponding add from an updateDocument) can
+      // sneak into the commit point:
+      toCommit = (SegmentInfos) segmentInfos.clone();
+      pendingCommitChangeCount = changeCount;
+      // This protects the segmentInfos we are now going
+      // to commit. This is important in case, eg, while
+      // we are trying to sync all referenced files, a
+      // merge completes which would otherwise have
+      // removed the files we are now syncing.
+      deleter.incRef(toCommit, false);
+    }
+    boolean success = false;
+    try {
+      if (anySegmentsFlushed) {
+        maybeMerge();
+      } 
+      success = true;
+    } finally {
+      if (!success) {
+        deleter.decRef(toCommit);
+      }
+    }
 
-    startCommit(commitUserData);
+    startCommit(toCommit, commitUserData);
   }
 
   // Used only by commit, below; lock order is commitLock -> IW
@@ -3602,6 +3635,7 @@
       } else if (infoStream != null) {
         message("don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed());
       }
+      
 
       doAfterFlush();
       flushCount.incrementAndGet();
@@ -4460,7 +4494,7 @@
    *  if it wasn't already.  If that succeeds, then we
    *  prepare a new segments_N file but do not fully commit
    *  it. */
-  private void startCommit(Map<String,String> commitUserData) throws IOException {
+  private void startCommit(SegmentInfos toSync, Map<String,String> commitUserData) throws IOException {
 
     assert testPoint("startStartCommit");
     assert pendingCommit == null;
@@ -4474,17 +4508,16 @@
       if (infoStream != null)
         message("startCommit(): start");
 
-      final SegmentInfos toSync;
-      final long myChangeCount;
 
       synchronized(this) {
 
         assert lastCommitChangeCount <= changeCount;
-        myChangeCount = changeCount;
         
-        if (changeCount == lastCommitChangeCount) {
-          if (infoStream != null)
+        if (pendingCommitChangeCount == lastCommitChangeCount) {
+          if (infoStream != null) {
             message("  skip startCommit(): no changes pending");
+          }
+          deleter.decRef(toSync);
           return;
         }
         
@@ -4493,22 +4526,13 @@
         // referenced by toSync, in the background.
         
         if (infoStream != null)
-          message("startCommit index=" + segString(segmentInfos) + " changeCount=" + changeCount);
+          message("startCommit index=" + segString(toSync) + " changeCount=" + changeCount);
 
-        readerPool.commit(segmentInfos);
-        toSync = (SegmentInfos) segmentInfos.clone();
-
         assert filesExist(toSync);
         
-        if (commitUserData != null)
+        if (commitUserData != null) {
           toSync.setUserData(commitUserData);
-        
-        // This protects the segmentInfos we are now going
-        // to commit.  This is important in case, eg, while
-        // we are trying to sync all referenced files, a
-        // merge completes which would otherwise have
-        // removed the files we are now syncing.
-        deleter.incRef(toSync, false);
+        }
       }
 
       assert testPoint("midStartCommit");
@@ -4532,14 +4556,13 @@
           // (this method unwinds everything it did on
           // an exception)
           toSync.prepareCommit(directory);
-
+          pendingCommitSet = true;
           pendingCommit = toSync;
-          pendingCommitSet = true;
-          pendingCommitChangeCount = myChangeCount;
         }
 
-        if (infoStream != null)
+        if (infoStream != null) {
           message("done all syncs");
+        }
 
         assert testPoint("midStartCommitSuccess");
 
Index: lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
===================================================================
--- lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java	(revision 1155278)
+++ lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java	(working copy)
@@ -486,7 +486,7 @@
       fail("fake disk full IOExceptions not hit");
     } catch (IOException ioe) {
       // expected
-      assertTrue(ftdm.didFail1);
+      assertTrue(ftdm.didFail1 || ftdm.didFail2);
     }
     _TestUtil.checkIndex(dir);
     ftdm.clearDoFail();
