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 2014/08/29 23:51:42 UTC

svn commit: r1621392 [2/2] - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/lucene40/ lucene/core/src/java/org/apache/l...

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java Fri Aug 29 21:51:40 2014
@@ -29,6 +29,7 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.junit.Test;
@@ -101,6 +102,10 @@ public class TestSnapshotDeletionPolicy 
   }
 
   private void runTest(Random random, Directory dir) throws Exception {
+    // we use the IW unref'ed files check which is unaware of retries:
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
+    }
     // Run for ~1 seconds
     final long stopTime = System.currentTimeMillis() + 1000;
 
@@ -251,6 +256,10 @@ public class TestSnapshotDeletionPolicy 
     
     // Create 3 snapshots: snapshot0, snapshot1, snapshot2
     Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      // we verify some files get deleted
+      ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
+    }
     IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
     SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
     prepareIndexAndSnapshots(sdp, writer, numSnapshots);
@@ -275,6 +284,10 @@ public class TestSnapshotDeletionPolicy 
   @Test
   public void testMultiThreadedSnapshotting() throws Exception {
     Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      // test relies on files actually being deleted
+      ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
+    }
     final IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
     final SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
 
@@ -346,6 +359,10 @@ public class TestSnapshotDeletionPolicy 
   @Test
   public void testReleaseSnapshot() throws Exception {
     Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      // we rely upon existence of files
+      ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
+    }
     IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
     SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
     prepareIndexAndSnapshots(sdp, writer, 1);
@@ -395,6 +412,10 @@ public class TestSnapshotDeletionPolicy 
     // Tests the behavior of SDP when commits that are given at ctor are missing
     // on onInit().
     Directory dir = newDirectory();
+    if (dir instanceof MockDirectoryWrapper) {
+      // we rely upon existence of files
+      ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
+    }
     IndexWriter writer = new IndexWriter(dir, getConfig(random(), getDeletionPolicy()));
     SnapshotDeletionPolicy sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
     writer.addDocument(new Document());

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java Fri Aug 29 21:51:40 2014
@@ -30,11 +30,17 @@ public class TestDirectory extends BaseD
 
   @Override
   protected Directory getDirectory(File path) throws IOException {
+    final Directory dir;
     if (random().nextBoolean()) {
-      return newDirectory();
+      dir = newDirectory();
     } else {
-      return newFSDirectory(path);
+      dir = newFSDirectory(path);
     }
+    if (dir instanceof MockDirectoryWrapper) {
+      // test manipulates directory directly
+      ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
+    }
+    return dir;
   }
 
   // we wrap the directory in slow stuff, so only run nightly

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestRateLimitedDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestRateLimitedDirectoryWrapper.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestRateLimitedDirectoryWrapper.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestRateLimitedDirectoryWrapper.java Fri Aug 29 21:51:40 2014
@@ -23,7 +23,12 @@ public class TestRateLimitedDirectoryWra
 
   @Override
   protected Directory getDirectory(File path) {
-    RateLimitedDirectoryWrapper dir = new RateLimitedDirectoryWrapper(newFSDirectory(path));
+    Directory in = newFSDirectory(path);
+    if (in instanceof MockDirectoryWrapper) {
+      // test manipulates directory directly
+      ((MockDirectoryWrapper)in).setEnableVirusScanner(false);
+    }
+    RateLimitedDirectoryWrapper dir = new RateLimitedDirectoryWrapper(in);
     RateLimiter limiter = new RateLimiter.SimpleRateLimiter(.1 + 3*random().nextDouble());
     dir.setRateLimiter(limiter, IOContext.Context.MERGE);
     return dir;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Fri Aug 29 21:51:40 2014
@@ -95,6 +95,7 @@ public class TestFSTs extends LuceneTest
     super.setUp();
     dir = newMockDirectory();
     dir.setPreventDoubleWrite(false);
+    dir.setEnableVirusScanner(false);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/util/fst/TestFSTsMisc.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/util/fst/TestFSTsMisc.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/util/fst/TestFSTsMisc.java (original)
+++ lucene/dev/branches/branch_4x/lucene/misc/src/test/org/apache/lucene/util/fst/TestFSTsMisc.java Fri Aug 29 21:51:40 2014
@@ -46,6 +46,7 @@ public class TestFSTsMisc extends Lucene
     super.setUp();
     dir = newMockDirectory();
     dir.setPreventDoubleWrite(false);
+    dir.setEnableVirusScanner(false);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/replicator/src/java/org/apache/lucene/replicator/IndexReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/replicator/src/java/org/apache/lucene/replicator/IndexReplicationHandler.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/replicator/src/java/org/apache/lucene/replicator/IndexReplicationHandler.java (original)
+++ lucene/dev/branches/branch_4x/lucene/replicator/src/java/org/apache/lucene/replicator/IndexReplicationHandler.java Fri Aug 29 21:51:40 2014
@@ -35,6 +35,7 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.replicator.ReplicationClient.ReplicationHandler;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InfoStream;
 
 /**
@@ -122,12 +123,9 @@ public class IndexReplicationHandler imp
    */
   public static void cleanupFilesOnFailure(Directory dir, List<String> files) {
     for (String file : files) {
-      try {
-        dir.deleteFile(file);
-      } catch (Throwable t) {
-        // suppress any exception because if we're here, it means copy
-        // failed, and we must cleanup after ourselves.
-      }
+      // suppress any exception because if we're here, it means copy
+      // failed, and we must cleanup after ourselves.
+      IOUtils.deleteFilesIgnoringExceptions(dir, file);
     }
   }
   
@@ -155,11 +153,8 @@ public class IndexReplicationHandler imp
         for (String file : dir.listAll()) {
           if (!commitFiles.contains(file)
               && (matcher.reset(file).matches() || file.startsWith(IndexFileNames.SEGMENTS))) {
-            try {
-              dir.deleteFile(file);
-            } catch (Throwable t) {
-              // suppress, it's just a best effort
-            }
+            // suppress exceptions, it's just a best effort
+            IOUtils.deleteFilesIgnoringExceptions(dir, file);
           }
         }
       }
@@ -194,11 +189,7 @@ public class IndexReplicationHandler imp
     if (segmentsFile != null) {
       SegmentInfos.writeSegmentsGen(dir, SegmentInfos.generationFromSegmentsFileName(segmentsFile));
     } else {
-      try {
-        dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
-      } catch (Throwable t) {
-        // suppress any errors while deleting this file.
-      }
+      IOUtils.deleteFilesIgnoringExceptions(dir, IndexFileNames.SEGMENTS_GEN);
     }
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyRevisionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyRevisionTest.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyRevisionTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyRevisionTest.java Fri Aug 29 21:51:40 2014
@@ -36,6 +36,7 @@ import org.apache.lucene.replicator.Inde
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.Version;
 import org.junit.Test;
@@ -77,6 +78,10 @@ public class IndexAndTaxonomyRevisionTes
     
     Directory taxoDir = newDirectory();
     SnapshotDirectoryTaxonomyWriter taxoWriter = new SnapshotDirectoryTaxonomyWriter(taxoDir);
+    // we look to see that certain files are deleted:
+    if (indexDir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper)indexDir).setEnableVirusScanner(false);
+    }
     try {
       indexWriter.addDocument(newDocument(taxoWriter));
       indexWriter.commit();
@@ -96,6 +101,10 @@ public class IndexAndTaxonomyRevisionTes
       assertFalse(slowFileExists(indexDir, IndexFileNames.SEGMENTS + "_1"));
     } finally {
       IOUtils.close(indexWriter, taxoWriter, taxoDir, indexDir);
+      if (indexDir instanceof MockDirectoryWrapper) {
+        // set back to on for other tests
+        ((MockDirectoryWrapper)indexDir).setEnableVirusScanner(true);
+      }
     }
   }
   

Modified: lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexRevisionTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexRevisionTest.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexRevisionTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexRevisionTest.java Fri Aug 29 21:51:40 2014
@@ -30,6 +30,7 @@ import org.apache.lucene.index.SnapshotD
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.Version;
 import org.junit.Test;
@@ -71,6 +72,10 @@ public class IndexRevisionTest extends R
   @Test
   public void testRevisionRelease() throws Exception {
     Directory dir = newDirectory();
+    // we look to see that certain files are deleted:
+    if (dir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
+    }
     IndexWriterConfig conf = new IndexWriterConfig(Version.LATEST, null);
     conf.setIndexDeletionPolicy(new SnapshotDeletionPolicy(conf.getIndexDeletionPolicy()));
     IndexWriter writer = new IndexWriter(dir, conf);

Modified: lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/LocalReplicatorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/LocalReplicatorTest.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/LocalReplicatorTest.java (original)
+++ lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/LocalReplicatorTest.java Fri Aug 29 21:51:40 2014
@@ -32,6 +32,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.SnapshotDeletionPolicy;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.IOUtils;
 import org.junit.After;
 import org.junit.Before;
@@ -117,35 +118,57 @@ public class LocalReplicatorTest extends
   
   @Test
   public void testPublishSameRevision() throws IOException {
-    Revision rev = createRevision(1);
-    replicator.publish(rev);
-    SessionToken res = replicator.checkForUpdate(null);
-    assertNotNull(res);
-    assertEquals(rev.getVersion(), res.version);
-    replicator.release(res.id);
-    replicator.publish(new IndexRevision(sourceWriter));
-    res = replicator.checkForUpdate(res.version);
-    assertNull(res);
-    
-    // now make sure that publishing same revision doesn't leave revisions
-    // "locked", i.e. that replicator releases revisions even when they are not
-    // kept
-    replicator.publish(createRevision(2));
-    assertEquals(1, DirectoryReader.listCommits(sourceDir).size());
+    // we look to see that certain files are deleted:
+    if (sourceDir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper)sourceDir).setEnableVirusScanner(false);
+    }
+    try {
+      Revision rev = createRevision(1);
+      replicator.publish(rev);
+      SessionToken res = replicator.checkForUpdate(null);
+      assertNotNull(res);
+      assertEquals(rev.getVersion(), res.version);
+      replicator.release(res.id);
+      replicator.publish(new IndexRevision(sourceWriter));
+      res = replicator.checkForUpdate(res.version);
+      assertNull(res);
+      
+      // now make sure that publishing same revision doesn't leave revisions
+      // "locked", i.e. that replicator releases revisions even when they are not
+      // kept
+      replicator.publish(createRevision(2));
+      assertEquals(1, DirectoryReader.listCommits(sourceDir).size());
+    } finally {
+      if (sourceDir instanceof MockDirectoryWrapper) {
+        // set back to on for other tests
+        ((MockDirectoryWrapper)sourceDir).setEnableVirusScanner(true);
+      }
+    }
   }
   
   @Test
   public void testPublishOlderRev() throws IOException {
-    replicator.publish(createRevision(1));
-    Revision old = new IndexRevision(sourceWriter);
-    replicator.publish(createRevision(2));
+    // we look to see that certain files are deleted:
+    if (sourceDir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper)sourceDir).setEnableVirusScanner(false);
+    }
     try {
-      replicator.publish(old);
-      fail("should have failed to publish an older revision");
-    } catch (IllegalArgumentException e) {
-      // expected
+      replicator.publish(createRevision(1));
+      Revision old = new IndexRevision(sourceWriter);
+      replicator.publish(createRevision(2));
+      try {
+        replicator.publish(old);
+        fail("should have failed to publish an older revision");
+      } catch (IllegalArgumentException e) {
+        // expected
+      }
+      assertEquals(1, DirectoryReader.listCommits(sourceDir).size());
+    } finally {
+      if (sourceDir instanceof MockDirectoryWrapper) {
+        // set back to on for other tests
+        ((MockDirectoryWrapper)sourceDir).setEnableVirusScanner(true);
+      }
     }
-    assertEquals(1, DirectoryReader.listCommits(sourceDir).size());
   }
   
   @Test
@@ -186,12 +209,24 @@ public class LocalReplicatorTest extends
   
   @Test
   public void testRevisionRelease() throws Exception {
-    replicator.publish(createRevision(1));
-    assertTrue(slowFileExists(sourceDir, IndexFileNames.SEGMENTS + "_1"));
-    replicator.publish(createRevision(2));
-    // now the files of revision 1 can be deleted
-    assertTrue(slowFileExists(sourceDir, IndexFileNames.SEGMENTS + "_2"));
-    assertFalse("segments_1 should not be found in index directory after revision is released", slowFileExists(sourceDir, IndexFileNames.SEGMENTS + "_1"));
+    // we look to see that certain files are deleted:
+    if (sourceDir instanceof MockDirectoryWrapper) {
+      ((MockDirectoryWrapper)sourceDir).setEnableVirusScanner(false);
+    }
+    
+    try {
+      replicator.publish(createRevision(1));
+      assertTrue(slowFileExists(sourceDir, IndexFileNames.SEGMENTS + "_1"));
+      replicator.publish(createRevision(2));
+      // now the files of revision 1 can be deleted
+      assertTrue(slowFileExists(sourceDir, IndexFileNames.SEGMENTS + "_2"));
+      assertFalse("segments_1 should not be found in index directory after revision is released", slowFileExists(sourceDir, IndexFileNames.SEGMENTS + "_1"));
+    } finally {
+      if (sourceDir instanceof MockDirectoryWrapper) {
+        // set back to on for other tests
+        ((MockDirectoryWrapper)sourceDir).setEnableVirusScanner(true);
+      }
+    }
   }
   
 }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Fri Aug 29 21:51:40 2014
@@ -76,6 +76,7 @@ public class MockDirectoryWrapper extend
   boolean trackDiskUsage = false;
   boolean wrapLockFactory = true;
   boolean useSlowOpenClosers = true;
+  boolean enableVirusScanner = true;
   boolean allowRandomFileNotFoundException = true;
   boolean allowReadingFilesStillOpenForWrite = false;
   private Set<String> unSyncedFiles;
@@ -102,11 +103,16 @@ public class MockDirectoryWrapper extend
   // Only tracked if noDeleteOpenFile is true: if an attempt
   // is made to delete an open file, we enroll it here.
   private Set<String> openFilesDeleted;
+  
+  // only tracked if virus scanner is enabled:
+  // set of files it prevented deletion for
+  private Set<String> triedToDelete;
 
   private synchronized void init() {
     if (openFiles == null) {
       openFiles = new HashMap<>();
       openFilesDeleted = new HashSet<>();
+      triedToDelete = new HashSet<>();
     }
 
     if (createdFiles == null)
@@ -155,6 +161,18 @@ public class MockDirectoryWrapper extend
   public void setAllowReadingFilesStillOpenForWrite(boolean value) {
     allowReadingFilesStillOpenForWrite = value;
   }
+  
+  /** Returns true if the virus scanner is enabled */
+  public boolean getEnableVirusScanner() {
+    return enableVirusScanner;
+  }
+  
+  /** If set to true (the default), deleteFile sometimes
+   *  fails because a virus scanner is open.
+   */
+  public void setEnableVirusScanner(boolean value) {
+    this.enableVirusScanner = value;
+  }
 
   /**
    * Enum for controlling hard disk throttling.
@@ -466,6 +484,14 @@ public class MockDirectoryWrapper extend
         openFilesDeleted.remove(name);
       }
     }
+    if (!forced && enableVirusScanner && (randomState.nextInt(4) == 0)) {
+      triedToDelete.add(name);
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MDW: now refuse to delete file: " + name);
+      }
+      throw new IOException("cannot delete file: " + name + ", a virus scanner has it open");
+    }
+    triedToDelete.remove(name);
     in.deleteFile(name);
   }
 
@@ -655,7 +681,10 @@ public class MockDirectoryWrapper extend
   public synchronized void close() throws IOException {
     // files that we tried to delete, but couldn't because readers were open.
     // all that matters is that we tried! (they will eventually go away)
+    //   still open when we tried to delete
     Set<String> pendingDeletions = new HashSet<>(openFilesDeleted);
+    //   virus scanner when we tried to delete
+    pendingDeletions.addAll(triedToDelete);
     maybeYield();
     if (openFiles == null) {
       openFiles = new HashMap<>();

Modified: lucene/dev/branches/branch_4x/solr/test-framework/src/java/org/apache/solr/core/MockDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/test-framework/src/java/org/apache/solr/core/MockDirectoryFactory.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/test-framework/src/java/org/apache/solr/core/MockDirectoryFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/test-framework/src/java/org/apache/solr/core/MockDirectoryFactory.java Fri Aug 29 21:51:40 2014
@@ -63,6 +63,9 @@ public class MockDirectoryFactory extend
       // already been created.
       mockDirWrapper.setPreventDoubleWrite(false);
       
+      // snappuller & co don't seem ready for this:
+      mockDirWrapper.setEnableVirusScanner(false);
+      
       if (allowReadingFilesStillOpenForWrite) {
         mockDirWrapper.setAllowReadingFilesStillOpenForWrite(true);
       }

Modified: lucene/dev/branches/branch_4x/solr/test-framework/src/java/org/apache/solr/core/MockFSDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/test-framework/src/java/org/apache/solr/core/MockFSDirectoryFactory.java?rev=1621392&r1=1621391&r2=1621392&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/test-framework/src/java/org/apache/solr/core/MockFSDirectoryFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/test-framework/src/java/org/apache/solr/core/MockFSDirectoryFactory.java Fri Aug 29 21:51:40 2014
@@ -49,6 +49,7 @@ public class MockFSDirectoryFactory exte
     if (cdir instanceof MockDirectoryWrapper) {
       ((MockDirectoryWrapper)cdir).setAssertNoUnrefencedFilesOnClose(false);
       ((MockDirectoryWrapper)cdir).setPreventDoubleWrite(false);
+      ((MockDirectoryWrapper)cdir).setEnableVirusScanner(false);
     }
     return dir;
   }