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/06 14:27:05 UTC

[12/17] lucene-solr git commit: disable random insertion of VirusCheckingFS, and instead add it to a few tests

disable random insertion of VirusCheckingFS, and instead add it to a few tests


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

Branch: refs/heads/master
Commit: 85c546b742b008fda7ab9931a1fa18ac806406aa
Parents: dd6379c
Author: Mike McCandless <mi...@apache.org>
Authored: Fri Feb 5 09:42:54 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Fri Feb 5 09:42:54 2016 -0500

----------------------------------------------------------------------
 .../benchmark/byTask/TestPerfTasksLogic.java    |   2 -
 .../org/apache/lucene/store/FSDirectory.java    |  29 +-
 .../apache/lucene/index/TestDeletionPolicy.java |   2 -
 .../index/TestDemoParallelLeafReader.java       |   5 -
 .../lucene/index/TestDirectoryReader.java       | 336 +++++++++----------
 .../lucene/index/TestIndexFileDeleter.java      |  14 +-
 .../apache/lucene/index/TestIndexWriter.java    |   1 -
 .../index/TestIndexWriterDeleteByQuery.java     |   2 +-
 .../apache/lucene/index/TestStressIndexing.java |   2 +-
 .../apache/lucene/search/TestPointQueries.java  |  32 +-
 .../org/apache/lucene/store/TestDirectory.java  |   1 -
 .../lucene/store/TestNativeFSLockFactory.java   |   1 -
 .../lucene/store/TestSimpleFSLockFactory.java   |   1 -
 .../org/apache/lucene/util/TestIOUtils.java     |   2 -
 .../apache/lucene/util/TestOfflineSorter.java   |   2 -
 .../org/apache/lucene/util/fst/TestFSTs.java    |   2 -
 .../taxonomy/directory/TestAddTaxonomy.java     |   1 -
 .../writercache/TestCompactLabelToOrdinal.java  |   2 -
 .../IndexAndTaxonomyReplicationClientTest.java  |   2 -
 .../replicator/IndexReplicationClientTest.java  |   2 -
 .../replicator/http/HttpReplicatorTest.java     |   2 -
 .../analyzing/AnalyzingInfixSuggesterTest.java  |   2 -
 .../analyzing/BlendedInfixSuggesterTest.java    |   2 -
 .../apache/lucene/mockfile/VirusCheckingFS.java |   6 +-
 .../lucene/store/BaseDirectoryTestCase.java     |  66 +++-
 .../lucene/store/BaseLockFactoryTestCase.java   |   6 -
 .../lucene/store/MockDirectoryWrapper.java      |  33 +-
 .../org/apache/lucene/util/LuceneTestCase.java  |  34 +-
 .../util/TestRuleTemporaryFilesCleanup.java     |  16 -
 .../java/org/apache/lucene/util/TestUtil.java   |  51 +++
 30 files changed, 373 insertions(+), 286 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
----------------------------------------------------------------------
diff --git a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
index 0c82cb5..1e6d94d 100644
--- a/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
+++ b/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
@@ -440,8 +440,6 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
     IndexReader ir = DirectoryReader.open(benchmark.getRunData().getDirectory());
     assertEquals(numLines + " lines were created but " + ir.numDocs() + " docs are in the index", numLines, ir.numDocs());
     ir.close();
-
-    Files.delete(lineFile);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/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 14bb7af..38c8fdc 100644
--- a/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
@@ -222,10 +222,9 @@ public abstract class FSDirectory extends BaseDirectory {
     }
     
     String[] array = entries.toArray(new String[entries.size()]);
-
-    // Don't let filesystem specifics leak out of this abstraction:
+    // Directory.listAll javadocs state that we sort the results here, so we don't let filesystem
+    // specifics leak out of this abstraction:
     Arrays.sort(array);
-
     return array;
   }
 
@@ -238,6 +237,9 @@ public abstract class FSDirectory extends BaseDirectory {
   @Override
   public long fileLength(String name) throws IOException {
     ensureOpen();
+    if (pendingDeletes.contains(name)) {
+      throw new NoSuchFileException("file \"" + name + "\" is pending delete");
+    }
     return Files.size(directory.resolve(name));
   }
 
@@ -284,10 +286,13 @@ public abstract class FSDirectory extends BaseDirectory {
       fsync(name);
     }
   }
-  
+
   @Override
   public void renameFile(String source, String dest) throws IOException {
     ensureOpen();
+    if (pendingDeletes.contains(source)) {
+      throw new NoSuchFileException("file \"" + source + "\" is pending delete and cannot be moved");
+    }
     maybeDeletePendingFiles();
     Files.move(directory.resolve(source), directory.resolve(dest), StandardCopyOption.ATOMIC_MOVE);
     // TODO: should we move directory fsync to a separate 'syncMetadata' method?
@@ -298,7 +303,7 @@ public abstract class FSDirectory extends BaseDirectory {
   @Override
   public synchronized void close() throws IOException {
     isOpen = false;
-    maybeDeletePendingFiles();
+    deletePendingFiles();
   }
 
   /** @return the underlying filesystem directory */
@@ -333,19 +338,21 @@ public abstract class FSDirectory extends BaseDirectory {
 
   /** 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 {
+  public synchronized void deletePendingFiles() throws IOException {
+    if (pendingDeletes.isEmpty() == false) {
 
-    // TODO: we could fix IndexInputs from FSDirectory subclasses to call this when they are closed?
+      // TODO: we could fix IndexInputs from FSDirectory subclasses to call this when they are closed?
 
-    // Clone the set since we mutate it in privateDeleteFile:
-    for(String name : new HashSet<>(pendingDeletes)) {
-      privateDeleteFile(name);
+      // Clone the set since we mutate it in privateDeleteFile:
+      for(String name : new HashSet<>(pendingDeletes)) {
+        privateDeleteFile(name);
+      }
     }
   }
 
   private void maybeDeletePendingFiles() throws IOException {
     if (pendingDeletes.isEmpty() == false) {
-      // This is a silly heuristic to try to avoid O(N^2), where N = number of files pending deletion, behavior:
+      // This is a silly heuristic to try to avoid O(N^2), where N = number of files pending deletion, behaviour on Windows:
       int count = opsSinceLastDelete.incrementAndGet();
       if (count >= pendingDeletes.size()) {
         opsSinceLastDelete.addAndGet(-count);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/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 f15f673..1d7d305 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDeletionPolicy.java
@@ -35,7 +35,6 @@ import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.Version;
 
@@ -43,7 +42,6 @@ import org.apache.lucene.util.Version;
   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) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/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 985606f..fe659e0 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
@@ -896,7 +896,6 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
 
     // TODO: separate refresh thread, search threads, indexing threads
     Path root = createTempDir();
-    assumeFalse("we directly delete files", TestUtil.hasVirusChecker(root));
     ReindexingReader reindexer = getReindexerNewDVFields(root, currentSchemaGen);
     reindexer.commit();
 
@@ -967,7 +966,6 @@ 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++) {
@@ -1053,7 +1051,6 @@ 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;
 
@@ -1156,7 +1153,6 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
 
   public void testBasic() throws Exception {
     Path tempPath = createTempDir();
-    assumeFalse("we directly delete files", TestUtil.hasVirusChecker(tempPath));
     ReindexingReader reindexer = getReindexer(tempPath);
 
     // Start with initial empty commit:
@@ -1226,7 +1222,6 @@ 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/85c546b7/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 2213033..dafdcdd 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
@@ -162,174 +162,174 @@ public class TestDirectoryReader extends LuceneTestCase {
    * @throws Exception on error
    */
   public void testGetFieldNames() throws Exception {
-      Directory d = newDirectory();
-      // set up writer
-      IndexWriter writer = new IndexWriter(
-          d,
-          newIndexWriterConfig(new MockAnalyzer(random()))
-      );
+    Directory d = newDirectory();
+    // set up writer
+    IndexWriter writer = new IndexWriter(
+                                         d,
+                                         newIndexWriterConfig(new MockAnalyzer(random()))
+                                         );
 
-      Document doc = new Document();
+    Document doc = new Document();
 
-      FieldType customType3 = new FieldType();
-      customType3.setStored(true);
+    FieldType customType3 = new FieldType();
+    customType3.setStored(true);
       
+    doc.add(new StringField("keyword", "test1", Field.Store.YES));
+    doc.add(new TextField("text", "test1", Field.Store.YES));
+    doc.add(new Field("unindexed", "test1", customType3));
+    doc.add(new TextField("unstored","test1", Field.Store.NO));
+    writer.addDocument(doc);
+
+    writer.close();
+    // set up reader
+    DirectoryReader reader = DirectoryReader.open(d);
+    FieldInfos fieldInfos = MultiFields.getMergedFieldInfos(reader);
+    assertNotNull(fieldInfos.fieldInfo("keyword"));
+    assertNotNull(fieldInfos.fieldInfo("text"));
+    assertNotNull(fieldInfos.fieldInfo("unindexed"));
+    assertNotNull(fieldInfos.fieldInfo("unstored"));
+    reader.close();
+    // add more documents
+    writer = new IndexWriter(
+                             d,
+                             newIndexWriterConfig(new MockAnalyzer(random()))
+                             .setOpenMode(OpenMode.APPEND)
+                             .setMergePolicy(newLogMergePolicy())
+                             );
+    // want to get some more segments here
+    int mergeFactor = ((LogMergePolicy) writer.getConfig().getMergePolicy()).getMergeFactor();
+    for (int i = 0; i < 5*mergeFactor; i++) {
+      doc = new Document();
       doc.add(new StringField("keyword", "test1", Field.Store.YES));
       doc.add(new TextField("text", "test1", Field.Store.YES));
       doc.add(new Field("unindexed", "test1", customType3));
       doc.add(new TextField("unstored","test1", Field.Store.NO));
       writer.addDocument(doc);
+    }
+    // new fields are in some different segments (we hope)
+    for (int i = 0; i < 5*mergeFactor; i++) {
+      doc = new Document();
+      doc.add(new StringField("keyword2", "test1", Field.Store.YES));
+      doc.add(new TextField("text2", "test1", Field.Store.YES));
+      doc.add(new Field("unindexed2", "test1", customType3));
+      doc.add(new TextField("unstored2","test1", Field.Store.NO));
+      writer.addDocument(doc);
+    }
+    // new termvector fields
 
-      writer.close();
-      // set up reader
-      DirectoryReader reader = DirectoryReader.open(d);
-      FieldInfos fieldInfos = MultiFields.getMergedFieldInfos(reader);
-      assertNotNull(fieldInfos.fieldInfo("keyword"));
-      assertNotNull(fieldInfos.fieldInfo("text"));
-      assertNotNull(fieldInfos.fieldInfo("unindexed"));
-      assertNotNull(fieldInfos.fieldInfo("unstored"));
-      reader.close();
-      // add more documents
-      writer = new IndexWriter(
-          d,
-          newIndexWriterConfig(new MockAnalyzer(random()))
-             .setOpenMode(OpenMode.APPEND)
-             .setMergePolicy(newLogMergePolicy())
-      );
-      // want to get some more segments here
-      int mergeFactor = ((LogMergePolicy) writer.getConfig().getMergePolicy()).getMergeFactor();
-      for (int i = 0; i < 5*mergeFactor; i++) {
-        doc = new Document();
-        doc.add(new StringField("keyword", "test1", Field.Store.YES));
-        doc.add(new TextField("text", "test1", Field.Store.YES));
-        doc.add(new Field("unindexed", "test1", customType3));
-        doc.add(new TextField("unstored","test1", Field.Store.NO));
-        writer.addDocument(doc);
-      }
-      // new fields are in some different segments (we hope)
-      for (int i = 0; i < 5*mergeFactor; i++) {
-        doc = new Document();
-        doc.add(new StringField("keyword2", "test1", Field.Store.YES));
-        doc.add(new TextField("text2", "test1", Field.Store.YES));
-        doc.add(new Field("unindexed2", "test1", customType3));
-        doc.add(new TextField("unstored2","test1", Field.Store.NO));
-        writer.addDocument(doc);
-      }
-      // new termvector fields
-
-      FieldType customType5 = new FieldType(TextField.TYPE_STORED);
-      customType5.setStoreTermVectors(true);
-      FieldType customType6 = new FieldType(TextField.TYPE_STORED);
-      customType6.setStoreTermVectors(true);
-      customType6.setStoreTermVectorOffsets(true);
-      FieldType customType7 = new FieldType(TextField.TYPE_STORED);
-      customType7.setStoreTermVectors(true);
-      customType7.setStoreTermVectorPositions(true);
-      FieldType customType8 = new FieldType(TextField.TYPE_STORED);
-      customType8.setStoreTermVectors(true);
-      customType8.setStoreTermVectorOffsets(true);
-      customType8.setStoreTermVectorPositions(true);
+    FieldType customType5 = new FieldType(TextField.TYPE_STORED);
+    customType5.setStoreTermVectors(true);
+    FieldType customType6 = new FieldType(TextField.TYPE_STORED);
+    customType6.setStoreTermVectors(true);
+    customType6.setStoreTermVectorOffsets(true);
+    FieldType customType7 = new FieldType(TextField.TYPE_STORED);
+    customType7.setStoreTermVectors(true);
+    customType7.setStoreTermVectorPositions(true);
+    FieldType customType8 = new FieldType(TextField.TYPE_STORED);
+    customType8.setStoreTermVectors(true);
+    customType8.setStoreTermVectorOffsets(true);
+    customType8.setStoreTermVectorPositions(true);
       
-      for (int i = 0; i < 5*mergeFactor; i++) {
-        doc = new Document();
-        doc.add(new TextField("tvnot", "tvnot", Field.Store.YES));
-        doc.add(new Field("termvector", "termvector", customType5));
-        doc.add(new Field("tvoffset", "tvoffset", customType6));
-        doc.add(new Field("tvposition", "tvposition", customType7));
-        doc.add(new Field("tvpositionoffset", "tvpositionoffset", customType8));
-        writer.addDocument(doc);
-      }
+    for (int i = 0; i < 5*mergeFactor; i++) {
+      doc = new Document();
+      doc.add(new TextField("tvnot", "tvnot", Field.Store.YES));
+      doc.add(new Field("termvector", "termvector", customType5));
+      doc.add(new Field("tvoffset", "tvoffset", customType6));
+      doc.add(new Field("tvposition", "tvposition", customType7));
+      doc.add(new Field("tvpositionoffset", "tvpositionoffset", customType8));
+      writer.addDocument(doc);
+    }
       
-      writer.close();
+    writer.close();
 
-      // verify fields again
-      reader = DirectoryReader.open(d);
-      fieldInfos = MultiFields.getMergedFieldInfos(reader);
+    // verify fields again
+    reader = DirectoryReader.open(d);
+    fieldInfos = MultiFields.getMergedFieldInfos(reader);
 
-      Collection<String> allFieldNames = new HashSet<>();
-      Collection<String> indexedFieldNames = new HashSet<>();
-      Collection<String> notIndexedFieldNames = new HashSet<>();
-      Collection<String> tvFieldNames = new HashSet<>();
+    Collection<String> allFieldNames = new HashSet<>();
+    Collection<String> indexedFieldNames = new HashSet<>();
+    Collection<String> notIndexedFieldNames = new HashSet<>();
+    Collection<String> tvFieldNames = new HashSet<>();
 
-      for(FieldInfo fieldInfo : fieldInfos) {
-        final String name = fieldInfo.name;
-        allFieldNames.add(name);
-        if (fieldInfo.getIndexOptions() != IndexOptions.NONE) {
-          indexedFieldNames.add(name);
-        } else {
-          notIndexedFieldNames.add(name);
-        }
-        if (fieldInfo.hasVectors()) {
-          tvFieldNames.add(name);
-        }
+    for(FieldInfo fieldInfo : fieldInfos) {
+      final String name = fieldInfo.name;
+      allFieldNames.add(name);
+      if (fieldInfo.getIndexOptions() != IndexOptions.NONE) {
+        indexedFieldNames.add(name);
+      } else {
+        notIndexedFieldNames.add(name);
+      }
+      if (fieldInfo.hasVectors()) {
+        tvFieldNames.add(name);
       }
+    }
 
-      assertTrue(allFieldNames.contains("keyword"));
-      assertTrue(allFieldNames.contains("text"));
-      assertTrue(allFieldNames.contains("unindexed"));
-      assertTrue(allFieldNames.contains("unstored"));
-      assertTrue(allFieldNames.contains("keyword2"));
-      assertTrue(allFieldNames.contains("text2"));
-      assertTrue(allFieldNames.contains("unindexed2"));
-      assertTrue(allFieldNames.contains("unstored2"));
-      assertTrue(allFieldNames.contains("tvnot"));
-      assertTrue(allFieldNames.contains("termvector"));
-      assertTrue(allFieldNames.contains("tvposition"));
-      assertTrue(allFieldNames.contains("tvoffset"));
-      assertTrue(allFieldNames.contains("tvpositionoffset"));
+    assertTrue(allFieldNames.contains("keyword"));
+    assertTrue(allFieldNames.contains("text"));
+    assertTrue(allFieldNames.contains("unindexed"));
+    assertTrue(allFieldNames.contains("unstored"));
+    assertTrue(allFieldNames.contains("keyword2"));
+    assertTrue(allFieldNames.contains("text2"));
+    assertTrue(allFieldNames.contains("unindexed2"));
+    assertTrue(allFieldNames.contains("unstored2"));
+    assertTrue(allFieldNames.contains("tvnot"));
+    assertTrue(allFieldNames.contains("termvector"));
+    assertTrue(allFieldNames.contains("tvposition"));
+    assertTrue(allFieldNames.contains("tvoffset"));
+    assertTrue(allFieldNames.contains("tvpositionoffset"));
       
-      // verify that only indexed fields were returned
-      assertEquals(11, indexedFieldNames.size());    // 6 original + the 5 termvector fields 
-      assertTrue(indexedFieldNames.contains("keyword"));
-      assertTrue(indexedFieldNames.contains("text"));
-      assertTrue(indexedFieldNames.contains("unstored"));
-      assertTrue(indexedFieldNames.contains("keyword2"));
-      assertTrue(indexedFieldNames.contains("text2"));
-      assertTrue(indexedFieldNames.contains("unstored2"));
-      assertTrue(indexedFieldNames.contains("tvnot"));
-      assertTrue(indexedFieldNames.contains("termvector"));
-      assertTrue(indexedFieldNames.contains("tvposition"));
-      assertTrue(indexedFieldNames.contains("tvoffset"));
-      assertTrue(indexedFieldNames.contains("tvpositionoffset"));
+    // verify that only indexed fields were returned
+    assertEquals(11, indexedFieldNames.size());    // 6 original + the 5 termvector fields 
+    assertTrue(indexedFieldNames.contains("keyword"));
+    assertTrue(indexedFieldNames.contains("text"));
+    assertTrue(indexedFieldNames.contains("unstored"));
+    assertTrue(indexedFieldNames.contains("keyword2"));
+    assertTrue(indexedFieldNames.contains("text2"));
+    assertTrue(indexedFieldNames.contains("unstored2"));
+    assertTrue(indexedFieldNames.contains("tvnot"));
+    assertTrue(indexedFieldNames.contains("termvector"));
+    assertTrue(indexedFieldNames.contains("tvposition"));
+    assertTrue(indexedFieldNames.contains("tvoffset"));
+    assertTrue(indexedFieldNames.contains("tvpositionoffset"));
       
-      // verify that only unindexed fields were returned
-      assertEquals(2, notIndexedFieldNames.size());    // the following fields
-      assertTrue(notIndexedFieldNames.contains("unindexed"));
-      assertTrue(notIndexedFieldNames.contains("unindexed2"));
+    // verify that only unindexed fields were returned
+    assertEquals(2, notIndexedFieldNames.size());    // the following fields
+    assertTrue(notIndexedFieldNames.contains("unindexed"));
+    assertTrue(notIndexedFieldNames.contains("unindexed2"));
               
-      // verify index term vector fields  
-      assertEquals(tvFieldNames.toString(), 4, tvFieldNames.size());    // 4 field has term vector only
-      assertTrue(tvFieldNames.contains("termvector"));
+    // verify index term vector fields  
+    assertEquals(tvFieldNames.toString(), 4, tvFieldNames.size());    // 4 field has term vector only
+    assertTrue(tvFieldNames.contains("termvector"));
 
-      reader.close();
-      d.close();
+    reader.close();
+    d.close();
   }
 
-public void testTermVectors() throws Exception {
-  Directory d = newDirectory();
-  // set up writer
-  IndexWriter writer = new IndexWriter(
-      d,
-      newIndexWriterConfig(new MockAnalyzer(random()))
-          .setMergePolicy(newLogMergePolicy())
-  );
-  // want to get some more segments here
-  // new termvector fields
-  int mergeFactor = ((LogMergePolicy) writer.getConfig().getMergePolicy()).getMergeFactor();
-  FieldType customType5 = new FieldType(TextField.TYPE_STORED);
-  customType5.setStoreTermVectors(true);
-  FieldType customType6 = new FieldType(TextField.TYPE_STORED);
-  customType6.setStoreTermVectors(true);
-  customType6.setStoreTermVectorOffsets(true);
-  FieldType customType7 = new FieldType(TextField.TYPE_STORED);
-  customType7.setStoreTermVectors(true);
-  customType7.setStoreTermVectorPositions(true);
-  FieldType customType8 = new FieldType(TextField.TYPE_STORED);
-  customType8.setStoreTermVectors(true);
-  customType8.setStoreTermVectorOffsets(true);
-  customType8.setStoreTermVectorPositions(true);
-  for (int i = 0; i < 5 * mergeFactor; i++) {
-    Document doc = new Document();
+  public void testTermVectors() throws Exception {
+    Directory d = newDirectory();
+    // set up writer
+    IndexWriter writer = new IndexWriter(
+                                         d,
+                                         newIndexWriterConfig(new MockAnalyzer(random()))
+                                         .setMergePolicy(newLogMergePolicy())
+                                         );
+    // want to get some more segments here
+    // new termvector fields
+    int mergeFactor = ((LogMergePolicy) writer.getConfig().getMergePolicy()).getMergeFactor();
+    FieldType customType5 = new FieldType(TextField.TYPE_STORED);
+    customType5.setStoreTermVectors(true);
+    FieldType customType6 = new FieldType(TextField.TYPE_STORED);
+    customType6.setStoreTermVectors(true);
+    customType6.setStoreTermVectorOffsets(true);
+    FieldType customType7 = new FieldType(TextField.TYPE_STORED);
+    customType7.setStoreTermVectors(true);
+    customType7.setStoreTermVectorPositions(true);
+    FieldType customType8 = new FieldType(TextField.TYPE_STORED);
+    customType8.setStoreTermVectors(true);
+    customType8.setStoreTermVectorOffsets(true);
+    customType8.setStoreTermVectorPositions(true);
+    for (int i = 0; i < 5 * mergeFactor; i++) {
+      Document doc = new Document();
       doc.add(new TextField("tvnot", "one two two three three three", Field.Store.YES));
       doc.add(new Field("termvector", "one two two three three three", customType5));
       doc.add(new Field("tvoffset", "one two two three three three", customType6));
@@ -337,30 +337,29 @@ public void testTermVectors() throws Exception {
       doc.add(new Field("tvpositionoffset", "one two two three three three", customType8));
       
       writer.addDocument(doc);
+    }
+    writer.close();
+    d.close();
   }
-  writer.close();
-  d.close();
-}
 
-void assertTermDocsCount(String msg,
-                                   IndexReader reader,
-                                   Term term,
-                                   int expected)
-  throws IOException {
-  PostingsEnum tdocs = TestUtil.docs(random(), reader,
-      term.field(),
-      new BytesRef(term.text()),
-      null,
-      0);
-  int count = 0;
-  if (tdocs != null) {
-    while(tdocs.nextDoc()!= DocIdSetIterator.NO_MORE_DOCS) {
-      count++;
+  void assertTermDocsCount(String msg,
+                           IndexReader reader,
+                           Term term,
+                           int expected)
+    throws IOException {
+    PostingsEnum tdocs = TestUtil.docs(random(), reader,
+                                       term.field(),
+                                       new BytesRef(term.text()),
+                                       null,
+                                       0);
+    int count = 0;
+    if (tdocs != null) {
+      while(tdocs.nextDoc()!= DocIdSetIterator.NO_MORE_DOCS) {
+        count++;
+      }
     }
+    assertEquals(msg + ", count mismatch", expected, count);
   }
-  assertEquals(msg + ", count mismatch", expected, count);
-}
-
   
   public void testBinaryFields() throws IOException {
     Directory dir = newDirectory();
@@ -439,7 +438,6 @@ void assertTermDocsCount(String msg,
   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())));
@@ -472,10 +470,6 @@ void assertTermDocsCount(String msg,
   public void testOpenReaderAfterDelete() throws IOException {
     Path dirFile = createTempDir("deletetest");
     Directory dir = newFSDirectory(dirFile);
-    if (TestUtil.hasVirusChecker(dir)) {
-      dir.close();
-      assumeTrue("test deletes files directly", false);
-    }
     if (dir instanceof BaseDirectoryWrapper) {
       ((BaseDirectoryWrapper)dir).setCheckIndexOnClose(false); // we will hit NoSuchFileException in MDW since we nuked it!
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/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 8c2dd91..50daac3 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
@@ -39,6 +39,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
@@ -224,13 +225,8 @@ public class TestIndexFileDeleter extends LuceneTestCase {
   
   public void testVirusScannerDoesntCorruptIndex() throws IOException {
     Path path = createTempDir();
-    VirusCheckingFS fs = new VirusCheckingFS(path.getFileSystem(), random().nextLong());
-    FileSystem filesystem = fs.getFileSystem(URI.create("file:///"));
-    fs.disable();
-
-    Path path2 = new FilterPath(path, filesystem);
-
-    Directory dir = newFSDirectory(path2);
+    Directory dir = newFSDirectory(addVirusChecker(path));
+    TestUtil.disableVirusChecker(dir);
     
     // add empty commit
     new IndexWriter(dir, new IndexWriterConfig(null)).close();
@@ -238,12 +234,12 @@ public class TestIndexFileDeleter extends LuceneTestCase {
     dir.createOutput("_0.si", IOContext.DEFAULT).close();
 
     // start virus scanner
-    fs.enable();
+    TestUtil.enableVirusChecker(dir);
     
     IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null));
     iw.addDocument(new Document());
     // stop virus scanner
-    fs.disable();
+    TestUtil.disableVirusChecker(dir);
     iw.commit();
     iw.close();
     dir.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index d6b8f84..1c3568a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -1265,7 +1265,6 @@ public class TestIndexWriter extends LuceneTestCase {
     for(int iter=0;iter<2;iter++) {
       // relies on windows semantics
       Path path = createTempDir();
-      assumeFalse("test directly deletes files", TestUtil.hasVirusChecker(path));
       FileSystem fs = new WindowsFS(path.getFileSystem()).getFileSystem(URI.create("file:///"));
       Path indexPath = new FilterPath(path, fs);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDeleteByQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDeleteByQuery.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDeleteByQuery.java
index c36cc1c..e87610a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDeleteByQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDeleteByQuery.java
@@ -27,7 +27,7 @@ public class TestIndexWriterDeleteByQuery extends LuceneTestCase {
 
   // LUCENE-6379
   public void testDeleteMatchAllDocsQuery() throws Exception {
-    Directory dir = newDirectory();
+    Directory dir = newMaybeVirusCheckingDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
     Document doc = new Document();
     // Norms are disabled:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java b/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java
index 87a0ea4..13aeb0b 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing.java
@@ -164,7 +164,7 @@ public class TestStressIndexing extends LuceneTestCase {
     FSDirectory.
   */
   public void testStressIndexAndSearching() throws Exception {
-    Directory directory = newDirectory();
+    Directory directory = newMaybeVirusCheckingDirectory();
     if (directory instanceof MockDirectoryWrapper) {
       ((MockDirectoryWrapper) directory).setAssertNoUnrefencedFilesOnClose(true);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
index 2f6f38a..7e08683 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestPointQueries.java
@@ -175,9 +175,9 @@ public class TestPointQueries extends LuceneTestCase {
     iwc.setCodec(getCodec());
     Directory dir;
     if (values.length > 100000) {
-      dir = newFSDirectory(createTempDir("TestRangeTree"));
+      dir = newMaybeVirusCheckingFSDirectory(createTempDir("TestRangeTree"));
     } else {
-      dir = getDirectory();
+      dir = newMaybeVirusCheckingDirectory();
     }
 
     int missingPct = random().nextInt(100);
@@ -441,7 +441,7 @@ public class TestPointQueries extends LuceneTestCase {
     if (docValues.length > 100000) {
       dir = newFSDirectory(createTempDir("TestPointQueries"));
     } else {
-      dir = getDirectory();
+      dir = newDirectory();
     }
 
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
@@ -721,7 +721,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
   
   public void testMinMaxLong() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
@@ -760,7 +760,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   public void testBasicSortedSet() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
@@ -825,7 +825,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   public void testLongMinMaxNumeric() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
@@ -851,7 +851,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   public void testLongMinMaxSortedSet() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
@@ -878,7 +878,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   public void testSortedSetNoOrdsMatch() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
@@ -900,7 +900,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   public void testNumericNoValuesMatch() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
@@ -920,7 +920,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   public void testNoDocs() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
@@ -935,7 +935,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   public void testWrongNumDims() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
@@ -958,7 +958,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   public void testWrongNumBytes() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
@@ -982,7 +982,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   public void testAllPointDocsWereDeletedAndThenMergedAgain() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     IndexWriter w = new IndexWriter(dir, iwc);
@@ -1018,10 +1018,6 @@ public class TestPointQueries extends LuceneTestCase {
     IOUtils.close(w, dir);
   }
 
-  private static Directory getDirectory() {     
-    return newDirectory();
-  }
-
   private static Codec getCodec() {
     if (Codec.getDefault().getName().equals("Lucene60")) {
       int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
@@ -1052,7 +1048,7 @@ public class TestPointQueries extends LuceneTestCase {
   }
 
   public void testExactPointQuery() throws Exception {
-    Directory dir = getDirectory();
+    Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig();
     iwc.setCodec(getCodec());
     IndexWriter w = new IndexWriter(dir, iwc);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
index 9c68b9e..b07ab0b 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
@@ -32,7 +32,6 @@ public class TestDirectory extends LuceneTestCase {
   // path, can read, write, and lock files.
   public void testDirectInstantiation() throws Exception {
     final Path path = createTempDir("testDirectInstantiation");
-    assumeFalse("test deletes files through different FSDir instances", TestUtil.hasVirusChecker(path));
     
     final byte[] largeBuffer = new byte[random().nextInt(256*1024)], largeReadBuffer = new byte[largeBuffer.length];
     for (int i = 0; i < largeBuffer.length; i++) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/core/src/test/org/apache/lucene/store/TestNativeFSLockFactory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestNativeFSLockFactory.java b/lucene/core/src/test/org/apache/lucene/store/TestNativeFSLockFactory.java
index 7cb36d6..7201bad 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestNativeFSLockFactory.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestNativeFSLockFactory.java
@@ -82,7 +82,6 @@ public class TestNativeFSLockFactory extends BaseLockFactoryTestCase {
   public void testDeleteLockFile() throws IOException {
     try (Directory dir = getDirectory(createTempDir())) {
       assumeFalse("we must be able to delete an open file", TestUtil.hasWindowsFS(dir));
-      assumeFalse("we must be able to delete an open file", TestUtil.hasVirusChecker(dir));
 
       Lock lock = dir.obtainLock("test.lock");
       lock.ensureValid();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSLockFactory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSLockFactory.java b/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSLockFactory.java
index 2baab10..d33d01c 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSLockFactory.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSLockFactory.java
@@ -35,7 +35,6 @@ public class TestSimpleFSLockFactory extends BaseLockFactoryTestCase {
   public void testDeleteLockFile() throws IOException {
     Directory dir = getDirectory(createTempDir());
     try {
-      assumeFalse("test directly deletes lock files", TestUtil.hasVirusChecker(dir));
       Lock lock = dir.obtainLock("test.lock");
       lock.ensureValid();
     

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/core/src/test/org/apache/lucene/util/TestIOUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestIOUtils.java b/lucene/core/src/test/org/apache/lucene/util/TestIOUtils.java
index 726f4f8..2c284bd 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestIOUtils.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestIOUtils.java
@@ -38,11 +38,9 @@ import java.util.UUID;
 import org.apache.lucene.mockfile.FilterFileSystem;
 import org.apache.lucene.mockfile.FilterFileSystemProvider;
 import org.apache.lucene.mockfile.FilterPath;
-import org.apache.lucene.mockfile.VirusCheckingFS;
 import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 
 /** Simple test methods for IOUtils */
-@SuppressFileSystems("VirusCheckingFS")
 public class TestIOUtils extends LuceneTestCase {
   
   public void testDeleteFileIgnoringExceptions() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java b/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
index ca8e802..522fbf9 100644
--- a/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
@@ -30,7 +30,6 @@ import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.apache.lucene.util.OfflineSorter.BufferSize;
 import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.OfflineSorter.SortInfo;
@@ -38,7 +37,6 @@ import org.apache.lucene.util.OfflineSorter.SortInfo;
 /**
  * Tests for on-disk merge sorting.
  */
-@SuppressFileSystems("VirusCheckingFS")
 public class TestOfflineSorter extends LuceneTestCase {
   private Path tempDir;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
index ac948d7..6cf8bb8 100644
--- a/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
+++ b/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
@@ -66,7 +66,6 @@ import org.apache.lucene.util.IntsRefBuilder;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.automaton.Automaton;
@@ -84,7 +83,6 @@ import static org.apache.lucene.util.fst.FSTTester.simpleRandomString;
 import static org.apache.lucene.util.fst.FSTTester.toIntsRef;
 
 @SuppressCodecs({ "SimpleText", "Memory", "Direct" })
-@SuppressFileSystems({ "VirusCheckingFS" })
 @Slow
 public class TestFSTs extends LuceneTestCase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAddTaxonomy.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAddTaxonomy.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAddTaxonomy.java
index 7dc73e1..a860ec9 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAddTaxonomy.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/directory/TestAddTaxonomy.java
@@ -31,7 +31,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
 
-//@SuppressFileSystems("VirusCheckingFS")
 public class TestAddTaxonomy extends FacetTestCase {
 
   private void dotest(int ncats, final int range) throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCompactLabelToOrdinal.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCompactLabelToOrdinal.java b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCompactLabelToOrdinal.java
index bf67929..49fc9bd 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCompactLabelToOrdinal.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCompactLabelToOrdinal.java
@@ -29,10 +29,8 @@ import java.util.Random;
 
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.taxonomy.FacetLabel;
-import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.junit.Test;
 
-@SuppressFileSystems("VirusCheckingFS")
 public class TestCompactLabelToOrdinal extends FacetTestCase {
 
   @Test

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyReplicationClientTest.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyReplicationClientTest.java b/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyReplicationClientTest.java
index 26e68d32..219e68b 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyReplicationClientTest.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyReplicationClientTest.java
@@ -52,14 +52,12 @@ import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-@SuppressFileSystems("VirusCheckingFS")
 public class IndexAndTaxonomyReplicationClientTest extends ReplicatorTestCase {
   
   private static class IndexAndTaxonomyReadyCallback implements Callable<Boolean>, Closeable {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/replicator/src/test/org/apache/lucene/replicator/IndexReplicationClientTest.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/IndexReplicationClientTest.java b/lucene/replicator/src/test/org/apache/lucene/replicator/IndexReplicationClientTest.java
index bcbf9cb..3f91013 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/IndexReplicationClientTest.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/IndexReplicationClientTest.java
@@ -33,14 +33,12 @@ import org.apache.lucene.replicator.ReplicationClient.SourceDirectoryFactory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-@SuppressFileSystems("VirusCheckingFS")
 public class IndexReplicationClientTest extends ReplicatorTestCase {
   
   private static class IndexReadyCallback implements Callable<Boolean>, Closeable {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/replicator/src/test/org/apache/lucene/replicator/http/HttpReplicatorTest.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/http/HttpReplicatorTest.java b/lucene/replicator/src/test/org/apache/lucene/replicator/http/HttpReplicatorTest.java
index 4298af8..6283d5b 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/http/HttpReplicatorTest.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/http/HttpReplicatorTest.java
@@ -36,14 +36,12 @@ import org.apache.lucene.replicator.Replicator;
 import org.apache.lucene.replicator.ReplicatorTestCase;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.servlet.ServletHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.Before;
 import org.junit.Test;
 
-@SuppressFileSystems("VirusCheckingFS")
 public class HttpReplicatorTest extends ReplicatorTestCase {
   private Path clientWorkDir;
   private Replicator serverReplicator;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
index 8fc65d7..b51894d 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggesterTest.java
@@ -45,12 +45,10 @@ import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.junit.Test;
 
-@SuppressFileSystems("VirusCheckingFS")
 public class AnalyzingInfixSuggesterTest extends LuceneTestCase {
 
   public void testBasic() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
----------------------------------------------------------------------
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
index 470d6a2..614a1a2 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggesterTest.java
@@ -33,11 +33,9 @@ import org.apache.lucene.search.suggest.Input;
 import org.apache.lucene.search.suggest.InputArrayIterator;
 import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
-@SuppressFileSystems("VirusCheckingFS")
 public class BlendedInfixSuggesterTest extends LuceneTestCase {
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/test-framework/src/java/org/apache/lucene/mockfile/VirusCheckingFS.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/mockfile/VirusCheckingFS.java b/lucene/test-framework/src/java/org/apache/lucene/mockfile/VirusCheckingFS.java
index 8b34551..5d14a3b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/mockfile/VirusCheckingFS.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/mockfile/VirusCheckingFS.java
@@ -33,7 +33,7 @@ import org.apache.lucene.util.LuceneTestCase;
  */
 public class VirusCheckingFS extends FilterFileSystemProvider {
 
-  private boolean enabled = true;
+  private volatile boolean enabled = true;
 
   private final AtomicLong state;
 
@@ -49,6 +49,10 @@ public class VirusCheckingFS extends FilterFileSystemProvider {
     enabled = true;
   }
 
+  public boolean isEnabled() {
+    return enabled;
+  }
+
   public void disable() {
     enabled = false;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
index ef8bf70..892da1e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
@@ -668,7 +668,6 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
   // LUCENE-3382 -- make sure we get exception if the directory really does not exist.
   public void testNoDir() throws Throwable {
     Path tempDir = createTempDir("doesnotexist");
-    assumeFalse("test directly deletes files", TestUtil.hasVirusChecker(tempDir));
     IOUtils.rm(tempDir);
     Directory dir = getDirectory(tempDir);
     try {
@@ -827,7 +826,6 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
   // TODO: somehow change this test to 
   public void testFsyncDoesntCreateNewFiles() throws Exception {
     Path path = createTempDir("nocreate");
-    assumeFalse("we directly delete files", TestUtil.hasVirusChecker(path));
     Directory fsdir = getDirectory(path);
     
     // this test backdoors the directory via the filesystem. so it must be an FSDir (for now)
@@ -1227,4 +1225,68 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
       }
     }
   }
+
+  // Make sure the FSDirectory impl properly "emulates" deletions on filesystems (Windows) with buggy deleteFile:
+  public void testPendingDeletions() throws IOException {
+    try (Directory dir = getDirectory(addVirusChecker(createTempDir()))) {
+      assumeTrue("we can only install VirusCheckingFS on an FSDirectory", dir instanceof FSDirectory);
+      FSDirectory fsDir = (FSDirectory) dir;
+
+      // Keep trying until virus checker refuses to delete:
+      String fileName;
+      while (true) {
+        fileName = TestUtil.randomSimpleString(random());
+        if (fileName.length() == 0) {
+          continue;
+        }
+        try (IndexOutput out = dir.createOutput(fileName, IOContext.DEFAULT)) {
+        }
+        fsDir.deleteFile(fileName);
+        if (fsDir.checkPendingDeletions()) {
+          // good: virus checker struck and prevented deletion of fileName
+          break;
+        }
+      }
+
+      // Make sure listAll does NOT include the file:
+      assertFalse(Arrays.asList(fsDir.listAll()).contains(fileName));
+
+      // Make sure fileLength claims it's deleted:
+      try {
+        fsDir.fileLength(fileName);
+        fail("did not hit exception");
+      } catch (NoSuchFileException nsfe) {
+        // expected
+      }
+
+      // Make sure rename fails:
+      try {
+        fsDir.renameFile(fileName, "file2");
+        fail("did not hit exception");
+      } catch (NoSuchFileException nsfe) {
+        // expected
+      }
+
+      // Make sure delete fails:
+      try {
+        fsDir.deleteFile(fileName);
+        fail("did not hit exception");
+      } catch (NoSuchFileException nsfe) {
+        // expected
+      }
+
+      try {
+        fsDir.openInput(fileName, IOContext.DEFAULT);
+        fail("did not hit exception");
+      } catch (NoSuchFileException nsfe) {
+        // expected
+      }
+
+      // write the file again
+      try (IndexOutput out = dir.createOutput(fileName, IOContext.DEFAULT)) {
+      }
+      assertEquals(0, fsDir.fileLength(fileName));
+      assertTrue(Arrays.asList(fsDir.listAll()).contains(fileName));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java
index 5dfe3e8..2b6a6e3 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java
@@ -56,7 +56,6 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
   /** Test obtaining and releasing locks, checking validity */
   public void testBasics() throws IOException {
     Path tempPath = createTempDir();
-    assumeFalse("test works with lock files", TestUtil.hasVirusChecker(tempPath));
     Directory dir = getDirectory(tempPath);
     
     Lock l = dir.obtainLock("commit");
@@ -76,7 +75,6 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
   /** Test closing locks twice */
   public void testDoubleClose() throws IOException {
     Path tempPath = createTempDir();
-    assumeFalse("test works with lock files", TestUtil.hasVirusChecker(tempPath));
     Directory dir = getDirectory(tempPath);
     
     Lock l = dir.obtainLock("commit");
@@ -89,7 +87,6 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
   /** Test ensureValid returns true after acquire */
   public void testValidAfterAcquire() throws IOException {
     Path tempPath = createTempDir();
-    assumeFalse("test works with lock files", TestUtil.hasVirusChecker(tempPath));
     Directory dir = getDirectory(tempPath);
     Lock l = dir.obtainLock("commit");
     l.ensureValid(); // no exception
@@ -100,7 +97,6 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
   /** Test ensureValid throws exception after close */
   public void testInvalidAfterClose() throws IOException {
     Path tempPath = createTempDir();
-    assumeFalse("test works with lock files", TestUtil.hasVirusChecker(tempPath));
     Directory dir = getDirectory(tempPath);
     
     Lock l = dir.obtainLock("commit");
@@ -115,7 +111,6 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
   
   public void testObtainConcurrently() throws InterruptedException, IOException {
     Path tempPath = createTempDir();
-    assumeFalse("test works with lock files", TestUtil.hasVirusChecker(tempPath));
     final Directory directory = getDirectory(tempPath);
     final AtomicBoolean running = new AtomicBoolean(true);
     final AtomicInteger atomicCounter = new AtomicInteger(0);
@@ -165,7 +160,6 @@ public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
   // no unexpected exceptions are raised:
   public void testStressLocks() throws Exception {
     Path tempPath = createTempDir();
-    assumeFalse("test works with lock files", TestUtil.hasVirusChecker(tempPath));
     Directory dir = getDirectory(tempPath);
 
     // First create a 1 doc index:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
index b86c043..4287ca9 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
@@ -20,6 +20,7 @@ package org.apache.lucene.store;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.FileSystem;
 import java.nio.file.NoSuchFileException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -43,6 +44,8 @@ import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.NoDeletionPolicy;
+import org.apache.lucene.mockfile.FilterFileSystem;
+import org.apache.lucene.mockfile.VirusCheckingFS;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -270,7 +273,10 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
         f.close();
       } catch (Exception ignored) {}
     }
-    
+
+    // Maybe disable virus checker so it doesn't interfere with our efforts to corrupt files below:
+    boolean virusCheckerWasEnabled = TestUtil.disableVirusChecker(in);
+
     while(it.hasNext()) {
       String name = it.next();
       int damage = randomState.nextInt(5);
@@ -318,14 +324,6 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
           ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState));
           out.copyBytes(ii, ii.length());
           ii.close();
-        } catch (IOException ioe) {
-          // VirusCheckingFS may have blocked the delete, at which point FSDir cannot overwrite here
-          if (ioe.getMessage().equals("file \"" + name + "\" is pending delete and cannot be overwritten")) {
-            // OK
-            action = "deleted";
-          } else {
-            throw ioe;
-          }
         }
         deleteFile(tempFileName);
       } else if (damage == 3) {
@@ -336,16 +334,12 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
         // Totally truncate the file to zero bytes
         deleteFile(name);
         try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
-        } catch (IOException ioe) {
-          // VirusCheckingFS may have blocked the delete, at which point FSDir cannot overwrite here
-          if (ioe.getMessage().equals("file \"" + name + "\" is pending delete and cannot be overwritten")) {
-            // OK
-            action = "deleted";
-          } else {
-            throw ioe;
-          }
         }
       }
+      // Re-enable
+      if (virusCheckerWasEnabled) {
+        TestUtil.enableVirusChecker(in);
+      }
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockDirectoryWrapper: " + action + " unsynced file: " + name);
       }
@@ -746,11 +740,16 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
           
           // TODO: factor this out / share w/ TestIW.assertNoUnreferencedFiles
           if (assertNoUnreferencedFilesOnClose) {
+
             // now look for unreferenced files: discount ones that we tried to delete but could not
             Set<String> allFiles = new HashSet<>(Arrays.asList(listAll()));
             String[] startFiles = allFiles.toArray(new String[0]);
             IndexWriterConfig iwc = new IndexWriterConfig(null);
             iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
+
+            // We must do this before opening writer otherwise writer will be angry if there are pending deletions:
+            TestUtil.disableVirusChecker(in);
+
             new IndexWriter(in, iwc).rollback();
             String[] endFiles = in.listAll();
             

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index c7c45c4..ffcd80b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -30,6 +30,8 @@ import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
+import java.net.URI;
+import java.nio.file.FileSystem;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -64,7 +66,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.logging.Logger;
 
-import junit.framework.AssertionFailedError;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
@@ -77,6 +78,8 @@ import org.apache.lucene.index.*;
 import org.apache.lucene.index.IndexReader.ReaderClosedListener;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
+import org.apache.lucene.mockfile.FilterPath;
+import org.apache.lucene.mockfile.VirusCheckingFS;
 import org.apache.lucene.search.AssertingIndexSearcher;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
@@ -138,6 +141,8 @@ import com.carrotsearch.randomizedtesting.rules.NoClassHooksShadowingRule;
 import com.carrotsearch.randomizedtesting.rules.NoInstanceHooksOverridesRule;
 import com.carrotsearch.randomizedtesting.rules.StaticFieldsInvariantRule;
 
+import junit.framework.AssertionFailedError;
+
 import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsBoolean;
 import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAsInt;
 
@@ -1266,6 +1271,16 @@ public abstract class LuceneTestCase extends Assert {
   public static BaseDirectoryWrapper newDirectory() {
     return newDirectory(random());
   }
+
+  /** Like {@link newDirectory} except randomly the {@link VirusCheckingFS} may be installed */
+  public static BaseDirectoryWrapper newMaybeVirusCheckingDirectory() {
+    if (random().nextInt(5) == 4) {
+      Path path = addVirusChecker(createTempDir());
+      return newFSDirectory(path);
+    } else {
+      return newDirectory(random());
+    }
+  }
   
   /**
    * Returns a new Directory instance, using the specified random.
@@ -1303,6 +1318,15 @@ public abstract class LuceneTestCase extends Assert {
     return (MockDirectoryWrapper) newFSDirectory(f, lf, false);
   }
 
+  public static Path addVirusChecker(Path path) {
+    if (TestUtil.hasVirusChecker(path) == false) {
+      VirusCheckingFS fs = new VirusCheckingFS(path.getFileSystem(), random().nextLong());
+      FileSystem filesystem = fs.getFileSystem(URI.create("file:///"));
+      path = new FilterPath(path, filesystem);
+    }
+    return path;
+  }
+
   /**
    * Returns a new Directory instance, with contents copied from the
    * provided directory. See {@link #newDirectory()} for more
@@ -1317,6 +1341,14 @@ public abstract class LuceneTestCase extends Assert {
     return newFSDirectory(f, FSLockFactory.getDefault());
   }
 
+  /** Like {@link newFSDirectory(Path)}, but randomly insert {@link VirusCheckingFS} */
+  public static BaseDirectoryWrapper newMaybeVirusCheckingFSDirectory(Path f) {
+    if (random().nextInt(5) == 4) {
+      f = addVirusChecker(f);
+    }
+    return newFSDirectory(f, FSLockFactory.getDefault());
+  }
+
   /** Returns a new FSDirectory instance over the given file, which must be a folder. */
   public static BaseDirectoryWrapper newFSDirectory(Path f, LockFactory lf) {
     return newFSDirectory(f, lf, rarely());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java
index c864ec1..dc2b10e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java
@@ -39,7 +39,6 @@ import org.apache.lucene.mockfile.HandleLimitFS;
 import org.apache.lucene.mockfile.LeakFS;
 import org.apache.lucene.mockfile.ShuffleFS;
 import org.apache.lucene.mockfile.VerboseFS;
-import org.apache.lucene.mockfile.VirusCheckingFS;
 import org.apache.lucene.mockfile.WindowsFS;
 import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
 import org.apache.lucene.util.LuceneTestCase.SuppressFsync;
@@ -76,11 +75,6 @@ final class TestRuleTemporaryFilesCleanup extends TestRuleAdapter {
   private FileSystem fileSystem;
 
   /**
-   * Only set if the file system chain includes the VirusCheckingFS
-   */
-  private VirusCheckingFS virusCheckingFS;
-
-  /**
    * Suite failure marker.
    */
   private final TestRuleMarkFailure failureMarker;
@@ -180,11 +174,6 @@ final class TestRuleTemporaryFilesCleanup extends TestRuleAdapter {
       if (allowed(avoid, ExtrasFS.class)) {
         fs = new ExtrasFS(fs, random.nextInt(4) == 0, random.nextBoolean()).getFileSystem(null);
       }
-      if (allowed(avoid, VirusCheckingFS.class) && random.nextInt(10) == 1) {
-        // 10% of the time we swap in virus checking (acts-like-windows) FS:    
-        virusCheckingFS = new VirusCheckingFS(fs, random.nextLong());
-        fs = virusCheckingFS.getFileSystem(null);
-      }
     }
     if (LuceneTestCase.VERBOSE) {
       System.out.println("filesystem: " + fs.provider());
@@ -224,11 +213,6 @@ final class TestRuleTemporaryFilesCleanup extends TestRuleAdapter {
     // and leave them there.
     if (failureMarker.wasSuccessful()) {
 
-      if (virusCheckingFS != null) {
-        // Otherwise our IOUtils.rm below can fail:
-        virusCheckingFS.disable();
-      }
-
       try {
         IOUtils.rm(everything);
       } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/85c546b7/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index 0329f4b..50a2204 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -1303,6 +1303,19 @@ public final class TestUtil {
     return false;
   }
 
+  public static boolean hasWindowsFS(Path path) {
+    FileSystem fs = path.getFileSystem();
+    while (fs instanceof FilterFileSystem) {
+      FilterFileSystem ffs = (FilterFileSystem) fs;
+      if (ffs.getParent() instanceof WindowsFS) {
+        return true;
+      }
+      fs = ffs.getDelegate();
+    }
+
+    return false;
+  }
+
   public static boolean hasVirusChecker(Directory dir) {
     dir = FilterDirectory.unwrap(dir);
     if (dir instanceof FSDirectory) {
@@ -1324,4 +1337,42 @@ public final class TestUtil {
 
     return false;
   }
+
+  /** Returns true if VirusCheckingFS is in use and was in fact already enabled */
+  public static boolean disableVirusChecker(Directory in) {
+    Directory dir = FilterDirectory.unwrap(in);
+    if (dir instanceof FSDirectory) {
+
+      FileSystem fs = ((FSDirectory) dir).getDirectory().getFileSystem();
+      while (fs instanceof FilterFileSystem) {
+        FilterFileSystem ffs = (FilterFileSystem) fs;
+        if (ffs.getParent() instanceof VirusCheckingFS) {
+          VirusCheckingFS vfs = (VirusCheckingFS) ffs.getParent();
+          boolean isEnabled = vfs.isEnabled();
+          vfs.disable();
+          return isEnabled;
+        }
+        fs = ffs.getDelegate();
+      }
+    }
+
+    return false;
+  }
+
+  public static void enableVirusChecker(Directory in) {
+    Directory dir = FilterDirectory.unwrap(in);
+    if (dir instanceof FSDirectory) {
+
+      FileSystem fs = ((FSDirectory) dir).getDirectory().getFileSystem();
+      while (fs instanceof FilterFileSystem) {
+        FilterFileSystem ffs = (FilterFileSystem) fs;
+        if (ffs.getParent() instanceof VirusCheckingFS) {
+          VirusCheckingFS vfs = (VirusCheckingFS) ffs.getParent();
+          vfs.enable();
+          return;
+        }
+        fs = ffs.getDelegate();
+      }
+    }
+  }
 }