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/05/30 08:00:58 UTC

svn commit: r1682579 - in /lucene/dev/branches/lucene6508/lucene: core/src/java/org/apache/lucene/index/ core/src/java/org/apache/lucene/store/ core/src/test/org/apache/lucene/store/ test-framework/src/java/org/apache/lucene/store/

Author: rmuir
Date: Sat May 30 06:00:57 2015
New Revision: 1682579

URL: http://svn.apache.org/r1682579
Log:
LUCENE-6508: some steps towards back compat and tests

Added:
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java   (with props)
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SleepingLockFactory.java   (with props)
    lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestNativeFSLockFactory.java   (with props)
    lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSLockFactory.java   (with props)
    lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSingleInstanceLockFactory.java   (with props)
    lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSleepingLockFactory.java   (with props)
    lucene/dev/branches/lucene6508/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java   (with props)
Removed:
    lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestLock.java
Modified:
    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/store/Lock.java
    lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SimpleFSLockFactory.java

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=1682579&r1=1682578&r2=1682579&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 Sat May 30 06:00:57 2015
@@ -56,6 +56,7 @@ 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;
@@ -118,7 +119,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 IOException}.</p>
+  {@link LockObtainFailedException}.</p>
   
   <a name="deletionPolicy"></a>
   <p>Expert: <code>IndexWriter</code> allows an optional
@@ -4354,6 +4355,24 @@ 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
+   * @deprecated Use of this method can only lead to race conditions. Try
+   *             to actually obtain a lock instead.
+   */
+  @Deprecated
+  public static boolean isLocked(Directory directory) throws IOException {
+    try {
+      directory.obtainLock(WRITE_LOCK_NAME).close();
+      return false;
+    } catch (LockObtainFailedException failed) {
+      return true;
+    }
+  }
 
   /** If {@link DirectoryReader#open(IndexWriter,boolean)} has
    *  been called (ie, this writer is in near real-time

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=1682579&r1=1682578&r2=1682579&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 Sat May 30 06:00:57 2015
@@ -42,6 +42,8 @@ public abstract class Lock implements Cl
    * of the lock file, or other reasons.
    * <p>
    * {@inheritDoc} 
+   * @throws LockReleaseFailedException optional specific exception) if 
+   *         the lock could not be properly released.
    */
   public abstract void close() throws IOException;
   

Added: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java?rev=1682579&view=auto
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java (added)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/LockReleaseFailedException.java Sat May 30 06:00:57 2015
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+package org.apache.lucene.store;
+
+import java.io.IOException;
+
+/**
+ * This exception is thrown when the <code>write.lock</code>
+ * could not be released.
+ * @see Lock#close()
+ */
+public class LockReleaseFailedException extends IOException {
+  public LockReleaseFailedException(String message) {
+    super(message);
+  }
+  
+  public LockReleaseFailedException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

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=1682579&r1=1682578&r2=1682579&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 Sat May 30 06:00:57 2015
@@ -128,14 +128,14 @@ public final class SimpleFSLockFactory e
           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);
+          throw new LockReleaseFailedException("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 may still remain locked.
         try {
           Files.delete(path);
         } catch (Throwable exc) {
-          throw new IOException("Unable to remove lock file. Manual intervention is recommended", exc);
+          throw new LockReleaseFailedException("Unable to remove lock file. Manual intervention is recommended", exc);
         }
       } finally {
         closed = true;

Added: lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SleepingLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SleepingLockFactory.java?rev=1682579&view=auto
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SleepingLockFactory.java (added)
+++ lucene/dev/branches/lucene6508/lucene/core/src/java/org/apache/lucene/store/SleepingLockFactory.java Sat May 30 06:00:57 2015
@@ -0,0 +1,110 @@
+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;
+
+import org.apache.lucene.util.ThreadInterruptedException;
+
+/** 
+ * LockFactory that wraps another, and that sleeps and retries
+ * if obtaining the lock fails.
+ * <p>
+ * This is not a good idea.
+ */
+public final class SleepingLockFactory extends LockFactory {
+ 
+  /** 
+   * Pass this lockWaitTimeout to try forever to obtain the lock. 
+   */
+  public static final long LOCK_OBTAIN_WAIT_FOREVER = -1;
+  
+  /** 
+   * How long {@link #obtainLock} waits, in milliseconds,
+   * in between attempts to acquire the lock. 
+   */
+  public static long DEFAULT_POLL_INTERVAL = 1000;
+  
+  private final LockFactory delegate;
+  private final long lockWaitTimeout;
+  private final long pollInterval;
+  
+  /**
+   * Create a new SleepingLockFactory
+   * @param delegate        underlying lockfactory to wrap
+   * @param lockWaitTimeout length of time to wait in milliseconds 
+   *                        or {@link #LOCK_OBTAIN_WAIT_FOREVER} to retry forever.
+   */
+  public SleepingLockFactory(LockFactory delegate, long lockWaitTimeout) {
+    this(delegate, lockWaitTimeout, DEFAULT_POLL_INTERVAL);
+  }
+  
+  /**
+   * Create a new SleepingLockFactory
+   * @param delegate        underlying lockfactory to wrap
+   * @param lockWaitTimeout length of time to wait in milliseconds 
+   *                        or {@link #LOCK_OBTAIN_WAIT_FOREVER} to retry forever.
+   * @param pollInterval    poll once per this interval in milliseconds until
+   *                        {@code lockWaitTimeout} is exceeded.
+   */
+  public SleepingLockFactory(LockFactory delegate, long lockWaitTimeout, long pollInterval) {
+    this.delegate = delegate;
+    this.lockWaitTimeout = lockWaitTimeout;
+    this.pollInterval = pollInterval;
+    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 + ")");
+    }
+    if (pollInterval < 0) {
+      throw new IllegalArgumentException("pollInterval must be a non-negative number (got " + pollInterval + ")");
+    }
+  }
+
+  @Override
+  public Lock obtainLock(Directory dir, String lockName) throws IOException {
+    LockObtainFailedException failureReason = null;
+    long maxSleepCount = lockWaitTimeout / pollInterval;
+    long sleepCount = 0;
+    
+    do {
+      try {
+        return delegate.obtainLock(dir, lockName);
+      } catch (LockObtainFailedException failed) {
+        if (failureReason == null) {
+          failureReason = failed;
+        }
+      }
+      try {
+        Thread.sleep(pollInterval);
+      } catch (InterruptedException ie) {
+        throw new ThreadInterruptedException(ie);
+      }
+    } while (sleepCount++ < maxSleepCount || lockWaitTimeout == LOCK_OBTAIN_WAIT_FOREVER);
+    
+    // we failed to obtain the lock in the required time
+    String reason = "Lock obtain timed out: " + this.toString();
+    if (failureReason != null) {
+      reason += ": " + failureReason;
+    }
+    throw new LockObtainFailedException(reason, failureReason);
+  }
+
+  @Override
+  public String toString() {
+    return "SleepingLockFactory(impl=" + delegate + ")";
+  }
+}

Added: lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestNativeFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestNativeFSLockFactory.java?rev=1682579&view=auto
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestNativeFSLockFactory.java (added)
+++ lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestNativeFSLockFactory.java Sat May 30 06:00:57 2015
@@ -0,0 +1,32 @@
+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;
+import java.nio.file.Path;
+
+/** Simple tests for NativeFSLockFactory */
+public class TestNativeFSLockFactory extends BaseLockFactoryTestCase {
+
+  @Override
+  protected Directory getDirectory(Path path) throws IOException {
+    return newFSDirectory(path, NativeFSLockFactory.INSTANCE);
+  }
+  
+  // TODO: specific tests to this impl
+}

Added: lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSLockFactory.java?rev=1682579&view=auto
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSLockFactory.java (added)
+++ lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSimpleFSLockFactory.java Sat May 30 06:00:57 2015
@@ -0,0 +1,32 @@
+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;
+import java.nio.file.Path;
+
+/** Simple tests for SimpleFSLockFactory */
+public class TestSimpleFSLockFactory extends BaseLockFactoryTestCase {
+
+  @Override
+  protected Directory getDirectory(Path path) throws IOException {
+    return newFSDirectory(path, SimpleFSLockFactory.INSTANCE);
+  }
+  
+  // TODO: specific tests to this impl
+}

Added: lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSingleInstanceLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSingleInstanceLockFactory.java?rev=1682579&view=auto
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSingleInstanceLockFactory.java (added)
+++ lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSingleInstanceLockFactory.java Sat May 30 06:00:57 2015
@@ -0,0 +1,32 @@
+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;
+import java.nio.file.Path;
+
+/** Simple tests for SingleInstanceLockFactory */
+public class TestSingleInstanceLockFactory extends BaseLockFactoryTestCase {
+
+  @Override
+  protected Directory getDirectory(Path path) throws IOException {
+    return newDirectory(random(), new SingleInstanceLockFactory());
+  }
+  
+  // TODO: specific tests to this impl
+}

Added: lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSleepingLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSleepingLockFactory.java?rev=1682579&view=auto
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSleepingLockFactory.java (added)
+++ lucene/dev/branches/lucene6508/lucene/core/src/test/org/apache/lucene/store/TestSleepingLockFactory.java Sat May 30 06:00:57 2015
@@ -0,0 +1,45 @@
+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;
+import java.nio.file.Path;
+
+import org.apache.lucene.util.TestUtil;
+
+/** Simple tests for SleepingLockFactory */
+public class TestSleepingLockFactory extends BaseLockFactoryTestCase {
+
+  @Override
+  protected Directory getDirectory(Path path) throws IOException {
+    long lockWaitTimeout = TestUtil.nextLong(random(), 20, 100);
+    long pollInterval = TestUtil.nextLong(random(), 2, 10);
+    
+    int which = random().nextInt(3);
+    switch (which) {
+      case 0:
+        return newDirectory(random(), new SleepingLockFactory(new SingleInstanceLockFactory(), lockWaitTimeout, pollInterval));
+      case 1:
+        return newFSDirectory(path, new SleepingLockFactory(NativeFSLockFactory.INSTANCE, lockWaitTimeout, pollInterval));
+      default:
+        return newFSDirectory(path, new SleepingLockFactory(SimpleFSLockFactory.INSTANCE, lockWaitTimeout, pollInterval));
+    }
+  }
+  
+  // TODO: specific tests to this impl
+}

Added: lucene/dev/branches/lucene6508/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6508/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java?rev=1682579&view=auto
==============================================================================
--- lucene/dev/branches/lucene6508/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java (added)
+++ lucene/dev/branches/lucene6508/lucene/test-framework/src/java/org/apache/lucene/store/BaseLockFactoryTestCase.java Sat May 30 06:00:57 2015
@@ -0,0 +1,81 @@
+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;
+import java.nio.file.Path;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.lucene.util.LuceneTestCase;
+
+/** Base class for per-LockFactory tests. */
+public abstract class BaseLockFactoryTestCase extends LuceneTestCase {
+  
+  /** Subclass returns the Directory to be tested; if it's
+   *  an FS-based directory it should point to the specified
+   *  path, else it can ignore it. */
+  protected abstract Directory getDirectory(Path path) throws IOException;
+  
+  public void testObtainConcurrently() throws InterruptedException, IOException {
+    final Directory directory = getDirectory(createTempDir());
+    final AtomicBoolean running = new AtomicBoolean(true);
+    final AtomicInteger atomicCounter = new AtomicInteger(0);
+    final ReentrantLock assertingLock = new ReentrantLock();
+    int numThreads = 2 + random().nextInt(10);
+    final int runs = atLeast(10000);
+    CyclicBarrier barrier = new CyclicBarrier(numThreads);
+    Thread[] threads = new Thread[numThreads];
+    for (int i = 0; i < threads.length; i++) {
+      threads[i] = new Thread() {
+        @Override
+        public void run() {
+          try {
+            barrier.await();
+          } catch (Exception e) {
+            throw new RuntimeException(e);
+          }
+          while (running.get()) {
+            try (Lock lock = directory.obtainLock("foo.lock")) {
+              assertFalse(assertingLock.isLocked());
+              if (assertingLock.tryLock()) {
+                assertingLock.unlock();
+              } else {
+                fail();
+              }
+              assert lock != null; // stupid compiler
+            } catch (IOException ex) {
+              //
+            }
+            if (atomicCounter.incrementAndGet() > runs) {
+              running.set(false);
+            }
+          }
+        }
+      };
+      threads[i].start();
+    }
+
+    for (int i = 0; i < threads.length; i++) {
+      threads[i].join();
+    }
+    directory.close();
+  }
+}