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/03/23 19:03:56 UTC

svn commit: r1580557 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/ lucene/core/src/java/org/apache/lucene/store/ lucene/core/src/test/org/apache/lucene/index/ lucene/core/src/test/org/apache/...

Author: mikemccand
Date: Sun Mar 23 18:03:55 2014
New Revision: 1580557

URL: http://svn.apache.org/r1580557
Log:
LUCENE-5543 remove/deprecate Directory.fileExists

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt
    lucene/dev/branches/branch_4x/lucene/core/   (props changed)
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/Directory.java
    lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOutOfFileDescriptors.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestSnapshotDeletionPolicy.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java
    lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
    lucene/dev/branches/branch_4x/lucene/replicator/   (props changed)
    lucene/dev/branches/branch_4x/lucene/replicator/src/java/org/apache/lucene/replicator/IndexReplicationHandler.java
    lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexAndTaxonomyRevisionTest.java
    lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/IndexRevisionTest.java
    lucene/dev/branches/branch_4x/lucene/replicator/src/test/org/apache/lucene/replicator/LocalReplicatorTest.java
    lucene/dev/branches/branch_4x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
    lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/store/hdfs/HdfsDirectoryTest.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Sun Mar 23 18:03:55 2014
@@ -119,6 +119,8 @@ API Changes
   at an arbitrary position in the analysis chain. Use ThaiTokenizer and HMMChineseTokenizer
   instead. (Robert Muir)
 
+* LUCENE-5543: Remove/deprecate Directory.fileExists (Mike McCandless)
+
 Optimizations
 
 * LUCENE-5468: HunspellStemFilter uses 10 to 100x less RAM. It also loads

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java Sun Mar 23 18:03:55 2014
@@ -584,23 +584,20 @@ final class IndexFileDeleter implements 
       }
       directory.deleteFile(fileName);
     } catch (IOException e) {  // if delete fails
-      if (directory.fileExists(fileName)) {
+      // Some operating systems (e.g. Windows) don't
+      // permit a file to be deleted while it is opened
+      // for read (e.g. by another process or thread). So
+      // we assume that when a delete fails it is because
+      // the file is open in another process, and queue
+      // the file for subsequent deletion.
 
-        // Some operating systems (e.g. Windows) don't
-        // permit a file to be deleted while it is opened
-        // for read (e.g. by another process or thread). So
-        // we assume that when a delete fails it is because
-        // the file is open in another process, and queue
-        // the file for subsequent deletion.
-
-        if (infoStream.isEnabled("IFD")) {
-          infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
-        }
-        if (deletable == null) {
-          deletable = new ArrayList<>();
-        }
-        deletable.add(fileName);                  // add to deletable
+      if (infoStream.isEnabled("IFD")) {
+        infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString() + "; Will re-try later.");
       }
+      if (deletable == null) {
+        deletable = new ArrayList<>();
+      }
+      deletable.add(fileName);                  // add to deletable
     }
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Sun Mar 23 18:03:55 2014
@@ -18,8 +18,11 @@ package org.apache.lucene.index;
  */
 
 import java.io.Closeable;
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.NoSuchFileException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
@@ -29,8 +32,8 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
-import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -2845,7 +2848,7 @@ public class IndexWriter implements Clos
           continue;
         }
 
-        assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists; siFiles=" + siFiles;
+        assert !slowFileExists(directory, newFileName): "file \"" + newFileName + "\" already exists; siFiles=" + siFiles;
         assert !copiedFiles.contains(file): "file \"" + file + "\" is being copied more than once";
         copiedFiles.add(file);
         info.info.dir.copy(directory, file, newFileName, context);
@@ -2985,17 +2988,18 @@ public class IndexWriter implements Clos
         if (anySegmentsFlushed) {
           maybeMerge(MergeTrigger.FULL_FLUSH, UNBOUNDED_MAX_MERGE_SEGMENTS);
         }
+        startCommit(toCommit);
         success = true;
       } finally {
         if (!success) {
           synchronized (this) {
-            deleter.decRef(filesToCommit);
-            filesToCommit = null;
+            if (filesToCommit != null) {
+              deleter.decRef(filesToCommit);
+              filesToCommit = null;
+            }
           }
         }
       }
-      
-      startCommit(toCommit);
     }
   }
   
@@ -4370,7 +4374,7 @@ public class IndexWriter implements Clos
     
     Collection<String> files = toSync.files(directory, false);
     for(final String fileName: files) {
-      assert directory.fileExists(fileName): "file " + fileName + " does not exist";
+      assert slowFileExists(directory, fileName): "file " + fileName + " does not exist; files=" + Arrays.toString(directory.listAll());
       // If this trips it means we are missing a call to
       // .checkpoint somewhere, because by the time we
       // are called, deleter should know about every
@@ -4772,4 +4776,16 @@ public class IndexWriter implements Clos
     void process(IndexWriter writer, boolean triggerMerge, boolean clearBuffers) throws IOException;
   }
 
+  /** Used only by asserts: returns true if the file exists
+   *  (can be opened), false if it cannot be opened, and
+   *  (unlike Java's File.exists) throws IOException if
+   *  there's some unexpected error. */
+  private static boolean slowFileExists(Directory dir, String fileName) throws IOException {
+    try {
+      dir.openInput(fileName, IOContext.DEFAULT).close();
+      return true;
+    } catch (NoSuchFileException | FileNotFoundException e) {
+      return false;
+    }
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java Sun Mar 23 18:03:55 2014
@@ -206,8 +206,10 @@ public class PersistentSnapshotDeletionP
     
     if (nextWriteGen > 0) {
       String lastSaveFile = SNAPSHOTS_PREFIX + (nextWriteGen-1);
-      if (dir.fileExists(lastSaveFile)) {
+      try {
         dir.deleteFile(lastSaveFile);
+      } catch (IOException ioe) {
+        // OK: likely it didn't exist
       }
     }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Sun Mar 23 18:03:55 2014
@@ -867,8 +867,14 @@ public final class SegmentInfos implemen
                                                                                "",
                                                                                gen-1);
 
-            final boolean prevExists;
-            prevExists = directory.fileExists(prevSegmentFileName);
+            boolean prevExists;
+
+            try {
+              directory.openInput(prevSegmentFileName, IOContext.DEFAULT).close();
+              prevExists = true;
+            } catch (IOException ioe) {
+              prevExists = false;
+            }
 
             if (prevExists) {
               if (infoStream != null) {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/CompoundFileWriter.java Sun Mar 23 18:03:55 2014
@@ -231,9 +231,6 @@ final class CompoundFileWriter implement
         out = new DirectCFSIndexOutput(getOutput(), entry, false);
       } else {
         entry.dir = this.directory;
-        if (directory.fileExists(name)) {
-          throw new IllegalArgumentException("File " + name + " already exists");
-        }
         out = new DirectCFSIndexOutput(directory.createOutput(name, context), entry,
             true);
       }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/Directory.java Sun Mar 23 18:03:55 2014
@@ -53,7 +53,10 @@ public abstract class Directory implemen
    */
   public abstract String[] listAll() throws IOException;
 
-  /** Returns true iff a file with the given name exists. */
+  /** Returns true iff a file with the given name exists.
+   *
+   *  @deprecated This method will be removed in 5.0 */
+  @Deprecated
   public abstract boolean fileExists(String name)
        throws IOException;
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java Sun Mar 23 18:03:55 2014
@@ -163,7 +163,6 @@ public class NRTCachingDirectory extends
       System.out.println("nrtdir.deleteFile name=" + name);
     }
     if (cache.fileExists(name)) {
-      assert !delegate.fileExists(name): "name=" + name;
       cache.deleteFile(name);
     } else {
       delegate.deleteFile(name);
@@ -294,9 +293,6 @@ public class NRTCachingDirectory extends
         // Another thread beat us...
         return;
       }
-      if (delegate.fileExists(fileName)) {
-        throw new IOException("cannot uncache file=\"" + fileName + "\": it was separately also created in the delegate directory");
-      }
       final IOContext context = IOContext.DEFAULT;
       final IndexOutput out = delegate.createOutput(fileName, context);
       IndexInput in = null;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java Sun Mar 23 18:03:55 2014
@@ -682,12 +682,6 @@ public class TestCompoundFile extends Lu
     createSequenceFile(newDir, "d1", (byte) 0, 15);
     IndexOutput out = csw.createOutput("d.xyz", newIOContext(random()));
     out.writeInt(0);
-    try {
-      newDir.copy(csw, "d1", "d1", newIOContext(random()));
-      fail("file does already exist");
-    } catch (IllegalArgumentException e) {
-      //
-    }
     out.close();
     assertEquals(1, csw.listAll().length);
     assertEquals("d.xyz", csw.listAll()[0]);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java Sun Mar 23 18:03:55 2014
@@ -33,7 +33,6 @@ import org.apache.lucene.store.IOContext
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.TestUtil;
 
 public class TestCrashCausesCorruptIndex extends LuceneTestCase  {
 
@@ -83,7 +82,7 @@ public class TestCrashCausesCorruptIndex
     }
     // writes segments_3
     indexWriter.close();
-    assertFalse(realDirectory.fileExists("segments_2"));
+    assertFalse(slowFileExists(realDirectory, "segments_2"));
     crashAfterCreateOutput.close();
   }
     
@@ -103,7 +102,7 @@ public class TestCrashCausesCorruptIndex
     // however, to test the fix, the following lines should pass as well.
     indexWriter.addDocument(getDocument());
     indexWriter.close();
-    assertFalse(realDirectory.fileExists("segments_2"));
+    assertFalse(slowFileExists(realDirectory, "segments_2"));
     realDirectory.close();
   }
     

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Sun Mar 23 18:03:55 2014
@@ -116,8 +116,8 @@ public class TestIndexFileDeleter extend
     // Create a bogus cfs file shadowing a non-cfs segment:
     
     // TODO: assert is bogus (relies upon codec-specific filenames)
-    assertTrue(dir.fileExists("_3.fdt") || dir.fileExists("_3.fld"));
-    assertTrue(!dir.fileExists("_3.cfs"));
+    assertTrue(slowFileExists(dir, "_3.fdt") || slowFileExists(dir, "_3.fld"));
+    assertTrue(!slowFileExists(dir, "_3.cfs"));
     copyFile(dir, "_1.cfs", "_3.cfs");
     
     String[] filesPre = dir.listAll();

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java Sun Mar 23 18:03:55 2014
@@ -1145,6 +1145,9 @@ public class TestIndexWriter extends Luc
       }
 
       if (!failed) {
+        if (VERBOSE) {
+          System.out.println("TEST: now rollback");
+        }
         // clear interrupt state:
         Thread.interrupted();
         if (w != null) {
@@ -1173,7 +1176,12 @@ public class TestIndexWriter extends Luc
         }
       }
       try {
-        IOUtils.close(dir, adder);
+        IOUtils.close(dir);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      try {
+        IOUtils.close(adder);
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
@@ -1979,7 +1987,7 @@ public class TestIndexWriter extends Luc
       
       new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random()))).close();
       
-      assertTrue(dir.fileExists("myrandomfile"));
+      assertTrue(slowFileExists(dir, "myrandomfile"));
     } finally {
       dir.close();
     }
@@ -2057,14 +2065,14 @@ public class TestIndexWriter extends Luc
       
       new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random()))).close();
       
-      assertTrue(dir.fileExists("_a.frq"));
+      assertTrue(slowFileExists(dir, "_a.frq"));
       
       IndexWriter iw = new IndexWriter(dir, 
           newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())));
       iw.addDocument(new Document());
       iw.close();
       
-      assertFalse(dir.fileExists("_a.frq"));
+      assertFalse(slowFileExists(dir, "_a.frq"));
     } finally {
       dir.close();
     }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Sun Mar 23 18:03:55 2014
@@ -46,7 +46,6 @@ import org.apache.lucene.store.MockDirec
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.TestUtil;
 
 public class TestIndexWriterDelete extends LuceneTestCase {
 
@@ -738,7 +737,8 @@ public class TestIndexWriterDelete exten
             boolean seen = false;
             StackTraceElement[] trace = new Exception().getStackTrace();
             for (int i = 0; i < trace.length; i++) {
-              if ("applyDeletesAndUpdates".equals(trace[i].getMethodName())) {
+              if ("applyDeletesAndUpdates".equals(trace[i].getMethodName()) ||
+                  "slowFileExists".equals(trace[i].getMethodName())) {
                 seen = true;
                 break;
               }
@@ -1053,7 +1053,7 @@ public class TestIndexWriterDelete exten
       w.updateDocument(delTerm, doc);
       // Eventually segment 0 should get a del docs:
       // TODO: fix this test
-      if (dir.fileExists("_0_1.del") || dir.fileExists("_0_1.liv") ) {
+      if (slowFileExists(dir, "_0_1.del") || slowFileExists(dir, "_0_1.liv") ) {
         if (VERBOSE) {
           System.out.println("TEST: deletes created @ count=" + count);
         }
@@ -1099,7 +1099,7 @@ public class TestIndexWriterDelete exten
       w.updateDocument(delTerm, doc);
       // Eventually segment 0 should get a del docs:
       // TODO: fix this test
-      if (dir.fileExists("_0_1.del") || dir.fileExists("_0_1.liv")) {
+      if (slowFileExists(dir, "_0_1.del") || slowFileExists(dir, "_0_1.liv")) {
         break;
       }
       count++;
@@ -1146,7 +1146,7 @@ public class TestIndexWriterDelete exten
       w.updateDocument(new Term("id", ""+id), doc);
       docsInSegment.incrementAndGet();
       // TODO: fix this test
-      if (dir.fileExists("_0_1.del") || dir.fileExists("_0_1.liv")) {
+      if (slowFileExists(dir, "_0_1.del") || slowFileExists(dir, "_0_1.liv")) {
         if (VERBOSE) {
           System.out.println("TEST: deletes created @ id=" + id);
         }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOutOfFileDescriptors.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOutOfFileDescriptors.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOutOfFileDescriptors.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOutOfFileDescriptors.java Sun Mar 23 18:03:55 2014
@@ -83,6 +83,7 @@ public class TestIndexWriterOutOfFileDes
           }
           w.addDocument(docs.nextDoc());
         }
+        dir.setRandomIOExceptionRateOnOpen(0.0);
         w.close();
         w = null;
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java Sun Mar 23 18:03:55 2014
@@ -91,7 +91,7 @@ public class TestNeverDelete extends Luc
       allFiles.addAll(ic.getFileNames());
       // Make sure no old files were removed
       for(String fileName : allFiles) {
-        assertTrue("file " + fileName + " does not exist", d.fileExists(fileName));
+        assertTrue("file " + fileName + " does not exist", slowFileExists(d, fileName));
       }
       DirectoryReader r2 = DirectoryReader.openIfChanged(r);
       if (r2 != null) {

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=1580557&r1=1580556&r2=1580557&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 Sun Mar 23 18:03:55 2014
@@ -51,7 +51,7 @@ public class TestSnapshotDeletionPolicy 
 
   protected void checkSnapshotExists(Directory dir, IndexCommit c) throws Exception {
     String segFileName = c.getSegmentsFileName();
-    assertTrue("segments file not found in directory: " + segFileName, dir.fileExists(segFileName));
+    assertTrue("segments file not found in directory: " + segFileName, slowFileExists(dir, segFileName));
   }
 
   protected void checkMaxDoc(IndexCommit commit, int expectedMaxDoc) throws Exception {
@@ -338,7 +338,7 @@ public class TestSnapshotDeletionPolicy 
 
     // but 'snapshot1' files will still exist (need to release snapshot before they can be deleted).
     String segFileName = snapshots.get(1).getSegmentsFileName();
-    assertTrue("snapshot files should exist in the directory: " + segFileName, dir.fileExists(segFileName));
+    assertTrue("snapshot files should exist in the directory: " + segFileName, slowFileExists(dir, segFileName));
 
     dir.close();
   }
@@ -360,7 +360,7 @@ public class TestSnapshotDeletionPolicy 
     sdp.release(snapshots.get(0));
     writer.deleteUnusedFiles();
     writer.close();
-    assertFalse("segments file should not be found in dirctory: " + segFileName, dir.fileExists(segFileName));
+    assertFalse("segments file should not be found in dirctory: " + segFileName, slowFileExists(dir, segFileName));
     dir.close();
   }
 
@@ -409,7 +409,7 @@ public class TestSnapshotDeletionPolicy 
     // commit.
     new IndexWriter(dir, getConfig(random(), null)).close();
     
-    assertFalse("snapshotted commit should not exist", dir.fileExists(s1.getSegmentsFileName()));
+    assertFalse("snapshotted commit should not exist", slowFileExists(dir, s1.getSegmentsFileName()));
     dir.close();
   }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java Sun Mar 23 18:03:55 2014
@@ -500,7 +500,7 @@ public class TestControlledRealTimeReope
                 for (String name : ic.getFileNames()) {
                   //distribute, and backup
                   //System.out.println(names);
-                  assertTrue(dir.fileExists(name));
+                  assertTrue(slowFileExists(dir, name));
                 }
               } catch (Exception e) {
                 throw new RuntimeException(e);

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=1580557&r1=1580556&r2=1580557&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 Sun Mar 23 18:03:55 2014
@@ -74,7 +74,7 @@ public class TestDirectory extends Lucen
             //System.out.println("create:" + fileName);
             IndexOutput output = dir.createOutput(fileName, newIOContext(random()));
             output.close();
-            assertTrue(dir.fileExists(fileName));
+            assertTrue(slowFileExists(dir, fileName));
           } catch (IOException e) {
             throw new RuntimeException(e);
           }
@@ -160,7 +160,7 @@ public class TestDirectory extends Lucen
       for (int j=0; j<dirs.length; j++) {
         FSDirectory d2 = dirs[j];
         d2.ensureOpen();
-        assertTrue(d2.fileExists(fname));
+        assertTrue(slowFileExists(d2, fname));
         assertEquals(1 + largeBuffer.length, d2.fileLength(fname));
 
         // don't do read tests if unmapping is not supported!
@@ -186,7 +186,7 @@ public class TestDirectory extends Lucen
 
       for (int j=0; j<dirs.length; j++) {
         FSDirectory d2 = dirs[j];
-        assertFalse(d2.fileExists(fname));
+        assertFalse(slowFileExists(d2, fname));
       }
 
       Lock lock = dir.makeLock(lockname);
@@ -248,7 +248,7 @@ public class TestDirectory extends Lucen
     String name = "file";
     try {
       dir.createOutput(name, newIOContext(random())).close();
-      assertTrue(dir.fileExists(name));
+      assertTrue(slowFileExists(dir, name));
       assertTrue(Arrays.asList(dir.listAll()).contains(name));
     } finally {
       dir.close();
@@ -275,7 +275,7 @@ public class TestDirectory extends Lucen
     try {
       IndexOutput out = fsDir.createOutput("afile", newIOContext(random()));
       out.close();
-      assertTrue(fsDir.fileExists("afile"));
+      assertTrue(slowFileExists(fsDir, "afile"));
       try {
         new SimpleFSDirectory(new File(path, "afile"), null);
         fail("did not hit expected exception");

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java Sun Mar 23 18:03:55 2014
@@ -122,7 +122,7 @@ public class TestFileSwitchDirectory ext
     String name = "file";
     try {
       dir.createOutput(name, newIOContext(random())).close();
-      assertTrue(dir.fileExists(name));
+      assertTrue(slowFileExists(dir, name));
       assertTrue(Arrays.asList(dir.listAll()).contains(name));
     } finally {
       dir.close();
@@ -136,12 +136,6 @@ public class TestFileSwitchDirectory ext
     createSequenceFile(newDir, "d1", (byte) 0, 15);
     IndexOutput out = csw.createOutput("d.xyz", newIOContext(random()));
     out.writeInt(0);
-    try {
-      newDir.copy(csw, "d1", "d1", newIOContext(random()));
-      fail("file does already exist");
-    } catch (IllegalArgumentException e) {
-      //
-    }
     out.close();
     assertEquals(1, csw.listAll().length);
     assertEquals("d.xyz", csw.listAll()[0]);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java Sun Mar 23 18:03:55 2014
@@ -142,7 +142,7 @@ public class TestNRTCachingDirectory ext
     String name = "file";
     try {
       dir.createOutput(name, newIOContext(random())).close();
-      assertTrue(dir.fileExists(name));
+      assertTrue(slowFileExists(dir, name));
       assertTrue(Arrays.asList(dir.listAll()).contains(name));
     } finally {
       dir.close();
@@ -156,12 +156,6 @@ public class TestNRTCachingDirectory ext
     createSequenceFile(newDir, "d1", (byte) 0, 15);
     IndexOutput out = csw.createOutput("d.xyz", newIOContext(random()));
     out.writeInt(0);
-    try {
-      newDir.copy(csw, "d1", "d1", newIOContext(random()));
-      fail("file does already exist");
-    } catch (IllegalArgumentException e) {
-      //
-    }
     out.close();
     assertEquals(1, csw.listAll().length);
     assertEquals("d.xyz", csw.listAll()[0]);

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=1580557&r1=1580556&r2=1580557&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 Sun Mar 23 18:03:55 2014
@@ -123,9 +123,7 @@ public class IndexReplicationHandler imp
   public static void cleanupFilesOnFailure(Directory dir, List<String> files) {
     for (String file : files) {
       try {
-        if (dir.fileExists(file)) {
-          dir.deleteFile(file);
-        }
+        dir.deleteFile(file);
       } catch (Throwable t) {
         // suppress any exception because if we're here, it means copy
         // failed, and we must cleanup after ourselves.
@@ -194,9 +192,7 @@ public class IndexReplicationHandler imp
       SegmentInfos.writeSegmentsGen(dir, SegmentInfos.generationFromSegmentsFileName(segmentsFile));
     } else {
       try {
-        if (dir.fileExists(IndexFileNames.SEGMENTS_GEN)) {
-          dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
-        }
+        dir.deleteFile(IndexFileNames.SEGMENTS_GEN);
       } catch (Throwable t) {
         // suppress any errors while deleting this file.
       }

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=1580557&r1=1580556&r2=1580557&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 Sun Mar 23 18:03:55 2014
@@ -83,8 +83,8 @@ public class IndexAndTaxonomyRevisionTes
       Revision rev1 = new IndexAndTaxonomyRevision(indexWriter, taxoWriter);
       // releasing that revision should not delete the files
       rev1.release();
-      assertTrue(indexDir.fileExists(IndexFileNames.SEGMENTS + "_1"));
-      assertTrue(taxoDir.fileExists(IndexFileNames.SEGMENTS + "_1"));
+      assertTrue(slowFileExists(indexDir, IndexFileNames.SEGMENTS + "_1"));
+      assertTrue(slowFileExists(taxoDir, IndexFileNames.SEGMENTS + "_1"));
       
       rev1 = new IndexAndTaxonomyRevision(indexWriter, taxoWriter); // create revision again, so the files are snapshotted
       indexWriter.addDocument(newDocument(taxoWriter));
@@ -92,7 +92,7 @@ public class IndexAndTaxonomyRevisionTes
       taxoWriter.commit();
       assertNotNull(new IndexAndTaxonomyRevision(indexWriter, taxoWriter));
       rev1.release(); // this release should trigger the delete of segments_1
-      assertFalse(indexDir.fileExists(IndexFileNames.SEGMENTS + "_1"));
+      assertFalse(slowFileExists(indexDir, IndexFileNames.SEGMENTS + "_1"));
     } finally {
       IOUtils.close(indexWriter, taxoWriter, taxoDir, indexDir);
     }

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=1580557&r1=1580556&r2=1580557&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 Sun Mar 23 18:03:55 2014
@@ -79,14 +79,14 @@ public class IndexRevisionTest extends R
       Revision rev1 = new IndexRevision(writer);
       // releasing that revision should not delete the files
       rev1.release();
-      assertTrue(dir.fileExists(IndexFileNames.SEGMENTS + "_1"));
+      assertTrue(slowFileExists(dir, IndexFileNames.SEGMENTS + "_1"));
       
       rev1 = new IndexRevision(writer); // create revision again, so the files are snapshotted
       writer.addDocument(new Document());
       writer.commit();
       assertNotNull(new IndexRevision(writer));
       rev1.release(); // this release should trigger the delete of segments_1
-      assertFalse(dir.fileExists(IndexFileNames.SEGMENTS + "_1"));
+      assertFalse(slowFileExists(dir, IndexFileNames.SEGMENTS + "_1"));
     } finally {
       IOUtils.close(writer, dir);
     }

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=1580557&r1=1580556&r2=1580557&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 Sun Mar 23 18:03:55 2014
@@ -187,11 +187,11 @@ public class LocalReplicatorTest extends
   @Test
   public void testRevisionRelease() throws Exception {
     replicator.publish(createRevision(1));
-    assertTrue(sourceDir.fileExists(IndexFileNames.SEGMENTS + "_1"));
+    assertTrue(slowFileExists(sourceDir, IndexFileNames.SEGMENTS + "_1"));
     replicator.publish(createRevision(2));
     // now the files of revision 1 can be deleted
-    assertTrue(sourceDir.fileExists(IndexFileNames.SEGMENTS + "_2"));
-    assertFalse("segments_1 should not be found in index directory after revision is released", sourceDir.fileExists(IndexFileNames.SEGMENTS + "_1"));
+    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"));
   }
   
 }

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=1580557&r1=1580556&r2=1580557&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 Sun Mar 23 18:03:55 2014
@@ -251,7 +251,7 @@ public class MockDirectoryWrapper extend
         String tempFileName;
         while (true) {
           tempFileName = ""+randomState.nextInt();
-          if (!in.fileExists(tempFileName)) {
+          if (!LuceneTestCase.slowFileExists(in, tempFileName)) {
             break;
           }
         }
@@ -549,7 +549,7 @@ public class MockDirectoryWrapper extend
     if (failOnOpenInput) {
       maybeThrowDeterministicException();
     }
-    if (!in.fileExists(name)) {
+    if (!LuceneTestCase.slowFileExists(in, name)) {
       throw randomState.nextBoolean() ? new FileNotFoundException(name + " in dir=" + in) : new NoSuchFileException(name + " in dir=" + in);
     }
 
@@ -925,7 +925,7 @@ public class MockDirectoryWrapper extend
   public IndexInputSlicer createSlicer(final String name, IOContext context)
       throws IOException {
     maybeYield();
-    if (!in.fileExists(name)) {
+    if (!LuceneTestCase.slowFileExists(in, name)) {
       throw randomState.nextBoolean() ? new FileNotFoundException(name) : new NoSuchFileException(name);
     }
     // cannot open a file for input if it's still open for

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Sun Mar 23 18:03:55 2014
@@ -18,6 +18,7 @@ package org.apache.lucene.util;
  */
 
 import java.io.*;
+import java.nio.file.NoSuchFileException;
 import java.lang.annotation.*;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
@@ -2087,4 +2088,17 @@ public abstract class LuceneTestCase ext
     
     assertEquals(info, left, right);
   }
+
+  /** Returns true if the file exists (can be opened), false
+   *  if it cannot be opened, and (unlike Java's
+   *  File.exists) throws IOException if there's some
+   *  unexpected error. */
+  public static boolean slowFileExists(Directory dir, String fileName) throws IOException {
+    try {
+      dir.openInput(fileName, IOContext.DEFAULT).close();
+      return true;
+    } catch (NoSuchFileException | FileNotFoundException e) {
+      return false;
+    }
+  }
 }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java Sun Mar 23 18:03:55 2014
@@ -19,7 +19,9 @@ package org.apache.solr.core;
 
 import java.io.Closeable;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.NoSuchFileException;
 
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FlushInfo;
@@ -202,11 +204,11 @@ public abstract class DirectoryFactory i
   }
   
   public static long sizeOf(Directory directory, String file) throws IOException {
-    if (!directory.fileExists(file)) {
+    try {
+      return directory.fileLength(file);
+    } catch (FileNotFoundException | NoSuchFileException e) {
       return 0;
     }
-    
-    return directory.fileLength(file);
   }
   
   /**

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java Sun Mar 23 18:03:55 2014
@@ -17,6 +17,43 @@
 
 package org.apache.solr.core;
 
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Writer;
+import java.lang.reflect.Constructor;
+import java.net.URL;
+import java.nio.file.NoSuchFileException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.IdentityHashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.StringTokenizer;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.ParserConfigurationException;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.index.AtomicReaderContext;
@@ -32,8 +69,8 @@ import org.apache.solr.cloud.CloudDescri
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CommonParams.EchoParamStyle;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.NamedList;
@@ -42,6 +79,7 @@ import org.apache.solr.core.DirectoryFac
 import org.apache.solr.handler.SnapPuller;
 import org.apache.solr.handler.admin.ShowFileRequestHandler;
 import org.apache.solr.handler.component.DebugComponent;
+import org.apache.solr.handler.component.ExpandComponent;
 import org.apache.solr.handler.component.FacetComponent;
 import org.apache.solr.handler.component.HighlightComponent;
 import org.apache.solr.handler.component.MoreLikeThisComponent;
@@ -49,7 +87,6 @@ import org.apache.solr.handler.component
 import org.apache.solr.handler.component.RealTimeGetComponent;
 import org.apache.solr.handler.component.SearchComponent;
 import org.apache.solr.handler.component.StatsComponent;
-import org.apache.solr.handler.component.ExpandComponent;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.BinaryResponseWriter;
@@ -75,8 +112,8 @@ import org.apache.solr.search.SolrIndexS
 import org.apache.solr.search.ValueSourceParser;
 import org.apache.solr.update.DefaultSolrCoreState;
 import org.apache.solr.update.DirectUpdateHandler2;
-import org.apache.solr.update.SolrCoreState;
 import org.apache.solr.update.SolrCoreState.IndexWriterCloser;
+import org.apache.solr.update.SolrCoreState;
 import org.apache.solr.update.SolrIndexWriter;
 import org.apache.solr.update.UpdateHandler;
 import org.apache.solr.update.VersionInfo;
@@ -95,42 +132,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.xml.sax.SAXException;
 
-import javax.xml.parsers.ParserConfigurationException;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.Writer;
-import java.lang.reflect.Constructor;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.IdentityHashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.StringTokenizer;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.ReentrantLock;
-
-
 /**
  *
  */
@@ -255,9 +256,14 @@ public final class SolrCore implements S
     Directory dir = null;
     try {
       dir = getDirectoryFactory().get(getDataDir(), DirContext.META_DATA, getSolrConfig().indexConfig.lockType);
-      if (dir.fileExists(SnapPuller.INDEX_PROPERTIES)){
-        final IndexInput input = dir.openInput(SnapPuller.INDEX_PROPERTIES, IOContext.DEFAULT);
-  
+      IndexInput input;
+      try {
+        input = dir.openInput(SnapPuller.INDEX_PROPERTIES, IOContext.DEFAULT);
+      } catch (FileNotFoundException | NoSuchFileException e) {
+        input = null;
+      }
+
+      if (input != null) {
         final InputStream is = new PropertiesInputStream(input);
         try {
           p.load(new InputStreamReader(is, "UTF-8"));

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java Sun Mar 23 18:03:55 2014
@@ -18,6 +18,7 @@ package org.apache.solr.handler;
 
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
@@ -25,6 +26,7 @@ import java.io.OutputStream;
 import java.io.Writer;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
+import java.nio.file.NoSuchFileException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -43,19 +45,16 @@ import java.util.zip.Checksum;
 import java.util.zip.DeflaterOutputStream;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexDeletionPolicy;
-import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
-
-import static org.apache.lucene.util.IOUtils.CHARSET_UTF_8;
-
 import org.apache.solr.client.solrj.impl.BinaryResponseParser;
-import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
@@ -64,12 +63,12 @@ import org.apache.solr.common.util.Named
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.CloseHook;
+import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrDeletionPolicy;
 import org.apache.solr.core.SolrEventListener;
-import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.BinaryQueryResponseWriter;
 import org.apache.solr.response.SolrQueryResponse;
@@ -82,6 +81,8 @@ import org.apache.solr.util.plugin.SolrC
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.lucene.util.IOUtils.CHARSET_UTF_8;
+
 /**
  * <p> A Handler which provides a REST API for replication and serves replication requests from Slaves. <p/> </p>
  * <p>When running on the master, it provides the following commands <ol> <li>Get the current replicable index version
@@ -811,11 +812,14 @@ public class ReplicationHandler extends 
       try {
         dir = core.getDirectoryFactory().get(core.getDataDir(),
             DirContext.META_DATA, core.getSolrConfig().indexConfig.lockType);
-        if (!dir.fileExists(SnapPuller.REPLICATION_PROPERTIES)) {
+        IndexInput input;
+        try {
+          input = dir.openInput(
+            SnapPuller.REPLICATION_PROPERTIES, IOContext.DEFAULT);
+        } catch (FileNotFoundException | NoSuchFileException e) {
           return new Properties();
         }
-        final IndexInput input = dir.openInput(
-            SnapPuller.REPLICATION_PROPERTIES, IOContext.DEFAULT);
+
         try {
           final InputStream is = new PropertiesInputStream(input);
           Properties props = new Properties();

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/SnapPuller.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/SnapPuller.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/SnapPuller.java Sun Mar 23 18:03:55 2014
@@ -16,28 +16,8 @@
  */
 package org.apache.solr.handler;
 
-import static org.apache.lucene.util.IOUtils.CHARSET_UTF_8;
-import static org.apache.solr.handler.ReplicationHandler.ALIAS;
-import static org.apache.solr.handler.ReplicationHandler.CHECKSUM;
-import static org.apache.solr.handler.ReplicationHandler.CMD_DETAILS;
-import static org.apache.solr.handler.ReplicationHandler.CMD_GET_FILE;
-import static org.apache.solr.handler.ReplicationHandler.CMD_GET_FILE_LIST;
-import static org.apache.solr.handler.ReplicationHandler.CMD_INDEX_VERSION;
-import static org.apache.solr.handler.ReplicationHandler.COMMAND;
-import static org.apache.solr.handler.ReplicationHandler.COMPRESSION;
-import static org.apache.solr.handler.ReplicationHandler.CONF_FILES;
-import static org.apache.solr.handler.ReplicationHandler.CONF_FILE_SHORT;
-import static org.apache.solr.handler.ReplicationHandler.EXTERNAL;
-import static org.apache.solr.handler.ReplicationHandler.FILE;
-import static org.apache.solr.handler.ReplicationHandler.FILE_STREAM;
-import static org.apache.solr.handler.ReplicationHandler.GENERATION;
-import static org.apache.solr.handler.ReplicationHandler.INTERNAL;
-import static org.apache.solr.handler.ReplicationHandler.MASTER_URL;
-import static org.apache.solr.handler.ReplicationHandler.NAME;
-import static org.apache.solr.handler.ReplicationHandler.OFFSET;
-import static org.apache.solr.handler.ReplicationHandler.SIZE;
-
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -46,6 +26,7 @@ import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
+import java.nio.file.NoSuchFileException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -76,22 +57,23 @@ import org.apache.http.client.HttpClient
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.request.QueryRequest;
-import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.FastInputStream;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.DirectoryFactory.DirContext;
+import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.handler.ReplicationHandler.FileInfo;
@@ -108,6 +90,27 @@ import org.eclipse.jetty.util.log.Log;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.lucene.util.IOUtils.CHARSET_UTF_8;
+import static org.apache.solr.handler.ReplicationHandler.ALIAS;
+import static org.apache.solr.handler.ReplicationHandler.CHECKSUM;
+import static org.apache.solr.handler.ReplicationHandler.CMD_DETAILS;
+import static org.apache.solr.handler.ReplicationHandler.CMD_GET_FILE;
+import static org.apache.solr.handler.ReplicationHandler.CMD_GET_FILE_LIST;
+import static org.apache.solr.handler.ReplicationHandler.CMD_INDEX_VERSION;
+import static org.apache.solr.handler.ReplicationHandler.COMMAND;
+import static org.apache.solr.handler.ReplicationHandler.COMPRESSION;
+import static org.apache.solr.handler.ReplicationHandler.CONF_FILES;
+import static org.apache.solr.handler.ReplicationHandler.CONF_FILE_SHORT;
+import static org.apache.solr.handler.ReplicationHandler.EXTERNAL;
+import static org.apache.solr.handler.ReplicationHandler.FILE;
+import static org.apache.solr.handler.ReplicationHandler.FILE_STREAM;
+import static org.apache.solr.handler.ReplicationHandler.GENERATION;
+import static org.apache.solr.handler.ReplicationHandler.INTERNAL;
+import static org.apache.solr.handler.ReplicationHandler.MASTER_URL;
+import static org.apache.solr.handler.ReplicationHandler.NAME;
+import static org.apache.solr.handler.ReplicationHandler.OFFSET;
+import static org.apache.solr.handler.ReplicationHandler.SIZE;
+
 /**
  * <p/> Provides functionality of downloading changed index files as well as config files and a timer for scheduling fetches from the
  * master. </p>
@@ -763,7 +766,7 @@ public class SnapPuller {
       LOG.debug("Download files to dir: " + Arrays.asList(indexDir.listAll()));
     }
     for (Map<String,Object> file : filesToDownload) {
-      if (!indexDir.fileExists((String) file.get(NAME))
+      if (!slowFileExists(indexDir, (String) file.get(NAME))
           || downloadCompleteIndex) {
         dirFileFetcher = new DirectoryFileFetcher(tmpIndexDir, file,
             (String) file.get(NAME), false, latestGeneration);
@@ -777,6 +780,19 @@ public class SnapPuller {
     }
   }
 
+  /** Returns true if the file exists (can be opened), false
+   *  if it cannot be opened, and (unlike Java's
+   *  File.exists) throws IOException if there's some
+   *  unexpected error. */
+  private static boolean slowFileExists(Directory dir, String fileName) throws IOException {
+    try {
+      dir.openInput(fileName, IOContext.DEFAULT).close();
+      return true;
+    } catch (NoSuchFileException | FileNotFoundException e) {
+      return false;
+    }
+  }  
+
   /**
    * All the files which are common between master and slave must have same size else we assume they are
    * not compatible (stale).
@@ -786,7 +802,7 @@ public class SnapPuller {
    */
   private boolean isIndexStale(Directory dir) throws IOException {
     for (Map<String, Object> file : filesToDownload) {
-      if (dir.fileExists((String) file.get(NAME))
+      if (slowFileExists(dir, (String) file.get(NAME))
               && dir.fileLength((String) file.get(NAME)) != (Long) file.get(SIZE)) {
         LOG.warn("File " + file.get(NAME) + " expected to be " + file.get(SIZE)
             + " while it is " + dir.fileLength((String) file.get(NAME)));
@@ -806,7 +822,7 @@ public class SnapPuller {
     LOG.debug("Moving file: {}", fname);
     boolean success = false;
     try {
-      if (indexDir.fileExists(fname)) {
+      if (slowFileExists(indexDir, fname)) {
         LOG.info("Skipping move file - it already exists:" + fname);
         return true;
       }
@@ -924,7 +940,7 @@ public class SnapPuller {
     Directory dir = null;
     try {
       dir = solrCore.getDirectoryFactory().get(solrCore.getDataDir(), DirContext.META_DATA, solrCore.getSolrConfig().indexConfig.lockType);
-      if (dir.fileExists(SnapPuller.INDEX_PROPERTIES)){
+      if (slowFileExists(dir, SnapPuller.INDEX_PROPERTIES)){
         final IndexInput input = dir.openInput(SnapPuller.INDEX_PROPERTIES, DirectoryFactory.IOCONTEXT_NO_CACHE);
   
         final InputStream is = new PropertiesInputStream(input);

Modified: lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/store/hdfs/HdfsDirectoryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/store/hdfs/HdfsDirectoryTest.java?rev=1580557&r1=1580556&r2=1580557&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/store/hdfs/HdfsDirectoryTest.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/store/hdfs/HdfsDirectoryTest.java Sun Mar 23 18:03:55 2014
@@ -118,10 +118,10 @@ public class HdfsDirectoryTest extends S
     assertEquals(12345, input1.readInt());
     input1.close();
 
-    assertFalse(directory.fileExists("testing.test.other"));
-    assertTrue(directory.fileExists("testing.test"));
+    assertFalse(slowFileExists(directory, "testing.test.other"));
+    assertTrue(slowFileExists(directory, "testing.test"));
     directory.deleteFile("testing.test");
-    assertFalse(directory.fileExists("testing.test"));
+    assertFalse(slowFileExists(directory, "testing.test"));
   }
   
   @Test