Index: src/main/java/org/infinispan/lucene/InfinispanLock.java =================================================================== --- src/main/java/org/infinispan/lucene/InfinispanLock.java (revision 0) +++ src/main/java/org/infinispan/lucene/InfinispanLock.java (revision 0) @@ -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; + +import java.io.IOException; + +import javax.transaction.TransactionManager; + +import org.apache.lucene.store.Lock; +import org.infinispan.AdvancedCache; +import org.infinispan.Cache; +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 InfinispanLock extends Lock { + + private static final Log log = LogFactory.getLog(InfinispanLock.class); + + private final AdvancedCache cache; + private final String lockName; + private final String indexName; +// private final TransactionManager tm; // not available yet + private final CacheKey keyOfLock; + + InfinispanLock(Cache 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} + */ + public boolean obtain() throws IOException { + Object previousValue = cache.putIfAbsent(keyOfLock, keyOfLock); + if (previousValue == null) { + if (log.isTraceEnabled()) { + log.trace("Lock: {0} acquired for index: {1}", lockName, indexName); + } + return true; + } + else { + if (log.isTraceEnabled()) { + log.trace("Lock: {0} not aquired for index: {1}, was taken already.", lockName, indexName); + } + return false; + } + } + + /** + * {@inheritDoc} + */ + public void release() throws IOException { + clearLock(); + } + + /** + * Removes the lock, without committing pending changes or involving transactions. + * Used by Lucene at Directory creation: we expect the lock to no exist. + */ + public void clearLock() { + cache.remove(keyOfLock); + if (log.isTraceEnabled()) { + log.trace("Lock removed for index: {0}", indexName); + } + } + + @Override + public boolean isLocked() { + throw new org.jboss.util.NotImplementedException("FIXME NYI isLocked"); + } + +} \ No newline at end of file Property changes on: src/main/java/org/infinispan/lucene/InfinispanLock.java ___________________________________________________________________ Added: svn:eol-style + LF Added: svn:keywords + Id Revision Index: src/main/java/org/infinispan/lucene/InfinispanLockFactory.java =================================================================== --- src/main/java/org/infinispan/lucene/InfinispanLockFactory.java (revision 981) +++ src/main/java/org/infinispan/lucene/InfinispanLockFactory.java (working copy) @@ -22,11 +22,10 @@ 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.index.IndexWriter; import org.apache.lucene.store.LockFactory; import org.infinispan.Cache; import org.infinispan.CacheException; @@ -36,6 +35,7 @@ /** * Factory for locks obtained in InfinispanDirectory * + * @author Sanne Grinovero * @since 4.0 * @author Lukasz Moren * @see org.infinispan.lucene.InfinispanDirectory @@ -44,183 +44,60 @@ public class InfinispanLockFactory extends LockFactory { private static final Log log = LogFactory.getLog(InfinispanLockFactory.class); + static final String DEF_LOCK_NAME = IndexWriter.WRITE_LOCK_NAME; - private Cache cache; - private String indexName; + private final Cache cache; + private final String indexName; + private final TransactionManager tm; + private final InfinispanLock defLock; public InfinispanLockFactory(Cache 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()); +// } + tm = null; + defLock = new InfinispanLock(cache, indexName, DEF_LOCK_NAME, tm); } /** * {@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); - } + public InfinispanLock makeLock(String lockName) { + InfinispanLock 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 InfinispanLock(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 { - 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 cache; - private String lockName; - private String indexName; - - final TransactionManager tm; - - InfinispanLock(Cache 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()); - } + //Same special care as above for locks named DEF_LOCK_NAME: + if (DEF_LOCK_NAME.equals(lockName)) { + defLock.clearLock(); } - - /** - * {@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; + else { + new InfinispanLock(cache, indexName, lockName, tm).clearLock(); } - - /** - * {@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; + if (log.isTraceEnabled()) { + log.trace("Removed lock: {0} for index {1}", lockName, indexName); } } - + } Index: src/test/java/org/infinispan/lucene/ConcurrentHashMapCacheTestFactory.java =================================================================== --- src/test/java/org/infinispan/lucene/ConcurrentHashMapCacheTestFactory.java (revision 0) +++ src/test/java/org/infinispan/lucene/ConcurrentHashMapCacheTestFactory.java (revision 0) @@ -0,0 +1,50 @@ +/* + * 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 org.infinispan.Cache; + +/** + * ConcurrentHashMapCacheTestFactory. + * + * @author Sanne Grinovero + * @since 4.0 + */ +public class ConcurrentHashMapCacheTestFactory implements TestCacheFactory { + + private final LocalCacheMock fakeCache = new LocalCacheMock(); + + @Override + public Cache cache(int nodeIndex) { + return fakeCache; + } + + @Override + public void initialize() throws Throwable { + } + + @Override + public void shutdown() throws Throwable { + fakeCache.clear(); + } + +} Property changes on: src/test/java/org/infinispan/lucene/ConcurrentHashMapCacheTestFactory.java ___________________________________________________________________ Added: svn:eol-style + LF Added: svn:keywords + Id Revision Index: src/test/java/org/infinispan/lucene/InfinispanLockFactoryStressTest.java =================================================================== --- src/test/java/org/infinispan/lucene/InfinispanLockFactoryStressTest.java (revision 0) +++ src/test/java/org/infinispan/lucene/InfinispanLockFactoryStressTest.java (revision 0) @@ -0,0 +1,231 @@ +/* + * 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 java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.lucene.store.LockObtainFailedException; +import org.infinispan.Cache; +import org.infinispan.util.logging.Log; +import org.infinispan.util.logging.LogFactory; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +/** + * InfinispanLockFactoryStressTest. Inspired by Lucene's LockStressTest, just not needing to + * coordinate the different threads by network. + * + * @see org.apache.lucene.store.VerifyingLockFactory + * @see org.apache.lucene.store.LockVerifyServer + * @author Sanne Grinovero + * @since 4.0 + */ +@Test(groups = "profiling", testName = "lucene.InfinispanDirectoryTest") +public class InfinispanLockFactoryStressTest { + + private static final Log log = LogFactory.getLog(InfinispanLockFactoryStressTest.class); + + private static final int STRESS_TIME_MINUTES = 1; + private static final int NODES_NUM = 5; + private static final int THREADS_PER_NODE = 10; + private static final String INDEXNAME = "indexName"; + private static final int SLEEP_MILLISECONDS = 100; + +// private final TestCacheFactory cacheFactory = new MultiNodeTestCacheFactory(NODES_NUM, "lucene"); + private final TestCacheFactory cacheFactory = new ConcurrentHashMapCacheTestFactory(); +// private final TestCacheFactory cacheFactory = new LocalISPNCacheTestFactory(false); + + @BeforeClass + public void prepareCaches() throws Throwable { + cacheFactory.initialize(); + } + + @AfterClass + public void cleanup() throws Throwable { + cacheFactory.shutdown(); + } + + @Test + @SuppressWarnings("unchecked") + public void testCachesAreCommunicating() { + Cache cacheZero = cacheFactory.cache(0); + FileCacheKey key = new FileCacheKey("someIndex", "this is a lock file", true); + for (int nodeCounter = 0; nodeCounter < NODES_NUM; nodeCounter++) { + Cache cache = cacheFactory.cache(nodeCounter); + assert cache.containsKey(key) == false; + } + cacheZero.put(key, key); + for (int nodeCounter = 0; nodeCounter < NODES_NUM; nodeCounter++) { + Cache cache = cacheFactory.cache(nodeCounter); + Object retrieved = cache.get(key); + assert retrieved != null; + assert retrieved.equals(key); + } + cacheZero.remove(key); + for (int nodeCounter = 0; nodeCounter < NODES_NUM; nodeCounter++) { + Cache cache = cacheFactory.cache(nodeCounter); + assert cache.containsKey(key) == false; + } + } + + @Test + @SuppressWarnings("unchecked") + public void testConcurrentLocking() throws IOException, InterruptedException { + SharedStats stats = new SharedStats(); + ThreadPoolExecutor executor = (ThreadPoolExecutor) Executors.newFixedThreadPool(NODES_NUM * THREADS_PER_NODE); + for (int nodeCounter = 0; nodeCounter < NODES_NUM; nodeCounter++) { + Cache cache = cacheFactory.cache(nodeCounter); + createNode(cache, executor, stats); + } + executor.shutdown(); + Thread.sleep(5000); + System.out.println("\nSituation after 5 seconds:"); + System.out.println(stats.toString()); + executor.awaitTermination(STRESS_TIME_MINUTES, TimeUnit.MINUTES); + stats.quitAllThreadsNicely(); + executor.awaitTermination(10, TimeUnit.SECONDS); //give some time to awake and quit + System.out.println("\nFinal situation:"); + System.out.println(stats.toString()); + assert stats.getFailedThreadsCount() == 0 : "at least one thread has seen unexpected state"; + assert stats.lockAcquiredCounter.get() > 0 : "the lock should have been taken at least once"; + assert stats.lockAcquiredCounter.get() == stats.lockReleasedCounter.get() : "there's a mismatch in acquires and releases count"; + assert stats.lockOwnersCounter.get() == 0 : "the lock is still held at test finish"; + } + + @SuppressWarnings("unchecked") + private void createNode(Cache cache, Executor executor, SharedStats stats) { + for (int threadCounter = 0; threadCounter < THREADS_PER_NODE; threadCounter++) { + InfinispanLockFactory lockFactory = new InfinispanLockFactory(cache, INDEXNAME); + LockFactoryStressingRunnable thread = new LockFactoryStressingRunnable(lockFactory, stats); + stats.threads.add(thread); + executor.execute(thread); + } + } + + private static class LockFactoryStressingRunnable implements Runnable { + + private final InfinispanLockFactory lockFactory; + private final SharedStats stats; + + private volatile boolean failed = false; + private InfinispanLock lock; + public volatile boolean quit = false; + + public LockFactoryStressingRunnable(InfinispanLockFactory lockFactory, SharedStats stats) { + this.lockFactory = lockFactory; + this.stats = stats; + } + + @Override + public void run() { + lock = lockFactory.makeLock(InfinispanLockFactory.DEF_LOCK_NAME); + while (!(failed || quit || Thread.interrupted())) { + try { + doCycle(); + } catch (IOException e) { + failed = true; + log.error(e); + } + } + } + + private void doCycle() throws IOException { + boolean weOwnTheLock = false; + try { + weOwnTheLock = lock.obtain(); + } catch (LockObtainFailedException e) { + stats.lockFailsCounter.incrementAndGet(); + } + if (weOwnTheLock) { + boolean lockIsFine = stats.lockOwnersCounter.compareAndSet(0, 1); + System.out.print("L"); + stats.lockAcquiredCounter.incrementAndGet(); + checkIsTrue(lockIsFine, "I got the lock, some other thread is owning the lock AS WELL."); + sleep(); + lockIsFine = stats.lockOwnersCounter.compareAndSet(1, 0); + checkIsTrue(lockIsFine, "Some other thread changed the lock count while I was having it!"); + System.out.print("R"); + lock.release(); + stats.lockReleasedCounter.incrementAndGet(); + } else { + sleep(); + } + } + + private void sleep() { + try { + Thread.sleep(SLEEP_MILLISECONDS); + } catch (InterruptedException e) { + // no-op: waking up is good enough + } + } + + private void checkIsTrue(boolean assertion, String message) { + if (assertion == false) { + failed = true; + log.error(message); + } + } + + } + + public static class SharedStats { + + final List threads = new ArrayList(); + final AtomicInteger lockFailsCounter = new AtomicInteger(0); + final AtomicInteger lockAcquiredCounter = new AtomicInteger(0); + final AtomicInteger lockReleasedCounter = new AtomicInteger(0); + final AtomicInteger lockOwnersCounter = new AtomicInteger(0); + + int getFailedThreadsCount() { + int counter = 0; + for (LockFactoryStressingRunnable thread : threads) { + if (thread.failed) { + counter++; + } + } + return counter; + } + + public void quitAllThreadsNicely() { + for (LockFactoryStressingRunnable thread : threads) { + thread.quit = true; + } + } + + public String toString() { + return "\n\tLock acquires which timed out:\t" + lockFailsCounter.get() + "\n\tLocks acquired:\t" + + lockAcquiredCounter.get() + "\n\tLocks released:\t" + lockReleasedCounter.get() + + "\n\tIllegal states detected:\t" + getFailedThreadsCount(); + } + + } + +} Property changes on: src/test/java/org/infinispan/lucene/InfinispanLockFactoryStressTest.java ___________________________________________________________________ Added: svn:eol-style + LF Added: svn:keywords + Id Revision Index: src/test/java/org/infinispan/lucene/LocalCacheMock.java =================================================================== --- src/test/java/org/infinispan/lucene/LocalCacheMock.java (revision 0) +++ src/test/java/org/infinispan/lucene/LocalCacheMock.java (revision 0) @@ -0,0 +1,505 @@ +/* + * 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.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; + +import org.infinispan.AdvancedCache; +import org.infinispan.Cache; +import org.infinispan.batch.BatchContainer; +import org.infinispan.config.Configuration; +import org.infinispan.container.DataContainer; +import org.infinispan.context.Flag; +import org.infinispan.context.InvocationContextContainer; +import org.infinispan.eviction.EvictionManager; +import org.infinispan.factories.ComponentRegistry; +import org.infinispan.interceptors.base.CommandInterceptor; +import org.infinispan.lifecycle.ComponentStatus; +import org.infinispan.manager.CacheManager; +import org.infinispan.remoting.rpc.RpcManager; +import org.infinispan.util.concurrent.NotifyingFuture; + +/** + * LocalCacheMock: implements only what is provided by ConcurrentHashMap, extending ConcurrentHashMap. + * Used in tests to compare the Cache implementation against the ConcurrentMap API. + * getAdvancedCache() returns this but doesn't implement any additional feature besides overloading + * methods having Flag options, discarding the flags. + * + * @since 4.0 + * @see java.util.concurrent.ConcurrentHashMap + * @author Sanne Grinovero + */ +@SuppressWarnings("serial") +public class LocalCacheMock extends ConcurrentHashMap implements Cache, AdvancedCache { + + @Override + public NotifyingFuture clearAsync() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: clearAsync"); + } + + @Override + public void compact() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: compact"); + } + + @Override + public void endBatch(boolean successful) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: endBatch"); + } + + @Override + public void evict(CacheKey key) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: evict"); + } + + @Override + public AdvancedCache getAdvancedCache() { + return this; + } + + @Override + public CacheManager getCacheManager() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getCacheManager"); + } + + @Override + public Configuration getConfiguration() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getConfiguration"); + } + + @Override + public String getName() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getName"); + } + + @Override + public ComponentStatus getStatus() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getStatus"); + } + + @Override + public String getVersion() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getVersion"); + } + + @Override + public Object put(CacheKey key, Object value, long lifespan, TimeUnit unit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: put"); + } + + @Override + public Object put(CacheKey key, Object value, long lifespan, TimeUnit lifespanUnit, long maxIdleTime, + TimeUnit maxIdleTimeUnit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: put"); + } + + @Override + public void putAll(Map map, long lifespan, TimeUnit unit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAll"); + } + + @Override + public void putAll(Map map, long lifespan, TimeUnit lifespanUnit, + long maxIdleTime, TimeUnit maxIdleTimeUnit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAll"); + } + + @Override + public NotifyingFuture putAllAsync(Map data) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAllAsync"); + } + + @Override + public NotifyingFuture putAllAsync(Map data, long lifespan, TimeUnit unit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAllAsync"); + } + + @Override + public NotifyingFuture putAllAsync(Map data, long lifespan, + TimeUnit lifespanUnit, long maxIdle, TimeUnit maxIdleUnit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAllAsync"); + } + + @Override + public NotifyingFuture putAsync(CacheKey key, Object value) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAsync"); + } + + @Override + public NotifyingFuture putAsync(CacheKey key, Object value, long lifespan, TimeUnit unit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAsync"); + } + + @Override + public NotifyingFuture putAsync(CacheKey key, Object value, long lifespan, TimeUnit lifespanUnit, + long maxIdle, TimeUnit maxIdleUnit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAsync"); + } + + @Override + public void putForExternalRead(CacheKey key, Object value) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putForExternalRead"); + } + + @Override + public Object putIfAbsent(CacheKey key, Object value, long lifespan, TimeUnit unit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putIfAbsent"); + } + + @Override + public Object putIfAbsent(CacheKey key, Object value, long lifespan, TimeUnit lifespanUnit, long maxIdleTime, + TimeUnit maxIdleTimeUnit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putIfAbsent"); + } + + @Override + public NotifyingFuture putIfAbsentAsync(CacheKey key, Object value) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putIfAbsentAsync"); + } + + @Override + public NotifyingFuture putIfAbsentAsync(CacheKey key, Object value, long lifespan, TimeUnit unit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putIfAbsentAsync"); + } + + @Override + public NotifyingFuture putIfAbsentAsync(CacheKey key, Object value, long lifespan, TimeUnit lifespanUnit, + long maxIdle, TimeUnit maxIdleUnit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putIfAbsentAsync"); + } + + @Override + public NotifyingFuture removeAsync(Object key) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: removeAsync"); + } + + @Override + public NotifyingFuture removeAsync(Object key, Object value) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: removeAsync"); + } + + @Override + public Object replace(CacheKey key, Object value, long lifespan, TimeUnit unit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replace"); + } + + @Override + public boolean replace(CacheKey key, Object oldValue, Object value, long lifespan, TimeUnit unit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replace"); + } + + @Override + public Object replace(CacheKey key, Object value, long lifespan, TimeUnit lifespanUnit, long maxIdleTime, + TimeUnit maxIdleTimeUnit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replace"); + } + + @Override + public boolean replace(CacheKey key, Object oldValue, Object value, long lifespan, TimeUnit lifespanUnit, + long maxIdleTime, TimeUnit maxIdleTimeUnit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replace"); + } + + @Override + public NotifyingFuture replaceAsync(CacheKey key, Object value) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replaceAsync"); + } + + @Override + public NotifyingFuture replaceAsync(CacheKey key, Object value, long lifespan, TimeUnit unit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replaceAsync"); + } + + @Override + public NotifyingFuture replaceAsync(CacheKey key, Object value, long lifespan, TimeUnit lifespanUnit, + long maxIdle, TimeUnit maxIdleUnit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replaceAsync"); + } + + @Override + public NotifyingFuture replaceAsync(CacheKey key, Object oldValue, Object newValue) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replaceAsync"); + } + + @Override + public NotifyingFuture replaceAsync(CacheKey key, Object oldValue, Object newValue, long lifespan, + TimeUnit unit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replaceAsync"); + } + + @Override + public NotifyingFuture replaceAsync(CacheKey key, Object oldValue, Object newValue, long lifespan, + TimeUnit lifespanUnit, long maxIdle, TimeUnit maxIdleUnit) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replaceAsync"); + } + + @Override + public boolean startBatch() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: startBatch"); + } + + @Override + public void start() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: start"); + } + + @Override + public void stop() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: stop"); + } + + @Override + public void addListener(Object listener) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: addListener"); + } + + @Override + public Set getListeners() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getListeners"); + } + + @Override + public void removeListener(Object listener) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: removeListener"); + } + + @Override + public void addInterceptor(CommandInterceptor i, int position) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: addInterceptor"); + } + + @Override + public void addInterceptorAfter(CommandInterceptor i, Class afterInterceptor) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: addInterceptorAfter"); + } + + @Override + public void addInterceptorBefore(CommandInterceptor i, Class beforeInterceptor) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: addInterceptorBefore"); + } + + @Override + public void clear(Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: clear"); + } + + @Override + public NotifyingFuture clearAsync(Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: clearAsync"); + } + + @Override + public boolean containsKey(Object key, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: containsKey"); + } + + @Override + public Object get(Object key, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: get"); + } + + @Override + public BatchContainer getBatchContainer() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getBatchContainer"); + } + + @Override + public ComponentRegistry getComponentRegistry() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getComponentRegistry"); + } + + @Override + public DataContainer getDataContainer() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getDataContainer"); + } + + @Override + public EvictionManager getEvictionManager() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getEvictionManager"); + } + + @Override + public List getInterceptorChain() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getInterceptorChain"); + } + + @Override + public InvocationContextContainer getInvocationContextContainer() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getInvocationContextContainer"); + } + + @Override + public RpcManager getRpcManager() { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: getRpcManager"); + } + + @Override + public void lock(CacheKey key) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: lock"); + } + + @Override + public void lock(Collection keys) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: lock"); + } + + @Override + public Object put(CacheKey key, Object value, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: put"); + } + + @Override + public Object put(CacheKey key, Object value, long lifespan, TimeUnit lifespanUnit, long maxIdleTime, + TimeUnit maxIdleTimeUnit, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: put"); + } + + @Override + public void putAll(Map map, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAll"); + } + + @Override + public void putAll(Map map, long lifespan, TimeUnit lifespanUnit, + long maxIdleTime, TimeUnit maxIdleTimeUnit, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAll"); + } + + @Override + public NotifyingFuture putAllAsync(Map map, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAllAsync"); + } + + @Override + public NotifyingFuture putAllAsync(Map map, long lifespan, + TimeUnit lifespanUnit, long maxIdleTime, TimeUnit maxIdleTimeUnit, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAllAsync"); + } + + @Override + public NotifyingFuture putAsync(CacheKey key, Object value, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAsync"); + } + + @Override + public NotifyingFuture putAsync(CacheKey key, Object value, long lifespan, TimeUnit lifespanUnit, + long maxIdleTime, TimeUnit maxIdleTimeUnit, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putAsync"); + } + + @Override + public void putForExternalRead(CacheKey key, Object value, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putForExternalRead"); + } + + @Override + public Object putIfAbsent(CacheKey key, Object value, Flag... flags) { + return putIfAbsent(key, value); + } + + @Override + public Object putIfAbsent(CacheKey key, Object value, long lifespan, TimeUnit lifespanUnit, long maxIdleTime, + TimeUnit maxIdleTimeUnit, Flag... flags) { + return putIfAbsent(key, value); + } + + @Override + public NotifyingFuture putIfAbsentAsync(CacheKey key, Object value, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putIfAbsentAsync"); + } + + @Override + public NotifyingFuture putIfAbsentAsync(CacheKey key, Object value, long lifespan, TimeUnit lifespanUnit, + long maxIdleTime, TimeUnit maxIdleTimeUnit, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: putIfAbsentAsync"); + } + + @Override + public Object remove(Object key, Flag... flags) { + return remove(key); + } + + @Override + public NotifyingFuture removeAsync(Object key, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: removeAsync"); + } + + @Override + public void removeInterceptor(int position) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: removeInterceptor"); + } + + @Override + public void removeInterceptor(Class interceptorType) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: removeInterceptor"); + } + + @Override + public Object replace(CacheKey k, Object v, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replace"); + } + + @Override + public boolean replace(CacheKey k, Object oV, Object nV, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replace"); + } + + @Override + public Object replace(CacheKey k, Object v, long lifespan, TimeUnit lifespanUnit, long maxIdle, + TimeUnit maxIdleUnit, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replace"); + } + + @Override + public boolean replace(CacheKey k, Object oV, Object nV, long lifespan, TimeUnit lifespanUnit, long maxIdle, + TimeUnit maxIdleUnit, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replace"); + } + + @Override + public NotifyingFuture replaceAsync(CacheKey k, Object v, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replaceAsync"); + } + + @Override + public NotifyingFuture replaceAsync(CacheKey k, Object oV, Object nV, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replaceAsync"); + } + + @Override + public NotifyingFuture replaceAsync(CacheKey k, Object v, long lifespan, TimeUnit lifespanUnit, + long maxIdle, TimeUnit maxIdleUnit, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replaceAsync"); + } + + @Override + public NotifyingFuture replaceAsync(CacheKey k, Object oV, Object nV, long lifespan, TimeUnit lifespanUnit, + long maxIdle, TimeUnit maxIdleUnit, Flag... flags) { + throw new org.jboss.util.NotImplementedException("Not implemented on purpose: replaceAsync"); + } + +} Property changes on: src/test/java/org/infinispan/lucene/LocalCacheMock.java ___________________________________________________________________ Added: svn:eol-style + LF Added: svn:keywords + Id Revision Index: src/test/java/org/infinispan/lucene/LocalISPNCacheTestFactory.java =================================================================== --- src/test/java/org/infinispan/lucene/LocalISPNCacheTestFactory.java (revision 0) +++ src/test/java/org/infinispan/lucene/LocalISPNCacheTestFactory.java (revision 0) @@ -0,0 +1,71 @@ +/* + * 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 org.infinispan.Cache; +import org.infinispan.manager.CacheManager; +import org.infinispan.test.SingleCacheManagerTest; +import org.infinispan.test.fwk.TestCacheManagerFactory; +import org.testng.annotations.Test; + +/** + * LocalISPNCacheTestFactory. + * + * @author Sanne Grinovero + * @since 4.0 + */ +@Test(enabled=false)//this is not a test, but inherits from a testcase so disable it +public class LocalISPNCacheTestFactory extends SingleCacheManagerTest implements TestCacheFactory { + + private final boolean useTransaction; + + public LocalISPNCacheTestFactory(boolean useTransaction) { + super(); + this.useTransaction = useTransaction; + } + + @Override + public Cache cache(int nodeIndex) { + return cache; + } + + @Override + public void initialize() throws Throwable { + createBeforeClass(); + createBeforeMethod(); + } + + @Override + public void shutdown() throws Throwable { + clearContent(); + destroyAfterMethod(); + destroyAfterClass(); + } + + @Override + protected CacheManager createCacheManager() throws Exception { + CacheManager cm = TestCacheManagerFactory.createLocalCacheManager(useTransaction); + cache = cm.getCache(); + return cm; + } + +} Property changes on: src/test/java/org/infinispan/lucene/LocalISPNCacheTestFactory.java ___________________________________________________________________ Added: svn:eol-style + LF Added: svn:keywords + Id Revision Index: src/test/java/org/infinispan/lucene/MultiNodeTestCacheFactory.java =================================================================== --- src/test/java/org/infinispan/lucene/MultiNodeTestCacheFactory.java (revision 0) +++ src/test/java/org/infinispan/lucene/MultiNodeTestCacheFactory.java (revision 0) @@ -0,0 +1,70 @@ +/* + * 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 org.infinispan.Cache; +import org.infinispan.config.Configuration; +import org.infinispan.test.MultipleCacheManagersTest; + +/** + * MultiNodeTestCacheFactory used to be able to switch to different + * + * @author Sanne Grinovero + * @since 4.0 + */ +public class MultiNodeTestCacheFactory extends MultipleCacheManagersTest implements TestCacheFactory { + + private final int numberOfNodes; + private final String cacheName; + + public MultiNodeTestCacheFactory(int numberOfNodes, String cacheName) { + super(); + this.numberOfNodes = numberOfNodes; + this.cacheName = cacheName; + } + + @Override + @SuppressWarnings("unchecked") + public Cache cache(int managerIndex) { + return cache(managerIndex, cacheName); + } + + @Override + public void initialize() throws Throwable { + createBeforeClass(); + createBeforeMethod(); + } + + @Override + public void shutdown() throws Throwable { + clearContent(); + destroy(); + } + + @Override + protected void createCacheManagers() throws Throwable { + Configuration configuration = CacheTestSupport.createTestConfiguration(); + createClusteredCaches(numberOfNodes, "lucene", configuration); + log.debug("Using distributed cache. Cluster view ready."); + } + +} Property changes on: src/test/java/org/infinispan/lucene/MultiNodeTestCacheFactory.java ___________________________________________________________________ Added: svn:eol-style + LF Added: svn:keywords + Id Revision Index: src/test/java/org/infinispan/lucene/TestCacheFactory.java =================================================================== --- src/test/java/org/infinispan/lucene/TestCacheFactory.java (revision 0) +++ src/test/java/org/infinispan/lucene/TestCacheFactory.java (revision 0) @@ -0,0 +1,40 @@ +/* + * 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 org.infinispan.Cache; + +/** + * TestCacheFactory. + * + * @author Sanne Grinovero + * @since 4.0 + */ +public interface TestCacheFactory { + + void initialize() throws Throwable; + + void shutdown() throws Throwable; + + Cache cache(int nodeIndex); + +} Property changes on: src/test/java/org/infinispan/lucene/TestCacheFactory.java ___________________________________________________________________ Added: svn:eol-style + LF Added: svn:keywords + Id Revision