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 2015/04/21 13:33:52 UTC

svn commit: r1675097 - in /lucene/dev/trunk/lucene/test-framework/src: java/org/apache/lucene/mockfile/ java/org/apache/lucene/util/ test/org/apache/lucene/mockfile/

Author: rmuir
Date: Tue Apr 21 11:33:52 2015
New Revision: 1675097

URL: http://svn.apache.org/r1675097
Log:
LUCENE-6442: Add mockfs with unpredictable but deterministic file listing order

Added:
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/MockFileSystemTestCase.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/ShuffleFS.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestDisableFsyncFS.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestExtrasFS.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestHandleLimitFS.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestLeakFS.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestShuffleFS.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestVerboseFS.java   (with props)
    lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestWindowsFS.java   (with props)
Removed:
    lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestMockFilesystems.java
Modified:
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/ExtrasFS.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/package-info.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/ExtrasFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/ExtrasFS.java?rev=1675097&r1=1675096&r2=1675097&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/ExtrasFS.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/ExtrasFS.java Tue Apr 21 11:33:52 2015
@@ -22,10 +22,6 @@ import java.nio.file.FileSystem;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.attribute.FileAttribute;
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicLong;
-
-import com.carrotsearch.randomizedtesting.RandomizedContext;
 
 /** 
  * Adds extra files/subdirectories when directories are created.
@@ -39,14 +35,19 @@ import com.carrotsearch.randomizedtestin
  * All other filesystem operations are passed thru as normal.
  */
 public class ExtrasFS extends FilterFileSystemProvider {
-  final int seed;
+  final boolean active;
+  final boolean createDirectory;
   
   /** 
    * Create a new instance, wrapping {@code delegate}.
+   * @param active {@code true} if we should create extra files
+   * @param createDirectory {@code true} if we should create directories instead of files.
+   *        Ignored if {@code active} is {@code false}.
    */
-  public ExtrasFS(FileSystem delegate, Random random) {
+  public ExtrasFS(FileSystem delegate, boolean active, boolean createDirectory) {
     super("extras://", delegate);
-    this.seed = random.nextInt();
+    this.active = active;
+    this.createDirectory = createDirectory;
   }
 
   @Override
@@ -54,17 +55,11 @@ public class ExtrasFS extends FilterFile
     super.createDirectory(dir, attrs);   
     // ok, we created the directory successfully.
     
-    // a little funky: we only look at hashcode (well-defined) of the target class name.
-    // using a generator won't reproduce, because we are a per-class resource.
-    // using hashing on filenames won't reproduce, because many of the names rely on other things
-    // the test class did.
-    // so a test gets terrorized with extras or gets none at all depending on the initial seed.
-    int hash = RandomizedContext.current().getTargetClass().toString().hashCode() ^ seed;
-    if ((hash & 3) == 0) {
+    if (active) {
       // lets add a bogus file... if this fails, we don't care, its best effort.
       try {
         Path target = dir.resolve("extra0");
-        if (hash < 0) {
+        if (createDirectory) {
           super.createDirectory(target);
         } else {
           Files.createFile(target);

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/MockFileSystemTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/MockFileSystemTestCase.java?rev=1675097&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/MockFileSystemTestCase.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/MockFileSystemTestCase.java Tue Apr 21 11:33:52 2015
@@ -0,0 +1,116 @@
+package org.apache.lucene.mockfile;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.nio.charset.Charset;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
+
+/** 
+ * Base class for testing mockfilesystems. This tests things
+ * that really need to work: Path equals()/hashcode(), directory listing
+ * glob and filtering, URI conversion, etc.
+ */
+@SuppressFileSystems("*") // we suppress random filesystems and do tests explicitly.
+public abstract class MockFileSystemTestCase extends LuceneTestCase {
+  
+  /** wraps Path with custom behavior */
+  protected abstract Path wrap(Path path);
+
+  /** Test that Path.hashcode/equals are sane */
+  public void testHashCodeEquals() throws IOException {
+    Path dir = wrap(createTempDir());
+
+    Path f1 = dir.resolve("file1");
+    Path f1Again = dir.resolve("file1");
+    Path f2 = dir.resolve("file2");
+    
+    assertEquals(f1, f1);
+    assertFalse(f1.equals(null));
+    assertEquals(f1, f1Again);
+    assertEquals(f1.hashCode(), f1Again.hashCode());
+    assertFalse(f1.equals(f2));
+    dir.getFileSystem().close();
+  }
+  
+  /** Test that URIs are not corrumpted */
+  public void testURI() throws IOException {
+    Path dir = wrap(createTempDir());
+
+    Path f1 = dir.resolve("file1");
+    URI uri = f1.toUri();
+    Path f2 = dir.getFileSystem().provider().getPath(uri);
+    assertEquals(f1, f2);
+    
+    assumeTrue(Charset.defaultCharset().name() + " can't encode chinese", 
+               Charset.defaultCharset().newEncoder().canEncode("中国"));
+    Path f3 = dir.resolve("中国");
+    URI uri2 = f3.toUri();
+    Path f4 = dir.getFileSystem().provider().getPath(uri2);
+    assertEquals(f3, f4);
+    dir.getFileSystem().close();
+  }
+  
+  /** Tests that newDirectoryStream with a filter works correctly */
+  public void testDirectoryStreamFiltered() throws IOException {
+    Path dir = wrap(createTempDir());
+
+    OutputStream file = Files.newOutputStream(dir.resolve("file1"));
+    file.write(5);
+    file.close();
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir)) {
+      int count = 0;
+      for (Path path : stream) {
+        assertTrue(path instanceof FilterPath);
+        if (!path.getFileName().toString().startsWith("extra")) {
+          count++;
+        }
+      }
+      assertEquals(1, count);
+    }
+    dir.getFileSystem().close();
+  }
+
+  /** Tests that newDirectoryStream with globbing works correctly */
+  public void testDirectoryStreamGlobFiltered() throws IOException {
+    Path dir = wrap(createTempDir());
+
+    OutputStream file = Files.newOutputStream(dir.resolve("foo"));
+    file.write(5);
+    file.close();
+    file = Files.newOutputStream(dir.resolve("bar"));
+    file.write(5);
+    file.close();
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir, "f*")) {
+      int count = 0;
+      for (Path path : stream) {
+        assertTrue(path instanceof FilterPath);
+        ++count;
+      }
+      assertEquals(1, count);
+    }
+    dir.getFileSystem().close();
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/ShuffleFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/ShuffleFS.java?rev=1675097&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/ShuffleFS.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/ShuffleFS.java Tue Apr 21 11:33:52 2015
@@ -0,0 +1,70 @@
+package org.apache.lucene.mockfile;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.nio.file.DirectoryStream;
+import java.nio.file.DirectoryStream.Filter;
+import java.nio.file.FileSystem;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * Gives an unpredictable, but deterministic order to directory listings.
+ * <p>
+ * This can be useful if for instance, you have build servers on
+ * linux but developers are using macs.
+ */
+public class ShuffleFS extends FilterFileSystemProvider {
+  final long seed;
+  
+  /** 
+   * Create a new instance, wrapping {@code delegate}.
+   */
+  public ShuffleFS(FileSystem delegate, long seed) {
+    super("shuffle://", delegate);
+    this.seed = seed;
+  }
+
+  @Override
+  public DirectoryStream<Path> newDirectoryStream(Path dir, Filter<? super Path> filter) throws IOException {
+    try (DirectoryStream<Path> stream = super.newDirectoryStream(dir, filter)) {
+      // read complete directory listing
+      List<Path> contents = new ArrayList<>();
+      for (Path path : stream) {
+        contents.add(path);
+      }
+      // sort first based only on filename
+      Collections.sort(contents, (path1, path2) -> path1.getFileName().toString().compareTo(path2.getFileName().toString()));
+      // sort based on current class seed
+      Collections.shuffle(contents, new Random(seed));
+      return new DirectoryStream<Path>() {
+        @Override
+        public Iterator<Path> iterator() {
+          return contents.iterator();
+        }
+        @Override
+        public void close() throws IOException {}        
+      };
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/package-info.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/package-info.java?rev=1675097&r1=1675096&r2=1675097&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/package-info.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/package-info.java Tue Apr 21 11:33:52 2015
@@ -24,6 +24,8 @@
  *   <li>{@link org.apache.lucene.mockfile.VerboseFS}: Prints destructive filesystem operations to infostream.
  *   <li>{@link org.apache.lucene.mockfile.WindowsFS}: Acts like windows.
  *   <li>{@link org.apache.lucene.mockfile.DisableFsyncFS}: Makes actual fsync calls a no-op.
+ *   <li>{@link org.apache.lucene.mockfile.ExtrasFS}: Adds 'bonus' files to directories.
+ *   <li>{@link org.apache.lucene.mockfile.ShuffleFS}: Directory listings in an unpredictable but deterministic order.
  * </ul>
  */
 package org.apache.lucene.mockfile;

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=1675097&r1=1675096&r2=1675097&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 Tue Apr 21 11:33:52 2015
@@ -20,6 +20,7 @@ import org.apache.lucene.mockfile.Disabl
 import org.apache.lucene.mockfile.ExtrasFS;
 import org.apache.lucene.mockfile.HandleLimitFS;
 import org.apache.lucene.mockfile.LeakFS;
+import org.apache.lucene.mockfile.ShuffleFS;
 import org.apache.lucene.mockfile.VerboseFS;
 import org.apache.lucene.mockfile.WindowsFS;
 import org.apache.lucene.util.LuceneTestCase.SuppressFileSystems;
@@ -147,6 +148,13 @@ final class TestRuleTemporaryFilesCleanu
       }
     }
     
+    // impacts test reproducibility across platforms.
+    if (random.nextInt(100) > 0) {
+      if (allowed(avoid, ShuffleFS.class)) {
+        fs = new ShuffleFS(fs, random.nextLong()).getFileSystem(null);
+      }
+    }
+    
     // otherwise, wrap with mockfilesystems for additional checks. some 
     // of these have side effects (e.g. concurrency) so it doesn't always happen.
     if (random.nextInt(10) > 0) {
@@ -164,7 +172,7 @@ final class TestRuleTemporaryFilesCleanu
         }
       }
       if (allowed(avoid, ExtrasFS.class)) {
-        fs = new ExtrasFS(fs, new Random(random.nextLong())).getFileSystem(null);
+        fs = new ExtrasFS(fs, random.nextInt(4) == 0, random.nextBoolean()).getFileSystem(null);
       }
     }
     if (LuceneTestCase.VERBOSE) {

Added: lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestDisableFsyncFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestDisableFsyncFS.java?rev=1675097&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestDisableFsyncFS.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestDisableFsyncFS.java Tue Apr 21 11:33:52 2015
@@ -0,0 +1,50 @@
+package org.apache.lucene.mockfile;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.FileSystem;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+
+/** Basic tests for DisableFsyncFS */
+public class TestDisableFsyncFS extends MockFileSystemTestCase {
+  
+  @Override
+  protected Path wrap(Path path) {
+    FileSystem fs = new DisableFsyncFS(path.getFileSystem()).getFileSystem(URI.create("file:///"));
+    return new FilterPath(path, fs);
+  }
+  
+  /** Test that we don't corrumpt fsync: it just doesnt happen */
+  public void testFsyncWorks() throws Exception {
+    Path dir = wrap(createTempDir());
+    
+    FileChannel file = FileChannel.open(dir.resolve("file"), 
+                                        StandardOpenOption.CREATE_NEW, 
+                                        StandardOpenOption.READ, 
+                                        StandardOpenOption.WRITE);
+    byte bytes[] = new byte[128];
+    random().nextBytes(bytes);
+    file.write(ByteBuffer.wrap(bytes));
+    file.force(true);
+    file.close();
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestExtrasFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestExtrasFS.java?rev=1675097&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestExtrasFS.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestExtrasFS.java Tue Apr 21 11:33:52 2015
@@ -0,0 +1,82 @@
+package org.apache.lucene.mockfile;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.net.URI;
+import java.nio.file.DirectoryStream;
+import java.nio.file.FileSystem;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/** Basic tests for ExtrasFS */
+public class TestExtrasFS extends MockFileSystemTestCase {
+  
+  @Override
+  protected Path wrap(Path path) {
+    return wrap(path, random().nextBoolean(), random().nextBoolean());
+  }
+  
+  Path wrap(Path path, boolean active, boolean createDirectory) {
+    FileSystem fs = new ExtrasFS(path.getFileSystem(), active, createDirectory).getFileSystem(URI.create("file:///"));
+    return new FilterPath(path, fs);
+  }
+  
+  /** test where extra file is created */
+  public void testExtraFile() throws Exception {
+    Path dir = wrap(createTempDir(), true, false);
+    Files.createDirectory(dir.resolve("foobar"));
+    
+    List<String> seen = new ArrayList<>();
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir.resolve("foobar"))) {
+      for (Path path : stream) {
+        seen.add(path.getFileName().toString());
+      }
+    }
+    assertEquals(Arrays.asList("extra0"), seen);
+    assertTrue(Files.isRegularFile(dir.resolve("foobar").resolve("extra0")));
+  }
+  
+  /** test where extra directory is created */
+  public void testExtraDirectory() throws Exception {
+    Path dir = wrap(createTempDir(), true, true);
+    Files.createDirectory(dir.resolve("foobar"));
+    
+    List<String> seen = new ArrayList<>();
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir.resolve("foobar"))) {
+      for (Path path : stream) {
+        seen.add(path.getFileName().toString());
+      }
+    }
+    assertEquals(Arrays.asList("extra0"), seen);
+    assertTrue(Files.isDirectory(dir.resolve("foobar").resolve("extra0")));
+  }
+  
+  /** test where no extras are created: its a no-op */
+  public void testNoExtras() throws Exception {
+    Path dir = wrap(createTempDir(), false, false);
+    Files.createDirectory(dir.resolve("foobar"));
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir.resolve("foobar"))) {
+      for (Path path : stream) {
+        fail("should not have found file: " + path);
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestHandleLimitFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestHandleLimitFS.java?rev=1675097&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestHandleLimitFS.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestHandleLimitFS.java Tue Apr 21 11:33:52 2015
@@ -0,0 +1,67 @@
+package org.apache.lucene.mockfile;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.FileSystem;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.util.IOUtils;
+
+/** Basic tests for HandleLimitFS */
+public class TestHandleLimitFS extends MockFileSystemTestCase {
+  
+  @Override
+  protected Path wrap(Path path) {
+    return wrap(path, 4096);
+  }
+  
+  Path wrap(Path path, int limit) {
+    FileSystem fs = new HandleLimitFS(path.getFileSystem(), limit).getFileSystem(URI.create("file:///"));
+    return new FilterPath(path, fs);
+  }
+  
+  /** set a limit at n files, then open more than that and ensure we hit exception */
+  public void testTooManyOpenFiles() throws IOException {
+    int n = 60;
+
+    Path dir = wrap(createTempDir(), n);
+    
+    // create open files to exact limit
+    List<Closeable> toClose = new ArrayList<>();
+    for (int i = 0; i < n; i++) {
+      Path p = Files.createTempFile(dir, null, null);
+      toClose.add(Files.newOutputStream(p));
+    }
+    
+    // now exceed
+    try {
+      Files.newOutputStream(Files.createTempFile(dir, null, null));
+      fail("didn't hit exception");
+    } catch (IOException e) {
+      assertTrue(e.getMessage().contains("Too many open files"));
+    }
+    
+    IOUtils.close(toClose);
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestLeakFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestLeakFS.java?rev=1675097&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestLeakFS.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestLeakFS.java Tue Apr 21 11:33:52 2015
@@ -0,0 +1,121 @@
+package org.apache.lucene.mockfile;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.nio.channels.AsynchronousFileChannel;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.FileSystem;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+/** Basic tests for LeakFS */
+public class TestLeakFS extends MockFileSystemTestCase {
+  
+  @Override
+  protected Path wrap(Path path) {
+    FileSystem fs = new LeakFS(path.getFileSystem()).getFileSystem(URI.create("file:///"));
+    return new FilterPath(path, fs);
+  }
+  
+  /** Test leaks via Files.newInputStream */
+  public void testLeakInputStream() throws IOException {
+    Path dir = wrap(createTempDir());
+    
+    OutputStream file = Files.newOutputStream(dir.resolve("stillopen"));
+    file.write(5);
+    file.close();
+    InputStream leak = Files.newInputStream(dir.resolve("stillopen"));
+    try {
+      dir.getFileSystem().close();
+      fail("should have gotten exception");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("file handle leaks"));
+    }
+    leak.close();
+  }
+  
+  /** Test leaks via Files.newOutputStream */
+  public void testLeakOutputStream() throws IOException {
+    Path dir = wrap(createTempDir());
+    
+    OutputStream leak = Files.newOutputStream(dir.resolve("leaky"));
+    try {
+      dir.getFileSystem().close();
+      fail("should have gotten exception");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("file handle leaks"));
+    }
+    leak.close();
+  }
+  
+  /** Test leaks via FileChannel.open */
+  public void testLeakFileChannel() throws IOException {
+    Path dir = wrap(createTempDir());
+    
+    OutputStream file = Files.newOutputStream(dir.resolve("stillopen"));
+    file.write(5);
+    file.close();
+    FileChannel leak = FileChannel.open(dir.resolve("stillopen"));
+    try {
+      dir.getFileSystem().close();
+      fail("should have gotten exception");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("file handle leaks"));
+    }
+    leak.close();
+  }
+  
+  /** Test leaks via AsynchronousFileChannel.open */
+  public void testLeakAsyncFileChannel() throws IOException {
+    Path dir = wrap(createTempDir());
+    
+    OutputStream file = Files.newOutputStream(dir.resolve("stillopen"));
+    file.write(5);
+    file.close();
+    AsynchronousFileChannel leak = AsynchronousFileChannel.open(dir.resolve("stillopen"));
+    try {
+      dir.getFileSystem().close();
+      fail("should have gotten exception");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("file handle leaks"));
+    }
+    leak.close();
+  }
+  
+  /** Test leaks via Files.newByteChannel */
+  public void testLeakByteChannel() throws IOException {
+    Path dir = wrap(createTempDir());
+    
+    OutputStream file = Files.newOutputStream(dir.resolve("stillopen"));
+    file.write(5);
+    file.close();
+    SeekableByteChannel leak = Files.newByteChannel(dir.resolve("stillopen"));
+    try {
+      dir.getFileSystem().close();
+      fail("should have gotten exception");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains("file handle leaks"));
+    }
+    leak.close();
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestShuffleFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestShuffleFS.java?rev=1675097&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestShuffleFS.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestShuffleFS.java Tue Apr 21 11:33:52 2015
@@ -0,0 +1,170 @@
+package org.apache.lucene.mockfile;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.DirectoryStream;
+import java.nio.file.FileSystem;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+/** Basic tests for ShuffleFS */
+public class TestShuffleFS extends MockFileSystemTestCase {
+  
+  @Override
+  protected Path wrap(Path path) {
+    return wrap(path, random().nextLong());
+  }
+  
+  Path wrap(Path path, long seed) {
+    FileSystem fs = new ShuffleFS(path.getFileSystem(), seed).getFileSystem(URI.create("file:///"));
+    return new FilterPath(path, fs);
+  }
+  
+  /** test that we return directory listings correctly */
+  public void testShuffleWorks() throws IOException {
+    Path dir = wrap(createTempDir());
+    
+    Files.createFile(dir.resolve("file1"));
+    Files.createFile(dir.resolve("file2"));
+    Files.createFile(dir.resolve("file3"));
+    
+    List<Path> seen = new ArrayList<>();
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir)) {
+      for (Path path : stream) {
+        seen.add(path);
+      }
+    }
+    
+    assertEquals(3, seen.size());
+    assertTrue(seen.contains(dir.resolve("file1")));
+    assertTrue(seen.contains(dir.resolve("file2")));
+    assertTrue(seen.contains(dir.resolve("file3")));
+  }
+  
+  /** test that we change order of directory listings */
+  public void testActuallyShuffles() throws IOException {
+    Path dir = createTempDir();
+    for (int i = 0; i < 100; i++) {
+      Files.createFile(dir.resolve("file" + i));
+    }
+    List<String> expected = new ArrayList<>();
+    
+    // get the raw listing from the actual filesystem
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir)) {
+      for (Path path : stream) {
+        expected.add(path.getFileName().toString());
+      }
+    }
+    
+    // shuffle until the order changes.
+    for (int i = 0; i < 10000; i++) {
+      Path wrapped = wrap(dir, random().nextLong());
+    
+      List<String> seen = new ArrayList<>();
+      try (DirectoryStream<Path> stream = Files.newDirectoryStream(wrapped)) {
+        for (Path path : stream) {
+          seen.add(path.getFileName().toString());
+        }
+      }
+      
+      // we should always see the same files.
+      assertEquals(new HashSet<>(expected), new HashSet<>(seen));
+      if (!expected.equals(seen)) {
+        return;
+      }
+    }
+    fail("ordering never changed");
+  }
+  
+  /** 
+   * shuffle underlying contents randomly with different seeds,
+   * and ensure shuffling that again with the same seed is consistent.
+   */
+  public void testConsistentOrder() throws IOException {
+    Path raw = createTempDir();
+    for (int i = 0; i < 100; i++) {
+      Files.createFile(raw.resolve("file" + i));
+    }
+    
+    long seed = random().nextLong();
+    Path dirExpected = wrap(raw, seed);
+
+    // get the shuffled listing for the seed.
+    List<String> expected = new ArrayList<>();
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(dirExpected)) {
+      for (Path path : stream) {
+        expected.add(path.getFileName().toString());
+      }
+    }
+    
+    // shuffle wrapping a different scrambled ordering each time, it should always be the same.
+    for (int i = 0; i < 100; i++) {
+      Path scrambled = wrap(raw, random().nextLong());
+      Path ordered = wrap(scrambled, seed);
+    
+      List<String> seen = new ArrayList<>();
+      try (DirectoryStream<Path> stream = Files.newDirectoryStream(ordered)) {
+        for (Path path : stream) {
+          seen.add(path.getFileName().toString());
+        }
+      }
+      
+      // we should always see the same files in the same order
+      assertEquals(expected, seen);
+    }
+  }
+  
+  /** 
+   * test that we give a consistent order 
+   * for the same file names within different directories 
+   */
+  public void testFileNameOnly() throws IOException {
+    Path dir = wrap(createTempDir());
+    
+    Files.createFile(dir.resolve("file1"));
+    Files.createFile(dir.resolve("file2"));
+    Files.createFile(dir.resolve("file3"));
+    
+    List<String> expected = new ArrayList<>();
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir)) {
+      for (Path path : stream) {
+        expected.add(path.getFileName().toString());
+      }
+    }
+    
+    Path subdir = dir.resolve("subdir");
+    Files.createDirectory(subdir);
+    Files.createFile(subdir.resolve("file3"));
+    Files.createFile(subdir.resolve("file2"));
+    Files.createFile(subdir.resolve("file1"));
+    
+    List<String> actual = new ArrayList<>();
+    try (DirectoryStream<Path> stream = Files.newDirectoryStream(subdir)) {
+      for (Path path : stream) {
+        actual.add(path.getFileName().toString());
+      }
+    }
+    
+    assertEquals(expected, actual);
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestVerboseFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestVerboseFS.java?rev=1675097&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestVerboseFS.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestVerboseFS.java Tue Apr 21 11:33:52 2015
@@ -0,0 +1,224 @@
+package org.apache.lucene.mockfile;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.nio.channels.AsynchronousFileChannel;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.FileSystem;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.lucene.util.InfoStream;
+
+/** Basic tests for VerboseFS */
+public class TestVerboseFS extends MockFileSystemTestCase {
+  
+  @Override
+  protected Path wrap(Path path) {
+    return wrap(path, InfoStream.NO_OUTPUT);
+  }
+  
+  Path wrap(Path path, InfoStream stream) {
+    FileSystem fs = new VerboseFS(path.getFileSystem(), stream).getFileSystem(URI.create("file:///"));
+    return new FilterPath(path, fs);
+  }
+  
+  /** InfoStream that looks for a substring and indicates if it saw it */
+  static class InfoStreamListener extends InfoStream {
+    /** True if we saw the message */
+    final AtomicBoolean seenMessage = new AtomicBoolean(false);
+    /** Expected message */ 
+    final String messageStartsWith;
+    
+    InfoStreamListener(String messageStartsWith) {
+      this.messageStartsWith = messageStartsWith;
+    }
+    
+    @Override
+    public void close() throws IOException {}
+
+    @Override
+    public void message(String component, String message) {
+      if ("FS".equals(component) && message.startsWith(messageStartsWith)) {
+        seenMessage.set(true);
+      }
+    }
+
+    @Override
+    public boolean isEnabled(String component) {
+      return true;
+    }
+    
+    boolean sawMessage() {
+      return seenMessage.get();
+    }
+  }
+  
+  /** Test createDirectory */
+  public void testCreateDirectory() throws IOException {
+    InfoStreamListener stream = new InfoStreamListener("createDirectory");
+    Path dir = wrap(createTempDir(), stream);
+    Files.createDirectory(dir.resolve("subdir"));
+    assertTrue(stream.sawMessage());
+
+    try {
+      Files.createDirectory(dir.resolve("subdir"));
+      fail("didn't get expected exception");
+    } catch (IOException expected) {}
+  }
+  
+  /** Test delete */
+  public void testDelete() throws IOException {
+    InfoStreamListener stream = new InfoStreamListener("delete");
+    Path dir = wrap(createTempDir(), stream);
+    Files.createFile(dir.resolve("foobar"));
+    Files.delete(dir.resolve("foobar"));
+    assertTrue(stream.sawMessage());
+
+    try {
+      Files.delete(dir.resolve("foobar"));
+      fail("didn't get expected exception");
+    } catch (IOException expected) {}
+  }
+  
+  /** Test deleteIfExists */
+  public void testDeleteIfExists() throws IOException {
+    InfoStreamListener stream = new InfoStreamListener("deleteIfExists");
+    Path dir = wrap(createTempDir(), stream);
+    Files.createFile(dir.resolve("foobar"));
+    Files.deleteIfExists(dir.resolve("foobar"));
+    assertTrue(stream.sawMessage());
+
+    // no exception
+    Files.deleteIfExists(dir.resolve("foobar"));
+  }
+  
+  /** Test copy */
+  public void testCopy() throws IOException {
+    InfoStreamListener stream = new InfoStreamListener("copy");
+    Path dir = wrap(createTempDir(), stream);
+    Files.createFile(dir.resolve("foobar"));
+    Files.copy(dir.resolve("foobar"), dir.resolve("baz"));
+    assertTrue(stream.sawMessage());
+
+    try {
+      Files.copy(dir.resolve("nonexistent"), dir.resolve("something"));
+      fail("didn't get expected exception");
+    } catch (IOException expected) {}
+  }
+  
+  /** Test move */
+  public void testMove() throws IOException {
+    InfoStreamListener stream = new InfoStreamListener("move");
+    Path dir = wrap(createTempDir(), stream);
+    Files.createFile(dir.resolve("foobar"));
+    Files.move(dir.resolve("foobar"), dir.resolve("baz"));
+    assertTrue(stream.sawMessage());
+
+    try {
+      Files.move(dir.resolve("nonexistent"), dir.resolve("something"));
+      fail("didn't get expected exception");
+    } catch (IOException expected) {}
+  }
+  
+  /** Test newOutputStream */
+  public void testNewOutputStream() throws IOException {
+    InfoStreamListener stream = new InfoStreamListener("newOutputStream");
+    Path dir = wrap(createTempDir(), stream);
+    OutputStream file = Files.newOutputStream(dir.resolve("output"));
+    assertTrue(stream.sawMessage());
+    file.close();
+    
+    try {
+      Files.newOutputStream(dir.resolve("output"), StandardOpenOption.CREATE_NEW);
+      fail("didn't get expected exception");
+    } catch (IOException expected) {}
+  }
+  
+  /** Test FileChannel.open */
+  public void testFileChannel() throws IOException {
+    InfoStreamListener stream = new InfoStreamListener("newFileChannel");
+    Path dir = wrap(createTempDir(), stream);
+    FileChannel channel = FileChannel.open(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
+    assertTrue(stream.sawMessage());
+    channel.close();
+    
+    try {
+      FileChannel.open(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
+      fail("didn't get expected exception");
+    } catch (IOException expected) {}
+  }
+  
+  /** Test AsynchronousFileChannel.open */
+  public void testAsyncFileChannel() throws IOException {
+    InfoStreamListener stream = new InfoStreamListener("newAsynchronousFileChannel");
+    Path dir = wrap(createTempDir(), stream);
+    AsynchronousFileChannel channel = AsynchronousFileChannel.open(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
+    assertTrue(stream.sawMessage());
+    channel.close();
+    
+    try {
+      AsynchronousFileChannel.open(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
+      fail("didn't get expected exception");
+    } catch (IOException expected) {}
+  }
+  
+  /** Test newByteChannel */
+  public void testByteChannel() throws IOException {
+    InfoStreamListener stream = new InfoStreamListener("newByteChannel");
+    Path dir = wrap(createTempDir(), stream);
+    SeekableByteChannel channel = Files.newByteChannel(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
+    assertTrue(stream.sawMessage());
+    channel.close();
+    
+    try {
+      Files.newByteChannel(dir.resolve("foobar"), StandardOpenOption.CREATE_NEW, StandardOpenOption.READ, StandardOpenOption.WRITE);
+      fail("didn't get expected exception");
+    } catch (IOException expected) {}
+  }
+  
+  /** Test that verbose does not corrumpt file not found exceptions */
+  public void testVerboseFSNoSuchFileException() throws IOException {
+    Path dir = wrap(createTempDir());
+    try {
+      AsynchronousFileChannel.open(dir.resolve("doesNotExist.rip"));
+      fail("did not hit exception");
+    } catch (NoSuchFileException nsfe) {
+      // expected
+    }
+    try {
+      FileChannel.open(dir.resolve("doesNotExist.rip"));
+      fail("did not hit exception");
+    } catch (NoSuchFileException nsfe) {
+      // expected
+    }
+    try {
+      Files.newByteChannel(dir.resolve("stillopen"));
+      fail("did not hit exception");
+    } catch (NoSuchFileException nsfe) {
+      // expected
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestWindowsFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestWindowsFS.java?rev=1675097&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestWindowsFS.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/test/org/apache/lucene/mockfile/TestWindowsFS.java Tue Apr 21 11:33:52 2015
@@ -0,0 +1,98 @@
+package org.apache.lucene.mockfile;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.nio.file.FileSystem;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardCopyOption;
+
+import org.apache.lucene.util.Constants;
+
+/** Basic tests for WindowsFS */
+public class TestWindowsFS extends MockFileSystemTestCase {
+  
+  // currently we don't emulate windows well enough to work on windows!
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    assumeFalse("windows is not supported", Constants.WINDOWS);
+  }
+
+  @Override
+  protected Path wrap(Path path) {
+    FileSystem fs = new WindowsFS(path.getFileSystem()).getFileSystem(URI.create("file:///"));
+    return new FilterPath(path, fs);
+  }
+  
+  /** Test Files.delete fails if a file has an open inputstream against it */
+  public void testDeleteOpenFile() throws IOException {
+    Path dir = wrap(createTempDir());
+    
+    OutputStream file = Files.newOutputStream(dir.resolve("stillopen"));
+    file.write(5);
+    file.close();
+    InputStream is = Files.newInputStream(dir.resolve("stillopen"));
+    try {
+      Files.delete(dir.resolve("stillopen"));
+      fail("should have gotten exception");
+    } catch (IOException e) {
+      assertTrue(e.getMessage().contains("access denied"));
+    }
+    is.close();
+  }
+  
+  /** Test Files.deleteIfExists fails if a file has an open inputstream against it */
+  public void testDeleteIfExistsOpenFile() throws IOException {
+    Path dir = wrap(createTempDir());
+    
+    OutputStream file = Files.newOutputStream(dir.resolve("stillopen"));
+    file.write(5);
+    file.close();
+    InputStream is = Files.newInputStream(dir.resolve("stillopen"));
+    try {
+      Files.deleteIfExists(dir.resolve("stillopen"));
+      fail("should have gotten exception");
+    } catch (IOException e) {
+      assertTrue(e.getMessage().contains("access denied"));
+    }
+    is.close();
+  }
+  
+  /** Test Files.rename fails if a file has an open inputstream against it */
+  // TODO: what does windows do here?
+  public void testRenameOpenFile() throws IOException {
+    Path dir = wrap(createTempDir());
+    
+    OutputStream file = Files.newOutputStream(dir.resolve("stillopen"));
+    file.write(5);
+    file.close();
+    InputStream is = Files.newInputStream(dir.resolve("stillopen"));
+    try {
+      Files.move(dir.resolve("stillopen"), dir.resolve("target"), StandardCopyOption.ATOMIC_MOVE);
+      fail("should have gotten exception");
+    } catch (IOException e) {
+      assertTrue(e.getMessage().contains("access denied"));
+    }
+    is.close();
+  }
+}