You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2015/07/08 21:17:39 UTC

svn commit: r1689940 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/lucene50/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/test/org/apache/lucene/index/ lucene/test-framework/...

Author: mikemccand
Date: Wed Jul  8 19:17:38 2015
New Revision: 1689940

URL: http://svn.apache.org/r1689940
Log:
LUCENE-6616: IW lists files only once on init, IFD no longer suppresses FNFE, IFD deletes segments_N files last

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
    lucene/dev/branches/branch_5x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1689940&r1=1689939&r2=1689940&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Wed Jul  8 19:17:38 2015
@@ -272,6 +272,14 @@ Optimizations
 
 * LUCENE-6617: Reduce heap usage for small FSTs (Mike McCandless)
 
+* LUCENE-6616: IndexWriter now lists the files in the index directory
+  only once on init, and IndexFileDeleter no longer suppresses
+  FileNotFoundException and NoSuchFileException.  This also improves
+  IndexFileDeleter to delete segments_N files last, so that in the
+  presence of a virus checker, the index is never left in a state
+  where an expired segments_N references non-existing files (Robert
+  Muir, Mike McCandless)
+
 Build
 
 * LUCENE-6518: Don't report false thread leaks from IBM J9

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java?rev=1689940&r1=1689939&r2=1689940&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SegmentInfoFormat.java Wed Jul  8 19:17:38 2015
@@ -123,9 +123,10 @@ public class Lucene50SegmentInfoFormat e
   @Override
   public void write(Directory dir, SegmentInfo si, IOContext ioContext) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(si.name, "", Lucene50SegmentInfoFormat.SI_EXTENSION);
-    si.addFile(fileName);
 
     try (IndexOutput output = dir.createOutput(fileName, ioContext)) {
+      // Only add the file once we've successfully created it, else IFD assert can trip:
+      si.addFile(fileName);
       CodecUtil.writeIndexHeader(output, 
                                    Lucene50SegmentInfoFormat.CODEC_NAME, 
                                    Lucene50SegmentInfoFormat.VERSION_CURRENT,

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1689940&r1=1689939&r2=1689940&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Wed Jul  8 19:17:38 2015
@@ -396,7 +396,7 @@ final class DocumentsWriter implements C
     if (state.isActive() && state.dwpt == null) {
       final FieldInfos.Builder infos = new FieldInfos.Builder(
           writer.globalFieldNumberMap);
-      state.dwpt = new DocumentsWriterPerThread(writer.newSegmentName(), directoryOrig,
+      state.dwpt = new DocumentsWriterPerThread(writer, writer.newSegmentName(), directoryOrig,
                                                 directory, config, infoStream, deleteQueue, infos,
                                                 writer.pendingNumDocs, writer.enableTestPoints);
     }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1689940&r1=1689939&r2=1689940&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Wed Jul  8 19:17:38 2015
@@ -157,9 +157,11 @@ class DocumentsWriterPerThread {
   private final AtomicLong pendingNumDocs;
   private final LiveIndexWriterConfig indexWriterConfig;
   private final boolean enableTestPoints;
+  private final IndexWriter indexWriter;
   
-  public DocumentsWriterPerThread(String segmentName, Directory directoryOrig, Directory directory, LiveIndexWriterConfig indexWriterConfig, InfoStream infoStream, DocumentsWriterDeleteQueue deleteQueue,
+  public DocumentsWriterPerThread(IndexWriter writer, String segmentName, Directory directoryOrig, Directory directory, LiveIndexWriterConfig indexWriterConfig, InfoStream infoStream, DocumentsWriterDeleteQueue deleteQueue,
                                   FieldInfos.Builder fieldInfos, AtomicLong pendingNumDocs, boolean enableTestPoints) throws IOException {
+    this.indexWriter = writer;
     this.directoryOrig = directoryOrig;
     this.directory = new TrackingDirectoryWrapper(directory);
     this.fieldInfos = fieldInfos;
@@ -487,7 +489,7 @@ class DocumentsWriterPerThread {
       if (indexWriterConfig.getUseCompoundFile()) {
         Set<String> originalFiles = newSegment.info.files();
         // TODO: like addIndexes, we are relying on createCompoundFile to successfully cleanup...
-        IndexWriter.createCompoundFile(infoStream, new TrackingDirectoryWrapper(directory), newSegment.info, context);
+        indexWriter.createCompoundFile(infoStream, new TrackingDirectoryWrapper(directory), newSegment.info, context);
         filesToDelete.addAll(originalFiles);
         newSegment.info.setUseCompoundFile(true);
       }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1689940&r1=1689939&r2=1689940&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Wed Jul  8 19:17:38 2015
@@ -17,6 +17,12 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.InfoStream;
+
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -32,12 +38,6 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.regex.Matcher;
 
-import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.CollectionUtil;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.InfoStream;
-
 /*
  * This class keeps track of each SegmentInfos instance that
  * is still "live", either because it corresponds to a
@@ -80,7 +80,7 @@ final class IndexFileDeleter implements
   /* Files that we tried to delete but failed (likely
    * because they are open and we are running on Windows),
    * so we will retry them again later: */
-  private Set<String> deletable;
+  private final Set<String> deletable = new HashSet<>();
 
   /* Reference count for all files in the index.
    * Counts how many existing commits reference a file.
@@ -127,7 +127,7 @@ final class IndexFileDeleter implements
    * any files not referenced by any of the commits.
    * @throws IOException if there is a low-level IO error
    */
-  public IndexFileDeleter(Directory directoryOrig, Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos,
+  public IndexFileDeleter(String[] files, Directory directoryOrig, Directory directory, IndexDeletionPolicy policy, SegmentInfos segmentInfos,
                           InfoStream infoStream, IndexWriter writer, boolean initialIndexExists,
                           boolean isReaderInit) throws IOException {
     Objects.requireNonNull(writer);
@@ -149,7 +149,6 @@ final class IndexFileDeleter implements
     long currentGen = segmentInfos.getGeneration();
 
     CommitPoint currentCommitPoint = null;
-    String[] files = directory.listAll();
 
     if (currentSegmentsFile != null) {
       Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
@@ -168,40 +167,17 @@ final class IndexFileDeleter implements
             if (infoStream.isEnabled("IFD")) {
               infoStream.message("IFD", "init: load commit \"" + fileName + "\"");
             }
-            SegmentInfos sis = null;
-            try {
-              sis = SegmentInfos.readCommit(directoryOrig, fileName);
-            } catch (FileNotFoundException | NoSuchFileException e) {
-              // LUCENE-948: on NFS (and maybe others), if
-              // you have writers switching back and forth
-              // between machines, it's very likely that the
-              // dir listing will be stale and will claim a
-              // file segments_X exists when in fact it
-              // doesn't.  So, we catch this and handle it
-              // as if the file does not exist
-              if (infoStream.isEnabled("IFD")) {
-                infoStream.message("IFD", "init: hit FileNotFoundException when loading commit \"" + fileName + "\"; skipping this commit point");
-              }
-            } catch (IOException e) {
-              if (SegmentInfos.generationFromSegmentsFileName(fileName) <= currentGen && directory.fileLength(fileName) > 0) {
-                throw e;
-              } else {
-                // Most likely we are opening an index that
-                // has an aborted "future" commit, so suppress
-                // exc in this case
-              }
+            SegmentInfos sis = SegmentInfos.readCommit(directoryOrig, fileName);
+
+            final CommitPoint commitPoint = new CommitPoint(commitsToDelete, directoryOrig, sis);
+            if (sis.getGeneration() == segmentInfos.getGeneration()) {
+              currentCommitPoint = commitPoint;
             }
-            if (sis != null) {
-              final CommitPoint commitPoint = new CommitPoint(commitsToDelete, directoryOrig, sis);
-              if (sis.getGeneration() == segmentInfos.getGeneration()) {
-                currentCommitPoint = commitPoint;
-              }
-              commits.add(commitPoint);
-              incRef(sis, true);
+            commits.add(commitPoint);
+            incRef(sis, true);
               
-              if (lastSegmentInfos == null || sis.getGeneration() > lastSegmentInfos.getGeneration()) {
-                lastSegmentInfos = sis;
-              }
+            if (lastSegmentInfos == null || sis.getGeneration() > lastSegmentInfos.getGeneration()) {
+              lastSegmentInfos = sis;
             }
           }
         }
@@ -248,6 +224,10 @@ final class IndexFileDeleter implements
       RefCount rc = entry.getValue();
       final String fileName = entry.getKey();
       if (0 == rc.count) {
+        // A segments_N file should never have ref count 0 on init:
+        if (fileName.startsWith(IndexFileNames.SEGMENTS) && fileName.equals(IndexFileNames.OLD_SEGMENTS_GEN) == false) {
+          throw new IllegalStateException("file \"" + fileName + "\" has refCount=0, which should never happen on init");
+        }
         if (infoStream.isEnabled("IFD")) {
           infoStream.message("IFD", "init: removing unreferenced file \"" + fileName + "\"");
         }
@@ -469,17 +449,16 @@ final class IndexFileDeleter implements
         if (infoStream.isEnabled("IFD")) {
           infoStream.message("IFD", "refresh [prefix=" + segmentName + "]: removing newly created unreferenced file \"" + fileName + "\"");
         }
-        deleteFile(fileName);
+        deletable.add(fileName);
       }
     }
+
+    deletePendingFiles();
   }
 
   void refresh() throws IOException {
-    // Set to null so that we regenerate the list of pending
-    // files; else we can accumulate same file more than
-    // once
     assert locked();
-    deletable = null;
+    deletable.clear();
     refresh(null);
   }
 
@@ -522,22 +501,35 @@ final class IndexFileDeleter implements
 
   public void deletePendingFiles() {
     assert locked();
-    if (deletable != null) {
-      Set<String> oldDeletable = deletable;
-      deletable = null;
-      for(String fileName : oldDeletable) {
-        if (infoStream.isEnabled("IFD")) {
-          infoStream.message("IFD", "delete pending file " + fileName);
-        }
-        RefCount rc = refCounts.get(fileName);
-        if (rc != null && rc.count > 0) {
-          // LUCENE-5904: should never happen!  This means we are about to pending-delete a referenced index file
-          assert false: "fileName=" + fileName + " is in pending delete list but also has refCount=" + rc.count;
-        } else {
-          deleteFile(fileName);
+
+    // Clone the set because it will change as we iterate:
+    List<String> toDelete = new ArrayList<>(deletable);
+    
+    // First pass: delete any segments_N files.  We do these first to be certain stale commit points are removed
+    // before we remove any files they reference.  If any delete of segments_N fails, we leave all other files
+    // undeleted so index is never in a corrupt state:
+    for (String fileName : toDelete) {
+      RefCount rc = refCounts.get(fileName);
+      if (rc != null && rc.count > 0) {
+        // LUCENE-5904: should never happen!  This means we are about to pending-delete a referenced index file
+        throw new IllegalStateException("file \"" + fileName + "\" is in pending delete set but has non-zero refCount=" + rc.count);
+      } else if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+        if (deleteFile(fileName) == false) {
+          if (infoStream.isEnabled("IFD")) {
+            infoStream.message("IFD", "failed to remove commit point \"" + fileName + "\"; skipping deletion of all other pending files");
+          }
+          return;
         }
       }
     }
+
+    // Only delete other files if we were able to remove the segments_N files; this way we never
+    // leave a corrupt commit in the index even in the presense of virus checkers:
+    for(String fileName : toDelete) {
+      if (fileName.startsWith(IndexFileNames.SEGMENTS) == false) {
+        deleteFile(fileName);
+      }
+    }
   }
 
   /**
@@ -570,10 +562,6 @@ final class IndexFileDeleter implements
       infoStream.message("IFD", "now checkpoint \"" + writer.segString(writer.toLiveInfos(segmentInfos)) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
     }
 
-    // Try again now to delete any previously un-deletable
-    // files (because they were in use, on Windows):
-    deletePendingFiles();
-
     // Incref the files:
     incRef(segmentInfos, isCommit);
 
@@ -597,6 +585,7 @@ final class IndexFileDeleter implements
       // Save files so we can decr on next checkpoint/commit:
       lastFiles.addAll(segmentInfos.files(false));
     }
+
     if (infoStream.isEnabled("IFD")) {
       long t1 = System.nanoTime();
       infoStream.message("IFD", ((t1-t0)/1000000) + " msec to checkpoint");
@@ -645,6 +634,15 @@ final class IndexFileDeleter implements
       }
     }
 
+    try {
+      deletePendingFiles();
+    } catch (Throwable t) {
+      if (firstThrowable == null) {
+        // Save first exception and throw it in the end, but be sure to finish decRef all files
+        firstThrowable = t;
+      }
+    }
+
     // NOTE: does nothing if firstThrowable is null
     IOUtils.reThrowUnchecked(firstThrowable);
   }
@@ -659,9 +657,14 @@ final class IndexFileDeleter implements
       } catch (Throwable t) {
       }
     }
+
+    try {
+      deletePendingFiles();
+    } catch (Throwable t) {
+    }
   }
 
-  void decRef(String fileName) {
+  private void decRef(String fileName) {
     assert locked();
     RefCount rc = getRefCount(fileName);
     if (infoStream.isEnabled("IFD")) {
@@ -673,7 +676,7 @@ final class IndexFileDeleter implements
       // This file is no longer referenced by any past
       // commit points nor by the in-memory SegmentInfos:
       try {
-        deleteFile(fileName);
+        deletable.add(fileName);
       } finally {
         refCounts.remove(fileName);
       }
@@ -708,16 +711,9 @@ final class IndexFileDeleter implements
     return rc;
   }
 
-  void deleteFiles(List<String> files) {
-    assert locked();
-    for(final String file: files) {
-      deleteFile(file);
-    }
-  }
-
   /** Deletes the specified files, but only if they are new
    *  (have not yet been incref'd). */
-  void deleteNewFiles(Collection<String> files) {
+  void deleteNewFiles(Collection<String> files) throws IOException {
     assert locked();
     for (final String fileName: files) {
       // NOTE: it's very unusual yet possible for the
@@ -728,14 +724,19 @@ final class IndexFileDeleter implements
       // TestCrash.testCrashAfterReopen can hit this:
       if (!refCounts.containsKey(fileName) || refCounts.get(fileName).count == 0) {
         if (infoStream.isEnabled("IFD")) {
-          infoStream.message("IFD", "delete new file \"" + fileName + "\"");
+          infoStream.message("IFD", "will delete new file \"" + fileName + "\"");
         }
-        deleteFile(fileName);
+        deletable.add(fileName);
       }
     }
+
+    deletePendingFiles();
   }
 
-  void deleteFile(String fileName) {
+  /** Returns true if the delete succeeded. Otherwise, the fileName is
+   *  added to the deletable set so we will retry the delete later, and
+   *  we return false. */
+  private boolean deleteFile(String fileName) {
     assert locked();
     ensureOpen();
     try {
@@ -743,7 +744,14 @@ final class IndexFileDeleter implements
         infoStream.message("IFD", "delete \"" + fileName + "\"");
       }
       directory.deleteFile(fileName);
+      deletable.remove(fileName);
+      return true;
     } catch (IOException e) {  // if delete fails
+
+      // IndexWriter should only ask us to delete files it knows it wrote, so if we hit this, something is wrong!
+      assert e instanceof NoSuchFileException == false: "hit unexpected NoSuchFileException: file=" + fileName;
+      assert e instanceof FileNotFoundException == false: "hit unexpected FileNotFoundException: file=" + fileName;
+
       // Some operating systems (e.g. Windows) don't
       // permit a file to be deleted while it is opened
       // for read (e.g. by another process or thread). So
@@ -754,10 +762,8 @@ final class IndexFileDeleter implements
       if (infoStream.isEnabled("IFD")) {
         infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
       }
-      if (deletable == null) {
-        deletable = new HashSet<>();
-      }
-      deletable.add(fileName);                  // add to deletable
+      deletable.add(fileName);
+      return false;
     }
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1689940&r1=1689939&r2=1689940&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Wed Jul  8 19:17:38 2015
@@ -808,6 +808,8 @@ public class IndexWriter implements Clos
       boolean initialIndexExists = true;
       boolean fromReader = false;
 
+      String[] files = directory.listAll();
+
       // Set up our initial SegmentInfos:
       IndexCommit commit = config.getIndexCommit();
 
@@ -898,7 +900,6 @@ public class IndexWriter implements Clos
       } else {
         // Init from either the latest commit point, or an explicit prior commit point:
 
-        String[] files = directory.listAll();
         String lastSegmentsFile = SegmentInfos.getLastCommitSegmentsFileName(files);
         if (lastSegmentsFile == null) {
           throw new IndexNotFoundException("no segments* file found in " + directory + ": files: " + Arrays.toString(files));
@@ -942,8 +943,10 @@ public class IndexWriter implements Clos
 
       // Default deleter (for backwards compatibility) is
       // KeepOnlyLastCommitDeleter:
+
+      // Sync'd is silly here, but IFD asserts we sync'd on the IW instance:
       synchronized(this) {
-        deleter = new IndexFileDeleter(directoryOrig, directory,
+        deleter = new IndexFileDeleter(files, directoryOrig, directory,
                                        config.getIndexDeletionPolicy(),
                                        segmentInfos, infoStream, this,
                                        initialIndexExists, reader != null);
@@ -2507,7 +2510,8 @@ public class IndexWriter implements Clos
       } finally {
         if (!success) {
           for(SegmentCommitInfo sipc : infos) {
-            IOUtils.deleteFilesIgnoringExceptions(directory, sipc.files().toArray(new String[0]));
+            // Safe: these files must exist
+            deleteNewFiles(sipc.files());
           }
         }
       }
@@ -2524,12 +2528,8 @@ public class IndexWriter implements Clos
         } finally {
           if (!success) {
             for(SegmentCommitInfo sipc : infos) {
-              for(String file : sipc.files()) {
-                try {
-                  directory.deleteFile(file);
-                } catch (Throwable t) {
-                }
-              }
+              // Safe: these files must exist
+              deleteNewFiles(sipc.files());
             }
           }
         }
@@ -2642,7 +2642,8 @@ public class IndexWriter implements Clos
       boolean useCompoundFile;
       synchronized(this) { // Guard segmentInfos
         if (stopMerges) {
-          deleter.deleteNewFiles(infoPerCommit.files());
+          // Safe: these files must exist
+          deleteNewFiles(infoPerCommit.files());
           return;
         }
         ensureOpen();
@@ -2660,9 +2661,7 @@ public class IndexWriter implements Clos
         } finally {
           // delete new non cfs files directly: they were never
           // registered with IFD
-          synchronized(this) {
-            deleter.deleteNewFiles(filesToDelete);
-          }
+          deleteNewFiles(filesToDelete);
         }
         info.setUseCompoundFile(true);
       }
@@ -2688,7 +2687,8 @@ public class IndexWriter implements Clos
       // Register the new segment
       synchronized(this) {
         if (stopMerges) {
-          deleter.deleteNewFiles(info.files());
+          // Safe: these files must exist
+          deleteNewFiles(infoPerCommit.files());
           return;
         }
         ensureOpen();
@@ -2720,6 +2720,7 @@ public class IndexWriter implements Clos
 
     boolean success = false;
 
+    Set<String> copiedFiles = new HashSet<>();
     try {
       // Copy the segment's files
       for (String file: info.files()) {
@@ -2728,13 +2729,17 @@ public class IndexWriter implements Clos
         assert !slowFileExists(directory, newFileName): "file \"" + newFileName + "\" already exists; newInfo.files=" + newInfo.files();
 
         directory.copyFrom(info.info.dir, file, newFileName, context);
+        copiedFiles.add(newFileName);
       }
       success = true;
     } finally {
       if (!success) {
-        IOUtils.deleteFilesIgnoringExceptions(directory, newInfo.files().toArray(new String[0]));
+        // Safe: these files must exist
+        deleteNewFiles(copiedFiles);
       }
     }
+
+    assert copiedFiles.equals(newInfoPerCommit.files());
     
     return newInfoPerCommit;
   }
@@ -3483,7 +3488,9 @@ public class IndexWriter implements Clos
       // doing this  makes  MockDirWrapper angry in
       // TestNRTThreads (LUCENE-5434):
       readerPool.drop(merge.info);
-      deleter.deleteNewFiles(merge.info.files());
+
+      // Safe: these files must exist:
+      deleteNewFiles(merge.info.files());
       return false;
     }
 
@@ -3550,7 +3557,8 @@ public class IndexWriter implements Clos
     if (dropSegment) {
       assert !segmentInfos.contains(merge.info);
       readerPool.drop(merge.info);
-      deleter.deleteNewFiles(merge.info.files());
+      // Safe: these files must exist
+      deleteNewFiles(merge.info.files());
     }
 
     boolean success = false;
@@ -3653,9 +3661,10 @@ public class IndexWriter implements Clos
         }
       } finally {
         synchronized(this) {
+
           mergeFinish(merge);
 
-          if (!success) {
+          if (success == false) {
             if (infoStream.isEnabled("IW")) {
               infoStream.message("IW", "hit exception during merge");
             }
@@ -4133,31 +4142,27 @@ public class IndexWriter implements Clos
         try {
           createCompoundFile(infoStream, trackingCFSDir, merge.info.info, context);
           success = true;
-        } catch (IOException ioe) {
+        } catch (Throwable t) {
           synchronized(this) {
             if (merge.rateLimiter.getAbort()) {
-              // This can happen if rollback or close(false)
-              // is called -- fall through to logic below to
-              // remove the partially created CFS:
+              // This can happen if rollback is called while we were building
+              // our CFS -- fall through to logic below to remove the non-CFS
+              // merged files:
+              if (infoStream.isEnabled("IW")) {
+                infoStream.message("IW", "hit merge abort exception creating compound file during merge");
+              }
+              return 0;
             } else {
-              handleMergeException(ioe, merge);
+              handleMergeException(t, merge);
             }
           }
-        } catch (Throwable t) {
-          handleMergeException(t, merge);
         } finally {
-          if (!success) {
+          if (success == false) {
             if (infoStream.isEnabled("IW")) {
               infoStream.message("IW", "hit exception creating compound file during merge");
             }
-
-            synchronized(this) {
-              Set<String> cfsFiles = new HashSet<>(trackingCFSDir.getCreatedFiles());
-              for (String cfsFile : cfsFiles) {
-                deleter.deleteFile(cfsFile);
-              }
-              deleter.deleteNewFiles(merge.info.files());
-            }
+            // Safe: these files must exist
+            deleteNewFiles(merge.info.files());
           }
         }
 
@@ -4170,16 +4175,14 @@ public class IndexWriter implements Clos
 
           // delete new non cfs files directly: they were never
           // registered with IFD
-          deleter.deleteNewFiles(filesToRemove);
+          deleteNewFiles(filesToRemove);
 
           if (merge.rateLimiter.getAbort()) {
             if (infoStream.isEnabled("IW")) {
               infoStream.message("IW", "abort merge after building CFS");
             }
-            Set<String> cfsFiles = new HashSet<>(trackingCFSDir.getCreatedFiles());
-            for (String cfsFile : cfsFiles) {
-              deleter.deleteFile(cfsFile);
-            }
+            // Safe: these files must exist
+            deleteNewFiles(merge.info.files());
             return 0;
           }
         }
@@ -4202,9 +4205,8 @@ public class IndexWriter implements Clos
         success2 = true;
       } finally {
         if (!success2) {
-          synchronized(this) {
-            deleter.deleteNewFiles(merge.info.files());
-          }
+          // Safe: these files must exist
+          deleteNewFiles(merge.info.files());
         }
       }
 
@@ -4241,7 +4243,7 @@ public class IndexWriter implements Clos
     } finally {
       // Readers are already closed in commitMerge if we didn't hit
       // an exc:
-      if (!success) {
+      if (success == false) {
         closeMergeReaders(merge, true);
       }
     }
@@ -4651,8 +4653,7 @@ public class IndexWriter implements Clos
    * deletion files, this SegmentInfo must not reference such files when this
    * method is called, because they are not allowed within a compound file.
    */
-  static final void createCompoundFile(InfoStream infoStream, TrackingDirectoryWrapper directory, final SegmentInfo info, IOContext context)
-          throws IOException {
+  final void createCompoundFile(InfoStream infoStream, TrackingDirectoryWrapper directory, final SegmentInfo info, IOContext context) throws IOException {
 
     // maybe this check is not needed, but why take the risk?
     if (!directory.getCreatedFiles().isEmpty()) {
@@ -4669,16 +4670,13 @@ public class IndexWriter implements Clos
       success = true;
     } finally {
       if (!success) {
-        Set<String> cfsFiles = new HashSet<>(directory.getCreatedFiles());
-        for (String file : cfsFiles) {
-          IOUtils.deleteFilesIgnoringExceptions(directory, file);
-        }
+        // Safe: these files must exist
+        deleteNewFiles(directory.getCreatedFiles());
       }
     }
 
     // Replace all previous files with the CFS/CFE files:
-    Set<String> siFiles = new HashSet<>(directory.getCreatedFiles());
-    info.setFiles(siFiles);
+    info.setFiles(new HashSet<>(directory.getCreatedFiles()));
   }
   
   /**
@@ -4779,7 +4777,7 @@ public class IndexWriter implements Clos
    *  (can be opened), false if it cannot be opened, and
    *  (unlike Java's File.exists) throws IOException if
    *  there's some unexpected error. */
-  private static boolean slowFileExists(Directory dir, String fileName) throws IOException {
+  static boolean slowFileExists(Directory dir, String fileName) throws IOException {
     try {
       dir.openInput(fileName, IOContext.DEFAULT).close();
       return true;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestDoc.java?rev=1689940&r1=1689939&r2=1689940&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestDoc.java Wed Jul  8 19:17:38 2015
@@ -235,7 +235,7 @@ public class TestDoc extends LuceneTestC
       
     if (useCompoundFile) {
       Collection<String> filesToDelete = si.files();
-      IndexWriter.createCompoundFile(InfoStream.getDefault(), new TrackingDirectoryWrapper(dir), si, newIOContext(random()));
+      codec.compoundFormat().write(dir, si, context);
       si.setUseCompoundFile(true);
       for (final String fileToDelete : filesToDelete) {
         si1.info.dir.deleteFile(fileToDelete);

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1689940&r1=1689939&r2=1689940&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Wed Jul  8 19:17:38 2015
@@ -1467,33 +1467,6 @@ public class TestIndexWriter extends Luc
     dir.close();
   }
 
-  public void testNoSegmentFile() throws IOException {
-    BaseDirectoryWrapper dir = newDirectory(random(), NoLockFactory.INSTANCE);
-    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                           .setMaxBufferedDocs(2));
-
-    Document doc = new Document();
-    FieldType customType = new FieldType(TextField.TYPE_STORED);
-    customType.setStoreTermVectors(true);
-    customType.setStoreTermVectorPositions(true);
-    customType.setStoreTermVectorOffsets(true);
-    doc.add(newField("c", "val", customType));
-    w.addDocument(doc);
-    w.addDocument(doc);
-    IndexWriter w2 = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                            .setMaxBufferedDocs(2)
-                                            .setOpenMode(OpenMode.CREATE));
-
-    w2.close();
-    // If we don't do that, the test fails on Windows
-    w.rollback();
-
-    // This test leaves only segments.gen, which causes
-    // DirectoryReader.indexExists to return true:
-    dir.setCheckIndexOnClose(false);
-    dir.close();
-  }
-
   public void testNoUnwantedTVFiles() throws Exception {
 
     Directory dir = newDirectory();
@@ -2238,7 +2211,9 @@ public class TestIndexWriter extends Luc
       BaseDirectoryWrapper dir = newDirectory();
 
       // Create a corrupt first commit:
-      dir.createOutput("segments_0", IOContext.DEFAULT).close();
+      dir.createOutput(IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS,
+                                                             "",
+                                                             0), IOContext.DEFAULT).close();
 
       IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
       int mode = i/2;
@@ -2260,9 +2235,6 @@ public class TestIndexWriter extends Luc
         } else {
           new IndexWriter(dir, iwc).rollback();
         }
-        if (mode != 0) {
-          fail("expected exception");
-        }
       } catch (IOException ioe) {
         // OpenMode.APPEND should throw an exception since no
         // index exists:

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java?rev=1689940&r1=1689939&r2=1689940&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java Wed Jul  8 19:17:38 2015
@@ -227,7 +227,7 @@ public class TestIndexWriterOnDiskFull e
     }
 
     for(int iter=0;iter<3;iter++) {
-      
+
       if (VERBOSE) {
         System.out.println("TEST: iter=" + iter);
       }
@@ -251,7 +251,7 @@ public class TestIndexWriterOnDiskFull e
       
       while(!done) {
         if (VERBOSE) {
-          System.out.println("TEST: cycle...");
+          System.out.println("\nTEST: cycle...");
         }
         
         // Make a new dir that will enforce disk usage:

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java?rev=1689940&r1=1689939&r2=1689940&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java Wed Jul  8 19:17:38 2015
@@ -151,7 +151,7 @@ public abstract class BaseLockFactoryTes
   // no unexpected exceptions are raised:
   public void testStressLocks() throws Exception {
     Directory dir = getDirectory(createTempDir());
-    
+
     // First create a 1 doc index:
     IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())).setOpenMode(OpenMode.CREATE));
     addDoc(w);
@@ -190,6 +190,9 @@ public abstract class BaseLockFactoryTes
     public void run() {
       IndexWriter writer = null;
       for(int i=0;i<this.numIteration;i++) {
+        if (VERBOSE) {
+          System.out.println("TEST: WriterThread iter=" + i);
+        }
         try {
           writer = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())).setOpenMode(OpenMode.APPEND));
         } catch (LockObtainFailedException e) {