LUCENE-5958: OOM or exceptions during checkpoint make IndexWriter have a bad day

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/branch_4x@1625859 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 8eeb4a4..22c2f71 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -75,6 +75,11 @@
 * LUCENE-5934: Fix backwards compatibility for 4.0 indexes.
   (Ian Lea, Uwe Schindler, Robert Muir, Ryan Ernst)
 
+* LUCENE-5958: Don't let exceptions during checkpoint corrupt the index. 
+  Refactor existing OOM handling too, so you don't need to handle OOM special
+  for every IndexWriter method: instead such disasters will cause IW to close itself
+  defensively. (Robert Muir, Mike McCandless)
+
 Tests
 
 * LUCENE-5936: Add backcompat checks to verify what is tested matches known versions
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
index 62bccd2..c24912c 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
@@ -27,6 +27,7 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.regex.Matcher;
 
 import org.apache.lucene.store.AlreadyClosedException;
@@ -110,7 +111,6 @@
    *  infoStream is enabled */
   public static boolean VERBOSE_REF_COUNTS = false;
 
-  // Used only for assert
   private final IndexWriter writer;
 
   // called only from assert
@@ -127,6 +127,7 @@
    */
   public IndexFileDeleter(Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos,
                           InfoStream infoStream, IndexWriter writer, boolean initialIndexExists) throws IOException {
+    Objects.requireNonNull(writer);
     this.infoStream = infoStream;
     this.writer = writer;
 
@@ -350,10 +351,10 @@
   }
 
   private void ensureOpen() throws AlreadyClosedException {
-    if (writer == null) {
-      throw new AlreadyClosedException("this IndexWriter is closed");
-    } else {
-      writer.ensureOpen(false);
+    writer.ensureOpen(false);
+    // since we allow 'closing' state, we must still check this, we could be closing because we hit e.g. OOM
+    if (writer.tragedy != null) {
+      throw new AlreadyClosedException("refusing to delete any files: this IndexWriter hit an unrecoverable exception", writer.tragedy);
     }
   }
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 68cf77e..25a433a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -149,16 +149,11 @@
   {@link ConcurrentMergeScheduler}. </p>
 
   <a name="OOME"></a><p><b>NOTE</b>: if you hit an
-  OutOfMemoryError then IndexWriter will quietly record this
-  fact and block all future segment commits.  This is a
+  OutOfMemoryError, or disaster strikes during a checkpoint
+  then IndexWriter will close itself.  This is a
   defensive measure in case any internal state (buffered
-  documents and deletions) were corrupted.  Any subsequent
-  calls to {@link #commit()} will throw an
-  IllegalStateException.  The only course of action is to
-  call {@link #close()}, which internally will call {@link
-  #rollback()}, to undo any changes to the index since the
-  last commit.  You can also just call {@link #rollback()}
-  directly.</p>
+  documents, deletions, reference counts) were corrupted.  
+  Any subsequent calls will throw an AlreadyClosedException.</p>
 
   <a name="thread-safety"></a><p><b>NOTE</b>: {@link
   IndexWriter} instances are completely thread
@@ -248,7 +243,8 @@
    * IndexWriterConfig#setInfoStream(InfoStream)}).
    */
   public final static int MAX_TERM_LENGTH = DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8;
-  volatile private boolean hitOOM;
+  // when unrecoverable disaster strikes, we populate this with the reason that we had to close IndexWriter
+  volatile Throwable tragedy;
 
   private final Directory directory;  // where this index resides
   private final Analyzer analyzer;    // how to analyze text
@@ -430,7 +426,7 @@
             }
           }
         } catch (OutOfMemoryError oom) {
-          handleOOM(oom, "getReader");
+          tragicEvent(oom, "getReader");
           // never reached but javac disagrees:
           return null;
         } finally {
@@ -439,10 +435,12 @@
               infoStream.message("IW", "hit exception during NRT reader");
             }
           }
-          // Done: finish the full flush!
-          docWriter.finishFullFlush(success);
-          processEvents(false, true);
-          doAfterFlush();
+          if (tragedy == null) {
+            // Done: finish the full flush! (unless we hit OOM or something)
+            docWriter.finishFullFlush(success);
+            processEvents(false, true);
+            doAfterFlush();
+          }
         }
       }
       if (anySegmentFlushed) {
@@ -703,7 +701,7 @@
    */
   protected final void ensureOpen(boolean failIfClosing) throws AlreadyClosedException {
     if (closed || (failIfClosing && closing)) {
-      throw new AlreadyClosedException("this IndexWriter is closed");
+      throw new AlreadyClosedException("this IndexWriter is closed", tragedy);
     }
   }
 
@@ -1120,7 +1118,7 @@
       }
       assert docWriter.perThreadPool.numDeactivatedThreadStates() == docWriter.perThreadPool.getMaxThreadStates() : "" +  docWriter.perThreadPool.numDeactivatedThreadStates() + " " +  docWriter.perThreadPool.getMaxThreadStates();
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "closeInternal");
+      tragicEvent(oom, "closeInternal");
     } finally {
       synchronized(this) {
         closing = false;
@@ -1232,10 +1230,6 @@
    * replaced with the Unicode replacement character
    * U+FFFD.</p>
    *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
-   *
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
@@ -1251,10 +1245,6 @@
    * index and IndexWriter state after an Exception, and
    * flushing/merging temporary free space requirements.</p>
    *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
-   *
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
@@ -1294,10 +1284,6 @@
    * and will likely break them up.  Use such tools at your
    * own risk!
    *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
-   *
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    *
@@ -1370,7 +1356,7 @@
         }
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "updateDocuments");
+      tragicEvent(oom, "updateDocuments");
     }
   }
 
@@ -1454,10 +1440,6 @@
    * terms. All given deletes are applied and flushed atomically
    * at the same time.
    *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
-   *
    * @param terms array of terms to identify the documents
    * to be deleted
    * @throws CorruptIndexException if the index is corrupt
@@ -1470,7 +1452,7 @@
         processEvents(true, false);
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "deleteDocuments(Term..)");
+      tragicEvent(oom, "deleteDocuments(Term..)");
     }
   }
 
@@ -1478,10 +1460,6 @@
    * Deletes the document(s) matching any of the provided queries.
    * All given deletes are applied and flushed atomically at the same time.
    *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
-   *
    * @param queries array of queries to identify the documents
    * to be deleted
    * @throws CorruptIndexException if the index is corrupt
@@ -1494,7 +1472,7 @@
         processEvents(true, false);
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "deleteDocuments(Query..)");
+      tragicEvent(oom, "deleteDocuments(Query..)");
     }
   }
 
@@ -1505,10 +1483,6 @@
    * by a reader on the same index (flush may happen only after
    * the add).
    *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
-   *
    * @param term the term to identify the document(s) to be
    * deleted
    * @param doc the document to be added
@@ -1527,10 +1501,6 @@
    * by a reader on the same index (flush may happen only after
    * the add).
    *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
-   *
    * @param term the term to identify the document(s) to be
    * deleted
    * @param doc the document to be added
@@ -1556,7 +1526,7 @@
         }
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "updateDocument");
+      tragicEvent(oom, "updateDocument");
     }
   }
 
@@ -1565,11 +1535,6 @@
    * given <code>value</code>. You can only update fields that already exist in
    * the index, not add new fields through this method.
    * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
    * @param term
    *          the term to identify the document(s) to be updated
    * @param field
@@ -1591,7 +1556,7 @@
         processEvents(true, false);
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "updateNumericDocValue");
+      tragicEvent(oom, "updateNumericDocValue");
     }
   }
 
@@ -1604,11 +1569,6 @@
    * <b>NOTE:</b> this method currently replaces the existing value of all
    * affected documents with the new value.
    * 
-   * <p>
-   * <b>NOTE:</b> if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
    * @param term
    *          the term to identify the document(s) to be updated
    * @param field
@@ -1633,7 +1593,7 @@
         processEvents(true, false);
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "updateBinaryDocValue");
+      tragicEvent(oom, "updateBinaryDocValue");
     }
   }
   
@@ -1643,11 +1603,6 @@
    * {@link Term} to the same value. All updates are atomically applied and
    * flushed together.
    * 
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * </p>
-   * 
    * @param updates
    *          the updates to apply
    * @throws CorruptIndexException
@@ -1683,7 +1638,7 @@
         processEvents(true, false);
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "updateDocValues");
+      tragicEvent(oom, "updateDocValues");
     }
   }
   
@@ -1788,10 +1743,6 @@
    * newly created segments will not be merged unless you
    * call forceMerge again.</p>
    *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
-   *
    * <p><b>NOTE</b>: if you call {@link #close(boolean)}
    * with <tt>false</tt>, which aborts all running merges,
    * then any thread still running this method might hit a
@@ -1814,10 +1765,6 @@
    *  all merging completes.  This is only meaningful with a
    *  {@link MergeScheduler} that is able to run merges in
    *  background threads.
-   *
-   *  <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   *  you should immediately close the writer.  See <a
-   *  href="#OOME">above</a> for details.</p>
    */
   public void forceMerge(int maxNumSegments, boolean doWait) throws IOException {
     ensureOpen();
@@ -1859,8 +1806,8 @@
       synchronized(this) {
         while(true) {
 
-          if (hitOOM) {
-            throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete forceMerge");
+          if (tragedy != null) {
+            throw new IllegalStateException("this writer hit an unrecoverable error; cannot complete forceMerge", tragedy);
           }
 
           if (mergeExceptions.size() > 0) {
@@ -1915,10 +1862,6 @@
    *  {@link MergeScheduler} that is able to run merges in
    *  background threads.
    *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
-   *
    * <p><b>NOTE</b>: if you call {@link #close(boolean)}
    * with <tt>false</tt>, which aborts all running merges,
    * then any thread still running this method might hit a
@@ -1955,8 +1898,8 @@
         boolean running = true;
         while(running) {
 
-          if (hitOOM) {
-            throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete forceMergeDeletes");
+          if (tragedy != null) {
+            throw new IllegalStateException("this writer hit an unrecoverable error; cannot complete forceMergeDeletes", tragedy);
           }
 
           // Check each merge that MergePolicy asked us to
@@ -2005,10 +1948,6 @@
    *  <p><b>NOTE</b>: this method first flushes a new
    *  segment (if there are indexed documents), and applies
    *  all buffered deletes.
-   *
-   *  <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   *  you should immediately close the writer.  See <a
-   *  href="#OOME">above</a> for details.</p>
    */
   public void forceMergeDeletes() throws IOException {
     forceMergeDeletes(true);
@@ -2026,10 +1965,6 @@
    * 
    * This method will call the {@link MergePolicy} with
    * {@link MergeTrigger#EXPLICIT}.
-   *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
    */
   public final void maybeMerge() throws IOException {
     maybeMerge(config.getMergePolicy(), MergeTrigger.EXPLICIT, UNBOUNDED_MAX_MERGE_SEGMENTS);
@@ -2054,8 +1989,8 @@
       return false;
     }
 
-    // Do not start new merges if we've hit OOME
-    if (hitOOM) {
+    // Do not start new merges if disaster struck
+    if (tragedy != null) {
       return false;
     }
     boolean newMergesFound = false;
@@ -2216,7 +2151,7 @@
 
       success = true;
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "rollbackInternal");
+      tragicEvent(oom, "rollbackInternal");
     } finally {
       if (!success) {
         // Must not hold IW's lock while closing
@@ -2317,7 +2252,7 @@
             globalFieldNumberMap.clear();
             success = true;
           } catch (OutOfMemoryError oom) {
-            handleOOM(oom, "deleteAll");
+            tragicEvent(oom, "deleteAll");
           } finally {
             if (!success) {
               if (infoStream.isEnabled("IW")) {
@@ -2551,11 +2486,6 @@
    *
    * <p>This requires this index not be among those to be added.
    *
-   * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer. See <a
-   * href="#OOME">above</a> for details.
-   *
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    * @throws LockObtainFailedException if we were unable to
@@ -2654,7 +2584,7 @@
       successTop = true;
 
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "addIndexes(Directory...)");
+      tragicEvent(oom, "addIndexes(Directory...)");
     } finally {
       if (successTop) {
         IOUtils.close(locks);
@@ -2676,10 +2606,6 @@
    * free space required in the Directory, and non-CFS segments on an Exception.
    * 
    * <p>
-   * <b>NOTE</b>: if this method hits an OutOfMemoryError you should immediately
-   * close the writer. See <a href="#OOME">above</a> for details.
-   * 
-   * <p>
    * <b>NOTE:</b> empty segments are dropped by this method and not added to this
    * index.
    * 
@@ -2819,7 +2745,7 @@
         checkpoint();
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "addIndexes(IndexReader...)");
+      tragicEvent(oom, "addIndexes(IndexReader...)");
     }
     maybeMerge();
   }
@@ -2974,10 +2900,6 @@
    * <p>You can also just call {@link #commit()} directly
    *  without prepareCommit first in which case that method
    *  will internally call prepareCommit.
-   *
-   *  <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   *  you should immediately close the writer.  See <a
-   *  href="#OOME">above</a> for details.</p>
    */
   @Override
   public final void prepareCommit() throws IOException {
@@ -2994,8 +2916,8 @@
         infoStream.message("IW", "  index before flush " + segString());
       }
 
-      if (hitOOM) {
-        throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot commit");
+      if (tragedy != null) {
+        throw new IllegalStateException("this writer hit an unrecoverable error; cannot commit", tragedy);
       }
 
       if (pendingCommit != null) {
@@ -3061,7 +2983,7 @@
           }
         }
       } catch (OutOfMemoryError oom) {
-        handleOOM(oom, "prepareCommit");
+        tragicEvent(oom, "prepareCommit");
       }
      
       boolean success = false;
@@ -3134,10 +3056,6 @@
    * loss it may still lose data.  Lucene cannot guarantee
    * consistency on such devices.  </p>
    *
-   * <p><b>NOTE</b>: if this method hits an OutOfMemoryError
-   * you should immediately close the writer.  See <a
-   * href="#OOME">above</a> for details.</p>
-   *
    * @see #prepareCommit
    */
   @Override
@@ -3189,25 +3107,43 @@
 
   private synchronized final void finishCommit() throws IOException {
 
+    boolean success = false;
+    
     if (pendingCommit != null) {
       try {
         if (infoStream.isEnabled("IW")) {
           infoStream.message("IW", "commit: pendingCommit != null");
         }
         pendingCommit.finishCommit(directory);
-        if (infoStream.isEnabled("IW")) {
-          infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getSegmentsFileName() + "\"");
+        success = true;
+        // we committed, if anything goes wrong after this: we are screwed
+        try {
+          if (infoStream.isEnabled("IW")) {
+            infoStream.message("IW", "commit: wrote segments file \"" + pendingCommit.getSegmentsFileName() + "\"");
+          }
+          segmentInfos.updateGeneration(pendingCommit);
+          lastCommitChangeCount = pendingCommitChangeCount;
+          rollbackSegments = pendingCommit.createBackupSegmentInfos();
+          // NOTE: don't use this.checkpoint() here, because
+          // we do not want to increment changeCount:
+          deleter.checkpoint(pendingCommit, true);
+        } catch (Throwable tragedy) {
+          tragicEvent(tragedy, "finishCommit");
         }
-        segmentInfos.updateGeneration(pendingCommit);
-        lastCommitChangeCount = pendingCommitChangeCount;
-        rollbackSegments = pendingCommit.createBackupSegmentInfos();
-        // NOTE: don't use this.checkpoint() here, because
-        // we do not want to increment changeCount:
-        deleter.checkpoint(pendingCommit, true);
       } finally {
         // Matches the incRef done in prepareCommit:
         try {
-          deleter.decRef(filesToCommit);
+          if (success == false || tragedy == null) {
+            try {
+              deleter.decRef(filesToCommit);
+            } catch (Throwable t) {
+              // if the commit succeeded, we are in screwed state
+              // otherwise, throw our original exception
+              if (success) {
+                tragicEvent(tragedy, "finishCommit");
+              }
+            } 
+          }
         } finally {
           filesToCommit = null;
           pendingCommit = null;
@@ -3263,8 +3199,8 @@
   }
 
   private boolean doFlush(boolean applyAllDeletes) throws IOException {
-    if (hitOOM) {
-      throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot flush");
+    if (tragedy != null) {
+      throw new IllegalStateException("this writer hit an unrecoverable error; cannot flush", tragedy);
     }
 
     doBeforeFlush();
@@ -3299,7 +3235,7 @@
         return anySegmentFlushed;
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "doFlush");
+      tragicEvent(oom, "doFlush");
       // never hit
       return false;
     } finally {
@@ -3634,8 +3570,8 @@
 
     assert testPoint("startCommitMerge");
 
-    if (hitOOM) {
-      throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot complete merge");
+    if (tragedy != null) {
+      throw new IllegalStateException("this writer hit an unrecoverable error; cannot complete merge", tragedy);
     }
 
     if (infoStream.isEnabled("IW")) {
@@ -3852,7 +3788,7 @@
         }
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "merge");
+      tragicEvent(oom, "merge");
     }
     if (merge.info != null && !merge.isAborted()) {
       if (infoStream.isEnabled("IW")) {
@@ -3981,8 +3917,8 @@
     assert merge.registerDone;
     assert merge.maxNumSegments == -1 || merge.maxNumSegments > 0;
 
-    if (hitOOM) {
-      throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot merge");
+    if (tragedy != null) {
+      throw new IllegalStateException("this writer hit an unrecoverable error; cannot merge", tragedy);
     }
 
     if (merge.info != null) {
@@ -4512,8 +4448,8 @@
     assert testPoint("startStartCommit");
     assert pendingCommit == null;
 
-    if (hitOOM) {
-      throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot commit");
+    if (tragedy != null) {
+      throw new IllegalStateException("this writer hit an unrecoverable error; cannot commit", tragedy);
     }
 
     try {
@@ -4611,7 +4547,7 @@
         }
       }
     } catch (OutOfMemoryError oom) {
-      handleOOM(oom, "startCommit");
+      tragicEvent(oom, "startCommit");
     }
     assert testPoint("finishStartCommit");
   }
@@ -4663,12 +4599,26 @@
     public abstract void warm(AtomicReader reader) throws IOException;
   }
 
-  private void handleOOM(OutOfMemoryError oom, String location) {
+  private void tragicEvent(Throwable tragedy, String location) {
     if (infoStream.isEnabled("IW")) {
-      infoStream.message("IW", "hit OutOfMemoryError inside " + location);
+      infoStream.message("IW", "hit " + tragedy.getClass().getSimpleName() + " inside " + location);
     }
-    hitOOM = true;
-    throw oom;
+    // its possible you could have a really bad day
+    if (this.tragedy == null) {
+      this.tragedy = tragedy;
+    }
+    // if we are already closed (e.g. called by rollback), this will be a no-op.
+    synchronized(commitLock) {
+      if (closing == false) {
+        try {
+          rollback();
+        } catch (Throwable ignored) {
+          // it would be confusing to addSuppressed here, its unrelated to the disaster,
+          // and its possible our internal state is amiss anyway.
+        }
+      }
+    }
+    IOUtils.reThrowUnchecked(tragedy);
   }
 
   // Used only by assert for testing.  Current points:
diff --git a/lucene/core/src/java/org/apache/lucene/store/AlreadyClosedException.java b/lucene/core/src/java/org/apache/lucene/store/AlreadyClosedException.java
index f1a20cc..83eb7a5 100644
--- a/lucene/core/src/java/org/apache/lucene/store/AlreadyClosedException.java
+++ b/lucene/core/src/java/org/apache/lucene/store/AlreadyClosedException.java
@@ -25,4 +25,8 @@
   public AlreadyClosedException(String message) {
     super(message);
   }
+  
+  public AlreadyClosedException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
index 42e923f..64a6935 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
@@ -357,7 +357,7 @@
   }
 
   /**
-   * Simple utilty method that takes a previously caught
+   * Simple utility method that takes a previously caught
    * {@code Throwable} and rethrows either {@code
    * IOException} or an unchecked exception.  If the
    * argument is null then this method does nothing.
@@ -372,7 +372,7 @@
   }
 
   /**
-   * Simple utilty method that takes a previously caught
+   * Simple utility method that takes a previously caught
    * {@code Throwable} and rethrows it as an unchecked exception.
    * If the argument is null then this method does nothing.
    */
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
index 20d87af..290e9ad 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
@@ -26,6 +26,7 @@
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -480,6 +481,8 @@
       } catch (RuntimeException re) {
         if (re.getMessage().equals("fake fail")) {
           // ok
+        } else if (re instanceof AlreadyClosedException && re.getCause() != null && "fake fail".equals(re.getCause().getMessage())) {
+          break; // our test got unlucky, triggered our strange exception after successful finishCommit, caused a disaster!
         } else {
           throw re;
         }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOutOfMemory.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOutOfMemory.java
index 0ea18a4..87babee 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOutOfMemory.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOutOfMemory.java
@@ -38,6 +38,7 @@
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.TextField;
+import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.MockDirectoryWrapper.Failure;
 import org.apache.lucene.util.BytesRef;
@@ -143,17 +144,9 @@
               } else if (thingToDo == 2 && defaultCodecSupportsFieldUpdates()) {
                 iw.updateBinaryDocValue(new Term("id", Integer.toString(i)), "dv2", new BytesRef(Integer.toString(i+1)));
               }
-            } catch (OutOfMemoryError e) {
-              if (e.getMessage() != null && e.getMessage().startsWith("Fake OutOfMemoryError")) {
-                exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
-                e.printStackTrace(exceptionStream);
-                try {
-                  iw.rollback();
-                } catch (Throwable t) {}
-                continue STARTOVER;
-              } else {
-                Rethrow.rethrow(e);
-              }
+            } catch (OutOfMemoryError | AlreadyClosedException disaster) {
+              getOOM(disaster, iw, exceptionStream);
+              continue STARTOVER;
             }
           } else {
             // block docs
@@ -170,16 +163,8 @@
               if (random().nextBoolean()) {
                 iw.deleteDocuments(new Term("id", Integer.toString(i)), new Term("id", Integer.toString(-i)));
               }
-            } catch (OutOfMemoryError e) {
-              if (e.getMessage() != null && e.getMessage().startsWith("Fake OutOfMemoryError")) {
-                exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
-                e.printStackTrace(exceptionStream);
-              } else {
-                Rethrow.rethrow(e);
-              }
-              try {
-                iw.rollback();
-              } catch (Throwable t) {}
+            } catch (OutOfMemoryError | AlreadyClosedException disaster) {
+              getOOM(disaster, iw, exceptionStream);
               continue STARTOVER;
             }
           }
@@ -201,16 +186,8 @@
               if (DirectoryReader.indexExists(dir)) {
                 TestUtil.checkIndex(dir);
               }
-            } catch (OutOfMemoryError e) {
-              if (e.getMessage() != null && e.getMessage().startsWith("Fake OutOfMemoryError")) {
-                exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
-                e.printStackTrace(exceptionStream);
-              } else {
-                Rethrow.rethrow(e);
-              }
-              try {
-                iw.rollback();
-              } catch (Throwable t) {}
+            } catch (OutOfMemoryError | AlreadyClosedException disaster) {
+              getOOM(disaster, iw, exceptionStream);
               continue STARTOVER;
             }
           }
@@ -218,17 +195,9 @@
         
         try {
           iw.close();
-        } catch (OutOfMemoryError e) {
-          if (e.getMessage() != null && e.getMessage().startsWith("Fake OutOfMemoryError")) {
-            exceptionStream.println("\nTEST: got expected fake exc:" + e.getMessage());
-            e.printStackTrace(exceptionStream);
-            try {
-              iw.rollback();
-            } catch (Throwable t) {}
-            continue STARTOVER;
-          } else {
-            Rethrow.rethrow(e);
-          }
+        } catch (OutOfMemoryError | AlreadyClosedException disaster) {
+          getOOM(disaster, iw, exceptionStream);
+          continue STARTOVER;
         }
       } catch (Throwable t) {
         System.out.println("Unexpected exception: dumping fake-exception-log:...");
@@ -245,6 +214,27 @@
     }
   }
   
+  private OutOfMemoryError getOOM(Throwable disaster, IndexWriter writer, PrintStream log) {
+    Throwable e = disaster;
+    if (e instanceof AlreadyClosedException) {
+      e = e.getCause();
+    }
+    
+    if (e instanceof OutOfMemoryError && e.getMessage() != null && e.getMessage().startsWith("Fake OutOfMemoryError")) {
+      log.println("\nTEST: got expected fake exc:" + e.getMessage());
+      e.printStackTrace(log);
+      // TODO: remove rollback here, and add this assert to ensure "full OOM protection" anywhere IW does writes
+      // assertTrue("hit OOM but writer is still open, WTF: ", writer.isClosed());
+      try {
+        writer.rollback();
+      } catch (Throwable t) {}
+      return (OutOfMemoryError) e;
+    } else {
+      Rethrow.rethrow(disaster);
+      return null; // dead
+    }
+  }
+  
   public void testBasics() throws Exception {
     final Random r = new Random(random().nextLong());
     doTest(new Failure() {
@@ -265,7 +255,6 @@
     });
   }
   
-  @Ignore("LUCENE-5958: not yet")
   public void testCheckpoint() throws Exception {
     final Random r = new Random(random().nextLong());
     doTest(new Failure() {