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/11/25 16:21:19 UTC

svn commit: r1641632 [2/2] - in /lucene/dev/trunk/lucene: core/src/test/org/apache/lucene/mockfile/ test-framework/ test-framework/src/java/org/apache/lucene/mockfile/ test-framework/src/java/org/apache/lucene/util/

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/VerboseFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/VerboseFS.java?rev=1641632&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/VerboseFS.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/VerboseFS.java Tue Nov 25 15:21:18 2014
@@ -0,0 +1,315 @@
+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.nio.channels.AsynchronousFileChannel;
+import java.nio.channels.FileChannel;
+import java.nio.channels.SeekableByteChannel;
+import java.nio.file.CopyOption;
+import java.nio.file.DirectoryStream;
+import java.nio.file.DirectoryStream.Filter;
+import java.nio.file.FileSystem;
+import java.nio.file.LinkOption;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.SecureDirectoryStream;
+import java.nio.file.StandardOpenOption;
+import java.nio.file.attribute.FileAttribute;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.InfoStream;
+
+/** 
+ * FileSystem that records all major destructive filesystem activities.
+ */
+public class VerboseFS extends FilterFileSystemProvider {
+  final InfoStream infoStream;
+  final Path root;
+  
+  /**
+   * Create a new instance, recording major filesystem write activities
+   * (create, delete, etc) to the specified {@code InfoStream}.
+   * @param delegate delegate filesystem to wrap.
+   * @param infoStream infoStream to send messages to. The component for 
+   * messages is named "FS".
+   */
+  public VerboseFS(FileSystem delegate, InfoStream infoStream) {
+    super("verbose://", delegate);
+    this.infoStream = infoStream;
+    this.root = this.getFileSystem(null).getPath(".").toAbsolutePath().normalize();
+  }
+  
+  /** Records message, and rethrows exception if not null */
+  private void sop(String text, Throwable exception) throws IOException {
+    if (exception == null) {
+      if (infoStream.isEnabled("FS")) {
+        infoStream.message("FS", text);
+      }
+    } else {
+      if (infoStream.isEnabled("FS")) {
+        infoStream.message("FS", text + " (FAILED: " + exception + ")");
+      }
+      IOUtils.reThrow(exception);
+    }
+  }
+  
+  private String path(Path path) {
+    path = root.relativize(path.toAbsolutePath().normalize());
+    return path.toString();
+  }
+
+  @Override
+  public void createDirectory(Path dir, FileAttribute<?>... attrs) throws IOException {
+    Throwable exception = null;
+    try {
+      super.createDirectory(dir, attrs);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      sop("createDirectory: " + path(dir), exception);
+    }
+  }
+
+  @Override
+  public void delete(Path path) throws IOException {
+    Throwable exception = null;
+    try {
+      super.delete(path);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      sop("delete: " + path(path), exception);
+    }
+  }
+
+  @Override
+  public void copy(Path source, Path target, CopyOption... options) throws IOException {
+    Throwable exception = null;
+    try {
+      super.copy(source, target, options);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      sop("copy" + Arrays.toString(options) + ": " + path(source) + " -> " + path(target), exception);
+    }
+  }
+
+  @Override
+  public void move(Path source, Path target, CopyOption... options) throws IOException {
+    Throwable exception = null;
+    try {
+      super.move(source, target, options);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      sop("move" + Arrays.toString(options) + ": " + path(source) + " -> " + path(target), exception);
+    }
+  }
+
+  @Override
+  public void setAttribute(Path path, String attribute, Object value, LinkOption... options) throws IOException {
+    Throwable exception = null;
+    try {
+      super.setAttribute(path, attribute, value, options);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      sop("setAttribute[" + attribute + "=" + value + "]: " + path(path), exception);
+    }
+  }
+
+  @Override
+  public OutputStream newOutputStream(Path path, OpenOption... options) throws IOException {
+    Throwable exception = null;
+    try {
+      return super.newOutputStream(path, options);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      sop("newOutputStream" + Arrays.toString(options) + ": " + path(path), exception);
+    }
+    throw new AssertionError();
+  }
+  
+  private boolean containsDestructive(Set<? extends OpenOption> options) {
+    return (options.contains(StandardOpenOption.APPEND) ||
+            options.contains(StandardOpenOption.WRITE)  || 
+            options.contains(StandardOpenOption.DELETE_ON_CLOSE));
+  }
+
+  @Override
+  public FileChannel newFileChannel(Path path, Set<? extends OpenOption> options, FileAttribute<?>... attrs) throws IOException {
+    Throwable exception = null;
+    try {
+      return super.newFileChannel(path, options, attrs);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      if (containsDestructive(options)) {
+        sop("newFileChannel" + options + ": " + path(path), exception);
+      }
+    }
+    throw new AssertionError();
+  }
+
+  @Override
+  public AsynchronousFileChannel newAsynchronousFileChannel(Path path, Set<? extends OpenOption> options, ExecutorService executor, FileAttribute<?>... attrs) throws IOException {
+    Throwable exception = null;
+    try {
+      return super.newAsynchronousFileChannel(path, options, executor, attrs);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      if (containsDestructive(options)) {
+        sop("newAsynchronousFileChannel" + options + ": " + path(path), exception);
+      }
+    }
+    throw new AssertionError();
+  }
+
+  @Override
+  public SeekableByteChannel newByteChannel(Path path, Set<? extends OpenOption> options, FileAttribute<?>... attrs) throws IOException {
+    Throwable exception = null;
+    try {
+      return super.newByteChannel(path, options, attrs);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      if (containsDestructive(options)) {
+        sop("newByteChannel" + options + ": " + path(path), exception);
+      }
+    }
+    throw new AssertionError();
+  }
+
+  @Override
+  public void createSymbolicLink(Path link, Path target, FileAttribute<?>... attrs) throws IOException {
+    Throwable exception = null;
+    try {
+      super.createSymbolicLink(link, target, attrs);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      sop("createSymbolicLink: " + path(link) + " -> " + path(target), exception);
+    }
+  }
+
+  @Override
+  public void createLink(Path link, Path existing) throws IOException {
+    Throwable exception = null;
+    try {
+      super.createLink(link, existing);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      sop("createLink: " + path(link) + " -> " + path(existing), exception);
+    }
+  }
+
+  @Override
+  public boolean deleteIfExists(Path path) throws IOException {
+    Throwable exception = null;
+    try {
+      return super.deleteIfExists(path);
+    } catch (Throwable t) {
+      exception = t;
+    } finally {
+      sop("deleteIfExists: " + path(path), exception);
+    }
+    throw new AssertionError();
+  }
+
+  @Override
+  public DirectoryStream<Path> newDirectoryStream(Path dir, Filter<? super Path> filter) throws IOException {
+    DirectoryStream<Path> stream = super.newDirectoryStream(dir, filter);
+    if (stream instanceof SecureDirectoryStream) {
+      stream = new VerboseSecureDirectoryStream((SecureDirectoryStream<Path>)stream, dir);
+    }
+    return stream;
+  }
+  
+  class VerboseSecureDirectoryStream extends FilterSecureDirectoryStream<Path> {
+    final Path dir;
+    
+    VerboseSecureDirectoryStream(SecureDirectoryStream<Path> delegate, Path dir) {
+      super(delegate);
+      this.dir = dir;
+    }
+
+    @Override
+    public SecureDirectoryStream<Path> newDirectoryStream(Path path, LinkOption... options) throws IOException {
+      return new VerboseSecureDirectoryStream(super.newDirectoryStream(path, options), path);
+    }
+
+    @Override
+    public SeekableByteChannel newByteChannel(Path path, Set<? extends OpenOption> options, FileAttribute<?>... attrs) throws IOException {
+      Throwable exception = null;
+      try {
+        return super.newByteChannel(path, options, attrs);
+      } catch (Throwable t) {
+        exception = t;
+      } finally {
+        if (containsDestructive(options)) {
+          sop("newByteChannel[SECURE]" + options + ": " + path(path), exception);
+        }
+      }
+      throw new AssertionError();
+    }
+
+    @Override
+    public void deleteFile(Path path) throws IOException {
+      Throwable exception = null;
+      try {
+        super.deleteFile(path);
+      } catch (Throwable t) {
+        exception = t;
+      } finally {
+        sop("deleteFile[SECURE]: " + path(path), exception);
+      }
+    }
+
+    @Override
+    public void deleteDirectory(Path path) throws IOException {
+      Throwable exception = null;
+      try {
+        super.deleteDirectory(path);
+      } catch (Throwable t) {
+        exception = t;
+      } finally {
+        sop("deleteDirectory[SECURE]: " + path(path), exception);
+      }
+    }
+
+    @Override
+    public void move(Path srcpath, SecureDirectoryStream<Path> targetdir, Path targetpath) throws IOException {
+      Throwable exception = null;
+      try {
+        super.move(srcpath, targetdir, targetpath);
+      } catch (Throwable t) {
+        exception = t;
+      } finally {
+        sop("move[SECURE]: " + path(srcpath) + " -> " + path(targetpath), exception);
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/WindowsFS.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/WindowsFS.java?rev=1641632&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/WindowsFS.java (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/WindowsFS.java Tue Nov 25 15:21:18 2014
@@ -0,0 +1,129 @@
+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.CopyOption;
+import java.nio.file.FileSystem;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.attribute.BasicFileAttributeView;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.HashMap;
+import java.util.Map;
+
+/** 
+ * FileSystem that (imperfectly) acts like windows. 
+ * <p>
+ * Currently this filesystem only prevents deletion of open files.
+ */
+public class WindowsFS extends HandleTrackingFS {
+  private final Map<Object,Integer> openFiles = new HashMap<>();
+  
+  // TODO: try to make this as realistic as possible... it depends e.g. how you
+  // open files, if you map them, etc, if you can delete them (Uwe knows the rules)
+  
+  // TODO: add case-insensitivity
+  
+  /**
+   * Create a new instance, wrapping the delegate filesystem to
+   * act like Windows.
+   * @param delegate delegate filesystem to wrap.
+   */
+  public WindowsFS(FileSystem delegate) {
+    super("windows://", delegate);
+  }
+  
+  /** 
+   * Returns file "key" (e.g. inode) for the specified path 
+   */
+  private Object getKey(Path existing) throws IOException {
+    BasicFileAttributeView view = Files.getFileAttributeView(existing, BasicFileAttributeView.class);
+    BasicFileAttributes attributes = view.readAttributes();
+    return attributes.fileKey();
+  }
+
+  @Override
+  protected void onOpen(Path path, Object stream) throws IOException {
+    Object key = getKey(path);
+    synchronized (openFiles) {
+      Integer v = openFiles.get(key);
+      if (v != null) {
+        v = Integer.valueOf(v.intValue()+1);
+        openFiles.put(key, v);
+      } else {
+        openFiles.put(key, Integer.valueOf(1));
+      }
+    }
+  }
+
+  @Override
+  protected void onClose(Path path, Object stream) throws IOException {
+    Object key = getKey(path);
+    synchronized (openFiles) {
+      Integer v = openFiles.get(key);
+      if (v != null) {
+        if (v.intValue() == 1) {
+          openFiles.remove(key);
+        } else {
+          v = Integer.valueOf(v.intValue()-1);
+          openFiles.put(key, v);
+        }
+      }
+    }
+  }
+  
+  /** 
+   * Checks that its ok to delete {@code Path}. If the file
+   * is still open, it throws IOException("access denied").
+   */
+  private void checkDeleteAccess(Path path) throws IOException {
+    Object key = null;
+    try {
+      key = getKey(path);
+    } catch (Throwable ignore) {
+      // we don't care if the file doesn't exist
+    } 
+
+    if (key != null) {
+      synchronized(openFiles) {
+        if (openFiles.containsKey(key)) {
+          throw new IOException("access denied: " + path);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void delete(Path path) throws IOException {
+    checkDeleteAccess(path);
+    super.delete(path);
+  }
+
+  @Override
+  public void move(Path source, Path target, CopyOption... options) throws IOException {
+    checkDeleteAccess(source);
+    super.move(source, target, options);
+  }
+
+  @Override
+  public boolean deleteIfExists(Path path) throws IOException {
+    checkDeleteAccess(path);
+    return super.deleteIfExists(path);
+  }
+}

Added: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/package.html?rev=1641632&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/package.html (added)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/mockfile/package.html Tue Nov 25 15:21:18 2014
@@ -0,0 +1,34 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<html>
+<head>
+   <meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
+</head>
+<body>
+Support for testing/debugging with virtual filesystems 
+<p>
+The primary classes are:
+<ul>
+  <li>{@link org.apache.lucene.mockfile.LeakFS}: Fails tests if they leave open filehandles.
+  <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.
+</ul>
+</p>
+</body>
+</html>
\ No newline at end of file

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=1641632&r1=1641631&r2=1641632&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 Nov 25 15:21:18 2014
@@ -1,14 +1,21 @@
 package org.apache.lucene.util;
 
 import java.io.IOException;
+import java.net.URI;
+import java.nio.file.FileSystem;
+import java.nio.file.FileSystems;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
+import java.util.Random;
 
+import org.apache.lucene.mockfile.DisableFsyncFS;
+import org.apache.lucene.mockfile.LeakFS;
+import org.apache.lucene.mockfile.VerboseFS;
+import org.apache.lucene.mockfile.WindowsFS;
 import org.apache.lucene.util.LuceneTestCase.SuppressTempFileChecks;
 
 import com.carrotsearch.randomizedtesting.RandomizedContext;
@@ -52,6 +59,11 @@ final class TestRuleTemporaryFilesCleanu
    * Per-test class temporary folder.
    */
   private Path tempDirBase;
+  
+  /**
+   * Per-test filesystem
+   */
+  private FileSystem fileSystem;
 
   /**
    * Suite failure marker.
@@ -90,11 +102,34 @@ final class TestRuleTemporaryFilesCleanu
     super.before();
 
     assert tempDirBase == null;
+    fileSystem = initializeFileSystem();
     javaTempDir = initializeJavaTempDir();
   }
+  
+  private FileSystem initializeFileSystem() {
+    FileSystem fs = FileSystems.getDefault();
+    if (LuceneTestCase.VERBOSE) {
+      fs = new VerboseFS(fs, new TestRuleSetupAndRestoreClassEnv.ThreadNameFixingPrintStreamInfoStream(System.out)).getFileSystem(null);
+    }
+    Random random = RandomizedContext.current().getRandom();
+    // sometimes just use a bare filesystem
+    if (random.nextInt(10) > 0) {
+      fs = new DisableFsyncFS(fs).getFileSystem(null);
+      fs = new LeakFS(fs).getFileSystem(null);
+      // windows is currently slow
+      if (random.nextInt(10) == 0) {
+        fs = new WindowsFS(fs).getFileSystem(null);
+      }
+    }
+    if (LuceneTestCase.VERBOSE) {
+      System.out.println("filesystem: " + fs.provider());
+    }
+    return fs.provider().getFileSystem(URI.create("file:///"));
+  }
 
   private Path initializeJavaTempDir() throws IOException {
-    Path javaTempDir = Paths.get(System.getProperty("tempDir", System.getProperty("java.io.tmpdir")));
+    Path javaTempDir = fileSystem.getPath(System.getProperty("tempDir", System.getProperty("java.io.tmpdir")));
+    
     Files.createDirectories(javaTempDir);
 
     assert Files.isDirectory(javaTempDir) &&
@@ -134,6 +169,9 @@ final class TestRuleTemporaryFilesCleanu
         }
         throw e;
       }
+      if (fileSystem != FileSystems.getDefault()) {
+        fileSystem.close();
+      }
     } else {
       if (tempDirBasePath != null) {
         System.err.println("NOTE: leaving temporary files on disk at: " + tempDirBasePath);
@@ -171,7 +209,7 @@ final class TestRuleTemporaryFilesCleanu
     }
     return tempDirBase;
   }
-
+  
   /**
    * @see LuceneTestCase#createTempDir()
    */