[infinispan-commits] Infinispan SVN: r1095 - in trunk/lucene-directory/src: main/java/org/infinispan/lucene/locking and 2 other directories.

infinispan-commits at lists.jboss.org infinispan-commits at lists.jboss.org
Tue Nov 3 12:23:30 EST 2009


Author: sannegrinovero
Date: 2009-11-03 12:23:29 -0500 (Tue, 03 Nov 2009)
New Revision: 1095

Added:
   trunk/lucene-directory/src/main/java/org/infinispan/lucene/locking/
   trunk/lucene-directory/src/main/java/org/infinispan/lucene/locking/LuceneLockFactory.java
   trunk/lucene-directory/src/main/java/org/infinispan/lucene/locking/SharedLuceneLock.java
   trunk/lucene-directory/src/test/java/org/infinispan/lucene/locking/
   trunk/lucene-directory/src/test/java/org/infinispan/lucene/locking/LockManagerFunctionalTest.java
Removed:
   trunk/lucene-directory/src/main/java/org/infinispan/lucene/InfinispanLockFactory.java
Modified:
   trunk/lucene-directory/src/main/java/org/infinispan/lucene/FileCacheKey.java
   trunk/lucene-directory/src/main/java/org/infinispan/lucene/InfinispanDirectory.java
   trunk/lucene-directory/src/test/java/org/infinispan/lucene/CacheTestSupport.java
   trunk/lucene-directory/src/test/java/org/infinispan/lucene/InfinispanDirectoryStressTest.java
Log:
[ISPN-227] (Have Lucene LockManager use atomic operations)

Modified: trunk/lucene-directory/src/main/java/org/infinispan/lucene/FileCacheKey.java
===================================================================
--- trunk/lucene-directory/src/main/java/org/infinispan/lucene/FileCacheKey.java	2009-11-03 17:16:31 UTC (rev 1094)
+++ trunk/lucene-directory/src/main/java/org/infinispan/lucene/FileCacheKey.java	2009-11-03 17:23:29 UTC (rev 1095)
@@ -30,7 +30,7 @@
  * @author Lukasz Moren
  * @author Sanne Grinovero
  */
-final class FileCacheKey implements Serializable, CacheKey {
+public final class FileCacheKey implements Serializable, CacheKey {
 
    /** The serialVersionUID */
    private static final long serialVersionUID = -228474937509042691L;

Modified: trunk/lucene-directory/src/main/java/org/infinispan/lucene/InfinispanDirectory.java
===================================================================
--- trunk/lucene-directory/src/main/java/org/infinispan/lucene/InfinispanDirectory.java	2009-11-03 17:16:31 UTC (rev 1094)
+++ trunk/lucene-directory/src/main/java/org/infinispan/lucene/InfinispanDirectory.java	2009-11-03 17:23:29 UTC (rev 1095)
@@ -33,6 +33,7 @@
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.LockFactory;
 import org.infinispan.Cache;
+import org.infinispan.lucene.locking.LuceneLockFactory;
 import org.infinispan.util.logging.Log;
 import org.infinispan.util.logging.LogFactory;
 
@@ -40,12 +41,12 @@
  * Implementation that uses Infinispan to store Lucene indices.
  * 
  * Directory locking is assured with
- * {@link org.infinispan.lucene.InfinispanLockFactory.InfinispanLock}
+ * {@link org.infinispan.lucene.locking.SharedLuceneLock}
  * 
  * @since 4.0
  * @author Lukasz Moren
  * @author Sanne Grinovero
- * @see org.infinispan.lucene.InfinispanLockFactory
+ * @see org.infinispan.lucene.locking.LuceneLockFactory
  */
 // todo add support for ConcurrentMergeSheduler
 public class InfinispanDirectory extends Directory {
@@ -83,11 +84,11 @@
    }
 
    public InfinispanDirectory(Cache<CacheKey, Object> cache, String indexName, int chunkSize) {
-      this(cache, indexName, new InfinispanLockFactory(cache, indexName), chunkSize);
+      this(cache, indexName, new LuceneLockFactory(cache, indexName), chunkSize);
    }
 
    public InfinispanDirectory(Cache<CacheKey, Object> cache, String indexName) {
-      this(cache, indexName, new InfinispanLockFactory(cache, indexName), InfinispanIndexIO.DEFAULT_BUFFER_SIZE);
+      this(cache, indexName, new LuceneLockFactory(cache, indexName), InfinispanIndexIO.DEFAULT_BUFFER_SIZE);
    }
 
    public InfinispanDirectory(Cache<CacheKey, Object> cache) {

Deleted: trunk/lucene-directory/src/main/java/org/infinispan/lucene/InfinispanLockFactory.java
===================================================================
--- trunk/lucene-directory/src/main/java/org/infinispan/lucene/InfinispanLockFactory.java	2009-11-03 17:16:31 UTC (rev 1094)
+++ trunk/lucene-directory/src/main/java/org/infinispan/lucene/InfinispanLockFactory.java	2009-11-03 17:23:29 UTC (rev 1095)
@@ -1,226 +0,0 @@
-/*
- * JBoss, Home of Professional Open Source.
- * Copyright 2009, Red Hat Middleware LLC, and individual contributors
- * as indicated by the @author tags. See the copyright.txt file in the
- * distribution for a full listing of individual contributors.
- *
- * This is free software; you can redistribute it and/or modify it
- * under the terms of the GNU Lesser General Public License as
- * published by the Free Software Foundation; either version 2.1 of
- * the License, or (at your option) any later version.
- *
- * This software is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- * Lesser General Public License for more details.
- *
- * You should have received a copy of the GNU Lesser General Public
- * License along with this software; if not, write to the Free
- * Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA
- * 02110-1301 USA, or see the FSF site: http://www.fsf.org.
- */
-package org.infinispan.lucene;
-
-import java.io.IOException;
-import javax.transaction.RollbackException;
-import javax.transaction.Transaction;
-import javax.transaction.TransactionManager;
-
-import org.apache.lucene.store.Lock;
-import org.apache.lucene.store.LockFactory;
-import org.infinispan.Cache;
-import org.infinispan.CacheException;
-import org.infinispan.util.logging.Log;
-import org.infinispan.util.logging.LogFactory;
-
-/**
- * Factory for locks obtained in <code>InfinispanDirectory</code>
- * 
- * @since 4.0
- * @author Lukasz Moren
- * @see org.infinispan.lucene.InfinispanDirectory
- * @see org.infinispan.lucene.InfinispanLockFactory.InfinispanLock
- */
-public class InfinispanLockFactory extends LockFactory {
-
-   private static final Log log = LogFactory.getLog(InfinispanLockFactory.class);
-
-   private Cache<CacheKey, Object> cache;
-   private String indexName;
-
-   public InfinispanLockFactory(Cache<CacheKey, Object> cache, String indexName) {
-      this.cache = cache;
-      this.indexName = indexName;
-   }
-
-   /**
-    * {@inheritDoc}
-    */
-   public Lock makeLock(String lockName) {
-      try {
-         return new InfinispanLock(cache, indexName, lockName);
-      } finally {
-         if (log.isTraceEnabled()) {
-            log.trace("Created new lock: {0} for index {1}", lockName, indexName);
-         }
-      }
-   }
-
-   /**
-    * {@inheritDoc}
-    */
-   public void clearLock(String lockName) throws IOException {
-      try {
-         cache.remove(new FileCacheKey(indexName, lockName, true));
-      } finally {
-         if (log.isTraceEnabled()) {
-            log.trace("Removed lock: {0} for index {1}", lockName, indexName);
-         }
-      }
-   }
-
-   /**
-    * Interprocess Lucene index lock
-    * 
-    * @see org.apache.lucene.store.Directory#makeLock(String)
-    */
-   public static class InfinispanLock extends Lock {
-
-      private static final Log log = LogFactory.getLog(InfinispanLock.class);
-
-      private final Cache<CacheKey, Object> cache;
-      private String lockName;
-      private String indexName;
-
-      final TransactionManager tm;
-
-      InfinispanLock(Cache<CacheKey, Object> cache, String indexName, String lockName) {
-         this.cache = cache;
-         this.lockName = lockName;
-         this.indexName = indexName;
-
-         tm = cache.getAdvancedCache().getComponentRegistry().getComponent(TransactionManager.class);
-         if (tm == null) {
-            throw new CacheException(
-                     "Failed looking up TransactionManager, check if any transaction manager is associated with Infinispan cache: "
-                              + cache.getName());
-         }
-      }
-
-      /**
-       * {@inheritDoc}
-       */
-      public boolean obtain() throws IOException {
-         boolean acquired = false;
-
-         synchronized (cache) {
-            try {
-               // begin transaction for lock obtaining
-               tm.begin();
-               CacheKey lock = new FileCacheKey(indexName, lockName, true);
-               if (!cache.containsKey(lock)) {
-                  cache.put(lock, lock);
-                  acquired = true;
-               }
-            } catch (Exception e) {
-               log.error("Cannot obtain lock for: " + indexName, e);
-            } finally {
-               try {
-                  if (tm.getTransaction() != null) {
-                     if (acquired) {
-                        tm.commit();
-                        if (log.isTraceEnabled()) {
-                           log.trace("Lock: {0} acquired for index: {1} ", lockName, indexName);
-                        }
-                     } else {
-                        tm.rollback();
-                     }
-                  }
-               } catch (RollbackException e) {
-                  log.error("Cannot obtain lock for: " + indexName, e);
-                  acquired = false;
-               } catch (Exception e) {
-                  throw new CacheException(e);
-               }
-            }
-
-            if (acquired) {
-               try {
-                  // begin new transaction to batch all changes, tx commited when lock is released.
-                  tm.begin();
-                  if (log.isTraceEnabled()) {
-                     log.trace("Batch transaction started for index: {0}", indexName);
-                  }
-               } catch (Exception e) {
-                  log.error("Unable to start transaction", e);
-               }
-            }
-         }
-
-         return acquired;
-      }
-
-      /**
-       * {@inheritDoc}
-       */
-      public void release() throws IOException {
-         boolean removed = false;
-         synchronized (cache) {
-            try {
-               // commit changes in batch, transaction was started when lock was acquired
-               tm.commit();
-               if (log.isTraceEnabled()) {
-                  log.trace("Batch transaction commited for index: {0}", indexName);
-               }
-
-               tm.begin();
-               removed = cache.remove(new FileCacheKey(indexName, lockName, true)) != null;
-            } catch (Exception e) {
-               throw new CacheException("Unable to commit work done or release lock!", e);
-            } finally {
-               try {
-                  if (removed) {
-                     tm.commit();
-                     if (log.isTraceEnabled()) {
-                        log.trace("Lock: {0} removed for index: {1} ", lockName, indexName);
-                     }
-                  } else {
-                     tm.rollback();
-                  }
-               } catch (Exception e) {
-                  throw new CacheException("Unable to release lock!", e);
-               }
-            }
-         }
-      }
-
-      /**
-       * {@inheritDoc}
-       */
-      public boolean isLocked() {
-         boolean locked = false;
-         synchronized (cache) {
-            Transaction tx = null;
-            try {
-               // if there is an ongoing transaction we need to suspend it
-               if ((tx = tm.getTransaction()) != null) {
-                  tm.suspend();
-               }
-               locked = cache.containsKey(new FileCacheKey(indexName, lockName, true));
-            } catch (Exception e) {
-               log.error("Error in suspending transaction", e);
-            } finally {
-               if (tx != null) {
-                  try {
-                     tm.resume(tx);
-                  } catch (Exception e) {
-                     throw new CacheException("Unable to resume suspended transaction " + tx, e);
-                  }
-               }
-            }
-         }
-         return locked;
-      }
-   }
-
-}

Copied: trunk/lucene-directory/src/main/java/org/infinispan/lucene/locking/LuceneLockFactory.java (from rev 1090, trunk/lucene-directory/src/main/java/org/infinispan/lucene/InfinispanLockFactory.java)
===================================================================
--- trunk/lucene-directory/src/main/java/org/infinispan/lucene/locking/LuceneLockFactory.java	                        (rev 0)
+++ trunk/lucene-directory/src/main/java/org/infinispan/lucene/locking/LuceneLockFactory.java	2009-11-03 17:23:29 UTC (rev 1095)
@@ -0,0 +1,102 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * Copyright 2009, Red Hat Middleware LLC, and individual contributors
+ * as indicated by the @author tags. See the copyright.txt file in the
+ * distribution for a full listing of individual contributors.
+ *
+ * This is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU Lesser General Public License as
+ * published by the Free Software Foundation; either version 2.1 of
+ * the License, or (at your option) any later version.
+ *
+ * This software is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this software; if not, write to the Free
+ * Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA
+ * 02110-1301 USA, or see the FSF site: http://www.fsf.org.
+ */
+package org.infinispan.lucene.locking;
+
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.store.LockFactory;
+import org.infinispan.Cache;
+import org.infinispan.CacheException;
+import org.infinispan.lucene.CacheKey;
+import org.infinispan.util.logging.Log;
+import org.infinispan.util.logging.LogFactory;
+
+import javax.transaction.TransactionManager;
+import java.io.IOException;
+
+/**
+ * Factory for locks obtained in <code>InfinispanDirectory</code>
+ * 
+ * @since 4.0
+ * @author Sanne Grinovero
+ * @author Lukasz Moren
+ * @see org.infinispan.lucene.InfinispanDirectory
+ * @see org.infinispan.lucene.locking.SharedLuceneLock
+ */
+public class LuceneLockFactory extends LockFactory {
+
+   private static final Log log = LogFactory.getLog(LuceneLockFactory.class);
+   static final String DEF_LOCK_NAME = IndexWriter.WRITE_LOCK_NAME;
+
+   private final Cache<CacheKey, Object> cache;
+   private final String indexName;
+   private final TransactionManager tm;
+   private final SharedLuceneLock defLock;
+
+   public LuceneLockFactory(Cache<CacheKey, Object> cache, String indexName) {
+      this.cache = cache;
+      this.indexName = indexName;
+      tm = cache.getAdvancedCache().getComponentRegistry().getComponent(TransactionManager.class);
+      if (tm == null) {
+         throw new CacheException(
+                  "Failed looking up TransactionManager, check if any transaction manager is associated with Infinispan cache: "
+                           + cache.getName());
+      }
+      defLock = new SharedLuceneLock(cache, indexName, DEF_LOCK_NAME, tm);
+   }
+
+   /**
+    * {@inheritDoc}
+    */
+   public SharedLuceneLock makeLock(String lockName) {
+      SharedLuceneLock lock;
+      //It appears Lucene always uses the same name so we give locks
+      //having this name a special treatment:
+      if (DEF_LOCK_NAME.equals(lockName)) {
+         lock = defLock;
+      }
+      else {
+         // this branch is never taken with current Lucene version.
+         lock = new SharedLuceneLock(cache, indexName, lockName, tm);
+      }
+      if (log.isTraceEnabled()) {
+         log.trace("Lock prepared, not acquired: {0} for index {1}", lockName, indexName);
+      }
+      return lock;
+   }
+
+   /**
+    * {@inheritDoc}
+    */
+   public void clearLock(String lockName) throws IOException {
+      //Same special care as above for locks named DEF_LOCK_NAME:
+      if (DEF_LOCK_NAME.equals(lockName)) {
+         defLock.clearLockSuspending();
+      }
+      else {
+         new SharedLuceneLock(cache, indexName, lockName, tm).clearLockSuspending();
+      }
+      if (log.isTraceEnabled()) {
+         log.trace("Removed lock: {0} for index {1}", lockName, indexName);
+      }
+   }
+   
+}

Added: trunk/lucene-directory/src/main/java/org/infinispan/lucene/locking/SharedLuceneLock.java
===================================================================
--- trunk/lucene-directory/src/main/java/org/infinispan/lucene/locking/SharedLuceneLock.java	                        (rev 0)
+++ trunk/lucene-directory/src/main/java/org/infinispan/lucene/locking/SharedLuceneLock.java	2009-11-03 17:23:29 UTC (rev 1095)
@@ -0,0 +1,196 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * Copyright 2009, Red Hat Middleware LLC, and individual contributors
+ * as indicated by the @author tags. See the copyright.txt file in the
+ * distribution for a full listing of individual contributors.
+ *
+ * This is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU Lesser General Public License as
+ * published by the Free Software Foundation; either version 2.1 of
+ * the License, or (at your option) any later version.
+ *
+ * This software is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this software; if not, write to the Free
+ * Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA
+ * 02110-1301 USA, or see the FSF site: http://www.fsf.org.
+ */
+package org.infinispan.lucene.locking;
+
+import java.io.IOException;
+
+import javax.transaction.Transaction;
+import javax.transaction.TransactionManager;
+
+import org.apache.lucene.store.Lock;
+import org.infinispan.AdvancedCache;
+import org.infinispan.Cache;
+import org.infinispan.CacheException;
+import org.infinispan.context.Flag;
+import org.infinispan.lucene.CacheKey;
+import org.infinispan.lucene.FileCacheKey;
+import org.infinispan.util.logging.Log;
+import org.infinispan.util.logging.LogFactory;
+
+/**
+ * Inter-IndexWriter Lucene index lock based on Infinispan.
+ * 
+ * @since 4.0
+ * @author Sanne Grinovero
+ * @see org.apache.lucene.store.Lock
+ */
+class SharedLuceneLock extends Lock {
+
+   private static final Log log = LogFactory.getLog(SharedLuceneLock.class);
+   private static final Flag[] lockFlags = new Flag[]{Flag.SKIP_CACHE_STORE};
+
+   private final AdvancedCache<CacheKey, Object> cache;
+   private final String lockName;
+   private final String indexName;
+   private final TransactionManager tm;
+   private final FileCacheKey keyOfLock;
+
+   SharedLuceneLock(Cache<CacheKey, Object> cache, String indexName, String lockName, TransactionManager tm) {
+      this.cache = cache.getAdvancedCache();
+      this.lockName = lockName;
+      this.indexName = indexName;
+      this.tm = tm;
+      this.keyOfLock = new FileCacheKey(indexName, lockName, true);
+   }
+
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public boolean obtain() throws IOException {
+      Object previousValue = cache.putIfAbsent(keyOfLock, keyOfLock, lockFlags);
+      if (previousValue == null) {
+         if (log.isTraceEnabled()) {
+            log.trace("Lock: {0} acquired for index: {1}", lockName, indexName);
+         }
+         // we own the lock:
+         startTransaction();
+         return true;
+      } else {
+         if (log.isTraceEnabled()) {
+            log.trace("Lock: {0} not aquired for index: {1}, was taken already.", lockName, indexName);
+         }
+         return false;
+      }
+   }
+
+   /**
+    * {@inheritDoc}
+    */
+   @Override
+   public void release() throws IOException {
+      try {
+         commitTransactions();
+      }
+      finally {
+         clearLock();
+      }
+   }
+
+   /**
+    * Removes the lock, without committing pending changes or involving transactions. Used by Lucene
+    * at Directory creation: we expect the lock to not exist in this case.
+    */
+   private void clearLock() {
+      Object previousValue = cache.remove(keyOfLock, lockFlags);
+      if (previousValue!=null && log.isTraceEnabled()) {
+         log.trace("Lock removed for index: {0}", indexName);
+      }
+   }
+   
+   @Override
+   public boolean isLocked() {
+      boolean locked = false;
+      Transaction tx = null;
+      try {
+         // if there is an ongoing transaction we need to suspend it
+         if ((tx = tm.getTransaction()) != null) {
+            tm.suspend();
+         }
+         locked = cache.containsKey(keyOfLock, lockFlags);
+      } catch (Exception e) {
+         log.error("Error in suspending transaction", e);
+      } finally {
+         if (tx != null) {
+            try {
+               tm.resume(tx);
+            } catch (Exception e) {
+               throw new CacheException("Unable to resume suspended transaction " + tx, e);
+            }
+         }
+      }
+      return locked;
+   }
+   
+   /**
+    * Starts a new transaction. Used to batch changes in LuceneDirectory:
+    * a transaction is created at lock acquire, and closed on release.
+    * It's also committed and started again at each IndexWriter.commit();
+    * 
+    * @throws IOException wraps Infinispan exceptions to adapt to Lucene's API
+    */
+   private void startTransaction() throws IOException {
+      try {
+         tm.begin();
+      } catch (Exception e) {
+         log.error("Unable to start transaction", e);
+         throw new IOException("SharedLuceneLock could not start a transaction after having acquired the lock", e);
+      }
+      if (log.isTraceEnabled()) {
+         log.trace("Batch transaction started for index: {0}", indexName);
+      }
+   }
+   
+   /**
+    * Commits the existing transaction.
+    * It's illegal to call this if a transaction was not started.
+    * 
+    * @throws IOException wraps Infinispan exceptions to adapt to Lucene's API
+    */
+   private void commitTransactions() throws IOException {
+      try {
+         tm.commit();
+      } catch (Exception e) {
+         log.error("Unable to commit work done!", e);
+         throw new IOException("SharedLuceneLock could not commit a transaction", e);
+      }
+      if (log.isTraceEnabled()) {
+         log.trace("Batch transaction commited for index: {0}", indexName);
+      }
+   }
+
+   /**
+    * FIXME Comment this
+    * 
+    */
+   public void clearLockSuspending() {
+      Transaction tx = null;
+      try {
+         // if there is an ongoing transaction we need to suspend it
+         if ((tx = tm.getTransaction()) != null) {
+            tm.suspend();
+         }
+         clearLock();
+      } catch (Exception e) {
+         log.error("Error in suspending transaction", e);
+      } finally {
+         if (tx != null) {
+            try {
+               tm.resume(tx);
+            } catch (Exception e) {
+               throw new CacheException("Unable to resume suspended transaction " + tx, e);
+            }
+         }
+      }
+   }
+
+}
\ No newline at end of file


Property changes on: trunk/lucene-directory/src/main/java/org/infinispan/lucene/locking/SharedLuceneLock.java
___________________________________________________________________
Name: svn:keywords
   + Id Revision
Name: svn:eol-style
   + LF

Modified: trunk/lucene-directory/src/test/java/org/infinispan/lucene/CacheTestSupport.java
===================================================================
--- trunk/lucene-directory/src/test/java/org/infinispan/lucene/CacheTestSupport.java	2009-11-03 17:16:31 UTC (rev 1094)
+++ trunk/lucene-directory/src/test/java/org/infinispan/lucene/CacheTestSupport.java	2009-11-03 17:23:29 UTC (rev 1095)
@@ -51,7 +51,7 @@
       return TestCacheManagerFactory.createClusteredCacheManager( createTestConfiguration() );
    }
    
-   protected static Configuration createTestConfiguration() {
+   public static Configuration createTestConfiguration() {
       Configuration c = new Configuration();
       c.setCacheMode(Configuration.CacheMode.REPL_SYNC);
       c.setSyncReplTimeout(10000);

Modified: trunk/lucene-directory/src/test/java/org/infinispan/lucene/InfinispanDirectoryStressTest.java
===================================================================
--- trunk/lucene-directory/src/test/java/org/infinispan/lucene/InfinispanDirectoryStressTest.java	2009-11-03 17:16:31 UTC (rev 1094)
+++ trunk/lucene-directory/src/test/java/org/infinispan/lucene/InfinispanDirectoryStressTest.java	2009-11-03 17:23:29 UTC (rev 1095)
@@ -46,7 +46,7 @@
  * @author Sanne Grinovero
  */
 @SuppressWarnings("deprecation")
- at Test(groups = "profiling", testName = "lucene.InfinispanDirectoryTest")
+ at Test(groups = "profiling", testName = "lucene.InfinispanDirectoryStressTest")
 public class InfinispanDirectoryStressTest {
 
    private static final Log log = LogFactory.getLog(InfinispanDirectoryStressTest.class);

Added: trunk/lucene-directory/src/test/java/org/infinispan/lucene/locking/LockManagerFunctionalTest.java
===================================================================
--- trunk/lucene-directory/src/test/java/org/infinispan/lucene/locking/LockManagerFunctionalTest.java	                        (rev 0)
+++ trunk/lucene-directory/src/test/java/org/infinispan/lucene/locking/LockManagerFunctionalTest.java	2009-11-03 17:23:29 UTC (rev 1095)
@@ -0,0 +1,68 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * Copyright 2009, Red Hat Middleware LLC, and individual contributors
+ * as indicated by the @author tags. See the copyright.txt file in the
+ * distribution for a full listing of individual contributors.
+ *
+ * This is free software; you can redistribute it and/or modify it
+ * under the terms of the GNU Lesser General Public License as
+ * published by the Free Software Foundation; either version 2.1 of
+ * the License, or (at your option) any later version.
+ *
+ * This software is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
+ * Lesser General Public License for more details.
+ *
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this software; if not, write to the Free
+ * Software Foundation, Inc., 51 Franklin St, Fifth Floor, Boston, MA
+ * 02110-1301 USA, or see the FSF site: http://www.fsf.org.
+ */
+package org.infinispan.lucene.locking;
+
+import java.io.IOException;
+
+import org.infinispan.config.Configuration;
+import org.infinispan.lucene.CacheTestSupport;
+import org.infinispan.test.MultipleCacheManagersTest;
+import org.testng.annotations.Test;
+
+/**
+ * LockManagerFunctionalTest.
+ * 
+ * @author Sanne Grinovero
+ * @since 4.0
+ */
+ at Test(groups = "functional", testName = "lucene.locking.LockManagerFunctionalTest")
+public class LockManagerFunctionalTest extends MultipleCacheManagersTest {
+   
+   protected void createCacheManagers() throws Throwable {
+      Configuration replSync = CacheTestSupport.createTestConfiguration();
+      createClusteredCaches(2, "lucene", replSync);
+   }
+   
+   @Test
+   @SuppressWarnings("unchecked")
+   public void testLuceneIndexLocking() throws IOException {
+      final String commonIndexName = "myIndex";
+      LuceneLockFactory lockManagerA = new LuceneLockFactory(cache(0, "lucene"), commonIndexName);
+      LuceneLockFactory lockManagerB = new LuceneLockFactory(cache(1, "lucene"), commonIndexName);
+      LuceneLockFactory isolatedLockManager = new LuceneLockFactory(cache(0, "lucene"), "anotherIndex");
+      SharedLuceneLock luceneLockA = lockManagerA.makeLock(LuceneLockFactory.DEF_LOCK_NAME);
+      SharedLuceneLock luceneLockB = lockManagerB.makeLock(LuceneLockFactory.DEF_LOCK_NAME);
+      SharedLuceneLock anotherLock = isolatedLockManager.makeLock(LuceneLockFactory.DEF_LOCK_NAME);
+      
+      assert luceneLockA.obtain();
+      assert luceneLockB.isLocked();
+      assert ! anotherLock.isLocked();
+      assert ! luceneLockA.obtain();
+      assert ! luceneLockB.obtain();
+      luceneLockA.release();
+      assert ! luceneLockB.isLocked();
+      assert luceneLockB.obtain();
+      lockManagerA.clearLock(LuceneLockFactory.DEF_LOCK_NAME);
+      assert ! luceneLockB.isLocked();
+   }
+
+}


Property changes on: trunk/lucene-directory/src/test/java/org/infinispan/lucene/locking/LockManagerFunctionalTest.java
___________________________________________________________________
Name: svn:keywords
   + Id Revision
Name: svn:eol-style
   + LF



More information about the infinispan-commits mailing list