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 2016/02/03 10:23:17 UTC

[4/5] lucene-solr git commit: migrate current patch from svn

migrate current patch from svn


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/84f44589
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/84f44589
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/84f44589

Branch: refs/heads/lucene-6835
Commit: 84f44589f6f91250666b5f66f7c7667d347e1e3a
Parents: a48ba50
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 2 15:53:16 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 2 15:53:16 2016 -0500

----------------------------------------------------------------------
 .../lucene/analysis/hunspell/Dictionary.java    |   4 +-
 .../analysis/hunspell/Test64kAffixes.java       |   4 +-
 .../analysis/hunspell/TestAllDictionaries.java  |   6 +-
 .../analysis/hunspell/TestAllDictionaries2.java |   6 -
 .../analysis/hunspell/TestDictionary.java       |   6 +-
 .../hunspell/TestHunspellStemFilter.java        |   6 +-
 .../index/TestBackwardsCompatibility.java       |   6 -
 .../simpletext/SimpleTextCompoundFormat.java    |   2 +-
 .../codecs/lucene50/Lucene50CompoundReader.java |  16 +-
 .../apache/lucene/index/IndexFileDeleter.java   | 128 +--
 .../org/apache/lucene/index/IndexWriter.java    |  12 +-
 .../index/PersistentSnapshotDeletionPolicy.java |   2 +-
 .../java/org/apache/lucene/store/Directory.java |   6 +-
 .../org/apache/lucene/store/FSDirectory.java    | 114 ++-
 .../lucene/store/FileSwitchDirectory.java       |  17 +-
 .../apache/lucene/store/FilterDirectory.java    |   4 +-
 .../store/LockValidatingDirectoryWrapper.java   |   4 +-
 .../org/apache/lucene/store/MMapDirectory.java  |   1 +
 .../org/apache/lucene/store/NIOFSDirectory.java |   2 +-
 .../lucene/store/NRTCachingDirectory.java       |  27 +-
 .../org/apache/lucene/store/RAMDirectory.java   |  16 +-
 .../apache/lucene/store/SimpleFSDirectory.java  |   1 +
 .../lucene/store/TrackingDirectoryWrapper.java  |   9 +-
 .../java/org/apache/lucene/util/IOUtils.java    |  34 +-
 .../org/apache/lucene/util/bkd/BKDWriter.java   |   7 +-
 .../lucene/util/bkd/OfflinePointWriter.java     |   4 +-
 .../org/apache/lucene/index/TestAddIndexes.java |   2 -
 .../index/TestAllFilesCheckIndexHeader.java     |   5 -
 .../index/TestAllFilesDetectTruncation.java     |   5 -
 .../apache/lucene/index/TestAtomicUpdate.java   |   1 -
 .../index/TestBinaryDocValuesUpdates.java       |   4 -
 .../lucene/index/TestCodecHoldsOpenFiles.java   |  11 +-
 .../index/TestConcurrentMergeScheduler.java     |   4 -
 .../apache/lucene/index/TestDeletionPolicy.java |  44 +-
 .../index/TestDemoParallelLeafReader.java       |  13 +-
 .../lucene/index/TestDirectoryReader.java       |  54 +-
 .../lucene/index/TestDirectoryReaderReopen.java |  15 +-
 .../test/org/apache/lucene/index/TestDoc.java   |   9 +-
 .../apache/lucene/index/TestFieldsReader.java   |  65 +-
 .../lucene/index/TestIndexFileDeleter.java      |   4 +-
 .../apache/lucene/index/TestIndexWriter.java    | 928 +++++++++----------
 .../lucene/index/TestIndexWriterCommit.java     |  20 -
 .../lucene/index/TestIndexWriterDelete.java     |   9 +-
 .../lucene/index/TestIndexWriterExceptions.java |  17 +-
 .../lucene/index/TestIndexWriterForceMerge.java |   1 -
 .../lucene/index/TestIndexWriterFromReader.java |  17 +-
 .../lucene/index/TestIndexWriterOnDiskFull.java |   1 -
 .../lucene/index/TestNRTReaderCleanup.java      |  14 +-
 .../apache/lucene/index/TestNeverDelete.java    |   8 -
 .../index/TestNumericDocValuesUpdates.java      |   4 -
 .../apache/lucene/index/TestOmitPositions.java  |   5 +-
 .../org/apache/lucene/index/TestOmitTf.java     |   5 +-
 .../TestPersistentSnapshotDeletionPolicy.java   |   2 +-
 .../apache/lucene/index/TestRollingUpdates.java |   4 -
 .../index/TestSnapshotDeletionPolicy.java       |  22 +-
 .../lucene/index/TestSwappedIndexFiles.java     |   9 -
 .../apache/lucene/search/TestPointQueries.java  |  13 +-
 .../lucene/store/TestBufferedIndexInput.java    |  86 +-
 .../org/apache/lucene/store/TestDirectory.java  |  22 +-
 .../lucene/store/TestFileSwitchDirectory.java   |   1 -
 .../lucene/store/TestNativeFSLockFactory.java   |  23 +-
 .../apache/lucene/store/TestRAMDirectory.java   |   1 -
 .../lucene/store/TestSimpleFSLockFactory.java   |   5 +-
 .../store/TestTrackingDirectoryWrapper.java     |   2 +-
 .../apache/lucene/util/TestOfflineSorter.java   |  30 +-
 .../org/apache/lucene/util/bkd/TestBKD.java     |  11 +-
 .../org/apache/lucene/util/fst/Test2BFST.java   |   8 +-
 .../org/apache/lucene/util/fst/TestFSTs.java    |   5 +-
 .../lucene/util/packed/TestPackedInts.java      |   3 +-
 .../apache/lucene/util/fst/TestFSTsMisc.java    |   1 -
 .../IndexAndTaxonomyReplicationClientTest.java  |  17 +-
 .../IndexAndTaxonomyRevisionTest.java           |   8 -
 .../lucene/replicator/IndexRevisionTest.java    |   4 -
 .../lucene/replicator/LocalReplicatorTest.java  |  90 +-
 .../lucene/util/BaseGeoPointTestCase.java       |   9 -
 .../org/apache/lucene/geo3d/TestGeo3DPoint.java |  11 +-
 .../suggest/analyzing/AnalyzingSuggester.java   |   2 +-
 .../search/suggest/fst/ExternalRefSorter.java   |   3 +-
 .../search/suggest/fst/FSTCompletionLookup.java |   2 +-
 .../lucene/search/suggest/PersistenceTest.java  |   6 +-
 .../search/suggest/TestInputIterator.java       |   6 +-
 .../analyzing/AnalyzingSuggesterTest.java       |   6 +-
 .../suggest/analyzing/FuzzySuggesterTest.java   |   6 +-
 .../search/suggest/fst/BytesRefSortersTest.java |   3 -
 .../search/suggest/fst/FSTCompletionTest.java   |   6 +-
 .../search/suggest/fst/WFSTCompletionTest.java  |   6 +-
 .../index/BaseCompoundFormatTestCase.java       |   7 +-
 .../index/BaseIndexFileFormatTestCase.java      |  11 +-
 .../lucene/index/BasePointFormatTestCase.java   |  11 -
 .../index/BasePostingsFormatTestCase.java       |   2 -
 .../lucene/index/RandomPostingsTester.java      |   1 -
 .../ThreadedIndexingAndSearchingTestCase.java   |   1 -
 .../apache/lucene/mockfile/VirusCheckingFS.java |  65 ++
 .../lucene/store/BaseDirectoryTestCase.java     |  10 +-
 .../lucene/store/BaseLockFactoryTestCase.java   |  50 +-
 .../lucene/store/MockDirectoryWrapper.java      | 206 ++--
 .../util/TestRuleTemporaryFilesCleanup.java     |  19 +
 .../java/org/apache/lucene/util/TestUtil.java   |  46 +
 .../org/apache/lucene/util/fst/FSTTester.java   |   2 +-
 .../TestCompressingStoredFieldsFormat.java      |   4 -
 .../lucene/mockfile/TestVirusCheckingFS.java    |  62 ++
 .../apache/lucene/mockfile/TestWindowsFS.java   |   2 +-
 .../lucene/store/TestMockDirectoryWrapper.java  |   1 -
 .../org/apache/solr/core/DirectoryFactory.java  |   7 +-
 .../org/apache/solr/handler/IndexFetcher.java   |   4 +-
 .../org/apache/solr/handler/RestoreCore.java    |   3 +-
 .../solr/store/blockcache/BlockDirectory.java   |   9 +-
 .../apache/solr/store/hdfs/HdfsDirectory.java   |  10 +-
 .../solr/store/hdfs/HdfsDirectoryTest.java      |  14 +-
 .../apache/solr/core/MockDirectoryFactory.java  |   3 -
 .../solr/core/MockFSDirectoryFactory.java       |   1 -
 111 files changed, 1245 insertions(+), 1448 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
index d5db839..49c7045 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
@@ -877,7 +877,7 @@ public class Dictionary {
       success = true;
     } finally {
       if (success) {
-        tempDir.deleteFile(unsorted.getName());
+        tempDir.deleteFiles(Collections.singleton(unsorted.getName()));
       } else {
         IOUtils.deleteFilesIgnoringExceptions(tempDir, unsorted.getName());
       }
@@ -966,7 +966,7 @@ public class Dictionary {
       success2 = true;
     } finally {
       if (success2) {
-        tempDir.deleteFile(sorted);
+        tempDir.deleteFiles(Collections.singleton(sorted));
       } else {
         IOUtils.deleteFilesIgnoringExceptions(tempDir, sorted);
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/Test64kAffixes.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/Test64kAffixes.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/Test64kAffixes.java
index 6fa6ecf..20d4c47 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/Test64kAffixes.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/Test64kAffixes.java
@@ -28,6 +28,7 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
 
 /** Tests that &gt; 64k affixes actually works and doesnt overflow some internal int */
 public class Test64kAffixes extends LuceneTestCase {
@@ -54,9 +55,6 @@ public class Test64kAffixes extends LuceneTestCase {
     dictWriter.close();
     
     try (InputStream affStream = Files.newInputStream(affix); InputStream dictStream = Files.newInputStream(dict); Directory tempDir2 = newDirectory()) {
-      if (tempDir2 instanceof MockDirectoryWrapper) {
-        ((MockDirectoryWrapper) tempDir2).setEnableVirusScanner(false);
-      }
       Dictionary dictionary = new Dictionary(tempDir2, "dictionary", affStream, dictStream);
       Stemmer stemmer = new Stemmer(dictionary);
       // drinks should still stem to drink

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java
index 4a66430..5470ded 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java
@@ -215,10 +215,6 @@ public class TestAllDictionaries extends LuceneTestCase {
   }
 
   private Directory getDirectory() {
-    Directory dir = newDirectory();
-    if (dir instanceof MockDirectoryWrapper) {
-      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
-    }
-    return dir;
+    return newDirectory();
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries2.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries2.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries2.java
index efc5d66..1317cfc 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries2.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries2.java
@@ -189,9 +189,6 @@ public class TestAllDictionaries2 extends LuceneTestCase {
         try (InputStream dictionary = Files.newInputStream(dicEntry);
              InputStream affix = Files.newInputStream(affEntry);
              Directory tempDir = newDirectory()) {
-          if (tempDir instanceof MockDirectoryWrapper) {
-            ((MockDirectoryWrapper) tempDir).setEnableVirusScanner(false);
-          }
           Dictionary dic = new Dictionary(tempDir, "dictionary", affix, dictionary);
           System.out.println(tests[i] + "\t" + RamUsageTester.humanSizeOf(dic) + "\t(" +
                              "words=" + RamUsageTester.humanSizeOf(dic.words) + ", " +
@@ -226,9 +223,6 @@ public class TestAllDictionaries2 extends LuceneTestCase {
           try (InputStream dictionary = Files.newInputStream(dicEntry);
                InputStream affix = Files.newInputStream(affEntry);
                Directory tempDir = newDirectory()) {
-            if (tempDir instanceof MockDirectoryWrapper) {
-              ((MockDirectoryWrapper) tempDir).setEnableVirusScanner(false);
-            }
             new Dictionary(tempDir, "dictionary", affix, dictionary);
           } 
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java
index 39feb1d..1d162c0 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java
@@ -260,10 +260,6 @@ public class TestDictionary extends LuceneTestCase {
   }
 
   private Directory getDirectory() {
-    Directory dir = newDirectory();
-    if (dir instanceof MockDirectoryWrapper) {
-      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
-    }
-    return dir;
+    return newDirectory();
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilter.java
----------------------------------------------------------------------
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilter.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilter.java
index 69acc93..5cda81a 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilter.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilter.java
@@ -129,10 +129,6 @@ public class TestHunspellStemFilter extends BaseTokenStreamTestCase {
   }
 
   private static Directory getDirectory() {
-    Directory dir = newDirectory();
-    if (dir instanceof MockDirectoryWrapper) {
-      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
-    }
-    return dir;
+    return newDirectory();
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
index f61599d..87cdea6 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
@@ -1238,12 +1238,6 @@ public class TestBackwardsCompatibility extends LuceneTestCase {
         System.out.println("testUpgradeOldSingleSegmentIndexWithAdditions: index=" +name);
       }
       Directory dir = newDirectory(oldIndexDirs.get(name));
-      if (dir instanceof MockDirectoryWrapper) {
-        // we need to ensure we delete old commits for this test,
-        // otherwise IndexUpgrader gets angry
-        ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
-      }
-
       assertEquals("Original index must be single segment", 1, getNumberOfSegments(dir));
 
       // create a bunch of dummy segments

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
index c994df7..8398f66 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
@@ -148,7 +148,7 @@ public class SimpleTextCompoundFormat extends CompoundFormat {
       public void sync(Collection<String> names) { throw new UnsupportedOperationException(); }
       
       @Override
-      public void deleteFile(String name) { throw new UnsupportedOperationException(); }
+      public void deleteFiles(Collection<String> name) { throw new UnsupportedOperationException(); }
       
       @Override
       public void renameFile(String source, String dest) { throw new UnsupportedOperationException(); }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
index b1cfc5d..a6ccfd4 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
@@ -17,6 +17,13 @@ package org.apache.lucene.codecs.lucene50;
  * limitations under the License.
  */
 
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFileNames;
@@ -29,13 +36,6 @@ import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.util.IOUtils;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
 /**
  * Class for accessing a compound stream.
  * This class implements a directory, but is limited to only read operations.
@@ -161,7 +161,7 @@ final class Lucene50CompoundReader extends Directory {
   /** Not implemented
    * @throws UnsupportedOperationException always: not supported by CFS */
   @Override
-  public void deleteFile(String name) {
+  public void deleteFiles(Collection<String> name) {
     throw new UnsupportedOperationException();
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
----------------------------------------------------------------------
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 e2b29c9..569471e 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
@@ -79,11 +79,6 @@ import org.apache.lucene.util.InfoStream;
 
 final class IndexFileDeleter implements Closeable {
 
-  /* 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 final Set<String> deletable = new HashSet<>();
-
   /* Reference count for all files in the index.
    * Counts how many existing commits reference a file.
    **/
@@ -220,6 +215,7 @@ final class IndexFileDeleter implements Closeable {
     // Now delete anything with ref count at 0.  These are
     // presumably abandoned files eg due to crash of
     // IndexWriter.
+    Set<String> toDelete = new HashSet<>();
     for(Map.Entry<String, RefCount> entry : refCounts.entrySet() ) {
       RefCount rc = entry.getValue();
       final String fileName = entry.getKey();
@@ -231,10 +227,12 @@ final class IndexFileDeleter implements Closeable {
         if (infoStream.isEnabled("IFD")) {
           infoStream.message("IFD", "init: removing unreferenced file \"" + fileName + "\"");
         }
-        deleteFile(fileName);
+        toDelete.add(fileName);
       }
     }
 
+    deleteFiles(toDelete);
+
     // Finally, give policy a chance to remove things on
     // startup:
     policy.onInit(commits);
@@ -425,7 +423,7 @@ final class IndexFileDeleter implements Closeable {
    */
   void refresh() throws IOException {
     assert locked();
-    deletable.clear();
+    Set<String> toDelete = new HashSet<>();
 
     String[] files = directory.listAll();
 
@@ -445,15 +443,15 @@ final class IndexFileDeleter implements Closeable {
         if (infoStream.isEnabled("IFD")) {
           infoStream.message("IFD", "refresh: removing newly created unreferenced file \"" + fileName + "\"");
         }
-        deletable.add(fileName);
+        toDelete.add(fileName);
       }
     }
 
-    deletePendingFiles();
+    deleteFiles(toDelete);
   }
 
   @Override
-  public void close() {
+  public void close() throws IOException {
     // DecRef old files from the last checkpoint, if any:
     assert locked();
 
@@ -464,8 +462,6 @@ final class IndexFileDeleter implements Closeable {
         lastFiles.clear();
       }
     }
-
-    deletePendingFiles();
   }
 
   /**
@@ -489,39 +485,6 @@ final class IndexFileDeleter implements Closeable {
     }
   }
 
-  public void deletePendingFiles() {
-    assert locked();
-
-    // 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);
-      }
-    }
-  }
-
   /**
    * For definition of "check point" see IndexWriter comments:
    * "Clarification: Check Points (and commits)".
@@ -610,12 +573,15 @@ final class IndexFileDeleter implements Closeable {
   }
 
   /** Decrefs all provided files, even on exception; throws first exception hit, if any. */
-  void decRef(Collection<String> files) {
+  void decRef(Collection<String> files) throws IOException {
     assert locked();
+    Set<String> toDelete = new HashSet<>();
     Throwable firstThrowable = null;
     for(final String file : files) {
       try {
-        decRef(file);
+        if (decRef(file)) {
+          toDelete.add(file);
+        }
       } catch (Throwable t) {
         if (firstThrowable == null) {
           // Save first exception and throw it in the end, but be sure to finish decRef all files
@@ -625,7 +591,7 @@ final class IndexFileDeleter implements Closeable {
     }
 
     try {
-      deletePendingFiles();
+      deleteFiles(toDelete);
     } catch (Throwable t) {
       if (firstThrowable == null) {
         // Save first exception and throw it in the end, but be sure to finish decRef all files
@@ -634,27 +600,31 @@ final class IndexFileDeleter implements Closeable {
     }
 
     // NOTE: does nothing if firstThrowable is null
-    IOUtils.reThrowUnchecked(firstThrowable);
+    IOUtils.reThrow(firstThrowable);
   }
 
   /** Decrefs all provided files, ignoring any exceptions hit; call this if
    *  you are already handling an exception. */
   void decRefWhileHandlingException(Collection<String> files) {
     assert locked();
+    Set<String> toDelete = new HashSet<>();
     for(final String file : files) {
       try {
-        decRef(file);
+        if (decRef(file)) {
+          toDelete.add(file);
+        }
       } catch (Throwable t) {
       }
     }
 
     try {
-      deletePendingFiles();
+      deleteFiles(toDelete);
     } catch (Throwable t) {
     }
   }
 
-  private void decRef(String fileName) {
+  /** Returns true if the file should now be deleted. */
+  private boolean decRef(String fileName) {
     assert locked();
     RefCount rc = getRefCount(fileName);
     if (infoStream.isEnabled("IFD")) {
@@ -662,14 +632,13 @@ final class IndexFileDeleter implements Closeable {
         infoStream.message("IFD", "  DecRef \"" + fileName + "\": pre-decr count is " + rc.count);
       }
     }
-    if (0 == rc.DecRef()) {
+    if (rc.DecRef() == 0) {
       // This file is no longer referenced by any past
       // commit points nor by the in-memory SegmentInfos:
-      try {
-        deletable.add(fileName);
-      } finally {
-        refCounts.remove(fileName);
-      }
+      refCounts.remove(fileName);
+      return true;
+    } else {
+      return false;
     }
   }
 
@@ -692,8 +661,6 @@ final class IndexFileDeleter implements Closeable {
     RefCount rc;
     if (!refCounts.containsKey(fileName)) {
       rc = new RefCount(fileName);
-      // We should never incRef a file we are already wanting to delete:
-      assert deletable.contains(fileName) == false: "file \"" + fileName + "\" cannot be incRef'd: it's already pending delete";
       refCounts.put(fileName, rc);
     } else {
       rc = refCounts.get(fileName);
@@ -705,6 +672,7 @@ final class IndexFileDeleter implements Closeable {
    *  (have not yet been incref'd). */
   void deleteNewFiles(Collection<String> files) throws IOException {
     assert locked();
+    Set<String> toDelete = new HashSet<>();
     for (final String fileName: files) {
       // NOTE: it's very unusual yet possible for the
       // refCount to be present and 0: it can happen if you
@@ -716,45 +684,31 @@ final class IndexFileDeleter implements Closeable {
         if (infoStream.isEnabled("IFD")) {
           infoStream.message("IFD", "will delete new file \"" + fileName + "\"");
         }
-        deletable.add(fileName);
+        toDelete.add(fileName);
       }
     }
 
-    deletePendingFiles();
+    deleteFiles(toDelete);
   }
 
-  /** 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) {
+  private void deleteFiles(Collection<String> names) throws IOException {
     assert locked();
     ensureOpen();
+    if (names.isEmpty()) {
+      return;
+    }
     try {
       if (infoStream.isEnabled("IFD")) {
-        infoStream.message("IFD", "delete \"" + fileName + "\"");
+        infoStream.message("IFD", "delete \"" + names + "\"");
       }
-      directory.deleteFile(fileName);
-      deletable.remove(fileName);
-      return true;
-    } catch (IOException e) {  // if delete fails
-
+      directory.deleteFiles(names);
+    } catch (NoSuchFileException | FileNotFoundException e) {  // if delete fails
       // IndexWriter should only ask us to delete files it knows it wrote, so if we hit this, something is wrong!
-      // LUCENE-6684: we suppress this assert for Windows, since a file could be in a confusing "pending delete" state:
-      assert Constants.WINDOWS || e instanceof NoSuchFileException == false: "hit unexpected NoSuchFileException: file=" + fileName;
-      assert Constants.WINDOWS || 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
-      // we assume that when a delete fails it is because
-      // the file is open in another process, and queue
-      // the file for subsequent deletion.
-
-      if (infoStream.isEnabled("IFD")) {
-        infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
+      if (Constants.WINDOWS) {
+        // LUCENE-6684: we suppress this assert for Windows, since a file could be in a confusing "pending delete" state:
+      } else {
+        throw e;
       }
-      deletable.add(fileName);
-      return false;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
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 b05e15a..e62cafb 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -51,6 +51,7 @@ import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.FilterDirectory;
 import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
@@ -753,6 +754,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    *           IO error
    */
   public IndexWriter(Directory d, IndexWriterConfig conf) throws IOException {
+    if (d instanceof FSDirectory && ((FSDirectory) d).checkPendingDeletions()) {
+      throw new IllegalArgumentException("Directory still has pending deleted files");
+    }
+
     conf.setIndexWriter(this); // prevent reuse by other instances
     config = conf;
     infoStream = config.getInfoStream();
@@ -3569,8 +3574,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       }
     }
 
-    deleter.deletePendingFiles();
-
     if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "after commitMerge: " + segString());
     }
@@ -4616,14 +4619,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    */
   public synchronized void deleteUnusedFiles() throws IOException {
     ensureOpen(false);
-    deleter.deletePendingFiles();
     deleter.revisitPolicy();
   }
 
-  private synchronized void deletePendingFiles() throws IOException {
-    deleter.deletePendingFiles();
-  }
-  
   /**
    * NOTE: this method creates a compound file for all files returned by
    * info.files(). While, generally, this may include separate norms and

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java b/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
index 0464852..2a286ff 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
@@ -212,7 +212,7 @@ public class PersistentSnapshotDeletionPolicy extends SnapshotDeletionPolicy {
   private synchronized void clearPriorSnapshots() throws IOException {
     for(String file : dir.listAll()) {
       if (file.startsWith(SNAPSHOTS_PREFIX)) {
-        dir.deleteFile(file);
+        dir.deleteFiles(Collections.singleton(file));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/Directory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/Directory.java b/lucene/core/src/java/org/apache/lucene/store/Directory.java
index 8aa5fa1..b9e5ad4 100644
--- a/lucene/core/src/java/org/apache/lucene/store/Directory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/Directory.java
@@ -49,8 +49,9 @@ public abstract class Directory implements Closeable {
    */
   public abstract String[] listAll() throws IOException;
 
-  /** Removes an existing file in the directory. */
-  public abstract void deleteFile(String name) throws IOException;
+  /** Removes the specified files from the directory.  If an exception is thrown, behavior is undefined
+   *  (none, some or all of the files may have in fact been deleted). */
+  public abstract void deleteFiles(Collection<String> name) throws IOException;
 
   /**
    * Returns the length of a file in the directory. This method follows the
@@ -67,7 +68,6 @@ public abstract class Directory implements Closeable {
    */
   public abstract long fileLength(String name) throws IOException;
 
-
   /** Creates a new, empty file in the directory with the given name.
       Returns a stream writing this file. */
   public abstract IndexOutput createOutput(String name, IOContext context) throws IOException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java b/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
index d02c126..0e1d4e9 100644
--- a/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
@@ -17,19 +17,25 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
+import java.io.FileNotFoundException;
 import java.io.FilterOutputStream;
 import java.io.IOException;
 import java.nio.channels.ClosedChannelException; // javadoc @link
 import java.nio.file.DirectoryStream;
 import java.nio.file.FileAlreadyExistsException;
 import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
 import java.nio.file.OpenOption;
 import java.nio.file.Path;
 import java.nio.file.StandardCopyOption;
 import java.nio.file.StandardOpenOption;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -120,6 +126,12 @@ public abstract class FSDirectory extends BaseDirectory {
 
   protected final Path directory; // The underlying filesystem directory
 
+
+  /** Files we previously tried to delete, but hit exception (on Windows) last time we tried.
+   *  These files are in "pending delete" state, where we refuse to openInput or createOutput
+   *  them, nor include them in .listAll. */
+  protected final Set<String> pendingDeletes = Collections.newSetFromMap(new ConcurrentHashMap<String,Boolean>());
+
   /** Used to generate temp file names in {@link #createTempOutput}. */
   private final AtomicLong nextTempFileCounter = new AtomicLong();
 
@@ -193,11 +205,18 @@ public abstract class FSDirectory extends BaseDirectory {
    *
    *  @throws IOException if there was an I/O error during listing */
   public static String[] listAll(Path dir) throws IOException {
+    return listAll(dir, Collections.emptySet());
+  }
+
+  private static String[] listAll(Path dir, Set<String> skipNames) throws IOException {
     List<String> entries = new ArrayList<>();
     
     try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir)) {
       for (Path path : stream) {
-        entries.add(path.getFileName().toString());
+        String name = path.getFileName().toString();
+        if (skipNames.contains(name) == false) {
+          entries.add(name);
+        }
       }
     }
     
@@ -207,7 +226,7 @@ public abstract class FSDirectory extends BaseDirectory {
   @Override
   public String[] listAll() throws IOException {
     ensureOpen();
-    return listAll(directory);
+    return listAll(directory, pendingDeletes);
   }
 
   /** Returns the length in bytes of a file in the directory. */
@@ -219,9 +238,10 @@ public abstract class FSDirectory extends BaseDirectory {
 
   /** Removes an existing file in the directory. */
   @Override
-  public void deleteFile(String name) throws IOException {
+  public void deleteFiles(Collection<String> names) throws IOException {
     ensureOpen();
-    Files.delete(directory.resolve(name));
+    pendingDeletes.addAll(names);
+    deletePendingFiles();
   }
 
   /** Creates an IndexOutput for the file with the given name. */
@@ -247,9 +267,20 @@ public abstract class FSDirectory extends BaseDirectory {
   }
 
   protected void ensureCanWrite(String name) throws IOException {
+    deletePendingFiles();
+    if (pendingDeletes.contains(name)) {
+      throw new IOException("file \"" + name + "\" is pending delete and cannot be overwritten");
+    }
     Files.deleteIfExists(directory.resolve(name)); // delete existing, if any
   }
 
+  protected void ensureCanRead(String name) throws IOException {
+    deletePendingFiles();
+    if (pendingDeletes.contains(name)) {
+      throw new NoSuchFileException("file \"" + name + "\" is pending delete and cannot be overwritten");
+    }
+  }
+
   @Override
   public void sync(Collection<String> names) throws IOException {
     ensureOpen();
@@ -270,8 +301,9 @@ public abstract class FSDirectory extends BaseDirectory {
 
   /** Closes the store to future operations. */
   @Override
-  public synchronized void close() {
+  public synchronized void close() throws IOException {
     isOpen = false;
+    deletePendingFiles();
   }
 
   /** @return the underlying filesystem directory */
@@ -286,6 +318,74 @@ public abstract class FSDirectory extends BaseDirectory {
     return this.getClass().getSimpleName() + "@" + directory + " lockFactory=" + lockFactory;
   }
 
+  protected void fsync(String name) throws IOException {
+    deletePendingFiles();
+    IOUtils.fsync(directory.resolve(name), false);
+  }
+
+  /** Returns true if the file was successfully removed. */
+  private boolean deleteFile(String name) throws IOException {  
+    try {
+      Files.delete(directory.resolve(name));
+      pendingDeletes.remove(name);
+      return true;
+    } catch (NoSuchFileException | FileNotFoundException e) {
+      // We were asked to delete a non-existent file:
+      pendingDeletes.remove(name);
+      throw e;
+    } catch (IOException ioe) {
+      // On windows, a file delete can fail because there's still an open
+      // file handle against it.  We record this in pendingDeletes and
+      // try again later.
+
+      // TODO: this is hacky/lenient (we don't know which IOException this is), and
+      // it should only happen on filesystems that can do this, so really we should
+      // move this logic to WindowsDirectory or something
+
+      // TODO: can/should we do if (Constants.WINDOWS) here, else throw the exc?
+      // but what about a Linux box with a CIFS mount?
+      //System.out.println("FS.deleteFile failed (" + ioe + "): will retry later");
+      pendingDeletes.add(name);
+      return false;
+    }
+  }
+
+  /** Tries to delete any pending deleted files, and returns true if
+   *  there are still files that could not be deleted. */
+  public boolean checkPendingDeletions() throws IOException {
+    deletePendingFiles();
+    return pendingDeletes.isEmpty() == false;
+  }
+
+  /** Try to delete any pending files that we had previously tried to delete but failed
+   *  because we are on Windows and the files were still
+   *  held open. */
+  public void deletePendingFiles() throws IOException {
+    // TODO: we could fix IndexInputs from FSDirectory subclasses to call this when they are closed?
+
+    // Clone the set because it will change as we iterate:
+    List<String> toDelete = new ArrayList<>(pendingDeletes);
+
+    // 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) {
+      if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+        if (deleteFile(fileName) == false) {
+          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);
+      }
+    }
+  }
+
   final class FSIndexOutput extends OutputStreamIndexOutput {
     /**
      * The maximum chunk size is 8192 bytes, because file channel mallocs
@@ -312,8 +412,4 @@ public abstract class FSDirectory extends BaseDirectory {
       }, CHUNK_SIZE);
     }
   }
-
-  protected void fsync(String name) throws IOException {
-    IOUtils.fsync(directory.resolve(name), false);
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java b/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java
index 0b986f4..13bc217 100644
--- a/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java
@@ -140,8 +140,21 @@ public class FileSwitchDirectory extends Directory {
   }
 
   @Override
-  public void deleteFile(String name) throws IOException {
-    getDirectory(name).deleteFile(name);
+  public void deleteFiles(Collection<String> names) throws IOException {
+    Set<String> primaryToDelete = new HashSet<>();
+    Set<String> secondaryToDelete = new HashSet<>();
+    for(String name : names) {
+      if (getDirectory(name) == primaryDir) {
+        primaryToDelete.add(name);
+      } else {
+        secondaryToDelete.add(name);
+      }
+    }
+    try {
+      primaryDir.deleteFiles(primaryToDelete);
+    } finally {
+      secondaryDir.deleteFiles(secondaryToDelete);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java b/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java
index 7c550c1..9ee2928 100644
--- a/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java
@@ -58,8 +58,8 @@ public class FilterDirectory extends Directory {
   }
 
   @Override
-  public void deleteFile(String name) throws IOException {
-    in.deleteFile(name);
+  public void deleteFiles(Collection<String> names) throws IOException {
+    in.deleteFiles(names);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/LockValidatingDirectoryWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/LockValidatingDirectoryWrapper.java b/lucene/core/src/java/org/apache/lucene/store/LockValidatingDirectoryWrapper.java
index 389c56d..3ed659a 100644
--- a/lucene/core/src/java/org/apache/lucene/store/LockValidatingDirectoryWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/store/LockValidatingDirectoryWrapper.java
@@ -33,9 +33,9 @@ public final class LockValidatingDirectoryWrapper extends FilterDirectory {
   }
 
   @Override
-  public void deleteFile(String name) throws IOException {
+  public void deleteFiles(Collection<String> names) throws IOException {
     writeLock.ensureValid();
-    in.deleteFile(name);
+    in.deleteFiles(names);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java b/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
index 0808eb6..b6b4033 100644
--- a/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java
@@ -234,6 +234,7 @@ public class MMapDirectory extends FSDirectory {
   @Override
   public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
+    ensureCanRead(name);
     Path path = directory.resolve(name);
     try (FileChannel c = FileChannel.open(path, StandardOpenOption.READ)) {
       final String resourceDescription = "MMapIndexInput(path=\"" + path.toString() + "\")";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java b/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
index b739290..3234592 100644
--- a/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
@@ -74,10 +74,10 @@ public class NIOFSDirectory extends FSDirectory {
     this(path, FSLockFactory.getDefault());
   }
 
-  /** Creates an IndexInput for the file with the given name. */
   @Override
   public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
+    ensureCanRead(name);
     Path path = getDirectory().resolve(name);
     FileChannel fc = FileChannel.open(path, StandardOpenOption.READ);
     return new NIOFSIndexInput("NIOFSIndexInput(path=\"" + path + "\")", fc, context);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java b/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
index fd5e3d7..1b8404b 100644
--- a/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
@@ -112,14 +112,23 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
   }
 
   @Override
-  public synchronized void deleteFile(String name) throws IOException {
+  public synchronized void deleteFiles(Collection<String> names) throws IOException {
     if (VERBOSE) {
-      System.out.println("nrtdir.deleteFile name=" + name);
+      System.out.println("nrtdir.deleteFiles names=" + names);
     }
-    if (cache.fileNameExists(name)) {
-      cache.deleteFile(name);
-    } else {
-      in.deleteFile(name);
+    Set<String> cacheToDelete = new HashSet<>();
+    Set<String> toDelete = new HashSet<>();
+    for(String name : names) {
+      if (cache.fileNameExists(name)) {
+        cacheToDelete.add(name);
+      } else {
+        toDelete.add(name);
+      }
+    }
+    try {
+      cache.deleteFiles(cacheToDelete);
+    } finally {
+      in.deleteFiles(toDelete);
     }
   }
 
@@ -146,14 +155,14 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
         System.out.println("  to cache");
       }
       try {
-        in.deleteFile(name);
+        in.deleteFiles(Collections.singleton(name));
       } catch (IOException ioe) {
         // This is fine: file may not exist
       }
       return cache.createOutput(name, context);
     } else {
       try {
-        cache.deleteFile(name);
+        cache.deleteFiles(Collections.singleton(name));
       } catch (IOException ioe) {
         // This is fine: file may not exist
       }
@@ -323,7 +332,7 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
       synchronized(this) {
         // Must sync here because other sync methods have
         // if (cache.fileNameExists(name)) { ... } else { ... }:
-        cache.deleteFile(fileName);
+        cache.deleteFiles(Collections.singleton(fileName));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java b/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
index d1dc0d0..7be4679 100644
--- a/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/RAMDirectory.java
@@ -157,14 +157,16 @@ public class RAMDirectory extends BaseDirectory implements Accountable {
   }
   
   @Override
-  public void deleteFile(String name) throws IOException {
+  public void deleteFiles(Collection<String> names) throws IOException {
     ensureOpen();
-    RAMFile file = fileMap.remove(name);
-    if (file != null) {
-      file.directory = null;
-      sizeInBytes.addAndGet(-file.sizeInBytes);
-    } else {
-      throw new FileNotFoundException(name);
+    for(String name : names) {
+      RAMFile file = fileMap.remove(name);
+      if (file != null) {
+        file.directory = null;
+        sizeInBytes.addAndGet(-file.sizeInBytes);
+      } else {
+        throw new FileNotFoundException(name);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java b/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
index 2daf98f..dc7a92c 100644
--- a/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
@@ -72,6 +72,7 @@ public class SimpleFSDirectory extends FSDirectory {
   @Override
   public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
+    ensureCanRead(name);
     Path path = directory.resolve(name);
     SeekableByteChannel channel = Files.newByteChannel(path, StandardOpenOption.READ);
     return new SimpleFSIndexInput("SimpleFSIndexInput(path=\"" + path + "\")", channel, context);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/store/TrackingDirectoryWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/TrackingDirectoryWrapper.java b/lucene/core/src/java/org/apache/lucene/store/TrackingDirectoryWrapper.java
index aa7214c..49d70f8 100644
--- a/lucene/core/src/java/org/apache/lucene/store/TrackingDirectoryWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/store/TrackingDirectoryWrapper.java
@@ -18,6 +18,7 @@ package org.apache.lucene.store;
  */
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
@@ -33,9 +34,11 @@ public final class TrackingDirectoryWrapper extends FilterDirectory {
   }
 
   @Override
-  public void deleteFile(String name) throws IOException {
-    in.deleteFile(name);
-    createdFileNames.remove(name);
+  public void deleteFiles(Collection<String> names) throws IOException {
+    in.deleteFiles(names);
+    for(String name : names) {
+      createdFileNames.remove(name);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
----------------------------------------------------------------------
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 510545f..3be3d7f 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
@@ -38,8 +38,10 @@ import java.nio.file.StandardOpenOption;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
@@ -191,12 +193,10 @@ public final class IOUtils {
    * Note that the files should not be null.
    */
   public static void deleteFilesIgnoringExceptions(Directory dir, Collection<String> files) {
-    for (String name : files) {
-      try {
-        dir.deleteFile(name);
-      } catch (Throwable ignored) {
-        // ignore
-      }
+    try {
+      dir.deleteFiles(files);
+    } catch (Throwable ignored) {
+      // ignore
     }
   }
 
@@ -212,24 +212,18 @@ public final class IOUtils {
    * completes normally if there were no exceptions.
    * 
    * @param dir Directory to delete files from
-   * @param files file names to delete
+   * @param names file names to delete
    */
-  public static void deleteFiles(Directory dir, Collection<String> files) throws IOException {
-    Throwable th = null;
-    for (String name : files) {
+  public static void deleteFiles(Directory dir, Collection<String> names) throws IOException {
+    Set<String> nonNullNames = new HashSet<>();
+    for(String name : names) {
       if (name != null) {
-        try {
-          dir.deleteFile(name);
-        } catch (Throwable t) {
-          addSuppressed(th, t);
-          if (th == null) {
-            th = t;
-          }
-        }
+        nonNullNames.add(name);
       }
     }
-
-    reThrow(th);
+    if (names.isEmpty() == false) {
+      dir.deleteFiles(names);
+    }
   }
 
   public static void deleteFiles(Directory dir, String... files) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index e47830d..328ea2e 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -22,6 +22,7 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 
@@ -39,8 +40,8 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntroSorter;
 import org.apache.lucene.util.LongBitSet;
 import org.apache.lucene.util.NumericUtils;
-import org.apache.lucene.util.OfflineSorter;
 import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
+import org.apache.lucene.util.OfflineSorter;
 import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.StringHelper;
@@ -820,7 +821,7 @@ public class BKDWriter implements Closeable {
       //System.out.println("sort time: " + ((t1-t0)/1000000.0) + " msec");
 
       if (tempInput != null) {
-        tempDir.deleteFile(tempInput.getName());
+        tempDir.deleteFiles(Collections.singleton(tempInput.getName()));
         tempInput = null;
       } else {
         assert heapPointWriter != null;
@@ -913,7 +914,7 @@ public class BKDWriter implements Closeable {
       try {
         tempInput.close();
       } finally {
-        tempDir.deleteFile(tempInput.getName());
+        tempDir.deleteFiles(Collections.singleton(tempInput.getName()));
         tempInput = null;
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java
index 6cf1097..0751354 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/OfflinePointWriter.java
@@ -18,7 +18,7 @@ package org.apache.lucene.util.bkd;
  */
 
 import java.io.IOException;
-
+import java.util.Collections;
 
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -75,7 +75,7 @@ final class OfflinePointWriter implements PointWriter {
 
   @Override
   public void destroy() throws IOException {
-    tempDir.deleteFile(out.getName());
+    tempDir.deleteFiles(Collections.singleton(out.getName()));
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
index 778a7eb..4b32ec8 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAddIndexes.java
@@ -1103,7 +1103,6 @@ public class TestAddIndexes extends LuceneTestCase {
     }
   }
 
-
   // LUCENE-2790: tests that the non CFS files were deleted by addIndexes
   public void testNonCFSLeftovers() throws Exception {
     Directory[] dirs = new Directory[2];
@@ -1121,7 +1120,6 @@ public class TestAddIndexes extends LuceneTestCase {
     DirectoryReader[] readers = new DirectoryReader[] { DirectoryReader.open(dirs[0]), DirectoryReader.open(dirs[1]) };
     
     MockDirectoryWrapper dir = new MockDirectoryWrapper(random(), new RAMDirectory());
-    dir.setEnableVirusScanner(false); // we check for specific list of files
     IndexWriterConfig conf = new IndexWriterConfig(new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy(true));
     MergePolicy lmp = conf.getMergePolicy();
     // Force creation of CFS:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java
index 3031553..cafe0a3 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java
@@ -42,11 +42,6 @@ public class TestAllFilesCheckIndexHeader extends LuceneTestCase {
   public void test() throws Exception {
     Directory dir = newDirectory();
 
-    if (dir instanceof MockDirectoryWrapper) {
-      // otherwise we can have unref'd files left in the index that won't be visited when opening a reader and lead to scary looking false failures:
-      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
-    }
-
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
     conf.setCodec(TestUtil.getDefaultCodec());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java
index 347955d..bf5625a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java
@@ -42,11 +42,6 @@ public class TestAllFilesDetectTruncation extends LuceneTestCase {
   public void test() throws Exception {
     Directory dir = newDirectory();
 
-    if (dir instanceof MockDirectoryWrapper) {
-      // otherwise we can have unref'd files left in the index that won't be visited when opening a reader and lead to scary looking false failures:
-      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
-    }
-
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
     conf.setCodec(TestUtil.getDefaultCodec());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java b/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
index e801bcc..acf1ebe 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
@@ -181,6 +181,5 @@ public class TestAtomicUpdate extends LuceneTestCase {
     directory = newFSDirectory(dirPath);
     runTest(directory);
     directory.close();
-    IOUtils.rm(dirPath);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
index 9d72bd0..90d3c1c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java
@@ -1094,10 +1094,6 @@ public class TestBinaryDocValuesUpdates extends LuceneTestCase {
 
   public void testDeleteUnusedUpdatesFiles() throws Exception {
     Directory dir = newDirectory();
-    // test explicitly needs files to always be actually deleted
-    if (dir instanceof MockDirectoryWrapper) {
-      ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
-    }
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
     IndexWriter writer = new IndexWriter(dir, conf);
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestCodecHoldsOpenFiles.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestCodecHoldsOpenFiles.java b/lucene/core/src/test/org/apache/lucene/index/TestCodecHoldsOpenFiles.java
index 8077545..fa773fa 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestCodecHoldsOpenFiles.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestCodecHoldsOpenFiles.java
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.IntPoint;
@@ -47,15 +48,7 @@ public class TestCodecHoldsOpenFiles extends LuceneTestCase {
     w.commit();
     w.close();
 
-    for(String fileName : d.listAll()) {
-      try {
-        d.deleteFile(fileName);
-        // may succeed, e.g. if the file is completely read into RAM.
-      } catch (IOException ioe) {
-        // ignore: this means codec (correctly) is holding
-        // the file open
-      }
-    }
+    d.deleteFiles(Arrays.asList(d.listAll()));
 
     for(LeafReaderContext cxt : r.leaves()) {
       TestUtil.checkReader(cxt.reader());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java b/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
index 083e426..a4b81de 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
@@ -184,10 +184,6 @@ public class TestConcurrentMergeScheduler extends LuceneTestCase {
 
   public void testNoExtraFiles() throws IOException {
     Directory directory = newDirectory();
-    if (directory instanceof MockDirectoryWrapper) {
-      // test uses IW unref'ed helper which is unaware of retries
-      ((MockDirectoryWrapper)directory).setEnableVirusScanner(false);
-    }
     IndexWriter writer = new IndexWriter(directory, newIndexWriterConfig(new MockAnalyzer(random()))
                                                       .setMaxBufferedDocs(2));
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
index b56b1a0..6d7d1de 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
@@ -17,6 +17,15 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -27,23 +36,16 @@ import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.Version;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 /*
   Verify we can read the pre-2.1 file format, do searches
   against it, and add documents to it.
 */
-
+@SuppressFileSystems("VirusCheckingFS")
 public class TestDeletionPolicy extends LuceneTestCase {
   
   private void verifyCommitOrder(List<? extends IndexCommit> commits) {
@@ -223,10 +225,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
     final double SECONDS = 2.0;
 
     Directory dir = newDirectory();
-    if (dir instanceof MockDirectoryWrapper) {
-      // test manually deletes files
-      ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
-    }
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()))
         .setIndexDeletionPolicy(new ExpirationTimeDeletionPolicy(dir, SECONDS));
     MergePolicy mp = conf.getMergePolicy();
@@ -299,7 +297,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
         break;
       }
       
-      dir.deleteFile(IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", gen));
+      dir.deleteFiles(Collections.singleton(IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", gen)));
       gen--;
     }
 
@@ -319,10 +317,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
       boolean useCompoundFile = (pass % 2) != 0;
 
       Directory dir = newDirectory();
-      if (dir instanceof MockDirectoryWrapper) {
-        // test manually deletes files
-        ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
-      }
 
       IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()))
           .setIndexDeletionPolicy(new KeepAllDeletionPolicy(dir))
@@ -381,7 +375,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
       while(gen > 0) {
         IndexReader reader = DirectoryReader.open(dir);
         reader.close();
-        dir.deleteFile(IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", gen));
+        dir.deleteFiles(Collections.singleton(IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", gen)));
         gen--;
 
         if (gen > 0) {
@@ -570,10 +564,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
       boolean useCompoundFile = (pass % 2) != 0;
 
       Directory dir = newDirectory();
-      if (dir instanceof MockDirectoryWrapper) {
-        // test manually deletes files
-        ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
-      }
 
       KeepLastNDeletionPolicy policy = new KeepLastNDeletionPolicy(N);
       for(int j=0;j<N+1;j++) {
@@ -612,7 +602,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
           }
         }
         if (i < N) {
-          dir.deleteFile(IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", gen));
+          dir.deleteFiles(Collections.singleton(IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", gen)));
         }
         gen--;
       }
@@ -634,10 +624,6 @@ public class TestDeletionPolicy extends LuceneTestCase {
       boolean useCompoundFile = (pass % 2) != 0;
 
       Directory dir = newDirectory();
-      if (dir instanceof MockDirectoryWrapper) {
-        // test manually deletes files
-        ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
-      }
       IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()))
           .setOpenMode(OpenMode.CREATE)
           .setIndexDeletionPolicy(new KeepLastNDeletionPolicy(N))
@@ -730,7 +716,7 @@ public class TestDeletionPolicy extends LuceneTestCase {
           }
         }
         if (i < N) {
-          dir.deleteFile(IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", gen));
+          dir.deleteFiles(Collections.singleton(IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", gen)));
         }
         gen--;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
index 0d25f28..985606f 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
@@ -384,7 +384,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
 
             final Directory dir = openDirectory(leafIndex);
 
-            if (Files.exists(leafIndex.resolve("done")) == false) {
+            if (slowFileExists(dir, "done") == false) {
               if (DEBUG) System.out.println(Thread.currentThread().getName() + ": TEST: build segment index for " + leaf + " " + segIDGen + " (source: " + info.getDiagnostics().get("source") + ") dir=" + leafIndex);
 
               if (dir.listAll().length != 0) {
@@ -895,7 +895,9 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
     AtomicLong currentSchemaGen = new AtomicLong();
 
     // TODO: separate refresh thread, search threads, indexing threads
-    ReindexingReader reindexer = getReindexerNewDVFields(createTempDir(), currentSchemaGen);
+    Path root = createTempDir();
+    assumeFalse("we directly delete files", TestUtil.hasVirusChecker(root));
+    ReindexingReader reindexer = getReindexerNewDVFields(root, currentSchemaGen);
     reindexer.commit();
 
     Document doc = new Document();
@@ -965,6 +967,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
     int numDocs = atLeast(TEST_NIGHTLY ? 20000 : 1000);
     int maxID = 0;
     Path root = createTempDir();
+    assumeFalse("we directly delete files", TestUtil.hasVirusChecker(root));
     int refreshEveryNumDocs = 100;
     int commitCloseNumDocs = 1000;
     for(int i=0;i<numDocs;i++) {
@@ -1050,6 +1053,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
     int numDocs = atLeast(TEST_NIGHTLY ? 20000 : 1000);
     int maxID = 0;
     Path root = createTempDir();
+    assumeFalse("we directly delete files", TestUtil.hasVirusChecker(root));
     int refreshEveryNumDocs = 100;
     int commitCloseNumDocs = 1000;
 
@@ -1151,7 +1155,9 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
   }
 
   public void testBasic() throws Exception {
-    ReindexingReader reindexer = getReindexer(createTempDir());
+    Path tempPath = createTempDir();
+    assumeFalse("we directly delete files", TestUtil.hasVirusChecker(tempPath));
+    ReindexingReader reindexer = getReindexer(tempPath);
 
     // Start with initial empty commit:
     reindexer.commit();
@@ -1220,6 +1226,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
 
   public void testRandom() throws Exception {
     Path root = createTempDir();
+    assumeFalse("we directly delete files", TestUtil.hasVirusChecker(root));
     ReindexingReader reindexer = null;
 
     // TODO: separate refresh thread, search threads, indexing threads

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java b/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
index 00b424f..2c4f392 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
@@ -436,40 +436,43 @@ void assertTermDocsCount(String msg,
     rmDir(fileDirName);
   }*/
   
-public void testFilesOpenClose() throws IOException {
-      // Create initial data set
-      Path dirFile = createTempDir("TestIndexReader.testFilesOpenClose");
-      Directory dir = newFSDirectory(dirFile);
-      IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
-      addDoc(writer, "test");
-      writer.close();
-      dir.close();
+  public void testFilesOpenClose() throws IOException {
+    // Create initial data set
+    Path dirFile = createTempDir("TestIndexReader.testFilesOpenClose");
+    assumeFalse("test directly deletes files", TestUtil.hasVirusChecker(dirFile));
+    Directory dir = newFSDirectory(dirFile);
+    
+    IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
+    addDoc(writer, "test");
+    writer.close();
+    dir.close();
 
-      // Try to erase the data - this ensures that the writer closed all files
-      IOUtils.rm(dirFile);
-      dir = newFSDirectory(dirFile);
+    // Try to erase the data - this ensures that the writer closed all files
+    IOUtils.rm(dirFile);
+    dir = newFSDirectory(dirFile);
 
-      // Now create the data set again, just as before
-      writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
-                                       .setOpenMode(OpenMode.CREATE));
-      addDoc(writer, "test");
-      writer.close();
-      dir.close();
+    // Now create the data set again, just as before
+    writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random()))
+                              .setOpenMode(OpenMode.CREATE));
+    addDoc(writer, "test");
+    writer.close();
+    dir.close();
 
-      // Now open existing directory and test that reader closes all files
-      dir = newFSDirectory(dirFile);
-      DirectoryReader reader1 = DirectoryReader.open(dir);
-      reader1.close();
-      dir.close();
+    // Now open existing directory and test that reader closes all files
+    dir = newFSDirectory(dirFile);
+    DirectoryReader reader1 = DirectoryReader.open(dir);
+    reader1.close();
+    dir.close();
 
-      // The following will fail if reader did not close
-      // all files
-      IOUtils.rm(dirFile);
+    // The following will fail if reader did not close
+    // all files
+    IOUtils.rm(dirFile);
   }
 
   public void testOpenReaderAfterDelete() throws IOException {
     Path dirFile = createTempDir("deletetest");
     Directory dir = newFSDirectory(dirFile);
+    assumeFalse("test deletes files directly", TestUtil.hasVirusChecker(dir));
     if (dir instanceof BaseDirectoryWrapper) {
       ((BaseDirectoryWrapper)dir).setCheckIndexOnClose(false); // we will hit NoSuchFileException in MDW since we nuked it!
     }
@@ -717,7 +720,6 @@ public void testFilesOpenClose() throws IOException {
   // good exception
   public void testNoDir() throws Throwable {
     Path tempDir = createTempDir("doesnotexist");
-    IOUtils.rm(tempDir);
     Directory dir = newFSDirectory(tempDir);
     try {
       DirectoryReader.open(dir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java b/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java
index 2c3b134..934ec73 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReaderReopen.java
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -39,14 +40,12 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.MockDirectoryWrapper.FakeIOException;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
-
-
 public class TestDirectoryReaderReopen extends LuceneTestCase {
   
   public void testReopen() throws Exception {
@@ -625,10 +624,6 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
 
   public void testOverDecRefDuringReopen() throws Exception {
     MockDirectoryWrapper dir = newMockDirectory();
-    if (dir instanceof MockDirectoryWrapper) {
-      // ensure we produce enough of our exceptions
-      dir.setEnableVirusScanner(false);
-    }
 
     IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
     iwc.setCodec(TestUtil.getDefaultCodec());
@@ -713,7 +708,7 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
 
     // Blow away the index:
     for(String fileName : dir.listAll()) {
-      dir.deleteFile(fileName);
+      dir.deleteFiles(Collections.singleton(fileName));
     }
 
     w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
@@ -762,9 +757,7 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
     DirectoryReader r = DirectoryReader.open(dir);
 
     // Blow away the index:
-    for(String fileName : dir.listAll()) {
-      dir.deleteFile(fileName);
-    }
+    dir.deleteFiles(Arrays.asList(dir.listAll()));
 
     w = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
     doc = new Document();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
index 39fbec8..24ce405 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
@@ -48,6 +48,7 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.Version;
 
 /** JUnit adaptation of an older test case DocTest. */
@@ -120,8 +121,6 @@ public class TestDoc extends LuceneTestCase {
       // We create unreferenced files (we don't even write
       // a segments file):
       ((MockDirectoryWrapper) directory).setAssertNoUnrefencedFilesOnClose(false);
-      // this test itself deletes files (has no retry mechanism)
-      ((MockDirectoryWrapper) directory).setEnableVirusScanner(false);
     }
 
     IndexWriter writer = new IndexWriter(
@@ -164,8 +163,6 @@ public class TestDoc extends LuceneTestCase {
       // We create unreferenced files (we don't even write
       // a segments file):
       ((MockDirectoryWrapper) directory).setAssertNoUnrefencedFilesOnClose(false);
-      // this test itself deletes files (has no retry mechanism)
-      ((MockDirectoryWrapper) directory).setEnableVirusScanner(false);
     }
 
     writer = new IndexWriter(
@@ -237,9 +234,7 @@ public class TestDoc extends LuceneTestCase {
       Collection<String> filesToDelete = si.files();
       codec.compoundFormat().write(dir, si, context);
       si.setUseCompoundFile(true);
-      for (final String fileToDelete : filesToDelete) {
-        si1.info.dir.deleteFile(fileToDelete);
-      }
+      si1.info.dir.deleteFiles(filesToDelete);
     }
 
     return new SegmentCommitInfo(si, 0, -1L, -1L, -1L);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java b/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
index d9a1d2e..2d33f71 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
@@ -188,42 +188,37 @@ public class TestFieldsReader extends LuceneTestCase {
   public void testExceptions() throws Throwable {
     Path indexDir = createTempDir("testfieldswriterexceptions");
 
-    try {
-      Directory fsDir = newFSDirectory(indexDir);
-      FaultyFSDirectory dir = new FaultyFSDirectory(fsDir);
-      IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()))
-                                .setOpenMode(OpenMode.CREATE);
-      IndexWriter writer = new IndexWriter(dir, iwc);
-      for(int i=0;i<2;i++)
-        writer.addDocument(testDoc);
-      writer.forceMerge(1);
-      writer.close();
-
-      IndexReader reader = DirectoryReader.open(dir);
-      dir.startFailing();
-
-      boolean exc = false;
-
-      for(int i=0;i<2;i++) {
-        try {
-          reader.document(i);
-        } catch (IOException ioe) {
-          // expected
-          exc = true;
-        }
-        try {
-          reader.document(i);
-        } catch (IOException ioe) {
-          // expected
-          exc = true;
-        }
+    Directory fsDir = newFSDirectory(indexDir);
+    FaultyFSDirectory dir = new FaultyFSDirectory(fsDir);
+    IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()))
+      .setOpenMode(OpenMode.CREATE);
+    IndexWriter writer = new IndexWriter(dir, iwc);
+    for(int i=0;i<2;i++)
+      writer.addDocument(testDoc);
+    writer.forceMerge(1);
+    writer.close();
+
+    IndexReader reader = DirectoryReader.open(dir);
+    dir.startFailing();
+
+    boolean exc = false;
+
+    for(int i=0;i<2;i++) {
+      try {
+        reader.document(i);
+      } catch (IOException ioe) {
+        // expected
+        exc = true;
+      }
+      try {
+        reader.document(i);
+      } catch (IOException ioe) {
+        // expected
+        exc = true;
       }
-      assertTrue(exc);
-      reader.close();
-      dir.close();
-    } finally {
-      IOUtils.rm(indexDir);
     }
-
+    assertTrue(exc);
+    reader.close();
+    dir.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/84f44589/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
----------------------------------------------------------------------
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 5fa777b..fa9f1a7 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
@@ -34,6 +34,7 @@ import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.InfoStream;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
 
 /*
   Verify we can read the pre-2.1 file format, do searches
@@ -46,8 +47,6 @@ public class TestIndexFileDeleter extends LuceneTestCase {
     Directory dir = newDirectory();
     if (dir instanceof MockDirectoryWrapper) {
       ((MockDirectoryWrapper)dir).setPreventDoubleWrite(false);
-      // ensure we actually delete files
-      ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
     }
 
     MergePolicy mergePolicy = newLogMergePolicy(true, 10);
@@ -222,7 +221,6 @@ public class TestIndexFileDeleter extends LuceneTestCase {
   public void testVirusScannerDoesntCorruptIndex() throws IOException {
     MockDirectoryWrapper dir = newMockDirectory();
     dir.setPreventDoubleWrite(false); // we arent trying to test this
-    dir.setEnableVirusScanner(false); // we have our own to make test reproduce always
     
     // add empty commit
     new IndexWriter(dir, new IndexWriterConfig(null)).close();