You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2014/09/13 23:46:36 UTC

svn commit: r1624784 [4/7] - in /lucene/dev/trunk: lucene/ lucene/analysis/common/src/java/org/apache/lucene/analysis/charfilter/ lucene/analysis/common/src/java/org/apache/lucene/analysis/compound/ lucene/analysis/common/src/java/org/apache/lucene/ana...

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IOUtils.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IOUtils.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IOUtils.java Sat Sep 13 21:46:29 2014
@@ -21,9 +21,6 @@ import org.apache.lucene.store.Directory
 
 import java.io.BufferedReader;
 import java.io.Closeable;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
@@ -33,8 +30,12 @@ import java.nio.charset.Charset;
 import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.FileVisitResult;
+import java.nio.file.FileVisitor;
 import java.nio.file.Files;
+import java.nio.file.Path;
 import java.nio.file.StandardOpenOption;
+import java.nio.file.attribute.BasicFileAttributes;
 import java.util.Arrays;
 import java.util.LinkedHashMap;
 import java.util.Map;
@@ -154,34 +155,6 @@ public final class IOUtils {
         .onUnmappableCharacter(CodingErrorAction.REPORT);
     return new BufferedReader(new InputStreamReader(stream, charSetDecoder));
   }
-  
-  /**
-   * Opens a Reader for the given {@link File} using a {@link CharsetDecoder}.
-   * Unlike Java's defaults this reader will throw an exception if your it detects 
-   * the read charset doesn't match the expected {@link Charset}. 
-   * <p>
-   * Decoding readers are useful to load configuration files, stopword lists or synonym files
-   * to detect character set problems. However, its not recommended to use as a common purpose 
-   * reader.
-   * @param file the file to open a reader on
-   * @param charSet the expected charset
-   * @return a reader to read the given file
-   */
-  public static Reader getDecodingReader(File file, Charset charSet) throws IOException {
-    FileInputStream stream = null;
-    boolean success = false;
-    try {
-      stream = new FileInputStream(file);
-      final Reader reader = getDecodingReader(stream, charSet);
-      success = true;
-      return reader;
-
-    } finally {
-      if (!success) {
-        IOUtils.close(stream);
-      }
-    }
-  }
 
   /**
    * Opens a Reader for the given resource using a {@link CharsetDecoder}.
@@ -233,7 +206,7 @@ public final class IOUtils {
    * <p>
    * Some of the files may be null, if so they are ignored.
    */
-  public static void deleteFilesIgnoringExceptions(File... files) {
+  public static void deleteFilesIgnoringExceptions(Path... files) {
     deleteFilesIgnoringExceptions(Arrays.asList(files));
   }
   
@@ -242,11 +215,11 @@ public final class IOUtils {
    * <p>
    * Some of the files may be null, if so they are ignored.
    */
-  public static void deleteFilesIgnoringExceptions(Iterable<? extends File> files) {
-    for (File name : files) {
+  public static void deleteFilesIgnoringExceptions(Iterable<? extends Path> files) {
+    for (Path name : files) {
       if (name != null) {
         try {
-          Files.delete(name.toPath());
+          Files.delete(name);
         } catch (Throwable ignored) {
           // ignore
         }
@@ -255,7 +228,7 @@ public final class IOUtils {
   }
   
   /**
-   * Deletes all given <tt>File</tt>s, if they exist.  Some of the
+   * Deletes all given <tt>Path</tt>s, if they exist.  Some of the
    * <tt>File</tt>s may be null; they are
    * ignored.  After everything is deleted, the method either
    * throws the first exception it hit while deleting, or
@@ -263,12 +236,12 @@ public final class IOUtils {
    * 
    * @param files files to delete
    */
-  public static void deleteFilesIfExist(File... files) throws IOException {
+  public static void deleteFilesIfExist(Path... files) throws IOException {
     deleteFilesIfExist(Arrays.asList(files));
   }
   
   /**
-   * Deletes all given <tt>File</tt>s, if they exist.  Some of the
+   * Deletes all given <tt>Path</tt>s, if they exist.  Some of the
    * <tt>File</tt>s may be null; they are
    * ignored.  After everything is deleted, the method either
    * throws the first exception it hit while deleting, or
@@ -276,13 +249,13 @@ public final class IOUtils {
    * 
    * @param files files to delete
    */
-  public static void deleteFilesIfExist(Iterable<? extends File> files) throws IOException {
+  public static void deleteFilesIfExist(Iterable<? extends Path> files) throws IOException {
     Throwable th = null;
 
-    for (File file : files) {
+    for (Path file : files) {
       try {
         if (file != null) {
-          Files.deleteIfExists(file.toPath());
+          Files.deleteIfExists(file);
         }
       } catch (Throwable t) {
         addSuppressed(th, t);
@@ -301,13 +274,13 @@ public final class IOUtils {
    * @throws IOException if any of the given files (or their subhierarchy files in case
    * of directories) cannot be removed.
    */
-  public static void rm(File... locations) throws IOException {
-    LinkedHashMap<File,Throwable> unremoved = rm(new LinkedHashMap<File,Throwable>(), locations);
+  public static void rm(Path... locations) throws IOException {
+    LinkedHashMap<Path,Throwable> unremoved = rm(new LinkedHashMap<Path,Throwable>(), locations);
     if (!unremoved.isEmpty()) {
       StringBuilder b = new StringBuilder("Could not remove the following files (in the order of attempts):\n");
-      for (Map.Entry<File,Throwable> kv : unremoved.entrySet()) {
+      for (Map.Entry<Path,Throwable> kv : unremoved.entrySet()) {
         b.append("   ")
-         .append(kv.getKey().getAbsolutePath())
+         .append(kv.getKey().toAbsolutePath())
          .append(": ")
          .append(kv.getValue())
          .append("\n");
@@ -316,18 +289,50 @@ public final class IOUtils {
     }
   }
 
-  private static LinkedHashMap<File,Throwable> rm(LinkedHashMap<File,Throwable> unremoved, File... locations) {
+  private static LinkedHashMap<Path,Throwable> rm(final LinkedHashMap<Path,Throwable> unremoved, Path... locations) {
     if (locations != null) {
-      for (File location : locations) {
-        if (location != null && location.exists()) {
-          if (location.isDirectory()) {
-            rm(unremoved, location.listFiles());
-          }
-  
+      for (Path location : locations) {
+        // TODO: remove this leniency!
+        if (location != null && Files.exists(location)) {
           try {
-            Files.delete(location.toPath());
-          } catch (Throwable cause) {
-            unremoved.put(location, cause);
+            Files.walkFileTree(location, new FileVisitor<Path>() {            
+              @Override
+              public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException {
+                return FileVisitResult.CONTINUE;
+              }
+              
+              @Override
+              public FileVisitResult postVisitDirectory(Path dir, IOException impossible) throws IOException {
+                assert impossible == null;
+                
+                try {
+                  Files.delete(dir);
+                } catch (IOException e) {
+                  unremoved.put(dir, e);
+                }
+                return FileVisitResult.CONTINUE;
+              }
+              
+              @Override
+              public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
+                try {
+                  Files.delete(file);
+                } catch (IOException exc) {
+                  unremoved.put(file, exc);
+                }
+                return FileVisitResult.CONTINUE;
+              }
+              
+              @Override
+              public FileVisitResult visitFileFailed(Path file, IOException exc) throws IOException {
+                if (exc != null) {
+                  unremoved.put(file, exc);
+                }
+                return FileVisitResult.CONTINUE;
+              }
+            });
+          } catch (IOException impossible) {
+            throw new AssertionError("visitor threw exception", impossible);
           }
         }
       }
@@ -336,27 +341,6 @@ public final class IOUtils {
   }
 
   /**
-   * Copy one file's contents to another file. The target will be overwritten
-   * if it exists. The source must exist.
-   */
-  public static void copy(File source, File target) throws IOException {
-    FileInputStream fis = null;
-    FileOutputStream fos = null;
-    try {
-      fis = new FileInputStream(source);
-      fos = new FileOutputStream(target);
-      
-      final byte [] buffer = new byte [1024 * 8];
-      int len;
-      while ((len = fis.read(buffer)) > 0) {
-        fos.write(buffer, 0, len);
-      }
-    } finally {
-      close(fis, fos);
-    }
-  }
-
-  /**
    * Simple utilty method that takes a previously caught
    * {@code Throwable} and rethrows either {@code
    * IOException} or an unchecked exception.  If the
@@ -394,12 +378,12 @@ public final class IOUtils {
    * @param isDir if true, the given file is a directory (we open for read and ignore IOExceptions,
    *  because not all file systems and operating systems allow to fsync on a directory)
    */
-  public static void fsync(File fileToSync, boolean isDir) throws IOException {
+  public static void fsync(Path fileToSync, boolean isDir) throws IOException {
     IOException exc = null;
     
     // If the file is a directory we have to open read-only, for regular files we must open r/w for the fsync to have an effect.
     // See http://blog.httrack.com/blog/2013/11/15/everything-you-always-wanted-to-know-about-fsync/
-    try (final FileChannel file = FileChannel.open(fileToSync.toPath(), isDir ? StandardOpenOption.READ : StandardOpenOption.WRITE)) {
+    try (final FileChannel file = FileChannel.open(fileToSync, isDir ? StandardOpenOption.READ : StandardOpenOption.WRITE)) {
       for (int retry = 0; retry < 5; retry++) {
         try {
           file.force(true);

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java Sat Sep 13 21:46:29 2014
@@ -25,13 +25,11 @@ import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.EOFException;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
 import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardCopyOption;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
@@ -45,7 +43,7 @@ import java.util.Locale;
  *   <li>exactly the above count of bytes for the sequence to be sorted.
  * </ul>
  * 
- * @see #sort(File, File)
+ * @see #sort(Path, Path)
  * @lucene.experimental
  * @lucene.internal
  */
@@ -167,7 +165,7 @@ public final class OfflineSorter {
   }
 
   private final BufferSize ramBufferSize;
-  private final File tempDirectory;
+  private final Path tempDirectory;
   
   private final Counter bufferBytesUsed = Counter.newCounter();
   private final BytesRefArray buffer = new BytesRefArray(bufferBytesUsed);
@@ -201,7 +199,7 @@ public final class OfflineSorter {
   /**
    * All-details constructor.
    */
-  public OfflineSorter(Comparator<BytesRef> comparator, BufferSize ramBufferSize, File tempDirectory, int maxTempfiles) {
+  public OfflineSorter(Comparator<BytesRef> comparator, BufferSize ramBufferSize, Path tempDirectory, int maxTempfiles) {
     if (ramBufferSize.bytes < ABSOLUTE_MIN_SORT_BUFFER_SIZE) {
       throw new IllegalArgumentException(MIN_BUFFER_SIZE_MSG + ": " + ramBufferSize.bytes);
     }
@@ -220,13 +218,13 @@ public final class OfflineSorter {
    * Sort input to output, explicit hint for the buffer size. The amount of allocated
    * memory may deviate from the hint (may be smaller or larger).  
    */
-  public SortInfo sort(File input, File output) throws IOException {
+  public SortInfo sort(Path input, Path output) throws IOException {
     sortInfo = new SortInfo();
     sortInfo.totalTime = System.currentTimeMillis();
 
-    Files.deleteIfExists(output.toPath());
+    Files.deleteIfExists(output);
 
-    ArrayList<File> merges = new ArrayList<>();
+    ArrayList<Path> merges = new ArrayList<>();
     boolean success3 = false;
     try {
       ByteSequencesReader is = new ByteSequencesReader(input);
@@ -240,7 +238,7 @@ public final class OfflineSorter {
 
           // Handle intermediate merges.
           if (merges.size() == maxTempFiles) {
-            File intermediate = File.createTempFile("sort", "intermediate", tempDirectory);
+            Path intermediate = Files.createTempFile(tempDirectory, "sort", "intermediate");
             boolean success2 = false;
             try {
               mergePartitions(merges, intermediate);
@@ -267,11 +265,13 @@ public final class OfflineSorter {
 
       // One partition, try to rename or copy if unsuccessful.
       if (merges.size() == 1) {     
-        File single = merges.get(0);
+        Path single = merges.get(0);
         // If simple rename doesn't work this means the output is
         // on a different volume or something. Copy the input then.
-        if (!single.renameTo(output)) {
-          copy(single, output);
+        try {
+          Files.move(single, output, StandardCopyOption.ATOMIC_MOVE);
+        } catch (IOException | UnsupportedOperationException e) {
+          Files.copy(single, output);
         }
       } else { 
         // otherwise merge the partitions with a priority queue.
@@ -295,43 +295,23 @@ public final class OfflineSorter {
    * Returns the default temporary directory. By default, java.io.tmpdir. If not accessible
    * or not available, an IOException is thrown
    */
-  public static File defaultTempDir() throws IOException {
+  public static Path defaultTempDir() throws IOException {
     String tempDirPath = System.getProperty("java.io.tmpdir");
     if (tempDirPath == null) 
       throw new IOException("Java has no temporary folder property (java.io.tmpdir)?");
 
-    File tempDirectory = new File(tempDirPath);
-    if (!tempDirectory.exists() || !tempDirectory.canWrite()) {
+    Path tempDirectory = Paths.get(tempDirPath);
+    if (!Files.isWritable(tempDirectory)) {
       throw new IOException("Java's temporary folder not present or writeable?: " 
-          + tempDirectory.getAbsolutePath());
+          + tempDirectory.toAbsolutePath());
     }
     return tempDirectory;
   }
 
-  /**
-   * Copies one file to another.
-   */
-  private static void copy(File file, File output) throws IOException {
-    // 64kb copy buffer (empirical pick).
-    byte [] buffer = new byte [16 * 1024];
-    InputStream is = null;
-    OutputStream os = null;
-    try {
-      is = new FileInputStream(file);
-      os = new FileOutputStream(output);
-      int length;
-      while ((length = is.read(buffer)) > 0) {
-        os.write(buffer, 0, length);
-      }
-    } finally {
-      IOUtils.close(is, os);
-    }
-  }
-
   /** Sort a single partition in-memory. */
-  protected File sortPartition(int len) throws IOException {
+  protected Path sortPartition(int len) throws IOException {
     BytesRefArray data = this.buffer;
-    File tempFile = File.createTempFile("sort", "partition", tempDirectory);
+    Path tempFile = Files.createTempFile(tempDirectory, "sort", "partition");
 
     long start = System.currentTimeMillis();
     sortInfo.sortTime += (System.currentTimeMillis() - start);
@@ -356,7 +336,7 @@ public final class OfflineSorter {
   }
 
   /** Merge a list of sorted temporary files (partitions) into an output file */
-  void mergePartitions(List<File> merges, File outputFile) throws IOException {
+  void mergePartitions(List<Path> merges, Path outputFile) throws IOException {
     long start = System.currentTimeMillis();
 
     ByteSequencesWriter out = new ByteSequencesWriter(outputFile);
@@ -441,11 +421,11 @@ public final class OfflineSorter {
   public static class ByteSequencesWriter implements Closeable {
     private final DataOutput os;
 
-    /** Constructs a ByteSequencesWriter to the provided File */
-    public ByteSequencesWriter(File file) throws IOException {
+    /** Constructs a ByteSequencesWriter to the provided Path */
+    public ByteSequencesWriter(Path path) throws IOException {
       this(new DataOutputStream(
           new BufferedOutputStream(
-              new FileOutputStream(file))));
+              Files.newOutputStream(path))));
     }
 
     /** Constructs a ByteSequencesWriter to the provided DataOutput */
@@ -505,11 +485,11 @@ public final class OfflineSorter {
   public static class ByteSequencesReader implements Closeable {
     private final DataInput is;
 
-    /** Constructs a ByteSequencesReader from the provided File */
-    public ByteSequencesReader(File file) throws IOException {
+    /** Constructs a ByteSequencesReader from the provided Path */
+    public ByteSequencesReader(Path path) throws IOException {
       this(new DataInputStream(
           new BufferedInputStream(
-              new FileInputStream(file))));
+              Files.newInputStream(path))));
     }
 
     /** Constructs a ByteSequencesReader from the provided DataInput */

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Sat Sep 13 21:46:29 2014
@@ -19,12 +19,11 @@ package org.apache.lucene.util.fst;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -614,37 +613,18 @@ public final class FST<T> implements Acc
   /**
    * Writes an automaton to a file. 
    */
-  public void save(final File file) throws IOException {
-    boolean success = false;
-    OutputStream os = new BufferedOutputStream(new FileOutputStream(file));
-    try {
-      save(new OutputStreamDataOutput(os));
-      success = true;
-    } finally { 
-      if (success) { 
-        IOUtils.close(os);
-      } else {
-        IOUtils.closeWhileHandlingException(os); 
-      }
+  public void save(final Path path) throws IOException {
+    try (OutputStream os = Files.newOutputStream(path)) {
+      save(new OutputStreamDataOutput(new BufferedOutputStream(os)));
     }
   }
 
   /**
    * Reads an automaton from a file. 
    */
-  public static <T> FST<T> read(File file, Outputs<T> outputs) throws IOException {
-    InputStream is = new BufferedInputStream(new FileInputStream(file));
-    boolean success = false;
-    try {
-      FST<T> fst = new FST<>(new InputStreamDataInput(is), outputs);
-      success = true;
-      return fst;
-    } finally {
-      if (success) { 
-        IOUtils.close(is);
-      } else {
-        IOUtils.closeWhileHandlingException(is); 
-      }
+  public static <T> FST<T> read(Path path, Outputs<T> outputs) throws IOException {
+    try (InputStream is = Files.newInputStream(path)) {
+      return new FST<>(new InputStreamDataInput(new BufferedInputStream(is)), outputs);
     }
   }
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java Sat Sep 13 21:46:29 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.index;
  */
 
 import java.io.File;
+import java.nio.file.Path;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.*;
@@ -174,7 +175,7 @@ public class TestAtomicUpdate extends Lu
     directory.close();
 
     // Second in an FSDirectory:
-    File dirPath = createTempDir("lucene.test.atomic");
+    Path dirPath = createTempDir("lucene.test.atomic");
     directory = newFSDirectory(dirPath);
     runTest(directory);
     directory.close();

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCompoundFile.java Sat Sep 13 21:46:29 2014
@@ -30,8 +30,8 @@ import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 
 import static org.apache.lucene.store.TestHelper.isSimpleFSIndexInput;
 import static org.apache.lucene.store.TestHelper.isSimpleFSIndexInputOpen;
@@ -43,7 +43,7 @@ public class TestCompoundFile extends Lu
     @Override
     public void setUp() throws Exception {
        super.setUp();
-       File file = createTempDir("testIndex");
+       Path file = createTempDir("testIndex");
        // use a simple FSDir here, to be sure to have SimpleFSInputs
        dir = new SimpleFSDirectory(file,null);
     }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestCrashCausesCorruptIndex.java Sat Sep 13 21:46:29 2014
@@ -17,8 +17,8 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
@@ -35,7 +35,7 @@ import org.apache.lucene.util.LuceneTest
 
 public class TestCrashCausesCorruptIndex extends LuceneTestCase  {
 
-  File path;
+  Path path;
     
   /**
    * LUCENE-3627: This test fails.

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java Sat Sep 13 21:46:29 2014
@@ -22,6 +22,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -38,8 +39,8 @@ import org.apache.lucene.document.String
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.NoSuchDirectoryException;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -440,7 +441,7 @@ void assertTermDocsCount(String msg,
   
 public void testFilesOpenClose() throws IOException {
       // Create initial data set
-      File dirFile = createTempDir("TestIndexReader.testFilesOpenClose");
+      Path dirFile = createTempDir("TestIndexReader.testFilesOpenClose");
       Directory dir = newFSDirectory(dirFile);
       IndexWriter writer  = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())));
       addDoc(writer, "test");
@@ -470,8 +471,11 @@ public void testFilesOpenClose() throws 
   }
 
   public void testOpenReaderAfterDelete() throws IOException {
-    File dirFile = createTempDir("deletetest");
+    Path dirFile = createTempDir("deletetest");
     Directory dir = newFSDirectory(dirFile);
+    if (dir instanceof BaseDirectoryWrapper) {
+      ((BaseDirectoryWrapper)dir).setCheckIndexOnClose(false); // we will hit NoSuchFileException in MDW since we nuked it!
+    }
     try {
       DirectoryReader.open(dir);
       fail("expected FileNotFoundException/NoSuchFileException");
@@ -479,7 +483,7 @@ public void testFilesOpenClose() throws 
       // expected
     }
 
-    Files.delete(dirFile.toPath());
+    Files.delete(dirFile);
 
     // Make sure we still get a CorruptIndexException (not NPE):
     try {
@@ -717,13 +721,13 @@ public void testFilesOpenClose() throws 
   // DirectoryReader on a non-existent directory, you get a
   // good exception
   public void testNoDir() throws Throwable {
-    File tempDir = createTempDir("doesnotexist");
+    Path tempDir = createTempDir("doesnotexist");
     IOUtils.rm(tempDir);
     Directory dir = newFSDirectory(tempDir);
     try {
       DirectoryReader.open(dir);
       fail("did not hit expected exception");
-    } catch (NoSuchDirectoryException nsde) {
+    } catch (IndexNotFoundException nsde) {
       // expected
     }
     dir.close();
@@ -1053,8 +1057,8 @@ public void testFilesOpenClose() throws 
   }
 
   public void testIndexExistsOnNonExistentDirectory() throws Exception {
-    File tempDir = createTempDir("testIndexExistsOnNonExistentDirectory");
-    Files.delete(tempDir.toPath());
+    Path tempDir = createTempDir("testIndexExistsOnNonExistentDirectory");
+    Files.delete(tempDir);
     Directory dir = newFSDirectory(tempDir);
     assertFalse(DirectoryReader.indexExists(dir));
     dir.close();

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java Sat Sep 13 21:46:29 2014
@@ -16,9 +16,7 @@ package org.apache.lucene.index;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
+
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
@@ -27,6 +25,7 @@ import java.io.StringWriter;
 import java.io.Writer;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -50,9 +49,9 @@ import org.apache.lucene.util.Version;
 /** JUnit adaptation of an older test case DocTest. */
 public class TestDoc extends LuceneTestCase {
 
-    private File workDir;
-    private File indexDir;
-    private LinkedList<File> files;
+    private Path workDir;
+    private Path indexDir;
+    private LinkedList<Path> files;
 
     /** Set the test case. This test case needs
      *  a few text files created in the current working directory.
@@ -64,10 +63,7 @@ public class TestDoc extends LuceneTestC
           System.out.println("TEST: setUp");
         }
         workDir = createTempDir("TestDoc");
-        workDir.mkdirs();
-
         indexDir = createTempDir("testIndex");
-        indexDir.mkdirs();
 
         Directory directory = newFSDirectory(indexDir);
         directory.close();
@@ -82,18 +78,18 @@ public class TestDoc extends LuceneTestC
         ));
     }
 
-    private File createOutput(String name, String text) throws IOException {
+    private Path createOutput(String name, String text) throws IOException {
         Writer fw = null;
         PrintWriter pw = null;
 
         try {
-            File f = new File(workDir, name);
-            Files.deleteIfExists(f.toPath());
+            Path path = workDir.resolve(name);
+            Files.deleteIfExists(path);
 
-            fw = new OutputStreamWriter(new FileOutputStream(f), StandardCharsets.UTF_8);
+            fw = new OutputStreamWriter(Files.newOutputStream(path), StandardCharsets.UTF_8);
             pw = new PrintWriter(fw);
             pw.println(text);
-            return f;
+            return path;
 
         } finally {
             if (pw != null) pw.close();
@@ -203,9 +199,9 @@ public class TestDoc extends LuceneTestC
    private SegmentCommitInfo indexDoc(IndexWriter writer, String fileName)
    throws Exception
    {
-      File file = new File(workDir, fileName);
+      Path path = workDir.resolve(fileName);
       Document doc = new Document();
-      InputStreamReader is = new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8);
+      InputStreamReader is = new InputStreamReader(Files.newInputStream(path), StandardCharsets.UTF_8);
       doc.add(new TextField("contents", is));
       writer.addDocument(doc);
       writer.commit();

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java Sat Sep 13 21:46:29 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -110,7 +111,7 @@ public class TestFieldsReader extends Lu
     Directory fsDir;
     AtomicBoolean doFail = new AtomicBoolean();
 
-    public FaultyFSDirectory(File dir) {
+    public FaultyFSDirectory(Path dir) {
       fsDir = newFSDirectory(dir);
       lockFactory = fsDir.getLockFactory();
     }
@@ -220,7 +221,7 @@ public class TestFieldsReader extends Lu
 
   // LUCENE-1262
   public void testExceptions() throws Throwable {
-    File indexDir = createTempDir("testfieldswriterexceptions");
+    Path indexDir = createTempDir("testfieldswriterexceptions");
 
     try {
       FaultyFSDirectory dir = new FaultyFSDirectory(indexDir);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java Sat Sep 13 21:46:29 2014
@@ -18,15 +18,19 @@ package org.apache.lucene.index;
  *
  */
 
-import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.PrintStream;
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.util.Constants;
@@ -38,7 +42,7 @@ import com.carrotsearch.randomizedtestin
  * of execution, then runs checkindex to make sure its not corrupt.
  */
 public class TestIndexWriterOnJRECrash extends TestNRTThreads {
-  private File tempDir;
+  private Path tempDir;
   
   @Override
   public void setUp() throws Exception {
@@ -96,7 +100,7 @@ public class TestIndexWriterOnJRECrash e
     cmd.add("-Dtests.crashmode=true");
     // passing NIGHTLY to this test makes it run for much longer, easier to catch it in the act...
     cmd.add("-Dtests.nightly=true");
-    cmd.add("-DtempDir=" + tempDir.getPath());
+    cmd.add("-DtempDir=" + tempDir);
     cmd.add("-Dtests.seed=" + SeedUtils.formatSeed(random().nextLong()));
     cmd.add("-ea");
     cmd.add("-cp");
@@ -104,7 +108,7 @@ public class TestIndexWriterOnJRECrash e
     cmd.add("org.junit.runner.JUnitCore");
     cmd.add(getClass().getName());
     ProcessBuilder pb = new ProcessBuilder(cmd);
-    pb.directory(tempDir);
+    pb.directory(tempDir.toFile());
     pb.redirectErrorStream(true);
     Process p = pb.start();
 
@@ -147,30 +151,35 @@ public class TestIndexWriterOnJRECrash e
    * Recursively looks for indexes underneath <code>file</code>,
    * and runs checkindex on them. returns true if it found any indexes.
    */
-  public boolean checkIndexes(File file) throws IOException {
-    if (file.isDirectory()) {
-      BaseDirectoryWrapper dir = newFSDirectory(file);
-      dir.setCheckIndexOnClose(false); // don't double-checkindex
-      if (DirectoryReader.indexExists(dir)) {
-        if (VERBOSE) {
-          System.err.println("Checking index: " + file);
-        }
-        // LUCENE-4738: if we crashed while writing first
-        // commit it's possible index will be corrupt (by
-        // design we don't try to be smart about this case
-        // since that too risky):
-        if (SegmentInfos.getLastCommitGeneration(dir) > 1) {
-          TestUtil.checkIndex(dir);
+  public boolean checkIndexes(Path path) throws IOException {
+    final AtomicBoolean found = new AtomicBoolean();
+    Files.walkFileTree(path, new SimpleFileVisitor<Path>() {
+      @Override
+      public FileVisitResult postVisitDirectory(Path dirPath, IOException exc) throws IOException {
+        if (exc != null) {
+          throw exc;
+        } else {
+          BaseDirectoryWrapper dir = newFSDirectory(dirPath);
+          dir.setCheckIndexOnClose(false); // don't double-checkindex
+          if (DirectoryReader.indexExists(dir)) {
+            if (VERBOSE) {
+              System.err.println("Checking index: " + dirPath);
+            }
+            // LUCENE-4738: if we crashed while writing first
+            // commit it's possible index will be corrupt (by
+            // design we don't try to be smart about this case
+            // since that too risky):
+            if (SegmentInfos.getLastCommitGeneration(dir) > 1) {
+              TestUtil.checkIndex(dir);
+            }
+            dir.close();
+            found.set(true);
+          }
+          return FileVisitResult.CONTINUE;
         }
-        dir.close();
-        return true;
       }
-      dir.close();
-      for (File f : file.listFiles())
-        if (checkIndexes(f))
-          return true;
-    }
-    return false;
+    });
+    return found.get();
   }
 
   /**

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java Sat Sep 13 21:46:29 2014
@@ -17,7 +17,7 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.io.File;
+import java.nio.file.Path;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -36,7 +36,7 @@ import org.apache.lucene.util.TestUtil;
 public class TestNeverDelete extends LuceneTestCase {
 
   public void testIndexing() throws Exception {
-    final File tmpDir = createTempDir("TestNeverDelete");
+    final Path tmpDir = createTempDir("TestNeverDelete");
     final BaseDirectoryWrapper d = newFSDirectory(tmpDir);
 
     // We want to "see" files removed if Lucene removed

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java Sat Sep 13 21:46:29 2014
@@ -17,10 +17,8 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.OutputStream;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -45,21 +43,6 @@ import org.apache.lucene.util.ArrayUtil;
 
 public class TestBufferedIndexInput extends LuceneTestCase {
   
-  private static void writeBytes(File aFile, long size) throws IOException{
-    OutputStream stream = null;
-    try {
-      stream = new FileOutputStream(aFile);
-      for (int i = 0; i < size; i++) {
-        stream.write(byten(i));  
-      }
-      stream.flush();
-    } finally {
-      if (stream != null) {
-        stream.close();
-      }
-    }
-  }
-
   private static final long TEST_FILE_LENGTH = 100*1024;
  
   // Call readByte() repeatedly, past the buffer boundary, and see that it
@@ -228,7 +211,7 @@ public class TestBufferedIndexInput exte
     }
 
     public void testSetBufferSize() throws IOException {
-      File indexDir = createTempDir("testSetBufferSize");
+      Path indexDir = createTempDir("testSetBufferSize");
       MockFSDirectory dir = new MockFSDirectory(indexDir, random());
       try {
         IndexWriter writer = new IndexWriter(
@@ -292,7 +275,7 @@ public class TestBufferedIndexInput exte
 
       private Directory dir;
 
-      public MockFSDirectory(File path, Random rand) throws IOException {
+      public MockFSDirectory(Path path, Random rand) throws IOException {
         this.rand = rand;
         lockFactory = NoLockFactory.getNoLockFactory();
         dir = new SimpleFSDirectory(path, null);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java Sat Sep 13 21:46:29 2014
@@ -20,7 +20,9 @@ package org.apache.lucene.store;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
 import java.util.Arrays;
 import java.util.Collections;
 
@@ -30,7 +32,7 @@ import org.apache.lucene.util.TestUtil;
 public class TestDirectory extends BaseDirectoryTestCase {
 
   @Override
-  protected Directory getDirectory(File path) throws IOException {
+  protected Directory getDirectory(Path path) throws IOException {
     final Directory dir;
     if (random().nextBoolean()) {
       dir = newDirectory();
@@ -53,7 +55,7 @@ public class TestDirectory extends BaseD
   // Test that different instances of FSDirectory can coexist on the same
   // path, can read, write, and lock files.
   public void testDirectInstantiation() throws Exception {
-    final File path = createTempDir("testDirectInstantiation");
+    final Path path = createTempDir("testDirectInstantiation");
     
     final byte[] largeBuffer = new byte[random().nextInt(256*1024)], largeReadBuffer = new byte[largeBuffer.length];
     for (int i = 0; i < largeBuffer.length; i++) {
@@ -141,10 +143,9 @@ public class TestDirectory extends BaseD
 
   // LUCENE-1468
   public void testCopySubdir() throws Throwable {
-    File path = createTempDir("testsubdir");
+    Path path = createTempDir("testsubdir");
     try {
-      path.mkdirs();
-      new File(path, "subdir").mkdirs();
+      Files.createDirectory(path.resolve("subdir"));
       Directory fsDir = new SimpleFSDirectory(path, null);
       assertEquals(0, new RAMDirectory(fsDir, newIOContext(random())).listAll().length);
     } finally {
@@ -154,16 +155,16 @@ public class TestDirectory extends BaseD
 
   // LUCENE-1468
   public void testNotDirectory() throws Throwable {
-    File path = createTempDir("testnotdir");
+    Path path = createTempDir("testnotdir");
     Directory fsDir = new SimpleFSDirectory(path, null);
     try {
       IndexOutput out = fsDir.createOutput("afile", newIOContext(random()));
       out.close();
       assertTrue(slowFileExists(fsDir, "afile"));
       try {
-        new SimpleFSDirectory(new File(path, "afile"), null);
+        new SimpleFSDirectory(path.resolve("afile"), null);
         fail("did not hit expected exception");
-      } catch (NoSuchDirectoryException nsde) {
+      } catch (IOException nsde) {
         // Expected
       }
     } finally {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java Sat Sep 13 21:46:29 2014
@@ -17,8 +17,9 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
@@ -27,12 +28,12 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.compressing.CompressingStoredFieldsWriter;
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexNotFoundException;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.TestIndexWriterReader;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.TestUtil;
 
 public class TestFileSwitchDirectory extends BaseDirectoryTestCase {
 
@@ -85,12 +86,12 @@ public class TestFileSwitchDirectory ext
   }
   
   private Directory newFSSwitchDirectory(Set<String> primaryExtensions) throws IOException {
-    File primDir = createTempDir("foo");
-    File secondDir = createTempDir("bar");
+    Path primDir = createTempDir("foo");
+    Path secondDir = createTempDir("bar");
     return newFSSwitchDirectory(primDir, secondDir, primaryExtensions);
   }
 
-  private Directory newFSSwitchDirectory(File aDir, File bDir, Set<String> primaryExtensions) throws IOException {
+  private Directory newFSSwitchDirectory(Path aDir, Path bDir, Set<String> primaryExtensions) throws IOException {
     Directory a = new SimpleFSDirectory(aDir);
     Directory b = new SimpleFSDirectory(bDir);
     return new FileSwitchDirectory(primaryExtensions, a, b, true);
@@ -98,21 +99,21 @@ public class TestFileSwitchDirectory ext
   
   // LUCENE-3380 -- make sure we get exception if the directory really does not exist.
   public void testNoDir() throws Throwable {
-    File primDir = createTempDir("foo");
-    File secondDir = createTempDir("bar");
+    Path primDir = createTempDir("foo");
+    Path secondDir = createTempDir("bar");
     IOUtils.rm(primDir, secondDir);
     Directory dir = newFSSwitchDirectory(primDir, secondDir, Collections.<String>emptySet());
     try {
       DirectoryReader.open(dir);
       fail("did not hit expected exception");
-    } catch (NoSuchDirectoryException nsde) {
+    } catch (IndexNotFoundException nsde) {
       // expected
     }
     dir.close();
   }
 
   @Override
-  protected Directory getDirectory(File path) throws IOException {
+  protected Directory getDirectory(Path path) throws IOException {
     Set<String> extensions = new HashSet<String>();
     if (random().nextBoolean()) {
       extensions.add("cfs");

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java Sat Sep 13 21:46:29 2014
@@ -17,9 +17,9 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -38,7 +38,6 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
 
 public class TestLockFactory extends LuceneTestCase {
 
@@ -126,11 +125,6 @@ public class TestLockFactory extends Luc
             writer2.close();
         }
     }
-    
-    public void testSimpleFSLockFactory() throws IOException {
-      // test string file instantiation
-      new SimpleFSLockFactory("test");
-    }
 
     // Verify: do stress test, by opening IndexReaders and
     // IndexWriters over & over in 2 threads and making sure
@@ -146,11 +140,11 @@ public class TestLockFactory extends Luc
     // NativeFSLockFactory:
     @Nightly
     public void testStressLocksNativeFSLockFactory() throws Exception {
-      File dir = createTempDir("index.TestLockFactory7");
+      Path dir = createTempDir("index.TestLockFactory7");
       _testStressLocks(new NativeFSLockFactory(dir), dir);
     }
 
-    public void _testStressLocks(LockFactory lockFactory, File indexDir) throws Exception {
+    public void _testStressLocks(LockFactory lockFactory, Path indexDir) throws Exception {
         Directory dir = newFSDirectory(indexDir, lockFactory);
 
         // First create a 1 doc index:
@@ -202,22 +196,22 @@ public class TestLockFactory extends Luc
     
     // Verify: NativeFSLockFactory works correctly if the lock file exists
     public void testNativeFSLockFactoryLockExists() throws IOException {
-      File tempDir = createTempDir(LuceneTestCase.getTestClass().getSimpleName());
-      File lockFile = new File(tempDir, "test.lock");
-      lockFile.createNewFile();
+      Path tempDir = createTempDir(LuceneTestCase.getTestClass().getSimpleName());
+      Path lockFile = tempDir.resolve("test.lock");
+      Files.createFile(lockFile);
       
       Lock l = new NativeFSLockFactory(tempDir).makeLock("test.lock");
       assertTrue("failed to obtain lock", l.obtain());
       l.close();
       assertFalse("failed to release lock", l.isLocked());
-      Files.deleteIfExists(lockFile.toPath());
+      Files.deleteIfExists(lockFile);
     }
 
     // Verify: NativeFSLockFactory assigns null as lockPrefix if the lockDir is inside directory
     public void testNativeFSLockFactoryPrefix() throws IOException {
 
-      File fdir1 = createTempDir("TestLockFactory.8");
-      File fdir2 = createTempDir("TestLockFactory.8.Lockdir");
+      Path fdir1 = createTempDir("TestLockFactory.8");
+      Path fdir2 = createTempDir("TestLockFactory.8.Lockdir");
       Directory dir1 = newFSDirectory(fdir1, new NativeFSLockFactory(fdir1));
       // same directory, but locks are stored somewhere else. The prefix of the lock factory should != null
       Directory dir2 = newFSDirectory(fdir1, new NativeFSLockFactory(fdir2));
@@ -238,7 +232,7 @@ public class TestLockFactory extends Luc
     public void testDefaultFSLockFactoryPrefix() throws IOException {
 
       // Make sure we get null prefix, which wont happen if setLockFactory is ever called.
-      File dirName = createTempDir("TestLockFactory.10");
+      Path dirName = createTempDir("TestLockFactory.10");
 
       Directory dir = new SimpleFSDirectory(dirName);
       assertNull("Default lock prefix should be null", dir.getLockFactory().getLockPrefix());

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMmapDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMmapDirectory.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMmapDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMmapDirectory.java Sat Sep 13 21:46:29 2014
@@ -17,8 +17,8 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 
 /**
  * Tests MMapDirectory
@@ -26,7 +26,7 @@ import java.io.IOException;
 public class TestMmapDirectory extends BaseDirectoryTestCase {
 
   @Override
-  protected Directory getDirectory(File path) throws IOException {
+  protected Directory getDirectory(Path path) throws IOException {
     return new MMapDirectory(path);
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java Sat Sep 13 21:46:29 2014
@@ -17,8 +17,8 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.Random;
 
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -37,10 +37,9 @@ import org.apache.lucene.util.TestUtil;
  * Integer.MAX_VALUE in size using multiple byte buffers.
  */
 public class TestMultiMMap extends BaseDirectoryTestCase {
-  File workDir;
 
   @Override
-  protected Directory getDirectory(File path) throws IOException {
+  protected Directory getDirectory(Path path) throws IOException {
     return new MMapDirectory(path, null, 1<<TestUtil.nextInt(random(), 10, 28));
   }
   
@@ -335,7 +334,7 @@ public class TestMultiMMap extends BaseD
   }
   
   private void assertChunking(Random random, int chunkSize) throws Exception {
-    File path = createTempDir("mmap" + chunkSize);
+    Path path = createTempDir("mmap" + chunkSize);
     MMapDirectory mmapDir = new MMapDirectory(path, null, chunkSize);
     // we will map a lot, try to turn on the unmap hack
     if (MMapDirectory.UNMAP_SUPPORTED)

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestNIOFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestNIOFSDirectory.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestNIOFSDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestNIOFSDirectory.java Sat Sep 13 21:46:29 2014
@@ -17,8 +17,8 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 
 /**
  * Tests NIOFSDirectory
@@ -26,7 +26,7 @@ import java.io.IOException;
 public class TestNIOFSDirectory extends BaseDirectoryTestCase {
 
   @Override
-  protected Directory getDirectory(File path) throws IOException {
+  protected Directory getDirectory(Path path) throws IOException {
     return new NIOFSDirectory(path);
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java Sat Sep 13 21:46:29 2014
@@ -17,8 +17,8 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -43,7 +43,7 @@ public class TestNRTCachingDirectory ext
   // for the threads tests... maybe because of the synchronization in listAll?
   // would be good to investigate further...
   @Override
-  protected Directory getDirectory(File path) throws IOException {
+  protected Directory getDirectory(Path path) throws IOException {
     return new NRTCachingDirectory(new RAMDirectory(),
                                    .1 + 2.0*random().nextDouble(),
                                    .1 + 5.0*random().nextDouble());
@@ -115,7 +115,7 @@ public class TestNRTCachingDirectory ext
   public void verifyCompiles() throws Exception {
     Analyzer analyzer = null;
 
-    Directory fsDir = FSDirectory.open(new File("/path/to/index"));
+    Directory fsDir = FSDirectory.open(createTempDir("verify"));
     NRTCachingDirectory cachedFSDir = new NRTCachingDirectory(fsDir, 2.0, 25.0);
     IndexWriterConfig conf = new IndexWriterConfig(analyzer);
     IndexWriter writer = new IndexWriter(cachedFSDir, conf);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java Sat Sep 13 21:46:29 2014
@@ -17,8 +17,9 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
@@ -32,7 +33,6 @@ import org.apache.lucene.index.StoredDoc
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.util.English;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.TestUtil;
 
 /**
  * JUnit testcase to test RAMDirectory. RAMDirectory itself is used in many testcases,
@@ -41,15 +41,15 @@ import org.apache.lucene.util.TestUtil;
 public class TestRAMDirectory extends BaseDirectoryTestCase {
   
   @Override
-  protected Directory getDirectory(File path) {
+  protected Directory getDirectory(Path path) {
     return new RAMDirectory();
   }
   
   // add enough document so that the index will be larger than RAMDirectory.READ_BUFFER_SIZE
   private final int docsToAdd = 500;
 
-  private File buildIndex() throws IOException {
-    File path = createTempDir("buildIndex");
+  private Path buildIndex() throws IOException {
+    Path path = createTempDir("buildIndex");
     
     Directory dir = newFSDirectory(path);
     IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
@@ -70,11 +70,10 @@ public class TestRAMDirectory extends Ba
   
   // LUCENE-1468
   public void testCopySubdir() throws Throwable {
-    File path = createTempDir("testsubdir");
+    Path path = createTempDir("testsubdir");
     Directory fsDir = null;
     try {
-      path.mkdirs();
-      new File(path, "subdir").mkdirs();
+      Files.createDirectory(path.resolve("subdir"));
       fsDir = newFSDirectory(path);
       assertEquals(0, new RAMDirectory(fsDir, newIOContext(random())).listAll().length);
     } finally {
@@ -84,7 +83,7 @@ public class TestRAMDirectory extends Ba
   }
 
   public void testRAMDirectory () throws IOException {
-    File indexDir = buildIndex();
+    Path indexDir = buildIndex();
     
     Directory dir = newFSDirectory(indexDir);
     MockDirectoryWrapper ramDir = new MockDirectoryWrapper(random(), new RAMDirectory(dir, newIOContext(random())));
@@ -117,7 +116,7 @@ public class TestRAMDirectory extends Ba
   
   public void testRAMDirectorySize() throws IOException, InterruptedException {
 
-    File indexDir = buildIndex();
+    Path indexDir = buildIndex();
       
     Directory dir = newFSDirectory(indexDir);
     final MockDirectoryWrapper ramDir = new MockDirectoryWrapper(random(), new RAMDirectory(dir, newIOContext(random())));

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestRateLimitedDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestRateLimitedDirectoryWrapper.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestRateLimitedDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestRateLimitedDirectoryWrapper.java Sat Sep 13 21:46:29 2014
@@ -17,12 +17,12 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
+import java.nio.file.Path;
 
 public class TestRateLimitedDirectoryWrapper extends BaseDirectoryTestCase {
 
   @Override
-  protected Directory getDirectory(File path) {
+  protected Directory getDirectory(Path path) {
     Directory in = newFSDirectory(path);
     if (in instanceof MockDirectoryWrapper) {
       // test manipulates directory directly

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSDirectory.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSDirectory.java Sat Sep 13 21:46:29 2014
@@ -17,8 +17,8 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 
 /**
  * Tests SimpleFSDirectory
@@ -26,7 +26,7 @@ import java.io.IOException;
 public class TestSimpleFSDirectory extends BaseDirectoryTestCase {
 
   @Override
-  protected Directory getDirectory(File path) throws IOException {
+  protected Directory getDirectory(Path path) throws IOException {
     return new SimpleFSDirectory(path);
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java Sat Sep 13 21:46:29 2014
@@ -17,7 +17,7 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
+import java.nio.file.Path;
 
 import org.apache.lucene.document.Field;
 import org.apache.lucene.util.LuceneTestCase;
@@ -64,7 +64,7 @@ public class TestWindowsMMap extends Luc
     // sometimes the directory is not cleaned by rmDir, because on Windows it
     // may take some time until the files are finally dereferenced. So clean the
     // directory up front, or otherwise new IndexWriter will fail.
-    File dirPath = createTempDir("testLuceneMmap");
+    Path dirPath = createTempDir("testLuceneMmap");
     MMapDirectory dir = new MMapDirectory(dirPath, null);
     
     // plan to add a set of useful stopwords, consider changing some of the

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java Sat Sep 13 21:46:29 2014
@@ -18,10 +18,10 @@ package org.apache.lucene.util;
  */
 
 import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
@@ -32,13 +32,12 @@ import org.apache.lucene.util.OfflineSor
 import org.apache.lucene.util.OfflineSorter.BufferSize;
 import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
 import org.apache.lucene.util.OfflineSorter.SortInfo;
-import org.apache.lucene.util.TestUtil;
 
 /**
  * Tests for on-disk merge sorting.
  */
 public class TestOfflineSorter extends LuceneTestCase {
-  private File tempDir;
+  private Path tempDir;
 
   @Override
   public void setUp() throws Exception {
@@ -112,12 +111,12 @@ public class TestOfflineSorter extends L
    * Check sorting data on an instance of {@link OfflineSorter}.
    */
   private SortInfo checkSort(OfflineSorter sort, byte[][] data) throws IOException {
-    File unsorted = writeAll("unsorted", data);
+    Path unsorted = writeAll("unsorted", data);
 
     Arrays.sort(data, unsignedByteOrderComparator);
-    File golden = writeAll("golden", data);
+    Path golden = writeAll("golden", data);
 
-    File sorted = new File(tempDir, "sorted");
+    Path sorted = tempDir.resolve("sorted");
     SortInfo sortInfo = sort.sort(unsorted, sorted);
     //System.out.println("Input size [MB]: " + unsorted.length() / (1024 * 1024));
     //System.out.println(sortInfo);
@@ -129,14 +128,14 @@ public class TestOfflineSorter extends L
   /**
    * Make sure two files are byte-byte identical.
    */
-  private void assertFilesIdentical(File golden, File sorted) throws IOException {
-    assertEquals(golden.length(), sorted.length());
+  private void assertFilesIdentical(Path golden, Path sorted) throws IOException {
+    assertEquals(Files.size(golden), Files.size(sorted));
 
     byte [] buf1 = new byte [64 * 1024];
     byte [] buf2 = new byte [64 * 1024];
     int len;
-    DataInputStream is1 = new DataInputStream(new FileInputStream(golden));
-    DataInputStream is2 = new DataInputStream(new FileInputStream(sorted));
+    DataInputStream is1 = new DataInputStream(Files.newInputStream(golden));
+    DataInputStream is2 = new DataInputStream(Files.newInputStream(sorted));
     while ((len = is1.read(buf1)) > 0) {
       is2.readFully(buf2, 0, len);
       for (int i = 0; i < len; i++) {
@@ -146,8 +145,8 @@ public class TestOfflineSorter extends L
     IOUtils.close(is1, is2);
   }
 
-  private File writeAll(String name, byte[][] data) throws IOException {
-    File file = new File(tempDir, name);
+  private Path writeAll(String name, byte[][] data) throws IOException {
+    Path file = tempDir.resolve(name);
     ByteSequencesWriter w = new OfflineSorter.ByteSequencesWriter(file);
     for (byte [] datum : data) {
       w.write(datum);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/fst/TestFSTs.java Sat Sep 13 21:46:29 2014
@@ -18,15 +18,13 @@ package org.apache.lucene.util.fst;
  */
 
 import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
 import java.io.StringWriter;
 import java.io.Writer;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -313,7 +311,7 @@ public class TestFSTs extends LuceneTest
     analyzer.setMaxTokenLength(TestUtil.nextInt(random(), 1, IndexWriter.MAX_TERM_LENGTH));
 
     final IndexWriterConfig conf = newIndexWriterConfig(analyzer).setMaxBufferedDocs(-1).setRAMBufferSizeMB(64);
-    final File tempDir = createTempDir("fstlines");
+    final Path tempDir = createTempDir("fstlines");
     final Directory dir = newFSDirectory(tempDir);
     final IndexWriter writer = new IndexWriter(dir, conf);
     final long stopTime = System.currentTimeMillis() + RUN_TIME_MSEC;
@@ -458,14 +456,14 @@ public class TestFSTs extends LuceneTest
   }
 
   private static abstract class VisitTerms<T> {
-    private final String dirOut;
-    private final String wordsFileIn;
+    private final Path dirOut;
+    private final Path wordsFileIn;
     private int inputMode;
     private final Outputs<T> outputs;
     private final Builder<T> builder;
     private final boolean doPack;
 
-    public VisitTerms(String dirOut, String wordsFileIn, int inputMode, int prune, Outputs<T> outputs, boolean doPack, boolean noArcArrays) {
+    public VisitTerms(Path dirOut, Path wordsFileIn, int inputMode, int prune, Outputs<T> outputs, boolean doPack, boolean noArcArrays) {
       this.dirOut = dirOut;
       this.wordsFileIn = wordsFileIn;
       this.inputMode = inputMode;
@@ -478,7 +476,8 @@ public class TestFSTs extends LuceneTest
     protected abstract T getOutput(IntsRef input, int ord) throws IOException;
 
     public void run(int limit, boolean verify, boolean verifyByOutput) throws IOException {
-      BufferedReader is = new BufferedReader(new InputStreamReader(new FileInputStream(wordsFileIn), StandardCharsets.UTF_8), 65536);
+      
+      BufferedReader is = Files.newBufferedReader(wordsFileIn, StandardCharsets.UTF_8);
       try {
         final IntsRefBuilder intsRef = new IntsRefBuilder();
         long tStart = System.currentTimeMillis();
@@ -521,13 +520,13 @@ public class TestFSTs extends LuceneTest
 
         System.out.println(ord + " terms; " + fst.getNodeCount() + " nodes; " + fst.getArcCount() + " arcs; " + fst.getArcWithOutputCount() + " arcs w/ output; tot size " + fst.ramBytesUsed());
         if (fst.getNodeCount() < 100) {
-          Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), StandardCharsets.UTF_8);
+          Writer w = Files.newBufferedWriter(Paths.get("out.dot"), StandardCharsets.UTF_8);
           Util.toDot(fst, w, false, false);
           w.close();
           System.out.println("Wrote FST to out.dot");
         }
 
-        Directory dir = FSDirectory.open(new File(dirOut));
+        Directory dir = FSDirectory.open(dirOut);
         IndexOutput out = dir.createOutput("fst.bin", IOContext.DEFAULT);
         fst.save(out);
         out.close();
@@ -548,7 +547,7 @@ public class TestFSTs extends LuceneTest
         while(true) {
           for(int iter=0;iter<2;iter++) {
             is.close();
-            is = new BufferedReader(new InputStreamReader(new FileInputStream(wordsFileIn), StandardCharsets.UTF_8), 65536);
+            is = Files.newBufferedReader(wordsFileIn, StandardCharsets.UTF_8);
 
             ord = 0;
             tStart = System.currentTimeMillis();
@@ -622,8 +621,8 @@ public class TestFSTs extends LuceneTest
     boolean verify = true;
     boolean doPack = false;
     boolean noArcArrays = false;
-    String wordsFileIn = null;
-    String dirOut = null;
+    Path wordsFileIn = null;
+    Path dirOut = null;
 
     int idx = 0;
     while (idx < args.length) {
@@ -652,9 +651,9 @@ public class TestFSTs extends LuceneTest
         System.exit(-1);
       } else {
         if (wordsFileIn == null) {
-          wordsFileIn = args[idx];
+          wordsFileIn = Paths.get(args[idx]);
         } else if (dirOut == null) {
-          dirOut = args[idx];
+          dirOut = Paths.get(args[idx]);
         } else {
           System.err.println("Too many arguments, expected: input [output]");
           System.exit(-1);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestLeaveFilesIfTestFails.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestLeaveFilesIfTestFails.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestLeaveFilesIfTestFails.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestLeaveFilesIfTestFails.java Sat Sep 13 21:46:29 2014
@@ -17,15 +17,15 @@ package org.apache.lucene.util.junitcomp
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
+import java.nio.channels.SeekableByteChannel;
 import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
 
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.JUnitCore;
@@ -39,7 +39,7 @@ public class TestLeaveFilesIfTestFails e
   }
   
   public static class Nested1 extends WithNestedTests.AbstractNestedTest {
-    static File file;
+    static Path file;
     public void testDummy() {
       file = createTempDir("leftover");
       fail();
@@ -50,19 +50,19 @@ public class TestLeaveFilesIfTestFails e
   public void testLeaveFilesIfTestFails() throws IOException {
     Result r = JUnitCore.runClasses(Nested1.class);
     Assert.assertEquals(1, r.getFailureCount());
-    Assert.assertTrue(Nested1.file != null && Nested1.file.exists());
-    Files.delete(Nested1.file.toPath());
+    Assert.assertTrue(Nested1.file != null && Files.exists(Nested1.file));
+    Files.delete(Nested1.file);
   }
   
   public static class Nested2 extends WithNestedTests.AbstractNestedTest {
-    static File file;
-    static File parent;
-    static RandomAccessFile openFile;
+    static Path file;
+    static Path parent;
+    static SeekableByteChannel openFile;
 
     @SuppressWarnings("deprecation")
     public void testDummy() throws Exception {
-      file = new File(createTempDir("leftover"), "child.locked");
-      openFile = new RandomAccessFile(file, "rw");
+      file = createTempDir("leftover").resolve("child.locked");
+      openFile = Files.newByteChannel(file, StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE);
 
       parent = LuceneTestCase.getBaseTempDirForTestClass();
     }

Modified: lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java (original)
+++ lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java Sat Sep 13 21:46:29 2014
@@ -32,12 +32,16 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.FSDirectory;
 
 import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
 import java.util.Date;
 
 /** Index all text files under a directory.
@@ -75,9 +79,9 @@ public class IndexFiles {
       System.exit(1);
     }
 
-    final File docDir = new File(docsPath);
-    if (!docDir.exists() || !docDir.canRead()) {
-      System.out.println("Document directory '" +docDir.getAbsolutePath()+ "' does not exist or is not readable, please check the path");
+    final Path docDir = Paths.get(docsPath);
+    if (!Files.isReadable(docDir)) {
+      System.out.println("Document directory '" +docDir.toAbsolutePath()+ "' does not exist or is not readable, please check the path");
       System.exit(1);
     }
     
@@ -85,7 +89,7 @@ public class IndexFiles {
     try {
       System.out.println("Indexing to directory '" + indexPath + "'...");
 
-      Directory dir = FSDirectory.open(new File(indexPath));
+      Directory dir = FSDirectory.open(Paths.get(indexPath));
       Analyzer analyzer = new StandardAnalyzer();
       IndexWriterConfig iwc = new IndexWriterConfig(analyzer);
 
@@ -139,74 +143,65 @@ public class IndexFiles {
    * >WriteLineDocTask</a>.
    *  
    * @param writer Writer to the index where the given file/dir info will be stored
-   * @param file The file to index, or the directory to recurse into to find files to index
+   * @param path The file to index, or the directory to recurse into to find files to index
    * @throws IOException If there is a low-level I/O error
    */
-  static void indexDocs(IndexWriter writer, File file)
-    throws IOException {
-    // do not try to index files that cannot be read
-    if (file.canRead()) {
-      if (file.isDirectory()) {
-        String[] files = file.list();
-        // an IO error could occur
-        if (files != null) {
-          for (int i = 0; i < files.length; i++) {
-            indexDocs(writer, new File(file, files[i]));
+  static void indexDocs(final IndexWriter writer, Path path) throws IOException {
+    if (Files.isDirectory(path)) {
+      Files.walkFileTree(path, new SimpleFileVisitor<Path>() {
+        @Override
+        public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
+          try {
+            indexDoc(writer, file, attrs.lastModifiedTime().toMillis());
+          } catch (IOException ignore) {
+            // don't index files that can't be read.
           }
+          return FileVisitResult.CONTINUE;
         }
-      } else {
-
-        FileInputStream fis;
-        try {
-          fis = new FileInputStream(file);
-        } catch (FileNotFoundException fnfe) {
-          // at least on windows, some temporary files raise this exception with an "access denied" message
-          // checking if the file can be read doesn't help
-          return;
-        }
-
-        try {
-
-          // make a new, empty document
-          Document doc = new Document();
+      });
+    } else {
+      indexDoc(writer, path, Files.getLastModifiedTime(path).toMillis());
+    }
+  }
 
-          // Add the path of the file as a field named "path".  Use a
-          // field that is indexed (i.e. searchable), but don't tokenize 
-          // the field into separate words and don't index term frequency
-          // or positional information:
-          Field pathField = new StringField("path", file.getPath(), Field.Store.YES);
-          doc.add(pathField);
-
-          // Add the last modified date of the file a field named "modified".
-          // Use a LongField that is indexed (i.e. efficiently filterable with
-          // NumericRangeFilter).  This indexes to milli-second resolution, which
-          // is often too fine.  You could instead create a number based on
-          // year/month/day/hour/minutes/seconds, down the resolution you require.
-          // For example the long value 2011021714 would mean
-          // February 17, 2011, 2-3 PM.
-          doc.add(new LongField("modified", file.lastModified(), Field.Store.NO));
-
-          // Add the contents of the file to a field named "contents".  Specify a Reader,
-          // so that the text of the file is tokenized and indexed, but not stored.
-          // Note that FileReader expects the file to be in UTF-8 encoding.
-          // If that's not the case searching for special characters will fail.
-          doc.add(new TextField("contents", new BufferedReader(new InputStreamReader(fis, StandardCharsets.UTF_8))));
-
-          if (writer.getConfig().getOpenMode() == OpenMode.CREATE) {
-            // New index, so we just add the document (no old document can be there):
-            System.out.println("adding " + file);
-            writer.addDocument(doc);
-          } else {
-            // Existing index (an old copy of this document may have been indexed) so 
-            // we use updateDocument instead to replace the old one matching the exact 
-            // path, if present:
-            System.out.println("updating " + file);
-            writer.updateDocument(new Term("path", file.getPath()), doc);
-          }
-          
-        } finally {
-          fis.close();
-        }
+  /** Indexes a single document */
+  static void indexDoc(IndexWriter writer, Path file, long lastModified) throws IOException {
+    try (InputStream stream = Files.newInputStream(file)) {
+      // make a new, empty document
+      Document doc = new Document();
+      
+      // Add the path of the file as a field named "path".  Use a
+      // field that is indexed (i.e. searchable), but don't tokenize 
+      // the field into separate words and don't index term frequency
+      // or positional information:
+      Field pathField = new StringField("path", file.toString(), Field.Store.YES);
+      doc.add(pathField);
+      
+      // Add the last modified date of the file a field named "modified".
+      // Use a LongField that is indexed (i.e. efficiently filterable with
+      // NumericRangeFilter).  This indexes to milli-second resolution, which
+      // is often too fine.  You could instead create a number based on
+      // year/month/day/hour/minutes/seconds, down the resolution you require.
+      // For example the long value 2011021714 would mean
+      // February 17, 2011, 2-3 PM.
+      doc.add(new LongField("modified", lastModified, Field.Store.NO));
+      
+      // Add the contents of the file to a field named "contents".  Specify a Reader,
+      // so that the text of the file is tokenized and indexed, but not stored.
+      // Note that FileReader expects the file to be in UTF-8 encoding.
+      // If that's not the case searching for special characters will fail.
+      doc.add(new TextField("contents", new BufferedReader(new InputStreamReader(stream, StandardCharsets.UTF_8))));
+      
+      if (writer.getConfig().getOpenMode() == OpenMode.CREATE) {
+        // New index, so we just add the document (no old document can be there):
+        System.out.println("adding " + file);
+        writer.addDocument(doc);
+      } else {
+        // Existing index (an old copy of this document may have been indexed) so 
+        // we use updateDocument instead to replace the old one matching the exact 
+        // path, if present:
+        System.out.println("updating " + file);
+        writer.updateDocument(new Term("path", file.toString()), doc);
       }
     }
   }

Modified: lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java (original)
+++ lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java Sat Sep 13 21:46:29 2014
@@ -18,16 +18,15 @@ package org.apache.lucene.demo;
  */
 
 import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 import java.util.Date;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.StoredDocument;
@@ -88,13 +87,13 @@ public class SearchFiles {
       }
     }
     
-    IndexReader reader = DirectoryReader.open(FSDirectory.open(new File(index)));
+    IndexReader reader = DirectoryReader.open(FSDirectory.open(Paths.get(index)));
     IndexSearcher searcher = new IndexSearcher(reader);
     Analyzer analyzer = new StandardAnalyzer();
 
     BufferedReader in = null;
     if (queries != null) {
-      in = new BufferedReader(new InputStreamReader(new FileInputStream(queries), StandardCharsets.UTF_8));
+      in = Files.newBufferedReader(Paths.get(queries), StandardCharsets.UTF_8);
     } else {
       in = new BufferedReader(new InputStreamReader(System.in, StandardCharsets.UTF_8));
     }

Modified: lucene/dev/trunk/lucene/demo/src/test/org/apache/lucene/demo/TestDemo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/demo/src/test/org/apache/lucene/demo/TestDemo.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/demo/src/test/org/apache/lucene/demo/TestDemo.java (original)
+++ lucene/dev/trunk/lucene/demo/src/test/org/apache/lucene/demo/TestDemo.java Sat Sep 13 21:46:29 2014
@@ -21,19 +21,20 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.PrintStream;
 import java.nio.charset.Charset;
+import java.nio.file.Path;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 
 public class TestDemo extends LuceneTestCase {
 
-  private void testOneSearch(File indexPath, String query, int expectedHitCount) throws Exception {
+  private void testOneSearch(Path indexPath, String query, int expectedHitCount) throws Exception {
     PrintStream outSave = System.out;
     try {
       ByteArrayOutputStream bytes = new ByteArrayOutputStream();
       PrintStream fakeSystemOut = new PrintStream(bytes, false, Charset.defaultCharset().name());
       System.setOut(fakeSystemOut);
-      SearchFiles.main(new String[] {"-query", query, "-index", indexPath.getPath()});
+      SearchFiles.main(new String[] {"-query", query, "-index", indexPath.toString()});
       fakeSystemOut.flush();
       String output = bytes.toString(Charset.defaultCharset().name()); // intentionally use default encoding
       assertTrue("output=" + output, output.contains(expectedHitCount + " total matching documents"));
@@ -43,9 +44,9 @@ public class TestDemo extends LuceneTest
   }
 
   public void testIndexSearch() throws Exception {
-    File dir = getDataFile("test-files/docs");
-    File indexDir = createTempDir("ContribDemoTest");
-    IndexFiles.main(new String[] { "-create", "-docs", dir.getPath(), "-index", indexDir.getPath()});
+    Path dir = getDataPath("test-files/docs");
+    Path indexDir = createTempDir("ContribDemoTest");
+    IndexFiles.main(new String[] { "-create", "-docs", dir.toString(), "-index", indexDir.toString()});
     testOneSearch(indexDir, "apache", 3);
     testOneSearch(indexDir, "patent", 8);
     testOneSearch(indexDir, "lucene", 0);

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/PrintTaxonomyStats.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/PrintTaxonomyStats.java?rev=1624784&r1=1624783&r2=1624784&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/PrintTaxonomyStats.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/PrintTaxonomyStats.java Sat Sep 13 21:46:29 2014
@@ -17,9 +17,9 @@ package org.apache.lucene.facet.taxonomy
  * limitations under the License.
  */
 
-import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.nio.file.Paths;
 
 import org.apache.lucene.facet.taxonomy.TaxonomyReader.ChildrenIterator;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
@@ -50,7 +50,7 @@ public class PrintTaxonomyStats {
       System.out.println("\nUsage: java -classpath ... org.apache.lucene.facet.util.PrintTaxonomyStats [-printTree] /path/to/taxononmy/index\n");
       System.exit(1);
     }
-    Directory dir = FSDirectory.open(new File(path));
+    Directory dir = FSDirectory.open(Paths.get(path));
     TaxonomyReader r = new DirectoryTaxonomyReader(dir);
     printStats(r, System.out, printTree);
     r.close();