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/04 17:56:26 UTC

svn commit: r1622506 [1/2] - in /lucene/dev/trunk: lucene/ lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ lucene/analysis/common/src/test/org/apache/lucene/analysis/util/ lucene/analysis/stempel/src/test/org/egothor/stemmer/ lucen...

Author: rmuir
Date: Thu Sep  4 15:56:25 2014
New Revision: 1622506

URL: http://svn.apache.org/r1622506
Log:
LUCENE-5906: Use Files.delete everywhere instead of File.delete

Removed:
    lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/utils/FileUtils.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
    lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestFilesystemResourceLoader.java
    lucene/dev/trunk/lucene/analysis/stempel/src/test/org/egothor/stemmer/TestCompile.java
    lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/PerfRunData.java
    lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractReuters.java
    lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractWikipedia.java
    lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/BenchmarkTestCase.java
    lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
    lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/utils/StreamUtilsTest.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestAtomicUpdate.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestDoc.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestNeverDelete.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestFileSwitchDirectory.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/util/junitcompat/TestLeaveFilesIfTestFails.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCharBlockArray.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCompactLabelToOrdinal.java
    lucene/dev/trunk/lucene/replicator/src/java/org/apache/lucene/replicator/PerSessionDirectoryFactory.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
    lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/RemoveUponClose.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
    lucene/dev/trunk/lucene/tools/forbiddenApis/base.txt
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNonWritablePersistFile.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
    lucene/dev/trunk/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java
    lucene/dev/trunk/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java
    lucene/dev/trunk/solr/contrib/morphlines-core/src/java/org/apache/solr/morphlines/solr/FileUtils.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/rest/ManagedResourceStorage.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/TransactionLog.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/UpdateLog.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/util/VersionedFile.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZk2Test.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestCSVLoader.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/handler/TestReplicationHandlerBackup.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCollationField.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCollationFieldDocValues.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestManagedSchema.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestRecovery.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/servlet/CacheHeaderTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/store/blockcache/BlockDirectoryTest.java
    lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/TestLBHttpSolrServer.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/SolrTestCaseJ4.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Sep  4 15:56:25 2014
@@ -110,6 +110,10 @@ New Features
   PushPostingsWriterBase for single-pass push of docs/positions to the
   postings format.  (Mike McCandless)
 
+* LUCENE-5906: Use Files.delete everywhere instead of File.delete, so that
+  when things go wrong, you get a real exception message why.
+  (Uwe Schindler, Robert Muir)
+
 API Changes:
 
 * LUCENE-5900: Deprecated more constructors taking Version in *InfixSuggester and

Modified: lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java Thu Sep  4 15:56:25 2014
@@ -54,6 +54,7 @@ import java.nio.charset.Charset;
 import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.text.ParseException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -203,6 +204,7 @@ public class Dictionary {
     OutputStream out = new BufferedOutputStream(new FileOutputStream(aff));
     InputStream aff1 = null;
     InputStream aff2 = null;
+    boolean success = false;
     try {
       // copy contents of affix stream to temp file
       final byte [] buffer = new byte [1024 * 8];
@@ -228,9 +230,14 @@ public class Dictionary {
       words = b.finish();
       aliases = null; // no longer needed
       morphAliases = null; // no longer needed
+      success = true;
     } finally {
       IOUtils.closeWhileHandlingException(out, aff1, aff2);
-      aff.delete();
+      if (success) {
+        Files.delete(aff.toPath());
+      } else {
+        IOUtils.deleteFilesIgnoringExceptions(aff);
+      }
     }
   }
 
@@ -857,90 +864,107 @@ public class Dictionary {
         }
       }
     });
-    sorter.sort(unsorted, sorted);
-    unsorted.delete();
+    boolean success = false;
+    try {
+      sorter.sort(unsorted, sorted);
+      success = true;
+    } finally {
+      if (success) {
+        Files.delete(unsorted.toPath());
+      } else {
+        IOUtils.deleteFilesIgnoringExceptions(unsorted);
+      }
+    }
     
+    boolean success2 = false;
     ByteSequencesReader reader = new ByteSequencesReader(sorted);
-    BytesRefBuilder scratchLine = new BytesRefBuilder();
+    try {
+      BytesRefBuilder scratchLine = new BytesRefBuilder();
     
-    // TODO: the flags themselves can be double-chars (long) or also numeric
-    // either way the trick is to encode them as char... but they must be parsed differently
+      // TODO: the flags themselves can be double-chars (long) or also numeric
+      // either way the trick is to encode them as char... but they must be parsed differently
     
-    String currentEntry = null;
-    IntsRefBuilder currentOrds = new IntsRefBuilder();
+      String currentEntry = null;
+      IntsRefBuilder currentOrds = new IntsRefBuilder();
     
-    String line;
-    while (reader.read(scratchLine)) {
-      line = scratchLine.get().utf8ToString();
-      String entry;
-      char wordForm[];
-      int end;
-
-      int flagSep = line.indexOf(FLAG_SEPARATOR);
-      if (flagSep == -1) {
-        wordForm = NOFLAGS;
-        end = line.indexOf(MORPH_SEPARATOR);
-        entry = line.substring(0, end);
-      } else {
-        end = line.indexOf(MORPH_SEPARATOR);
-        String flagPart = line.substring(flagSep + 1, end);
-        if (aliasCount > 0) {
-          flagPart = getAliasValue(Integer.parseInt(flagPart));
-        } 
+      String line;
+      while (reader.read(scratchLine)) {
+        line = scratchLine.get().utf8ToString();
+        String entry;
+        char wordForm[];
+        int end;
+
+        int flagSep = line.indexOf(FLAG_SEPARATOR);
+        if (flagSep == -1) {
+          wordForm = NOFLAGS;
+          end = line.indexOf(MORPH_SEPARATOR);
+          entry = line.substring(0, end);
+        } else {
+          end = line.indexOf(MORPH_SEPARATOR);
+          String flagPart = line.substring(flagSep + 1, end);
+          if (aliasCount > 0) {
+            flagPart = getAliasValue(Integer.parseInt(flagPart));
+          } 
         
-        wordForm = flagParsingStrategy.parseFlags(flagPart);
-        Arrays.sort(wordForm);
-        entry = line.substring(0, flagSep);
-      }
-      // we possibly have morphological data
-      int stemExceptionID = 0;
-      if (hasStemExceptions && end+1 < line.length()) {
-        String stemException = parseStemException(line.substring(end+1));
-        if (stemException != null) {
-          if (stemExceptionCount == stemExceptions.length) {
-            int newSize = ArrayUtil.oversize(stemExceptionCount+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
-            stemExceptions = Arrays.copyOf(stemExceptions, newSize);
+          wordForm = flagParsingStrategy.parseFlags(flagPart);
+          Arrays.sort(wordForm);
+          entry = line.substring(0, flagSep);
+        }
+        // we possibly have morphological data
+        int stemExceptionID = 0;
+        if (hasStemExceptions && end+1 < line.length()) {
+          String stemException = parseStemException(line.substring(end+1));
+          if (stemException != null) {
+            if (stemExceptionCount == stemExceptions.length) {
+              int newSize = ArrayUtil.oversize(stemExceptionCount+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
+              stemExceptions = Arrays.copyOf(stemExceptions, newSize);
+            }
+            stemExceptionID = stemExceptionCount+1; // we use '0' to indicate no exception for the form
+            stemExceptions[stemExceptionCount++] = stemException;
           }
-          stemExceptionID = stemExceptionCount+1; // we use '0' to indicate no exception for the form
-          stemExceptions[stemExceptionCount++] = stemException;
         }
-      }
 
-      int cmp = currentEntry == null ? 1 : entry.compareTo(currentEntry);
-      if (cmp < 0) {
-        throw new IllegalArgumentException("out of order: " + entry + " < " + currentEntry);
-      } else {
-        encodeFlags(flagsScratch, wordForm);
-        int ord = flagLookup.add(flagsScratch.get());
-        if (ord < 0) {
-          // already exists in our hash
-          ord = (-ord)-1;
-        }
-        // finalize current entry, and switch "current" if necessary
-        if (cmp > 0 && currentEntry != null) {
-          Util.toUTF32(currentEntry, scratchInts);
-          words.add(scratchInts.get(), currentOrds.get());
-        }
-        // swap current
-        if (cmp > 0 || currentEntry == null) {
-          currentEntry = entry;
-          currentOrds = new IntsRefBuilder(); // must be this way
-        }
-        if (hasStemExceptions) {
-          currentOrds.append(ord);
-          currentOrds.append(stemExceptionID);
+        int cmp = currentEntry == null ? 1 : entry.compareTo(currentEntry);
+        if (cmp < 0) {
+          throw new IllegalArgumentException("out of order: " + entry + " < " + currentEntry);
         } else {
-          currentOrds.append(ord);
+          encodeFlags(flagsScratch, wordForm);
+          int ord = flagLookup.add(flagsScratch.get());
+          if (ord < 0) {
+            // already exists in our hash
+            ord = (-ord)-1;
+          }
+          // finalize current entry, and switch "current" if necessary
+          if (cmp > 0 && currentEntry != null) {
+            Util.toUTF32(currentEntry, scratchInts);
+            words.add(scratchInts.get(), currentOrds.get());
+          }
+          // swap current
+          if (cmp > 0 || currentEntry == null) {
+            currentEntry = entry;
+            currentOrds = new IntsRefBuilder(); // must be this way
+          }
+          if (hasStemExceptions) {
+            currentOrds.append(ord);
+            currentOrds.append(stemExceptionID);
+          } else {
+            currentOrds.append(ord);
+          }
         }
       }
-    }
-    
-    // finalize last entry
-    Util.toUTF32(currentEntry, scratchInts);
-    words.add(scratchInts.get(), currentOrds.get());
     
-    reader.close();
-    sorted.delete();
+      // finalize last entry
+      Util.toUTF32(currentEntry, scratchInts);
+      words.add(scratchInts.get(), currentOrds.get());
+      success2 = true;
+    } finally {
+      IOUtils.closeWhileHandlingException(reader);
+      if (success2) {
+        Files.delete(sorted.toPath());
+      } else {
+        IOUtils.deleteFilesIgnoringExceptions(sorted);
+      }
+    }
   }
   
   static char[] decodeFlags(BytesRef b) {

Modified: lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestFilesystemResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestFilesystemResourceLoader.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestFilesystemResourceLoader.java (original)
+++ lucene/dev/trunk/lucene/analysis/common/src/test/org/apache/lucene/analysis/util/TestFilesystemResourceLoader.java Thu Sep  4 15:56:25 2014
@@ -87,7 +87,7 @@ public class TestFilesystemResourceLoade
       assertClasspathDelegation(rl);
       assertNotFound(rl);
     } finally {
-      TestUtil.rm(base);
+      IOUtils.rm(base);
     }
   }
   

Modified: lucene/dev/trunk/lucene/analysis/stempel/src/test/org/egothor/stemmer/TestCompile.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/analysis/stempel/src/test/org/egothor/stemmer/TestCompile.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/analysis/stempel/src/test/org/egothor/stemmer/TestCompile.java (original)
+++ lucene/dev/trunk/lucene/analysis/stempel/src/test/org/egothor/stemmer/TestCompile.java Thu Sep  4 15:56:25 2014
@@ -66,6 +66,7 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.LineNumberReader;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.util.Locale;
 import java.util.StringTokenizer;
 
@@ -87,7 +88,7 @@ public class TestCompile extends LuceneT
     Trie trie = loadTrie(compiled);
     assertTrie(trie, path, true, true);
     assertTrie(trie, path, false, true);
-    new File(compiled).delete();
+    Files.delete(new File(compiled).toPath());
   }
   
   public void testCompileBackwards() throws Exception {
@@ -103,7 +104,7 @@ public class TestCompile extends LuceneT
     Trie trie = loadTrie(compiled);
     assertTrie(trie, path, true, true);
     assertTrie(trie, path, false, true);
-    new File(compiled).delete();
+    Files.delete(new File(compiled).toPath());
   }
   
   public void testCompileMulti() throws Exception {
@@ -119,7 +120,7 @@ public class TestCompile extends LuceneT
     Trie trie = loadTrie(compiled);
     assertTrie(trie, path, true, true);
     assertTrie(trie, path, false, true);
-    new File(compiled).delete();
+    Files.delete(new File(compiled).toPath());
   }
   
   static Trie loadTrie(String path) throws IOException {

Modified: lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/trunk/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Thu Sep  4 15:56:25 2014
@@ -355,7 +355,7 @@ public class TestBackwardsCompatibility 
       assertTrue(bos.toString(IOUtils.UTF_8).contains(IndexFormatTooOldException.class.getName()));
 
       dir.close();
-      TestUtil.rm(oldIndxeDir);
+      IOUtils.rm(oldIndxeDir);
     }
   }
   
@@ -666,7 +666,7 @@ public class TestBackwardsCompatibility 
   public File createIndex(String dirName, boolean doCFS, boolean fullyMerged) throws IOException {
     // we use a real directory name that is not cleaned up, because this method is only used to create backwards indexes:
     File indexDir = new File("/tmp/idx", dirName);
-    TestUtil.rm(indexDir);
+    IOUtils.rm(indexDir);
     Directory dir = newFSDirectory(indexDir);
     LogByteSizeMergePolicy mp = new LogByteSizeMergePolicy();
     mp.setNoCFSRatio(doCFS ? 1.0 : 0.0);

Modified: lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/PerfRunData.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/PerfRunData.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/PerfRunData.java (original)
+++ lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/byTask/PerfRunData.java Thu Sep  4 15:56:25 2014
@@ -37,7 +37,6 @@ import org.apache.lucene.benchmark.byTas
 import org.apache.lucene.benchmark.byTask.tasks.SearchTask;
 import org.apache.lucene.benchmark.byTask.utils.AnalyzerFactory;
 import org.apache.lucene.benchmark.byTask.utils.Config;
-import org.apache.lucene.benchmark.byTask.utils.FileUtils;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
 import org.apache.lucene.index.DirectoryReader;
@@ -195,7 +194,7 @@ public class PerfRunData implements Clos
       File workDir = new File(config.get("work.dir","work"));
       File indexDir = new File(workDir,dirName);
       if (eraseIndex && indexDir.exists()) {
-        FileUtils.fullyDelete(indexDir);
+        IOUtils.rm(indexDir);
       }
       indexDir.mkdirs();
       return FSDirectory.open(indexDir);

Modified: lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractReuters.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractReuters.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractReuters.java (original)
+++ lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractReuters.java Thu Sep  4 15:56:25 2014
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -37,12 +38,12 @@ public class ExtractReuters {
   private File outputDir;
   private static final String LINE_SEPARATOR = System.getProperty("line.separator");
 
-  public ExtractReuters(File reutersDir, File outputDir) {
+  public ExtractReuters(File reutersDir, File outputDir) throws IOException {
     this.reutersDir = reutersDir;
     this.outputDir = outputDir;
     System.out.println("Deleting all files in " + outputDir);
     for (File f : outputDir.listFiles()) {
-      f.delete();
+      Files.delete(f.toPath());
     }
   }
 
@@ -122,7 +123,7 @@ public class ExtractReuters {
     }
   }
 
-  public static void main(String[] args) {
+  public static void main(String[] args) throws Exception {
     if (args.length != 2) {
       usage("Wrong number of arguments ("+args.length+")");
       return;

Modified: lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractWikipedia.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractWikipedia.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractWikipedia.java (original)
+++ lucene/dev/trunk/lucene/benchmark/src/java/org/apache/lucene/benchmark/utils/ExtractWikipedia.java Thu Sep  4 15:56:25 2014
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.util.Properties;
 
 import org.apache.lucene.benchmark.byTask.feeds.ContentSource;
@@ -44,13 +45,13 @@ public class ExtractWikipedia {
   static final int BASE = 10;
   protected DocMaker docMaker;
 
-  public ExtractWikipedia(DocMaker docMaker, File outputDir) {
+  public ExtractWikipedia(DocMaker docMaker, File outputDir) throws IOException {
     this.outputDir = outputDir;
     this.docMaker = docMaker;
     System.out.println("Deleting all files in " + outputDir);
     File[] files = outputDir.listFiles();
     for (int i = 0; i < files.length; i++) {
-      files[i].delete();
+      Files.delete(files[i].toPath());
     }
   }
 

Modified: lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/BenchmarkTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/BenchmarkTestCase.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/BenchmarkTestCase.java (original)
+++ lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/BenchmarkTestCase.java Thu Sep  4 15:56:25 2014
@@ -38,8 +38,6 @@ public abstract class BenchmarkTestCase 
   @BeforeClass
   public static void beforeClassBenchmarkTestCase() {
     WORKDIR = createTempDir("benchmark");
-    WORKDIR.delete();
-    WORKDIR.mkdirs();
   }
   
   @AfterClass

Modified: lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java (original)
+++ lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java Thu Sep  4 15:56:25 2014
@@ -22,6 +22,7 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.InputStreamReader;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.text.Collator;
 import java.util.List;
 import java.util.Locale;
@@ -444,7 +445,7 @@ public class TestPerfTasksLogic extends 
     assertEquals(numLines + " lines were created but " + ir.numDocs() + " docs are in the index", numLines, ir.numDocs());
     ir.close();
 
-    lineFile.delete();
+    Files.delete(lineFile.toPath());
   }
   
   /**

Modified: lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/utils/StreamUtilsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/utils/StreamUtilsTest.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/utils/StreamUtilsTest.java (original)
+++ lucene/dev/trunk/lucene/benchmark/src/test/org/apache/lucene/benchmark/byTask/utils/StreamUtilsTest.java Thu Sep  4 15:56:25 2014
@@ -30,6 +30,7 @@ import java.nio.charset.StandardCharsets
 
 import org.apache.commons.compress.compressors.CompressorStreamFactory;
 import org.apache.lucene.benchmark.BenchmarkTestCase;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -136,14 +137,14 @@ public class StreamUtilsTest extends Ben
   public void setUp() throws Exception {
     super.setUp();
     testDir = new File(getWorkDir(),"ContentSourceTest");
-    TestUtil.rm(testDir);
+    IOUtils.rm(testDir);
     assertTrue(testDir.mkdirs());
   }
 
   @Override
   @After
   public void tearDown() throws Exception {
-    TestUtil.rm(testDir);
+    IOUtils.rm(testDir);
     super.tearDown();
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/FSDirectory.java Thu Sep  4 15:56:25 2014
@@ -26,6 +26,7 @@ import java.io.FileOutputStream;
 import java.io.FilenameFilter;
 import java.io.FilterOutputStream;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
@@ -251,8 +252,7 @@ public abstract class FSDirectory extend
   public void deleteFile(String name) throws IOException {
     ensureOpen();
     File file = new File(directory, name);
-    if (!file.delete())
-      throw new IOException("Cannot delete " + file);
+    Files.delete(file.toPath());
     staleFiles.remove(name);
   }
 
@@ -271,8 +271,7 @@ public abstract class FSDirectory extend
         throw new IOException("Cannot create directory: " + directory);
 
     File file = new File(directory, name);
-    if (file.exists() && !file.delete())          // delete existing, if any
-      throw new IOException("Cannot overwrite: " + file);
+    Files.deleteIfExists(file.toPath()); // delete existing, if any
   }
 
   /**

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java Thu Sep  4 15:56:25 2014
@@ -28,4 +28,8 @@ public class LockReleaseFailedException 
   public LockReleaseFailedException(String message) {
     super(message);
   }
+  
+  public LockReleaseFailedException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java Thu Sep  4 15:56:25 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.store;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
 
 /**
  * <p>Implements {@link LockFactory} using {@link
@@ -102,9 +103,7 @@ public class SimpleFSLockFactory extends
         lockName = lockPrefix + "-" + lockName;
       }
       File lockFile = new File(lockDir, lockName);
-      if (lockFile.exists() && !lockFile.delete()) {
-        throw new IOException("Cannot delete " + lockFile);
-      }
+      Files.deleteIfExists(lockFile.toPath());
     }
   }
 }
@@ -147,8 +146,11 @@ class SimpleFSLock extends Lock {
 
   @Override
   public void close() throws LockReleaseFailedException {
-    if (lockFile.exists() && !lockFile.delete()) {
-      throw new LockReleaseFailedException("failed to delete " + lockFile);
+    // TODO: wierd that clearLock() throws the raw IOException...
+    try {
+      Files.deleteIfExists(lockFile.toPath());
+    } catch (Throwable cause) {
+      throw new LockReleaseFailedException("failed to delete " + lockFile, cause);
     }
   }
 

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -33,7 +33,11 @@ import java.nio.charset.Charset;
 import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.Map;
 
 /** This class emulates the new Java 7 "Try-With-Resources" statement.
  * Remove once Lucene is on Java 7.
@@ -69,22 +73,7 @@ public final class IOUtils {
    *          objects to call <tt>close()</tt> on
    */
   public static void close(Closeable... objects) throws IOException {
-    Throwable th = null;
-
-    for (Closeable object : objects) {
-      try {
-        if (object != null) {
-          object.close();
-        }
-      } catch (Throwable t) {
-        addSuppressed(th, t);
-        if (th == null) {
-          th = t;
-        }
-      }
-    }
-
-    reThrow(th);
+    close(Arrays.asList(objects));
   }
   
   /**
@@ -118,14 +107,7 @@ public final class IOUtils {
    *          objects to call <tt>close()</tt> on
    */
   public static void closeWhileHandlingException(Closeable... objects) {
-    for (Closeable object : objects) {
-      try {
-        if (object != null) {
-          object.close();
-        }
-      } catch (Throwable t) {
-      }
-    }
+    closeWhileHandlingException(Arrays.asList(objects));
   }
   
   /**
@@ -245,6 +227,113 @@ public final class IOUtils {
       }
     }
   }
+  
+  /**
+   * Deletes all given files, suppressing all thrown IOExceptions.
+   * <p>
+   * Some of the files may be null, if so they are ignored.
+   */
+  public static void deleteFilesIgnoringExceptions(File... files) {
+    deleteFilesIgnoringExceptions(Arrays.asList(files));
+  }
+  
+  /**
+   * Deletes all given files, suppressing all thrown IOExceptions.
+   * <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) {
+      if (name != null) {
+        try {
+          Files.delete(name.toPath());
+        } catch (Throwable ignored) {
+          // ignore
+        }
+      }
+    }
+  }
+  
+  /**
+   * Deletes all given <tt>File</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
+   * completes normally if there were no exceptions.
+   * 
+   * @param files files to delete
+   */
+  public static void deleteFilesIfExist(File... files) throws IOException {
+    deleteFilesIfExist(Arrays.asList(files));
+  }
+  
+  /**
+   * Deletes all given <tt>File</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
+   * completes normally if there were no exceptions.
+   * 
+   * @param files files to delete
+   */
+  public static void deleteFilesIfExist(Iterable<? extends File> files) throws IOException {
+    Throwable th = null;
+
+    for (File file : files) {
+      try {
+        if (file != null) {
+          Files.deleteIfExists(file.toPath());
+        }
+      } catch (Throwable t) {
+        addSuppressed(th, t);
+        if (th == null) {
+          th = t;
+        }
+      }
+    }
+
+    reThrow(th);
+  }
+  
+  /**
+   * Deletes one or more files or directories (and everything underneath it).
+   * 
+   * @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);
+    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()) {
+        b.append("   ")
+         .append(kv.getKey().getAbsolutePath())
+         .append(": ")
+         .append(kv.getValue())
+         .append("\n");
+      }
+      throw new IOException(b.toString());
+    }
+  }
+
+  private static LinkedHashMap<File,Throwable> rm(LinkedHashMap<File,Throwable> unremoved, File... locations) {
+    if (locations != null) {
+      for (File location : locations) {
+        if (location != null && location.exists()) {
+          if (location.isDirectory()) {
+            rm(unremoved, location.listFiles());
+          }
+  
+          try {
+            Files.delete(location.toPath());
+          } catch (Throwable cause) {
+            unremoved.put(location, cause);
+          }
+        }
+      }
+    }
+    return unremoved;
+  }
 
   /**
    * Copy one file's contents to another file. The target will be overwritten

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -31,6 +31,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
@@ -223,10 +224,10 @@ public final class OfflineSorter {
     sortInfo = new SortInfo();
     sortInfo.totalTime = System.currentTimeMillis();
 
-    output.delete();
+    Files.deleteIfExists(output.toPath());
 
     ArrayList<File> merges = new ArrayList<>();
-    boolean success2 = false;
+    boolean success3 = false;
     try {
       ByteSequencesReader is = new ByteSequencesReader(input);
       boolean success = false;
@@ -240,11 +241,15 @@ public final class OfflineSorter {
           // Handle intermediate merges.
           if (merges.size() == maxTempFiles) {
             File intermediate = File.createTempFile("sort", "intermediate", tempDirectory);
+            boolean success2 = false;
             try {
               mergePartitions(merges, intermediate);
+              success2 = true;
             } finally {
-              for (File file : merges) {
-                file.delete();
+              if (success2) {
+                IOUtils.deleteFilesIfExist(merges);
+              } else {
+                IOUtils.deleteFilesIgnoringExceptions(merges);
               }
               merges.clear();
               merges.add(intermediate);
@@ -272,13 +277,13 @@ public final class OfflineSorter {
         // otherwise merge the partitions with a priority queue.
         mergePartitions(merges, output);
       }
-      success2 = true;
+      success3 = true;
     } finally {
-      for (File file : merges) {
-        file.delete();
-      }
-      if (!success2) {
-        output.delete();
+      if (success3) {
+        IOUtils.deleteFilesIfExist(merges);
+      } else {
+        IOUtils.deleteFilesIgnoringExceptions(merges);
+        IOUtils.deleteFilesIgnoringExceptions(output);
       }
     }
 

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -178,6 +178,6 @@ public class TestAtomicUpdate extends Lu
     directory = newFSDirectory(dirPath);
     runTest(directory);
     directory.close();
-    TestUtil.rm(dirPath);
+    IOUtils.rm(dirPath);
   }
 }

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
 import java.util.Collection;
 import java.util.HashSet;
@@ -41,6 +42,7 @@ 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;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.junit.Assume;
@@ -446,7 +448,7 @@ public void testFilesOpenClose() throws 
       dir.close();
 
       // Try to erase the data - this ensures that the writer closed all files
-      TestUtil.rm(dirFile);
+      IOUtils.rm(dirFile);
       dir = newFSDirectory(dirFile);
 
       // Now create the data set again, just as before
@@ -464,7 +466,7 @@ public void testFilesOpenClose() throws 
 
       // The following will fail if reader did not close
       // all files
-      TestUtil.rm(dirFile);
+      IOUtils.rm(dirFile);
   }
 
   public void testOpenReaderAfterDelete() throws IOException {
@@ -477,7 +479,7 @@ public void testFilesOpenClose() throws 
       // expected
     }
 
-    dirFile.delete();
+    Files.delete(dirFile.toPath());
 
     // Make sure we still get a CorruptIndexException (not NPE):
     try {
@@ -716,7 +718,7 @@ public void testFilesOpenClose() throws 
   // good exception
   public void testNoDir() throws Throwable {
     File tempDir = createTempDir("doesnotexist");
-    TestUtil.rm(tempDir);
+    IOUtils.rm(tempDir);
     Directory dir = newFSDirectory(tempDir);
     try {
       DirectoryReader.open(dir);
@@ -1052,7 +1054,7 @@ public void testFilesOpenClose() throws 
 
   public void testIndexExistsOnNonExistentDirectory() throws Exception {
     File tempDir = createTempDir("testIndexExistsOnNonExistentDirectory");
-    tempDir.delete();
+    Files.delete(tempDir.toPath());
     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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -26,6 +26,7 @@ import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.io.Writer;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
@@ -87,7 +88,7 @@ public class TestDoc extends LuceneTestC
 
         try {
             File f = new File(workDir, name);
-            if (f.exists()) f.delete();
+            Files.deleteIfExists(f.toPath());
 
             fw = new OutputStreamWriter(new FileOutputStream(f), StandardCharsets.UTF_8);
             pw = new PrintWriter(fw);

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -34,6 +34,7 @@ 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.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.junit.AfterClass;
@@ -249,7 +250,7 @@ public class TestFieldsReader extends Lu
       reader.close();
       dir.close();
     } finally {
-      TestUtil.rm(indexDir);
+      IOUtils.rm(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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -44,8 +44,6 @@ public class TestIndexWriterOnJRECrash e
   public void setUp() throws Exception {
     super.setUp();
     tempDir = createTempDir("jrecrash");
-    tempDir.delete();
-    tempDir.mkdir();
   }
   
   @Override @Nightly

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -26,6 +26,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -107,6 +108,6 @@ public class TestNeverDelete extends Luc
     w.close();
     d.close();
 
-    TestUtil.rm(tmpDir);
+    IOUtils.rm(tmpDir);
   }
 }

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -38,6 +38,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.ArrayUtil;
@@ -279,7 +280,7 @@ public class TestBufferedIndexInput exte
         writer.close();
         reader.close();
       } finally {
-        TestUtil.rm(indexDir);
+        IOUtils.rm(indexDir);
       }
     }
 

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -24,6 +24,7 @@ import java.nio.file.NoSuchFileException
 import java.util.Arrays;
 import java.util.Collections;
 
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
 
 public class TestDirectory extends BaseDirectoryTestCase {
@@ -135,7 +136,7 @@ public class TestDirectory extends BaseD
       assertFalse(dir.isOpen);
     }
     
-    TestUtil.rm(path);
+    IOUtils.rm(path);
   }
 
   // LUCENE-1468
@@ -147,7 +148,7 @@ public class TestDirectory extends BaseD
       Directory fsDir = new SimpleFSDirectory(path, null);
       assertEquals(0, new RAMDirectory(fsDir, newIOContext(random())).listAll().length);
     } finally {
-      TestUtil.rm(path);
+      IOUtils.rm(path);
     }
   }
 
@@ -167,7 +168,7 @@ public class TestDirectory extends BaseD
       }
     } finally {
       fsDir.close();
-      TestUtil.rm(path);
+      IOUtils.rm(path);
     }
   }
 }

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -31,6 +31,7 @@ import org.apache.lucene.index.IndexRead
 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 {
@@ -99,8 +100,7 @@ public class TestFileSwitchDirectory ext
   public void testNoDir() throws Throwable {
     File primDir = createTempDir("foo");
     File secondDir = createTempDir("bar");
-    TestUtil.rm(primDir);
-    TestUtil.rm(secondDir);
+    IOUtils.rm(primDir, secondDir);
     Directory dir = newFSSwitchDirectory(primDir, secondDir, Collections.<String>emptySet());
     try {
       DirectoryReader.open(dir);

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.store;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -35,6 +36,7 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.search.IndexSearcher;
 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;
 
@@ -170,7 +172,7 @@ public class TestLockFactory extends Luc
 
         dir.close();
         // Cleanup
-        TestUtil.rm(indexDir);
+        IOUtils.rm(indexDir);
     }
 
     // Verify: NativeFSLockFactory works correctly
@@ -208,9 +210,7 @@ public class TestLockFactory extends Luc
       assertTrue("failed to obtain lock", l.obtain());
       l.close();
       assertFalse("failed to release lock", l.isLocked());
-      if (lockFile.exists()) {
-        lockFile.delete();
-      }
+      Files.deleteIfExists(lockFile.toPath());
     }
 
     // Verify: NativeFSLockFactory assigns null as lockPrefix if the lockDir is inside directory
@@ -230,8 +230,7 @@ public class TestLockFactory extends Luc
 
       dir1.close();
       dir2.close();
-      TestUtil.rm(fdir1);
-      TestUtil.rm(fdir2);
+      IOUtils.rm(fdir1, fdir2);
     }
 
     // Verify: default LockFactory has no prefix (ie
@@ -253,7 +252,7 @@ public class TestLockFactory extends Luc
       assertNull("Default lock prefix should be null", dir.getLockFactory().getLockPrefix());
       dir.close();
  
-      TestUtil.rm(dirName);
+      IOUtils.rm(dirName);
     }
 
     private class WriterThread extends Thread { 

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -78,8 +78,8 @@ public class TestRAMDirectory extends Ba
       fsDir = newFSDirectory(path);
       assertEquals(0, new RAMDirectory(fsDir, newIOContext(random())).listAll().length);
     } finally {
-      TestUtil.rm(path);
       IOUtils.close(fsDir);
+      IOUtils.rm(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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -65,7 +65,6 @@ public class TestWindowsMMap extends Luc
     // 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");
-    rmDir(dirPath);
     MMapDirectory dir = new MMapDirectory(dirPath, null);
     
     // plan to add a set of useful stopwords, consider changing some of the
@@ -88,16 +87,5 @@ public class TestWindowsMMap extends Luc
     
     reader.close();
     writer.close();
-    rmDir(dirPath);
-  }
-
-  private void rmDir(File dir) {
-    if (!dir.exists()) {
-      return;
-    }
-    for (File file : dir.listFiles()) {
-      file.delete();
-    }
-    dir.delete();
   }
 }

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -44,14 +44,12 @@ public class TestOfflineSorter extends L
   public void setUp() throws Exception {
     super.setUp();
     tempDir = createTempDir("mergesort");
-    TestUtil.rm(tempDir);
-    tempDir.mkdirs();
   }
   
   @Override
   public void tearDown() throws Exception {
     if (tempDir != null)
-      TestUtil.rm(tempDir);
+      IOUtils.rm(tempDir);
     super.tearDown();
   }
 

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=1622506&r1=1622505&r2=1622506&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 Thu Sep  4 15:56:25 2014
@@ -20,8 +20,10 @@ package org.apache.lucene.util.junitcomp
 import java.io.File;
 import java.io.IOException;
 import java.io.RandomAccessFile;
+import java.nio.file.Files;
 
 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;
@@ -45,11 +47,11 @@ public class TestLeaveFilesIfTestFails e
   }
 
   @Test
-  public void testLeaveFilesIfTestFails() {
+  public void testLeaveFilesIfTestFails() throws IOException {
     Result r = JUnitCore.runClasses(Nested1.class);
     Assert.assertEquals(1, r.getFailureCount());
     Assert.assertTrue(Nested1.file != null && Nested1.file.exists());
-    Nested1.file.delete();
+    Files.delete(Nested1.file.toPath());
   }
   
   public static class Nested2 extends WithNestedTests.AbstractNestedTest {
@@ -75,6 +77,6 @@ public class TestLeaveFilesIfTestFails e
     Assert.assertEquals(1, r.getFailureCount());
 
     Nested2.openFile.close();
-    TestUtil.rm(Nested2.parent);
+    IOUtils.rm(Nested2.parent);
   }  
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java Thu Sep  4 15:56:25 2014
@@ -9,6 +9,7 @@ import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -941,9 +942,7 @@ public class DirectoryTaxonomyWriter imp
       in.close();
 
       // Delete the temporary file, which is no longer needed.
-      if (!tmpfile.delete()) {
-        tmpfile.deleteOnExit();
-      }
+      Files.delete(tmpfile.toPath());
 
       return map;
     }

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCharBlockArray.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCharBlockArray.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCharBlockArray.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCharBlockArray.java Thu Sep  4 15:56:25 2014
@@ -96,7 +96,6 @@ public class TestCharBlockArray extends 
     array = CharBlockArray.open(in);
     assertEqualsInternal("GrowingCharArray<->StringBuilder mismatch after flush/load.", builder, array);
     in.close();
-    f.delete();
   }
 
   private static void assertEqualsInternal(String msg, StringBuilder expected, CharBlockArray actual) {

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCompactLabelToOrdinal.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCompactLabelToOrdinal.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCompactLabelToOrdinal.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/taxonomy/writercache/TestCompactLabelToOrdinal.java Thu Sep  4 15:56:25 2014
@@ -5,6 +5,7 @@ import java.nio.ByteBuffer;
 import java.nio.charset.CharsetDecoder;
 import java.nio.charset.CodingErrorAction;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
@@ -76,7 +77,7 @@ public class TestCompactLabelToOrdinal e
       if (i > 0 && i % flushInterval == 0) {
         compact.flush(f);    
         compact = CompactLabelToOrdinal.open(f, 0.15f, 3);
-        assertTrue(f.delete());
+        Files.delete(f.toPath());
         if (flushInterval < (n / 10)) {
           flushInterval *= 10;
         }

Modified: lucene/dev/trunk/lucene/replicator/src/java/org/apache/lucene/replicator/PerSessionDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/replicator/src/java/org/apache/lucene/replicator/PerSessionDirectoryFactory.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/replicator/src/java/org/apache/lucene/replicator/PerSessionDirectoryFactory.java (original)
+++ lucene/dev/trunk/lucene/replicator/src/java/org/apache/lucene/replicator/PerSessionDirectoryFactory.java Thu Sep  4 15:56:25 2014
@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.apache.lucene.replicator.ReplicationClient.SourceDirectoryFactory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.util.IOUtils;
 
 /**
  * A {@link SourceDirectoryFactory} which returns {@link FSDirectory} under a
@@ -40,19 +41,6 @@ public class PerSessionDirectoryFactory 
     this.workDir = workDir;
   }
   
-  private void rm(File file) throws IOException {
-    if (file.isDirectory()) {
-      for (File f : file.listFiles()) {
-        rm(f);
-      }
-    }
-    
-    // This should be either an empty directory, or a file
-    if (!file.delete() && file.exists()) {
-      throw new IOException("failed to delete " + file);
-    }
-  }
-  
   @Override
   public Directory getDirectory(String sessionID, String source) throws IOException {
     File sessionDir = new File(workDir, sessionID);
@@ -71,7 +59,7 @@ public class PerSessionDirectoryFactory 
     if (sessionID.isEmpty()) { // protect against deleting workDir entirely!
       throw new IllegalArgumentException("sessionID cannot be empty");
     }
-    rm(new File(workDir, sessionID));
+    IOUtils.rm(new File(workDir, sessionID));
   }
   
 }

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java Thu Sep  4 15:56:25 2014
@@ -202,12 +202,16 @@ public class SortedInputIterator impleme
   }
   
   private void close() throws IOException {
-    IOUtils.close(reader);
-    if (tempInput != null) {
-      tempInput.delete();
-    }
-    if (tempSorted != null) {
-      tempSorted.delete();
+    boolean success = false;
+    try {
+      IOUtils.close(reader);
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.deleteFilesIfExist(tempInput, tempSorted);
+      } else {
+        IOUtils.deleteFilesIgnoringExceptions(tempInput, tempSorted);
+      }
     }
   }
   

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Thu Sep  4 15:56:25 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.search.suggest
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -500,7 +501,7 @@ public class AnalyzingSuggester extends 
       new OfflineSorter(new AnalyzingComparator(hasPayloads)).sort(tempInput, tempSorted);
 
       // Free disk space:
-      tempInput.delete();
+      Files.delete(tempInput.toPath());
 
       reader = new OfflineSorter.ByteSequencesReader(tempSorted);
      
@@ -593,14 +594,13 @@ public class AnalyzingSuggester extends 
       
       success = true;
     } finally {
+      IOUtils.closeWhileHandlingException(reader, writer);
+      
       if (success) {
-        IOUtils.close(reader, writer);
+        IOUtils.deleteFilesIfExist(tempInput, tempSorted);
       } else {
-        IOUtils.closeWhileHandlingException(reader, writer);
+        IOUtils.deleteFilesIgnoringExceptions(tempInput, tempSorted);
       }
-      
-      tempInput.delete();
-      tempSorted.delete();
     }
   }
 

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java Thu Sep  4 15:56:25 2014
@@ -376,23 +376,12 @@ public class FreeTextSuggester extends L
     } finally {
       try {
         if (success) {
-          IOUtils.close(reader);
+          IOUtils.close(reader, dir);
         } else {
-          IOUtils.closeWhileHandlingException(reader, writer);
+          IOUtils.closeWhileHandlingException(reader, writer, dir);
         }
       } finally {
-        for(String file : dir.listAll()) {
-          File path = new File(tempIndexPath, file);
-          if (path.delete() == false) {
-            throw new IllegalStateException("failed to remove " + path);
-          }
-        }
-
-        if (tempIndexPath.delete() == false) {
-          throw new IllegalStateException("failed to remove " + tempIndexPath);
-        }
-
-        dir.close();
+        IOUtils.rm(tempIndexPath);
       }
     }
   }

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java Thu Sep  4 15:56:25 2014
@@ -20,6 +20,7 @@ package org.apache.lucene.search.suggest
 import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.util.Comparator;
 
 import org.apache.lucene.util.BytesRef;
@@ -61,9 +62,18 @@ public class ExternalRefSorter implement
       
       sorted = File.createTempFile("RefSorter-", ".sorted",
           OfflineSorter.defaultTempDir());
-      sort.sort(input, sorted);
+      boolean success = false;
+      try {
+        sort.sort(input, sorted);
+        success = true;
+      } finally {
+        if (success) {
+          Files.delete(input.toPath());
+        } else {
+          IOUtils.deleteFilesIgnoringExceptions(input);
+        }
+      }
       
-      input.delete();
       input = null;
     }
     
@@ -82,11 +92,16 @@ public class ExternalRefSorter implement
    */
   @Override
   public void close() throws IOException {
+    boolean success = false;
     try {
       closeWriter();
+      success = true;
     } finally {
-      if (input != null) input.delete();
-      if (sorted != null) sorted.delete();
+      if (success) {
+        IOUtils.deleteFilesIfExist(input, sorted);
+      } else {
+        IOUtils.deleteFilesIgnoringExceptions(input, sorted);
+      }
     }
   }
   

Modified: lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java (original)
+++ lucene/dev/trunk/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java Thu Sep  4 15:56:25 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.search.suggest
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
@@ -189,7 +190,7 @@ public class FSTCompletionLookup extends
       // We don't know the distribution of scores and we need to bucket them, so we'll sort
       // and divide into equal buckets.
       SortInfo info = new OfflineSorter().sort(tempInput, tempSorted);
-      tempInput.delete();
+      Files.delete(tempInput.toPath());
       FSTCompletionBuilder builder = new FSTCompletionBuilder(
           buckets, sorter = new ExternalRefSorter(new OfflineSorter()), sharedTailLength);
 
@@ -231,13 +232,13 @@ public class FSTCompletionLookup extends
       
       success = true;
     } finally {
-      if (success) 
-        IOUtils.close(reader, writer, sorter);
-      else 
-        IOUtils.closeWhileHandlingException(reader, writer, sorter);
+      IOUtils.closeWhileHandlingException(reader, writer, sorter);
 
-      tempInput.delete();
-      tempSorted.delete();
+      if (success) {
+        Files.delete(tempSorted.toPath());
+      } else {
+        IOUtils.deleteFilesIgnoringExceptions(tempInput, tempSorted);
+      }
     }
   }
   

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Thu Sep  4 15:56:25 2014
@@ -1356,7 +1356,7 @@ public abstract class BasePostingsFormat
 
     fieldsProducer.close();
     dir.close();
-    TestUtil.rm(path);
+    IOUtils.rm(path);
   }
 
   public void testDocsOnly() throws Exception {
@@ -1405,7 +1405,7 @@ public abstract class BasePostingsFormat
       fieldsProducer = null;
 
       dir.close();
-      TestUtil.rm(path);
+      IOUtils.rm(path);
     }
   }
   

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Thu Sep  4 15:56:25 2014
@@ -42,6 +42,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FailOnNonBulkMergesInfoStream;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NamedThreadFactory;
@@ -653,7 +654,7 @@ public abstract class ThreadedIndexingAn
 
     TestUtil.checkIndex(dir);
     dir.close();
-    TestUtil.rm(tempDir);
+    IOUtils.rm(tempDir);
 
     if (VERBOSE) {
       System.out.println("TEST: done [" + (System.currentTimeMillis()-t0) + " ms]");

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java Thu Sep  4 15:56:25 2014
@@ -21,6 +21,7 @@ import java.io.EOFException;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.nio.file.NoSuchFileException;
 import java.util.Arrays;
 import java.util.Collections;
@@ -453,7 +454,7 @@ public abstract class BaseDirectoryTestC
    *  mkdir the underling directory in the filesystem. */
   public void testDontCreate() throws Throwable {
     File path = createTempDir("doesnotexist");
-    TestUtil.rm(path);
+    IOUtils.rm(path);
     assertTrue(!path.exists());
     Directory dir = getDirectory(path);
     assertTrue(!path.exists());
@@ -559,7 +560,7 @@ public abstract class BaseDirectoryTestC
   // LUCENE-3382 -- make sure we get exception if the directory really does not exist.
   public void testNoDir() throws Throwable {
     File tempDir = createTempDir("doesnotexist");
-    TestUtil.rm(tempDir);
+    IOUtils.rm(tempDir);
     Directory dir = getDirectory(tempDir);
     try {
       DirectoryReader.open(dir);
@@ -767,7 +768,7 @@ public abstract class BaseDirectoryTestC
     out.close();
     
     // delete it
-    assertTrue(new File(path, "afile").delete());
+    Files.delete(new File(path, "afile").toPath());
     
     // directory is empty
     assertEquals(0, fsdir.listAll().length);

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/RemoveUponClose.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/RemoveUponClose.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/RemoveUponClose.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/RemoveUponClose.java Thu Sep  4 15:56:25 2014
@@ -46,7 +46,7 @@ final class RemoveUponClose implements C
     if (failureMarker.wasSuccessful()) {
       if (file.exists()) {
         try {
-          TestUtil.rm(file);
+          IOUtils.rm(file);
         } catch (IOException e) {
           throw new IOException(
               "Could not remove temporary location '" 

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java Thu Sep  4 15:56:25 2014
@@ -122,7 +122,7 @@ final class TestRuleTemporaryFilesCleanu
     // and leave them there.
     if (failureMarker.wasSuccessful()) {
       try {
-        TestUtil.rm(everything);
+        IOUtils.rm(everything);
       } catch (IOException e) {
         Class<?> suiteClass = RandomizedContext.current().getTargetClass();
         if (suiteClass.isAnnotationPresent(SuppressTempFileChecks.class)) {

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Thu Sep  4 15:56:25 2014
@@ -28,10 +28,11 @@ import java.io.PrintStream;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.nio.CharBuffer;
+import java.nio.file.Files;
 import java.util.Arrays;
 import java.util.Enumeration;
 import java.util.HashMap;
-import java.util.LinkedHashSet;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
@@ -99,48 +100,12 @@ public final class TestUtil {
     //
   }
 
-  /**
-   * Deletes one or more files or directories (and everything underneath it).
-   * 
-   * @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 {
-    LinkedHashSet<File> unremoved = rm(new LinkedHashSet<File>(), locations);
-    if (!unremoved.isEmpty()) {
-      StringBuilder b = new StringBuilder("Could not remove the following files (in the order of attempts):\n");
-      for (File f : unremoved) {
-        b.append("   ")
-         .append(f.getAbsolutePath())
-         .append("\n");
-      }
-      throw new IOException(b.toString());
-    }
-  }
-
-  private static LinkedHashSet<File> rm(LinkedHashSet<File> unremoved, File... locations) {
-    if (locations != null) {
-      for (File location : locations) {
-        if (location != null && location.exists()) {
-          if (location.isDirectory()) {
-            rm(unremoved, location.listFiles());
-          }
-  
-          if (!location.delete()) {
-            unremoved.add(location);
-          }
-        }
-      }
-    }
-    return unremoved;
-  }
-
   /** 
    * Convenience method unzipping zipName into destDir, cleaning up 
    * destDir first. 
    */
   public static void unzip(File zipName, File destDir) throws IOException {
-    rm(destDir);
+    IOUtils.rm(destDir);
     destDir.mkdir();
 
     ZipFile zipFile = new ZipFile(zipName);

Modified: lucene/dev/trunk/lucene/tools/forbiddenApis/base.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/tools/forbiddenApis/base.txt?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/tools/forbiddenApis/base.txt (original)
+++ lucene/dev/trunk/lucene/tools/forbiddenApis/base.txt Thu Sep  4 15:56:25 2014
@@ -29,3 +29,5 @@ java.util.Properties#store(java.io.Outpu
 
 java.lang.Character#codePointBefore(char[],int) @ Implicit start offset is error-prone when the char[] is a buffer and the first chars are random chars
 java.lang.Character#codePointAt(char[],int) @ Implicit end offset is error-prone when the char[] is a buffer and the last chars are random chars
+
+java.io.File#delete() @ use Files.delete for real exception, IOUtils.deleteFilesIgnoringExceptions if you dont care

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractSqlEntityProcessorTestCase.java Thu Sep  4 15:56:25 2014
@@ -1,6 +1,7 @@
 package org.apache.solr.handler.dataimport;
 
 import java.io.File;
+import java.nio.file.Files;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -60,7 +61,7 @@ public abstract class AbstractSqlEntityP
   } 
   
   @After
-  public void afterSqlEntitiyProcessorTestCase() {
+  public void afterSqlEntitiyProcessorTestCase() throws Exception {
     useSimpleCaches = false;
     countryEntity = false;
     countryCached = false;
@@ -74,8 +75,8 @@ public abstract class AbstractSqlEntityP
     //If an Assume was tripped while setting up the test, 
     //the file might not ever have been created...
     if(fileLocation!=null) {
-      new File(fileLocation + File.separatorChar + fileName).delete();
-      new File(fileLocation).delete();
+      Files.deleteIfExists(new File(fileLocation + File.separatorChar + fileName).toPath());
+      Files.deleteIfExists(new File(fileLocation).toPath());
     }
   }
   

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java Thu Sep  4 15:56:25 2014
@@ -24,6 +24,7 @@ import org.junit.Test;
 import java.io.File;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.text.SimpleDateFormat;
 import java.util.*;
 
@@ -65,7 +66,7 @@ public class TestFileListEntityProcessor
   @Test
   public void testBiggerSmallerFiles() throws IOException {
     File tmpdir = File.createTempFile("test", "tmp", createTempDir());
-    tmpdir.delete();
+    Files.delete(tmpdir.toPath());
     tmpdir.mkdir();
 
     long minLength = Long.MAX_VALUE;

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNonWritablePersistFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNonWritablePersistFile.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNonWritablePersistFile.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNonWritablePersistFile.java Thu Sep  4 15:56:25 2014
@@ -92,7 +92,7 @@ public class TestNonWritablePersistFile 
       runFullImport(dataConfig_delta);
       assertQ(req("id:1"), "//*[@numFound='0']");
     } finally {
-      f.delete();
+      f.setWritable(true);
     }
   }  
 }

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java Thu Sep  4 15:56:25 2014
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -335,7 +336,7 @@ public class TestSolrEntityProcessorEndT
     }
 
     public void tearDown() throws Exception {
-      TestUtil.rm(homeDir);
+      IOUtils.rm(homeDir);
     }
   }
   

Modified: lucene/dev/trunk/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java (original)
+++ lucene/dev/trunk/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/SolrOutputFormat.java Thu Sep  4 15:56:25 2014
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Locale;
@@ -224,7 +225,7 @@ public class SolrOutputFormat<K, V> exte
                                    // to store in the zip file
     }
 
-    out.delete();
+    Files.deleteIfExists(out.toPath());
     int subst = dir.toString().length();
     ZipOutputStream zos = new ZipOutputStream(new FileOutputStream(out));
     byte[] buf = new byte[1024];

Modified: lucene/dev/trunk/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java?rev=1622506&r1=1622505&r2=1622506&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java (original)
+++ lucene/dev/trunk/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MRUnitBase.java Thu Sep  4 15:56:25 2014
@@ -18,6 +18,7 @@ package org.apache.solr.hadoop;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.util.Locale;
 
 import org.apache.commons.io.FileUtils;
@@ -44,7 +45,7 @@ public abstract class MRUnitBase extends
 
   @AfterClass
   public static void teardownClass() throws Exception {
-    if (solrHomeZip != null) solrHomeZip.delete();
+    if (solrHomeZip != null) Files.delete(solrHomeZip.toPath());
   }
   
   protected void setupHadoopConfig(Configuration config) throws IOException {