You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2015/05/29 13:31:40 UTC

svn commit: r1682422 - in /lucene/dev/branches/lucene6508/lucene: codecs/src/java/org/apache/lucene/codecs/simpletext/ core/src/java/org/apache/lucene/codecs/lucene50/ core/src/java/org/apache/lucene/index/ core/src/java/org/apache/lucene/store/ facet/...

Author: uschindler
Date: Fri May 29 11:31:39 2015
New Revision: 1682422

URL: http://svn.apache.org/r1682422
Log:
LUCENE-6508: Initial commit of Robert's and Uwe's code

Added:
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockValidatingDirectoryWrapper.java   (with props)
Removed:
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockObtainFailedException.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java
Modified:
    lucene/dev/branches/lucene6508/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/Directory.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FSLockFactory.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/Lock.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockFactory.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/NoLockFactory.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/VerifyingLockFactory.java
    lucene/dev/branches/lucene6508/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java

Modified: lucene/dev/branches/lucene6508/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java (original)
+++ lucene/dev/branches/lucene6508/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java Fri May 29 11:31:39 2015
@@ -151,7 +151,7 @@ public class SimpleTextCompoundFormat ex
       public void renameFile(String source, String dest) { throw new UnsupportedOperationException(); }
       
       @Override
-      public Lock makeLock(String name) { throw new UnsupportedOperationException(); }
+      public Lock obtainLock(String name) { throw new UnsupportedOperationException(); }
     };
   }
 

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java Fri May 29 11:31:39 2015
@@ -179,7 +179,7 @@ final class Lucene50CompoundReader exten
   }
   
   @Override
-  public Lock makeLock(String name) {
+  public Lock obtainLock(String name) {
     throw new UnsupportedOperationException();
   }
 

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Fri May 29 11:31:39 2015
@@ -47,7 +47,6 @@ import org.apache.lucene.store.FSDirecto
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.Lock;
-import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -356,7 +355,7 @@ public class CheckIndex implements Close
 
   /** Create a new CheckIndex on the directory. */
   public CheckIndex(Directory dir) throws IOException {
-    this(dir, dir.makeLock(IndexWriter.WRITE_LOCK_NAME));
+    this(dir, dir.obtainLock(IndexWriter.WRITE_LOCK_NAME));
   }
   
   /** 
@@ -370,9 +369,6 @@ public class CheckIndex implements Close
     this.dir = dir;
     this.writeLock = writeLock;
     this.infoStream = null;
-    if (!writeLock.obtain(IndexWriterConfig.WRITE_LOCK_TIMEOUT)) { // obtain write lock
-      throw new LockObtainFailedException("Index locked for write: " + writeLock);
-    }
   }
   
   private void ensureOpen() {

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Fri May 29 11:31:39 2015
@@ -56,10 +56,10 @@ import org.apache.lucene.store.FlushInfo
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.Lock;
-import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.store.MergeInfo;
 import org.apache.lucene.store.RateLimitedIndexOutput;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
+import org.apache.lucene.store.LockValidatingDirectoryWrapper;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -118,9 +118,7 @@ import org.apache.lucene.util.Version;
 
   <p>Opening an <code>IndexWriter</code> creates a lock file for the directory in use. Trying to open
   another <code>IndexWriter</code> on the same directory will lead to a
-  {@link LockObtainFailedException}. The {@link LockObtainFailedException}
-  is also thrown if an IndexReader on the same directory is used to delete documents
-  from the index.</p>
+  {@link IOException}.</p>
   
   <a name="deletionPolicy"></a>
   <p>Expert: <code>IndexWriter</code> allows an optional
@@ -755,11 +753,13 @@ public class IndexWriter implements Clos
     conf.setIndexWriter(this); // prevent reuse by other instances
     config = conf;
 
-    directory = d;
+    writeLock = d.obtainLock(WRITE_LOCK_NAME);
+
+    directory = new LockValidatingDirectoryWrapper(d, writeLock);
 
     // Directory we use for merging, so we can abort running merges, and so
     // merge schedulers can optionally rate-limit per-merge IO:
-    mergeDirectory = addMergeRateLimiters(d);
+    mergeDirectory = addMergeRateLimiters(directory);
 
     analyzer = config.getAnalyzer();
     infoStream = config.getInfoStream();
@@ -770,11 +770,6 @@ public class IndexWriter implements Clos
     bufferedUpdatesStream = new BufferedUpdatesStream(infoStream);
     poolReaders = config.getReaderPooling();
 
-    writeLock = directory.makeLock(WRITE_LOCK_NAME);
-
-    if (!writeLock.obtain(config.getWriteLockTimeout())) // obtain write lock
-      throw new LockObtainFailedException("Index locked for write: " + writeLock);
-
     boolean success = false;
     try {
       OpenMode mode = config.getOpenMode();
@@ -2288,13 +2283,13 @@ public class IndexWriter implements Clos
     for(int i=0;i<dirs.length;i++) {
       boolean success = false;
       try {
-        Lock lock = dirs[i].makeLock(WRITE_LOCK_NAME);
+        Lock lock = dirs[i].obtainLock(WRITE_LOCK_NAME);
         locks.add(lock);
-        lock.obtain(config.getWriteLockTimeout());
         success = true;
       } finally {
         if (success == false) {
           // Release all previously acquired locks:
+          // nocommit: addSuppressed
           IOUtils.closeWhileHandlingException(locks);
         }
       }
@@ -2334,8 +2329,6 @@ public class IndexWriter implements Clos
    *
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
-   * @throws LockObtainFailedException if we were unable to
-   *   acquire the write lock in at least one directory
    * @throws IllegalArgumentException if addIndexes would cause
    *   the index to exceed {@link #MAX_DOCS}
    */
@@ -4360,16 +4353,6 @@ public class IndexWriter implements Clos
     testPoint("finishStartCommit");
   }
 
-  /**
-   * Returns <code>true</code> iff the index in the named directory is
-   * currently locked.
-   * @param directory the directory to check for a lock
-   * @throws IOException if there is a low-level IO error
-   */
-  public static boolean isLocked(Directory directory) throws IOException {
-    return directory.makeLock(WRITE_LOCK_NAME).isLocked();
-  }
-
   /** If {@link DirectoryReader#open(IndexWriter,boolean)} has
    *  been called (ie, this writer is in near real-time
    *  mode), then after a merge completes, this class can be

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java Fri May 29 11:31:39 2015
@@ -86,13 +86,6 @@ public final class IndexWriterConfig ext
    */
   public final static double DEFAULT_RAM_BUFFER_SIZE_MB = 16.0;
 
-  /**
-   * Default value for the write lock timeout (1,000 ms).
-   *
-   * @see #setDefaultWriteLockTimeout(long)
-   */
-  public static long WRITE_LOCK_TIMEOUT = 1000;
-
   /** Default setting for {@link #setReaderPooling}. */
   public final static boolean DEFAULT_READER_POOLING = false;
 
@@ -113,24 +106,6 @@ public final class IndexWriterConfig ext
   /** Default value for whether calls to {@link IndexWriter#close()} include a commit. */
   public final static boolean DEFAULT_COMMIT_ON_CLOSE = true;
   
-  /**
-   * Sets the default (for any instance) maximum time to wait for a write lock
-   * (in milliseconds).
-   */
-  public static void setDefaultWriteLockTimeout(long writeLockTimeout) {
-    WRITE_LOCK_TIMEOUT = writeLockTimeout;
-  }
-
-  /**
-   * Returns the default write lock timeout for newly instantiated
-   * IndexWriterConfigs.
-   *
-   * @see #setDefaultWriteLockTimeout(long)
-   */
-  public static long getDefaultWriteLockTimeout() {
-    return WRITE_LOCK_TIMEOUT;
-  }
-
   // indicates whether this config instance is already attached to a writer.
   // not final so that it can be cloned properly.
   private SetOnce<IndexWriter> writer = new SetOnce<>();
@@ -263,22 +238,6 @@ public final class IndexWriterConfig ext
   }
 
   /**
-   * Sets the maximum time to wait for a write lock (in milliseconds) for this
-   * instance. You can change the default value for all instances by calling
-   * {@link #setDefaultWriteLockTimeout(long)}.
-   *
-   * <p>Only takes effect when IndexWriter is first created. */
-  public IndexWriterConfig setWriteLockTimeout(long writeLockTimeout) {
-    this.writeLockTimeout = writeLockTimeout;
-    return this;
-  }
-
-  @Override
-  public long getWriteLockTimeout() {
-    return writeLockTimeout;
-  }
-
-  /**
    * Set the {@link Codec}.
    * 
    * <p>

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java Fri May 29 11:31:39 2015
@@ -60,9 +60,6 @@ public class LiveIndexWriterConfig {
   /** {@link MergeScheduler} to use for running merges. */
   protected volatile MergeScheduler mergeScheduler;
 
-  /** Timeout when trying to obtain the write lock on init. */
-  protected volatile long writeLockTimeout;
-
   /** {@link IndexingChain} that determines how documents are
    *  indexed. */
   protected volatile IndexingChain indexingChain;
@@ -110,7 +107,6 @@ public class LiveIndexWriterConfig {
     openMode = OpenMode.CREATE_OR_APPEND;
     similarity = IndexSearcher.getDefaultSimilarity();
     mergeScheduler = new ConcurrentMergeScheduler();
-    writeLockTimeout = IndexWriterConfig.WRITE_LOCK_TIMEOUT;
     indexingChain = DocumentsWriterPerThread.defaultIndexingChain;
     codec = Codec.getDefault();
     if (codec == null) {
@@ -350,15 +346,6 @@ public class LiveIndexWriterConfig {
   public MergeScheduler getMergeScheduler() {
     return mergeScheduler;
   }
-
-  /**
-   * Returns allowed timeout when acquiring the write lock.
-   *
-   * @see IndexWriterConfig#setWriteLockTimeout(long)
-   */
-  public long getWriteLockTimeout() {
-    return writeLockTimeout;
-  }
   
   /** Returns the current {@link Codec}. */
   public Codec getCodec() {
@@ -481,8 +468,6 @@ public class LiveIndexWriterConfig {
     sb.append("openMode=").append(getOpenMode()).append("\n");
     sb.append("similarity=").append(getSimilarity().getClass().getName()).append("\n");
     sb.append("mergeScheduler=").append(getMergeScheduler()).append("\n");
-    sb.append("default WRITE_LOCK_TIMEOUT=").append(IndexWriterConfig.WRITE_LOCK_TIMEOUT).append("\n");
-    sb.append("writeLockTimeout=").append(getWriteLockTimeout()).append("\n");
     sb.append("codec=").append(getCodec()).append("\n");
     sb.append("infoStream=").append(getInfoStream().getClass().getName()).append("\n");
     sb.append("mergePolicy=").append(getMergePolicy()).append("\n");

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/BaseDirectory.java Fri May 29 11:31:39 2015
@@ -1,5 +1,7 @@
 package org.apache.lucene.store;
 
+import java.io.IOException;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -40,8 +42,8 @@ public abstract class BaseDirectory exte
   }
 
   @Override
-  public final Lock makeLock(String name) {
-    return lockFactory.makeLock(this, name);
+  public final Lock obtainLock(String name) throws IOException {
+    return lockFactory.obtainLock(this, name);
   }
 
   @Override

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/Directory.java Fri May 29 11:31:39 2015
@@ -50,8 +50,7 @@ public abstract class Directory implemen
   public abstract String[] listAll() throws IOException;
 
   /** Removes an existing file in the directory. */
-  public abstract void deleteFile(String name)
-       throws IOException;
+  public abstract void deleteFile(String name) throws IOException;
 
   /**
    * Returns the length of a file in the directory. This method follows the
@@ -110,10 +109,12 @@ public abstract class Directory implemen
     return new BufferedChecksumIndexInput(openInput(name, context));
   }
   
-  /** Construct a {@link Lock}.
+  /** 
+   * Returns an obtained {@link Lock}.
    * @param name the name of the lock file
+   * @throws IOException if the lock could not be obtained
    */
-  public abstract Lock makeLock(String name);
+  public abstract Lock obtainLock(String name) throws IOException;
 
   /** Closes the store. */
   @Override

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FSLockFactory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FSLockFactory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FSLockFactory.java Fri May 29 11:31:39 2015
@@ -1,5 +1,7 @@
 package org.apache.lucene.store;
 
+import java.io.IOException;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -32,14 +34,17 @@ public abstract class FSLockFactory exte
   }
 
   @Override
-  public final Lock makeLock(Directory dir, String lockName) {
+  public final Lock obtainLock(Directory dir, String lockName) throws IOException {
     if (!(dir instanceof FSDirectory)) {
       throw new UnsupportedOperationException(getClass().getSimpleName() + " can only be used with FSDirectory subclasses, got: " + dir);
     }
-    return makeFSLock((FSDirectory) dir, lockName);
+    return obtainFSLock((FSDirectory) dir, lockName);
   }
   
-  /** Implement this method to create a lock for a FSDirectory instance. */
-  protected abstract Lock makeFSLock(FSDirectory dir, String lockName);
+  /** 
+   * Implement this method to obtain a lock for a FSDirectory instance. 
+   * @throws IOException if the lock could not be obtained.
+   */
+  protected abstract Lock obtainFSLock(FSDirectory dir, String lockName) throws IOException;
 
 }

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FileSwitchDirectory.java Fri May 29 11:31:39 2015
@@ -71,8 +71,8 @@ public class FileSwitchDirectory extends
   }
   
   @Override
-  public Lock makeLock(String name) {
-    return getDirectory(name).makeLock(name);
+  public Lock obtainLock(String name) throws IOException {
+    return getDirectory(name).obtainLock(name);
   }
 
   @Override

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/FilterDirectory.java Fri May 29 11:31:39 2015
@@ -90,8 +90,8 @@ public class FilterDirectory extends Dir
   }
 
   @Override
-  public Lock makeLock(String name) {
-    return in.makeLock(name);
+  public Lock obtainLock(String name) throws IOException {
+    return in.obtainLock(name);
   }
 
   @Override

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/Lock.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/Lock.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/Lock.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/Lock.java Fri May 29 11:31:39 2015
@@ -20,126 +20,37 @@ package org.apache.lucene.store;
 import java.io.Closeable;
 import java.io.IOException;
 
-import org.apache.lucene.util.ThreadInterruptedException;
-
 /** An interprocess mutex lock.
  * <p>Typical use might look like:<pre class="prettyprint">
- * new Lock.With(directory.makeLock("my.lock")) {
- *     public Object doBody() {
- *       <i>... code to execute while locked ...</i>
- *     }
- *   }.run();
+ *   try (final Lock lock = directory.obtainLock("my.lock")) {
+ *     // ... code to execute while locked ...
+ *   }
  * </pre>
  *
- * @see Directory#makeLock(String)
+ * @see Directory#obtainLock(String)
  *
  * @lucene.internal
  */
 public abstract class Lock implements Closeable {
 
-  /** How long {@link #obtain(long)} waits, in milliseconds,
-   *  in between attempts to acquire the lock. */
-  public static long LOCK_POLL_INTERVAL = 1000;
-
-  /** Pass this value to {@link #obtain(long)} to try
-   *  forever to obtain the lock. */
-  public static final long LOCK_OBTAIN_WAIT_FOREVER = -1;
-
-  /** Attempts to obtain exclusive access and immediately return
-   *  upon success or failure.  Use {@link #close} to
-   *  release the lock.
-   * @return true iff exclusive access is obtained
-   */
-  public abstract boolean obtain() throws IOException;
-
-  /**
-   * If a lock obtain called, this failureReason may be set
-   * with the "root cause" Exception as to why the lock was
-   * not obtained.
+  /** 
+   * Releases exclusive access.
+   * <p>
+   * Note that exceptions thrown from close may require
+   * human intervention, as it may mean the lock was no
+   * longer valid, or that fs permissions prevent removal
+   * of the lock file, or other reasons.
+   * <p>
+   * {@inheritDoc} 
    */
-  protected Throwable failureReason;
-
-  /** Attempts to obtain an exclusive lock within amount of
-   *  time given. Polls once per {@link #LOCK_POLL_INTERVAL}
-   *  (currently 1000) milliseconds until lockWaitTimeout is
-   *  passed.
-   * @param lockWaitTimeout length of time to wait in
-   *        milliseconds or {@link
-   *        #LOCK_OBTAIN_WAIT_FOREVER} to retry forever
-   * @return true if lock was obtained
-   * @throws LockObtainFailedException if lock wait times out
-   * @throws IllegalArgumentException if lockWaitTimeout is
-   *         out of bounds
-   * @throws IOException if obtain() throws IOException
-   */
-  public final boolean obtain(long lockWaitTimeout) throws IOException {
-    failureReason = null;
-    boolean locked = obtain();
-    if (lockWaitTimeout < 0 && lockWaitTimeout != LOCK_OBTAIN_WAIT_FOREVER)
-      throw new IllegalArgumentException("lockWaitTimeout should be LOCK_OBTAIN_WAIT_FOREVER or a non-negative number (got " + lockWaitTimeout + ")");
-
-    long maxSleepCount = lockWaitTimeout / LOCK_POLL_INTERVAL;
-    long sleepCount = 0;
-    while (!locked) {
-      if (lockWaitTimeout != LOCK_OBTAIN_WAIT_FOREVER && sleepCount++ >= maxSleepCount) {
-        String reason = "Lock obtain timed out: " + this.toString();
-        if (failureReason != null) {
-          reason += ": " + failureReason;
-        }
-        throw new LockObtainFailedException(reason, failureReason);
-      }
-      try {
-        Thread.sleep(LOCK_POLL_INTERVAL);
-      } catch (InterruptedException ie) {
-        throw new ThreadInterruptedException(ie);
-      }
-      locked = obtain();
-    }
-    return locked;
-  }
-
-  /** Releases exclusive access. */
   public abstract void close() throws IOException;
-
-  /** Returns true if the resource is currently locked.  Note that one must
-   * still call {@link #obtain()} before using the resource. */
-  public abstract boolean isLocked() throws IOException;
-
-
-  /** Utility class for executing code with exclusive access. */
-  public abstract static class With {
-    private Lock lock;
-    private long lockWaitTimeout;
-
-
-    /** Constructs an executor that will grab the named lock. */
-    public With(Lock lock, long lockWaitTimeout) {
-      this.lock = lock;
-      this.lockWaitTimeout = lockWaitTimeout;
-    }
-
-    /** Code to execute with exclusive access. */
-    protected abstract Object doBody() throws IOException;
-
-    /** Calls {@link #doBody} while <i>lock</i> is obtained.  Blocks if lock
-     * cannot be obtained immediately.  Retries to obtain lock once per second
-     * until it is obtained, or until it has tried ten times. Lock is released when
-     * {@link #doBody} exits.
-     * @throws LockObtainFailedException if lock could not
-     * be obtained
-     * @throws IOException if {@link Lock#obtain} throws IOException
-     */
-    public Object run() throws IOException {
-      boolean locked = false;
-      try {
-         locked = lock.obtain(lockWaitTimeout);
-         return doBody();
-      } finally {
-        if (locked) {
-          lock.close();
-        }
-      }
-    }
-  }
-
+  
+  /** 
+   * Best effort check that this lock is still valid. Locks
+   * could become invalidated externally for a number of reasons,
+   * for example if a user deletes the lock file manually or
+   * when a network filesystem is in use. 
+   * @throws IOException if the lock is no longer valid.
+   */
+  public abstract void ensureValid() throws IOException;
 }

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockFactory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockFactory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockFactory.java Fri May 29 11:31:39 2015
@@ -1,5 +1,7 @@
 package org.apache.lucene.store;
 
+import java.io.IOException;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -46,9 +48,10 @@ package org.apache.lucene.store;
 public abstract class LockFactory {
 
   /**
-   * Return a new Lock instance identified by lockName.
+   * Return a new obtained Lock instance identified by lockName.
    * @param lockName name of the lock to be created.
+   * @throws IOException if the lock could not be obtained
    */
-  public abstract Lock makeLock(Directory dir, String lockName);
+  public abstract Lock obtainLock(Directory dir, String lockName) throws IOException;
 
 }

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockStressTest.java Fri May 29 11:31:39 2015
@@ -38,10 +38,12 @@ import org.apache.lucene.util.SuppressFo
  */ 
 
 public class LockStressTest {
+  
+  static final String LOCK_FILE_NAME = "test.lock";
 
   @SuppressForbidden(reason = "System.out required: command line tool")
+  @SuppressWarnings("try")
   public static void main(String[] args) throws Exception {
-
     if (args.length != 7) {
       System.out.println("Usage: java org.apache.lucene.store.LockStressTest myID verifierHost verifierPort lockFactoryClassName lockDirName sleepTimeMS count\n" +
                          "\n" +
@@ -91,7 +93,6 @@ public class LockStressTest {
       out.write(myID);
       out.flush();
       LockFactory verifyLF = new VerifyingLockFactory(lockFactory, in, out);
-      Lock l = verifyLF.makeLock(lockDir, "test.lock");
       final Random rnd = new Random();
       
       // wait for starting gun
@@ -100,25 +101,22 @@ public class LockStressTest {
       }
       
       for (int i = 0; i < count; i++) {
-        boolean obtained = false;
-        try {
-          obtained = l.obtain(rnd.nextInt(100) + 10);
-        } catch (LockObtainFailedException e) {}
-        
-        if (obtained) {
+        try (final Lock l = verifyLF.obtainLock(lockDir, LOCK_FILE_NAME)) {
           if (rnd.nextInt(10) == 0) {
             if (rnd.nextBoolean()) {
               verifyLF = new VerifyingLockFactory(getNewLockFactory(lockFactoryClassName), in, out);
             }
-            final Lock secondLock = verifyLF.makeLock(lockDir, "test.lock");
-            if (secondLock.obtain()) {
-              throw new IOException("Double Obtain");
+            try (final Lock secondLock = verifyLF.obtainLock(lockDir, LOCK_FILE_NAME)) {
+              throw new IOException("Double obtain");
+            } catch (IOException ioe) {
+              // pass
             }
           }
           Thread.sleep(sleepTimeMS);
-          l.close();
+        } catch (IOException ioe) {
+          // obtain failed
         }
-        
+
         if (i % 500 == 0) {
           System.out.println((i * 100. / count) + "% done.");
         }

Added: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockValidatingDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockValidatingDirectoryWrapper.java?rev=1682422&view=auto
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockValidatingDirectoryWrapper.java (added)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockValidatingDirectoryWrapper.java Fri May 29 11:31:39 2015
@@ -0,0 +1,57 @@
+package org.apache.lucene.store;
+
+/*
+ * 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;
+
+/** 
+ * This class makes a best-effort check that a provided {@link Lock}
+ * is valid before any destructive filesystem operation.
+ */
+public final class LockValidatingDirectoryWrapper extends FilterDirectory {
+  private final Lock writeLock;
+
+  public LockValidatingDirectoryWrapper(Directory in, Lock writeLock) {
+    super(in);
+    this.writeLock = writeLock;
+  }
+
+  @Override
+  public void deleteFile(String name) throws IOException {
+    writeLock.ensureValid();
+    in.deleteFile(name);
+  }
+
+  @Override
+  public IndexOutput createOutput(String name, IOContext context) throws IOException {
+    writeLock.ensureValid();
+    return in.createOutput(name, context);
+  }
+
+  @Override
+  public void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
+    writeLock.ensureValid();
+    in.copyFrom(from, src, dest, context);
+  }
+
+  @Override
+  public void renameFile(String source, String dest) throws IOException {
+    writeLock.ensureValid();
+    in.renameFile(source, dest);
+  }
+}
\ No newline at end of file

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/NativeFSLockFactory.java Fri May 29 11:31:39 2015
@@ -18,10 +18,12 @@ package org.apache.lucene.store;
  */
 
 import java.nio.channels.FileChannel;
-import java.nio.channels.OverlappingFileLockException;
+import java.nio.channels.FileLock;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.StandardOpenOption;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.nio.file.attribute.FileTime;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashSet;
@@ -77,136 +79,122 @@ public final class NativeFSLockFactory e
    */
   public static final NativeFSLockFactory INSTANCE = new NativeFSLockFactory();
 
+  private static final Set<String> LOCK_HELD = Collections.synchronizedSet(new HashSet<String>());
+
   private NativeFSLockFactory() {}
 
   @Override
-  protected Lock makeFSLock(FSDirectory dir, String lockName) {
-    return new NativeFSLock(dir.getDirectory(), lockName);
+  protected Lock obtainFSLock(FSDirectory dir, String lockName) throws IOException {
+    Path lockDir = dir.getDirectory();
+    
+    // Ensure that lockDir exists and is a directory.
+    // note: this will fail if lockDir is a symlink
+    Files.createDirectories(lockDir);
+    
+    Path lockFile = lockDir.resolve(lockName);
+
+    try {
+      Files.createFile(lockFile);
+    } catch (IOException ignore) {
+      // we must create the file to have a truly canonical path.
+      // if it's already created, we don't care. if it cant be created, it will fail below.
+    }
+    
+    // fails if the lock file does not exist
+    final Path realPath = lockFile.toRealPath();
+    
+    // used as a best-effort check, to see if the underlying file has changed
+    final FileTime creationTime = Files.readAttributes(realPath, BasicFileAttributes.class).creationTime();
+    
+    boolean obtained = false;
+    if (LOCK_HELD.add(realPath.toString())) {
+      FileChannel channel = null;
+      try {
+        channel = FileChannel.open(realPath, StandardOpenOption.CREATE, StandardOpenOption.WRITE);
+        FileLock lock = channel.tryLock();
+        if (lock != null) {
+          obtained = true;
+          return new NativeFSLock(lock, realPath, creationTime);
+        } else {
+          throw new IOException("Lock held by another program: " + realPath);
+        }
+      } finally {
+        if (obtained == false) { // not successful - clear up and move out
+          IOUtils.closeWhileHandlingException(channel); // nocommit: addSuppressed
+          clearLockHeld(realPath);  // clear LOCK_HELD last 
+        }
+      }
+    } else {
+      throw new IOException("Lock held by this virtual machine: " + realPath);
+    }
   }
   
+  private static final void clearLockHeld(Path path) throws IOException {
+    boolean remove = LOCK_HELD.remove(path.toString());
+    if (remove == false) {
+      throw new AlreadyClosedException("Lock path was cleared but never marked as held: " + path);
+    }
+  }
+    
   static final class NativeFSLock extends Lock {
-
+    private final FileLock lock;
     private final Path path;
-    private final Path lockDir;
-    private static final Set<String> LOCK_HELD = Collections.synchronizedSet(new HashSet<String>());
-
-    private FileChannel channel; // set when we have the lock
-    private Path realPath;       // unconditionally set in obtain(), for use in close()
-
-    public NativeFSLock(Path lockDir, String lockFileName) {
-      this.lockDir = lockDir;
-      path = lockDir.resolve(lockFileName);
+    private final FileTime creationTime;
+    private boolean closed;
+    
+    NativeFSLock(FileLock lock, Path path, FileTime creationTime) {
+      this.lock = lock;
+      this.path = path;
+      this.creationTime = creationTime;
     }
 
     @Override
-    public synchronized boolean obtain() throws IOException {
-
-      if (channel != null) {
-        // Our instance is already locked:
-        assert channel.isOpen();
-        assert realPath != null;
-        throw new LockObtainFailedException("this lock instance was already obtained");
+    public synchronized void ensureValid() throws IOException {
+      if (closed) {
+        throw new AlreadyClosedException("Lock instance already released: " + this);
+      }
+      // check we are still in the locks map (some debugger or something crazy didn't remove us)
+      if (!LOCK_HELD.contains(path.toString())) {
+        throw new AlreadyClosedException("Lock path unexpectedly cleared from map: " + this);
+      }
+      // check our lock wasn't invalidated.
+      if (!lock.isValid()) {
+        throw new AlreadyClosedException("FileLock invalidated by an external force: " + this);
+      }
+      // try to validate the underlying file descriptor.
+      // this will throw IOException if something is wrong.
+      long size = lock.channel().size();
+      if (size != 0) {
+        throw new AlreadyClosedException("Unexpected lock file size: " + size + ", (lock=" + this + ")");
+      }
+      // try to validate the backing file name, that it still exists,
+      // and has the same creation time as when we obtained the lock. 
+      // if it differs, someone deleted our lock file (and we are ineffective)
+      FileTime ctime = Files.readAttributes(path, BasicFileAttributes.class).creationTime(); 
+      if (!creationTime.equals(ctime)) {
+        throw new AlreadyClosedException("Underlying file changed by an external force at " + creationTime + ", (lock=" + this + ")");
       }
-
-      // Ensure that lockDir exists and is a directory.
-      Files.createDirectories(lockDir);
-      try {
-        Files.createFile(path);
-      } catch (IOException ignore) {
-        // we must create the file to have a truly canonical path.
-        // if it's already created, we don't care. if it cant be created, it will fail below.
-      }
-      realPath = path.toRealPath();
-      // Make sure nobody else in-process has this lock held
-      // already, and, mark it held if not:
-      // This is a pretty crazy workaround for some documented
-      // but yet awkward JVM behavior:
-      //
-      //   On some systems, closing a channel releases all locks held by the Java virtual machine on the underlying file
-      //   regardless of whether the locks were acquired via that channel or via another channel open on the same file.
-      //   It is strongly recommended that, within a program, a unique channel be used to acquire all locks on any given
-      //   file.
-      //
-      // This essentially means if we close "A" channel for a given file all locks might be released... the odd part
-      // is that we can't re-obtain the lock in the same JVM but from a different process if that happens. Nevertheless
-      // this is super trappy. See LUCENE-5738
-      boolean obtained = false;
-      if (LOCK_HELD.add(realPath.toString())) {
-        FileChannel ch = null;
-        try {
-          ch = FileChannel.open(realPath, StandardOpenOption.CREATE, StandardOpenOption.WRITE);
-          try {
-            if (ch.tryLock() != null) {
-              channel = ch;
-              obtained = true;
-            }
-          } catch (IOException | OverlappingFileLockException e) {
-            // At least on OS X, we will sometimes get an
-            // intermittent "Permission Denied" IOException,
-            // which seems to simply mean "you failed to get
-            // the lock".  But other IOExceptions could be
-            // "permanent" (eg, locking is not supported via
-            // the filesystem).  So, we record the failure
-            // reason here; the timeout obtain (usually the
-            // one calling us) will use this as "root cause"
-            // if it fails to get the lock.
-            failureReason = e;
-          }
-        } finally {
-          if (obtained == false) { // not successful - clear up and move out
-            IOUtils.closeWhileHandlingException(ch);
-            clearLockHeld(realPath);  // clear LOCK_HELD last 
-          }
-        }
-      }
-      return obtained;
     }
 
     @Override
     public synchronized void close() throws IOException {
-      if (channel != null) {
-        try {
-          IOUtils.close(channel);
-        } finally {
-          channel = null;
-          clearLockHeld(realPath); // clear LOCK_HELD last 
-        }
+      if (closed) {
+        return;
       }
-    }
-
-    private static final void clearLockHeld(Path path) {
-      boolean remove = LOCK_HELD.remove(path.toString());
-      assert remove : "Lock was cleared but never marked as held";
-    }
-
-    @Override
-    public synchronized boolean isLocked() {
-      // The test for is isLocked is not directly possible with native file locks:
-      
-      // First a shortcut, if a lock reference in this instance is available
-      if (channel != null) {
-        return true;
-      }
-      
-      // Look if lock file is definitely not present; if not, there can definitely be no lock!
-      if (Files.notExists(path)) { 
-        return false;
+      // NOTE: we don't validate, as unlike SimpleFSLockFactory, we can't break others locks
+      try (FileLock lock = this.lock;
+           FileChannel channel = lock.channel()) {
+        assert lock != null;
+        assert channel != null;
+      } finally {
+        closed = true;
+        clearLockHeld(path);
       }
-      
-      // Try to obtain and release (if was locked) the lock
-      try {
-        boolean obtained = obtain();
-        if (obtained) close();
-        return !obtained;
-      } catch (IOException ioe) {
-        return false;
-      }    
     }
 
     @Override
     public String toString() {
-      return "NativeFSLock@" + path;
+      return "NativeFSLock(path=" + path + ",impl=" + lock + ",ctime=" + creationTime + ")"; 
     }
   }
-
 }

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/NoLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/NoLockFactory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/NoLockFactory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/NoLockFactory.java Fri May 29 11:31:39 2015
@@ -37,23 +37,17 @@ public final class NoLockFactory extends
   private NoLockFactory() {}
 
   @Override
-  public Lock makeLock(Directory dir, String lockName) {
+  public Lock obtainLock(Directory dir, String lockName) {
     return SINGLETON_LOCK;
   }
   
   private static class NoLock extends Lock {
     @Override
-    public boolean obtain() throws IOException {
-      return true;
-    }
-
-    @Override
     public void close() {
     }
 
     @Override
-    public boolean isLocked() {
-      return false;
+    public void ensureValid() throws IOException {
     }
 
     @Override
@@ -61,5 +55,4 @@ public final class NoLockFactory extends
       return "NoLock";
     }
   }
-
 }

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java Fri May 29 11:31:39 2015
@@ -21,6 +21,8 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.nio.file.attribute.FileTime;
 
 /**
  * <p>Implements {@link LockFactory} using {@link
@@ -35,8 +37,8 @@ import java.nio.file.Path;
  * this API for locking is that the Lucene write lock may
  * not be released when the JVM exits abnormally.</p>
 
- * <p>When this happens, a {@link LockObtainFailedException}
- * is hit when trying to create a writer, in which case you
+ * <p>When this happens, an {@link IOException}
+ * is hit when trying to create a writer, in which case you may
  * need to explicitly clear the lock file first by
  * manually removing the file.  But, first be certain that
  * no writer is in fact writing to the index otherwise you
@@ -70,66 +72,78 @@ public final class SimpleFSLockFactory e
   private SimpleFSLockFactory() {}
 
   @Override
-  protected Lock makeFSLock(FSDirectory dir, String lockName) {
-    return new SimpleFSLock(dir.getDirectory(), lockName);
+  protected Lock obtainFSLock(FSDirectory dir, String lockName) throws IOException {
+    Path lockDir = dir.getDirectory();
+    
+    // Ensure that lockDir exists and is a directory.
+    // note: this will fail if lockDir is a symlink
+    Files.createDirectories(lockDir);
+    
+    Path lockFile = lockDir.resolve(lockName);
+    
+    // create the file: this will fail if it already exists
+    Files.createFile(lockFile);
+    
+    // used as a best-effort check, to see if the underlying file has changed
+    final FileTime creationTime = Files.readAttributes(lockFile, BasicFileAttributes.class).creationTime();
+    
+    return new SimpleFSLock(lockFile, creationTime);
   }
   
-  static class SimpleFSLock extends Lock {
-
-    Path lockFile;
-    Path lockDir;
-    boolean obtained = false;
-
-    public SimpleFSLock(Path lockDir, String lockFileName) {
-      this.lockDir = lockDir;
-      lockFile = lockDir.resolve(lockFileName);
+  static final class SimpleFSLock extends Lock {
+    private final Path path;
+    private final FileTime creationTime;
+    private boolean closed;
+
+    SimpleFSLock(Path path, FileTime creationTime) throws IOException {
+      this.path = path;
+      this.creationTime = creationTime;
     }
 
     @Override
-    public synchronized boolean obtain() throws IOException {
-      if (obtained) {
-        // Our instance is already locked:
-        throw new LockObtainFailedException("this lock instance was already obtained");
+    public synchronized void ensureValid() throws IOException {
+      if (closed) {
+        throw new AlreadyClosedException("Lock instance already released: " + this);
       }
-      
-      try {
-        Files.createDirectories(lockDir);
-        Files.createFile(lockFile);
-        obtained = true;
-      } catch (IOException ioe) {
-        // On Windows, on concurrent createNewFile, the 2nd process gets "access denied".
-        // In that case, the lock was not aquired successfully, so return false.
-        // We record the failure reason here; the obtain with timeout (usually the
-        // one calling us) will use this as "root cause" if it fails to get the lock.
-        failureReason = ioe;
+      // try to validate the backing file name, that it still exists,
+      // and has the same creation time as when we obtained the lock. 
+      // if it differs, someone deleted our lock file (and we are ineffective)
+      FileTime ctime = Files.readAttributes(path, BasicFileAttributes.class).creationTime(); 
+      if (!creationTime.equals(ctime)) {
+        throw new AlreadyClosedException("Underlying file changed by an external force at " + creationTime + ", (lock=" + this + ")");
       }
-
-      return obtained;
     }
 
     @Override
-    public synchronized void close() throws LockReleaseFailedException {
-      // TODO: wierd that clearLock() throws the raw IOException...
-      if (obtained) {
+    public synchronized void close() throws IOException {
+      if (closed) {
+        return;
+      }
+      try {
+        // NOTE: unlike NativeFSLockFactory, we can potentially delete someone else's
+        // lock if things have gone wrong. we do best-effort check (ensureValid) to
+        // avoid doing this.
         try {
-          Files.deleteIfExists(lockFile);
-        } catch (Throwable cause) {
-          throw new LockReleaseFailedException("failed to delete " + lockFile, cause);
-        } finally {
-          obtained = false;
+          ensureValid();
+        } catch (Throwable exc) {
+          // notify the user they may need to intervene.
+          throw new AlreadyClosedException("Lock file cannot be safely removed. Manual intervention is recommended.", exc);
         }
+        // we did a best effort check, now try to remove the file. if something goes wrong,
+        // we need to make it clear to the user that the directory my still remain locked.
+        try {
+          Files.delete(path);
+        } catch (Throwable exc) {
+          throw new IOException("Unable to remove lock file. Manual intervention is recommended", exc);
+        }
+      } finally {
+        closed = true;
       }
     }
 
     @Override
-    public boolean isLocked() {
-      return Files.exists(lockFile);
-    }
-
-    @Override
     public String toString() {
-      return "SimpleFSLock@" + lockFile;
+      return "SimpleFSLock(path=" + path + ",ctime=" + creationTime + ")";
     }
   }
-
 }

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SingleInstanceLockFactory.java Fri May 29 11:31:39 2015
@@ -24,7 +24,7 @@ import java.util.HashSet;
  * Implements {@link LockFactory} for a single in-process instance,
  * meaning all locking will take place through this one instance.
  * Only use this {@link LockFactory} when you are certain all
- * IndexReaders and IndexWriters for a given index are running
+ * IndexWriters for a given index are running
  * against a single shared in-process Directory instance.  This is
  * currently the default locking for RAMDirectory.
  *
@@ -33,51 +33,53 @@ import java.util.HashSet;
 
 public final class SingleInstanceLockFactory extends LockFactory {
 
-  private final HashSet<String> locks = new HashSet<>();
+  final HashSet<String> locks = new HashSet<>();
 
   @Override
-  public Lock makeLock(Directory dir, String lockName) {
-    return new SingleInstanceLock(locks, lockName);
+  public Lock obtainLock(Directory dir, String lockName) throws IOException {
+    synchronized (locks) {
+      if (locks.add(lockName)) {
+        return new SingleInstanceLock(lockName);
+      } else {
+        throw new IOException("lock instance already obtained: (dir=" + dir + ", lockName=" + lockName + ")");
+      }
+    }
   }
 
-  private static class SingleInstanceLock extends Lock {
-
+  private class SingleInstanceLock extends Lock {
     private final String lockName;
-    private final HashSet<String> locks;
-    private boolean obtained = false;
+    private boolean closed;
 
-    public SingleInstanceLock(HashSet<String> locks, String lockName) {
-      this.locks = locks;
+    public SingleInstanceLock(String lockName) {
       this.lockName = lockName;
     }
 
     @Override
-    public boolean obtain() throws IOException {
-      synchronized(locks) {
-        if (obtained) {
-          // Our instance is already locked:
-          throw new LockObtainFailedException("this lock instance was already obtained");
-        }
-        obtained = locks.add(lockName);
-
-        return obtained;
+    public synchronized void ensureValid() throws IOException {
+      if (closed) {
+        throw new AlreadyClosedException("Lock instance already released: " + this);
       }
-    }
-
-    @Override
-    public void close() {
-      synchronized(locks) {
-        if (obtained) {
-          locks.remove(lockName);
-          obtained = false;
+      // check we are still in the locks map (some debugger or something crazy didn't remove us)
+      synchronized (locks) {
+        if (!locks.contains(lockName)) {
+          throw new AlreadyClosedException("Lock instance was invalidated from map: " + this);
         }
       }
     }
 
     @Override
-    public boolean isLocked() {
-      synchronized(locks) {
-        return locks.contains(lockName);
+    public synchronized void close() throws IOException {
+      if (closed) {
+        return;
+      }
+      try {
+        synchronized (locks) {
+          if (!locks.remove(lockName)) {
+            throw new AlreadyClosedException("Lock was already released: " + this);
+          }
+        }
+      } finally {
+        closed = true;
       }
     }
 

Modified: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/VerifyingLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/VerifyingLockFactory.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/VerifyingLockFactory.java (original)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/VerifyingLockFactory.java Fri May 29 11:31:39 2015
@@ -43,10 +43,23 @@ public final class VerifyingLockFactory
 
   private class CheckedLock extends Lock {
     private final Lock lock;
-    private boolean obtained = false;
 
-    public CheckedLock(Lock lock) {
+    public CheckedLock(Lock lock) throws IOException {
       this.lock = lock;
+      verify((byte) 1);
+    }
+
+    @Override
+    public void ensureValid() throws IOException {
+      lock.ensureValid();
+    }
+
+    @Override
+    public void close() throws IOException {
+      try (Lock l = lock) {
+        l.ensureValid();
+        verify((byte) 0);
+      }
     }
 
     private void verify(byte message) throws IOException {
@@ -60,29 +73,6 @@ public final class VerifyingLockFactory
         throw new IOException("Protocol violation.");
       }
     }
-
-    @Override
-    public synchronized boolean obtain() throws IOException {
-      obtained = lock.obtain();
-      if (obtained) {
-        verify((byte) 1);
-      }
-      return obtained;
-    }
-
-    @Override
-    public synchronized boolean isLocked() throws IOException {
-      return lock.isLocked();
-    }
-
-    @Override
-    public synchronized void close() throws IOException {
-      if (obtained) {
-        assert isLocked();
-        verify((byte) 0);
-      }
-      lock.close();
-    }
   }
 
   /**
@@ -97,7 +87,7 @@ public final class VerifyingLockFactory
   }
 
   @Override
-  public Lock makeLock(Directory dir, String lockName) {
-    return new CheckedLock(lf.makeLock(dir, lockName));
+  public Lock obtainLock(Directory dir, String lockName) throws IOException {
+    return new CheckedLock(lf.obtainLock(dir, lockName));
   }
 }

Modified: lucene/dev/branches/lucene6508/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java?rev=1682422&r1=1682421&r2=1682422&view=diff
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java (original)
+++ lucene/dev/branches/lucene6508/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java Fri May 29 11:31:39 2015
@@ -43,7 +43,6 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.LockObtainFailedException; // javadocs
 import org.apache.lucene.util.BytesRef;
 
 /*
@@ -154,8 +153,6 @@ public class DirectoryTaxonomyWriter imp
    *    If null or missing, {@link #defaultTaxonomyWriterCache()} is used.
    * @throws CorruptIndexException
    *     if the taxonomy is corrupted.
-   * @throws LockObtainFailedException
-   *     if the taxonomy is locked by another writer.
    * @throws IOException
    *     if another error occurred.
    */