From: <tho...@us...> - 2011-03-03 21:27:09
|
Revision: 4267 http://bigdata.svn.sourceforge.net/bigdata/?rev=4267&view=rev Author: thompsonbry Date: 2011-03-03 21:27:02 +0000 (Thu, 03 Mar 2011) Log Message: ----------- Refactored the DefaultResourceLocator to remove the base class, which was simply a cache. Added a propertyCache to the DefaultResourceLocator to provide sharing of the materialized properties from the global row store across views which are backed by the same commit point. This sharing is only done for the local Journal right now as it depends on fast access to the commit time. This change could be extended to scale-out with a refactor to replace the native long transaction identifier with a thin interface capable of reporting both the transaction identifier and the commit point against which the transaction is reading. Added unit test for properties caching by the DefaultResourceLocator. Modified Journal to expose access to historical views of the global row store. Modified Paths: -------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/journal/Journal.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/AbstractResource.java branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/locator/DefaultResourceLocator.java branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/relation/locator/TestDefaultResourceLocator.java Removed Paths: ------------- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/locator/AbstractCachingResourceLocator.java Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/journal/Journal.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/journal/Journal.java 2011-03-03 18:38:51 UTC (rev 4266) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/journal/Journal.java 2011-03-03 21:27:02 UTC (rev 4267) @@ -1125,14 +1125,39 @@ /* * global row store. + */ + public SparseRowStore getGlobalRowStore() { + + return getGlobalRowStoreHelper().getGlobalRowStore(); + + } + + /** + * Return a view of the global row store as of the specified timestamp. This + * is mainly used to provide access to historical views. * + * @param timestamp + * The specified timestamp. + * + * @return The global row store view -or- <code>null</code> if no view + * exists as of that timestamp. + */ + public SparseRowStore getGlobalRowStore(final long timestamp) { + + return getGlobalRowStoreHelper().get(timestamp); + + } + + /** + * Return the {@link GlobalRowStoreHelper}. + * <p> * Note: An atomic reference provides us with a "lock" object which doubles * as a reference. We are not relying on its CAS properties. */ - public SparseRowStore getGlobalRowStore() { + private final GlobalRowStoreHelper getGlobalRowStoreHelper() { + + GlobalRowStoreHelper t = globalRowStoreHelper.get(); - GlobalRowStoreHelper t = globalRowStoreHelper.get(); - if (t == null) { synchronized (globalRowStoreHelper) { @@ -1151,14 +1176,14 @@ .set(t = new GlobalRowStoreHelper(this)); } - + } } - return globalRowStoreHelper.get().getGlobalRowStore(); + return globalRowStoreHelper.get(); + } - } final private AtomicReference<GlobalRowStoreHelper> globalRowStoreHelper = new AtomicReference<GlobalRowStoreHelper>(); /* Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/AbstractResource.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/AbstractResource.java 2011-03-03 18:38:51 UTC (rev 4266) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/AbstractResource.java 2011-03-03 21:27:02 UTC (rev 4267) @@ -47,7 +47,6 @@ import com.bigdata.journal.IIndexManager; import com.bigdata.journal.IResourceLock; import com.bigdata.journal.IResourceLockService; -import com.bigdata.rawstore.Bytes; import com.bigdata.rdf.rules.FastClosure; import com.bigdata.rdf.rules.FullClosure; import com.bigdata.rdf.rules.RuleFastClosure5; @@ -691,12 +690,12 @@ } // Write the map on the row store. - final Map afterMap = indexManager.getGlobalRowStore().write( - RelationSchema.INSTANCE, map); - + final Map<String, Object> afterMap = indexManager.getGlobalRowStore() + .write(RelationSchema.INSTANCE, map); + if(log.isDebugEnabled()) { - log.debug("Properties after write: "+afterMap); + log.debug("Properties after write: " + afterMap); } Deleted: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/locator/AbstractCachingResourceLocator.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/locator/AbstractCachingResourceLocator.java 2011-03-03 18:38:51 UTC (rev 4266) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/locator/AbstractCachingResourceLocator.java 2011-03-03 21:27:02 UTC (rev 4267) @@ -1,191 +0,0 @@ -/* - -Copyright (C) SYSTAP, LLC 2006-2008. All rights reserved. - -Contact: - SYSTAP, LLC - 4501 Tower Road - Greensboro, NC 27410 - lic...@bi... - -This program is free software; you can redistribute it and/or modify -it under the terms of the GNU General Public License as published by -the Free Software Foundation; version 2 of the License. - -This program 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 General Public License for more details. - -You should have received a copy of the GNU General Public License -along with this program; if not, write to the Free Software -Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA - -*/ -/* - * Created on Jun 30, 2008 - */ - -package com.bigdata.relation.locator; - -import java.lang.ref.WeakReference; -import java.util.concurrent.TimeUnit; - -import org.apache.log4j.Logger; - -import com.bigdata.cache.ConcurrentWeakValueCache; -import com.bigdata.cache.ConcurrentWeakValueCacheWithTimeout; -import com.bigdata.util.NT; - -/** - * Abstract base class for {@link IResourceLocator}s with caching. The cache - * uses {@link WeakReference}s so that cache entries will be cleared if the - * referenced item is cleared. - * - * @author <a href="mailto:tho...@us...">Bryan Thompson</a> - * @version $Id$ - */ -abstract public class AbstractCachingResourceLocator<T extends ILocatableResource> - implements IResourceLocator<T> { - - protected static final Logger log = Logger - .getLogger(AbstractCachingResourceLocator.class); - - protected static final boolean INFO = log.isInfoEnabled(); - - final private transient ConcurrentWeakValueCache<NT, T> cache; - - final private int capacity; - - /** - * The cache capacity. - */ - final public int capacity() { - - return capacity; - - } - - /** - * - * @param capacity - * The cache capacity. - * @param timeout - * The timeout in milliseconds for stale entries. - */ - protected AbstractCachingResourceLocator(final int capacity, - final long timeout) { - - if (capacity <= 0) - throw new IllegalArgumentException(); - - if (timeout < 0) - throw new IllegalArgumentException(); - - this.capacity = capacity; - -// this.cache = new WeakValueCache<NT, T>(new LRUCache<NT, T>(capacity)); - - this.cache = new ConcurrentWeakValueCacheWithTimeout<NT, T>(capacity, - TimeUnit.MILLISECONDS.toNanos(timeout)); - - } - - /** - * Looks up the resource in the cache (thread-safe since the underlying - * cache is thread-safe). - * - * @param namespace - * - * @param timestamp - * - * @return The relation -or- <code>null</code> iff it is not in the cache. - */ - protected T get(final String namespace, final long timestamp) { - - if (namespace == null) - throw new IllegalArgumentException(); - - final T r = cache.get(new NT(namespace, timestamp)); - - if (INFO) { - - log.info((r == null ? "miss" : "hit ") + ": namespace=" + namespace - + ", timestamp=" + timestamp); - - } - - return r; - - } - - /** - * Places the resource in the cache. - * <p> - * Note: The underlying cache is thread-safe. However, when adding an entry - * to the cache the caller MUST be synchronized on the named resource, use - * {@link #get(String, long)} to determine that there is no such entry in - * the cache, and then {@link #put(ILocatableResource)} the entry. - * <p> - * Note: Read committed views are allowed into the cache. - * <p> - * For a Journal, this depends on Journal#getIndex(name,timestamp) returning - * a ReadCommittedView for an index so that the view does in fact have - * read-committed semantics. - * <p> - * For a federation, read-committed semantics are achieved by the - * IClientIndex implementations since they always make standoff requests to - * one (or more) data services. Those requests allow the data service to - * resolve the then most recent view for the index for each request. - * - * @param resource - * The resource. - */ - protected void put(final T resource) { - - if (resource == null) - throw new IllegalArgumentException(); - - final String namespace = resource.getNamespace().toString(); - - final long timestamp = resource.getTimestamp(); - - if (INFO) { - - log.info("Caching: namespace=" + namespace + ", timestamp=" - + timestamp); - - } - - cache.put(new NT(namespace, timestamp), resource); - -// cache.put(new NT(namespace, timestamp), resource, false/* dirty */); - - } - - /** - * Clears any resource having the same namespace and timestamp from the - * cache. - * <p> - * Note: The caller MUST be synchronized on the named resource. - * - * @return <code>true</code> iff there was an entry in the cache for the - * same resource namespace and timestamp, in which case it was - * cleared from the cache. - */ - protected boolean clear(final String namespace, final long timestamp) { - - if (namespace == null) - throw new IllegalArgumentException(); - - if (cache.remove(new NT(namespace, timestamp)) != null) { - - return true; - - } - - return false; - - } - -} Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/locator/DefaultResourceLocator.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/locator/DefaultResourceLocator.java 2011-03-03 18:38:51 UTC (rev 4266) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/java/com/bigdata/relation/locator/DefaultResourceLocator.java 2011-03-03 21:27:02 UTC (rev 4267) @@ -33,24 +33,30 @@ import java.util.Properties; import java.util.UUID; import java.util.WeakHashMap; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import org.apache.log4j.Logger; import com.bigdata.btree.IIndex; +import com.bigdata.cache.ConcurrentWeakValueCache; +import com.bigdata.cache.ConcurrentWeakValueCacheWithTimeout; import com.bigdata.cache.LRUCache; import com.bigdata.concurrent.NamedLock; import com.bigdata.journal.AbstractTask; +import com.bigdata.journal.ICommitRecord; import com.bigdata.journal.IIndexManager; import com.bigdata.journal.IIndexStore; import com.bigdata.journal.Journal; import com.bigdata.journal.TemporaryStore; +import com.bigdata.journal.TimestampUtility; import com.bigdata.relation.AbstractResource; import com.bigdata.relation.IRelation; import com.bigdata.relation.RelationSchema; import com.bigdata.service.IBigdataFederation; import com.bigdata.sparse.SparseRowStore; +import com.bigdata.util.NT; /** * Generic implementation relies on a ctor for the resource with the following @@ -108,21 +114,27 @@ * @param <T> * The generic type of the [R]elation. */ -public class DefaultResourceLocator<T extends ILocatableResource> extends - AbstractCachingResourceLocator<T> implements IResourceLocator<T> { +public class DefaultResourceLocator<T extends ILocatableResource> // + implements IResourceLocator<T> { protected static final transient Logger log = Logger .getLogger(DefaultResourceLocator.class); - - protected static final boolean INFO = log.isInfoEnabled(); - - protected static final boolean DEBUG = log.isDebugEnabled(); protected final transient IIndexManager indexManager; private final IResourceLocator<T> delegate; - + /** + * Cache for recently located resources. + */ + final private transient ConcurrentWeakValueCache<NT, T> resourceCache; + + /** + * Cache for recently materialized properties from the GRS. + */ + final /*private*/ transient ConcurrentWeakValueCache<NT, Map<String,Object>> propertyCache; + + /** * Provides locks on a per-namespace basis for higher concurrency. */ private final transient NamedLock<String> namedLock = new NamedLock<String>(); @@ -170,8 +182,6 @@ final IResourceLocator<T> delegate, final int cacheCapacity, final long cacheTimeout) { - super(cacheCapacity, cacheTimeout); - if (indexManager == null) throw new IllegalArgumentException(); @@ -179,6 +189,18 @@ this.delegate = delegate;// MAY be null. + if (cacheCapacity <= 0) + throw new IllegalArgumentException(); + + if (cacheTimeout < 0) + throw new IllegalArgumentException(); + + this.resourceCache = new ConcurrentWeakValueCacheWithTimeout<NT, T>( + cacheCapacity, TimeUnit.MILLISECONDS.toNanos(cacheTimeout)); + + this.propertyCache = new ConcurrentWeakValueCacheWithTimeout<NT, Map<String, Object>>( + cacheCapacity, TimeUnit.MILLISECONDS.toNanos(cacheTimeout)); + } // @todo hotspot 2% total query time. @@ -187,46 +209,86 @@ if (namespace == null) throw new IllegalArgumentException(); - if (INFO) { + if (log.isInfoEnabled()) { - log.info("namespace=" + namespace+", timestamp="+timestamp); + log.info("namespace=" + namespace + ", timestamp=" + timestamp); } + T resource = null; + final NT nt; + + /* + * Note: The drawback with resolving the resource against the + * [commitTime] is that the views will be the same object instance and + * will have the timestamp associated with the [commitTime] rather than + * the caller's timestamp. This breaks the assumption that + * resource#getTimestamp() returns the transaction identifier for a + * read-only transaction. In order to fix that, we resort to sharing the + * Properties object instead of the resource view. + */ +// if (TimestampUtility.isReadOnly(timestamp) +// && indexManager instanceof Journal) { +// +// /* +// * If we are looking on a local Journal (standalone database) then +// * we resolve the caller's [timestamp] to the commit point against +// * which the resource will be located and handle caching of the +// * resource using that commit point. This is done in order to share +// * a read-only view of a resource with any request which would be +// * serviced by the same commit point. Any such views are read-only +// * and immutable. +// */ +// +// final Journal journal = (Journal) indexManager; +// +// // find the commit record on which we need to read. +// final long commitTime = journal.getCommitRecord( +// TimestampUtility.asHistoricalRead(timestamp)) +// .getTimestamp(); +// +// nt = new NT(namespace, commitTime); +// +// } else { + + nt = new NT(namespace, timestamp); +// +// } + // test cache: hotspot 93% of method time. - T resource = get(namespace, timestamp); + resource = resourceCache.get(nt); if (resource != null) { - if (DEBUG) + if (log.isDebugEnabled()) log.debug("cache hit: " + resource); // cache hit. return resource; } - + /* - * Since there was a cache miss, acquire a lock the named relation so - * that the locate + cache.put sequence will be atomic. + * Since there was a cache miss, acquire a lock for the named relation + * so that the locate + cache.put sequence will be atomic. */ final Lock lock = namedLock.acquireLock(namespace); try { // test cache now that we have the lock. - resource = get(namespace, timestamp); + resource = resourceCache.get(nt); if (resource != null) { - if (DEBUG) + if (log.isDebugEnabled()) log.debug("cache hit: " + resource); return resource; } - if (INFO) + if (log.isInfoEnabled()) log.info("cache miss: namespace=" + namespace + ", timestamp=" + timestamp); @@ -250,7 +312,7 @@ * resolve this request. */ - if(INFO) { + if(log.isInfoEnabled()) { log.info("Not found - passing to delegate: namespace=" + namespace + ", timestamp=" + timestamp); @@ -262,7 +324,7 @@ if (resource != null) { - if (INFO) { + if (log.isInfoEnabled()) { log.info("delegate answered: " + resource); @@ -275,15 +337,15 @@ } if (log.isInfoEnabled()) - log.info("Not found: namespace=" + namespace + ", timestamp=" - + timestamp); + log.info("Not found: namespace=" + namespace + + ", timestamp=" + timestamp); // not found. return null; } - if (DEBUG) { + if (log.isDebugEnabled()) { log.debug(properties.toString()); @@ -310,7 +372,7 @@ } - if (DEBUG) { + if (log.isDebugEnabled()) { log.debug("Implementation class=" + cls.getName()); @@ -321,8 +383,8 @@ properties); // Add to the cache. - put(resource); - + resourceCache.put(nt, resource); + return resource; } finally { @@ -373,7 +435,7 @@ * and removed from the [seeAlso] weak value cache. */ - if (INFO) + if (log.isInfoEnabled()) log.info("Closed? " + indexManager); } else { @@ -400,7 +462,7 @@ if (properties != null) { - if (INFO) { + if (log.isInfoEnabled()) { log.info("Found: namespace=" + namespace + " on " + indexManager); @@ -428,7 +490,7 @@ if (properties != null) { - if (INFO) { + if (log.isInfoEnabled()) { log.info("Found: namespace=" + namespace + " on " + indexManager); @@ -460,28 +522,19 @@ * @param indexManager * @param namespace * The resource identifier - this is the primary key. - * @param timestampIsIgnored + * @param timestamp * The timestamp of the resource view. * * @return The {@link Properties} iff there is a logical row for the given * namespace. - * - * @todo The timestamp of the resource view is currently ignored. This - * probably should be modified to use the corresponding view of the - * global row store rather than always using the read-committed / - * unisolated view. That would make the properties immutable for a - * historical resource view and thus more easily cached. However, - * it would also make it impossible to modify those properties for - * historical views as any changes would only apply to views whose - * commit time was after the change to the global row store. */ protected Properties locateResourceOn(final IIndexManager indexManager, - final String namespace, final long timestampIsIgnored) { + final String namespace, final long timestamp) { - if (INFO) { + if (log.isInfoEnabled()) { log.info("indexManager=" + indexManager + ", namespace=" - + namespace + ", timestamp=" + timestampIsIgnored); + + namespace + ", timestamp=" + timestamp); } @@ -489,23 +542,105 @@ * Look at the global row store view corresponding to the specified * timestamp. * - * @todo caching may be useful here for historical reads. + * Note: caching here is important in order to reduce the heap pressure + * associated with large numbers of concurrent historical reads against + * the same commit point when those reads are performed within read-only + * transactions and, hence, each read is performed with a DISTINCT + * timestamp. Since the timestamps are distinct, the resource [cache] + * will have cache misses. This code provides for a [propertyCache] + * which ensures that we share the materialized properties from the GRS + * across resource views backed by the same commit point (and also avoid + * unnecessary GRS reads). */ - final SparseRowStore rowStore = indexManager - .getGlobalRowStore(/*timestamp*/); - - final Map<String, Object> map = rowStore == null ? null : rowStore - .read(RelationSchema.INSTANCE, namespace); + final Map<String, Object> map; + if (TimestampUtility.isReadOnly(timestamp) + && !TimestampUtility.isReadCommitted(timestamp) + && indexManager instanceof Journal) { -// System.err.println("Reading properties: namespace="+namespace+", timestamp="+timestampIsIgnored); -// log.fatal("Reading properties: "+namespace,new RuntimeException()); - + final Journal journal = (Journal) indexManager; + + // find the commit record on which we need to read. + final ICommitRecord commitRecord = journal + .getCommitRecord(TimestampUtility + .asHistoricalRead(timestamp)); + + if (commitRecord != null) { + + // find the timestamp associated with that commit record. + final long commitTime = commitRecord.getTimestamp(); + + // Check the cache before materializing the properties from the + // GRS. + final Map<String, Object> cachedMap = propertyCache.get(new NT( + namespace, commitTime)); + + if (cachedMap != null) { + + // The properties are in the cache. + map = cachedMap; + + } else { + + // Use the GRS view as of that commit point. + final SparseRowStore rowStore = journal + .getGlobalRowStore(commitTime); + + // Read the properties from the GRS. + map = rowStore == null ? null : rowStore.read( + RelationSchema.INSTANCE, namespace); + + if (map != null) { + + // Stuff the properties into the cache. + propertyCache.put(new NT(namespace, commitTime), map); + + } + + } + + } else { + + /* + * No such commit record. + * + * @todo We can probably just return [null] for this case. + */ + + final SparseRowStore rowStore = indexManager + .getGlobalRowStore(/* timestamp */); + + // Read the properties from the GRS. + map = rowStore == null ? null : rowStore.read( + RelationSchema.INSTANCE, namespace); + + } + + } else { + + /* + * @todo The timestamp of the resource view is currently ignored. + * This probably should be modified to use the corresponding view of + * the global row store rather than always using the read-committed + * / unisolated view, which will require exposing a + * getGlobalRowStore(timestamp) method on IIndexStore. + */ + + final SparseRowStore rowStore = indexManager + .getGlobalRowStore(/* timestamp */); + + // Read the properties from the GRS. + map = rowStore == null ? null : rowStore.read( + RelationSchema.INSTANCE, namespace); + + } + if (map == null) { - if (DEBUG) { + if (log.isDebugEnabled()) { - log.debug("No properties: indexManager=" + indexManager - + ", namespace=" + namespace); + log.debug("Not found: indexManager=" + indexManager + + ", namespace=" + namespace + ", timestamp=" + + timestamp); } @@ -513,21 +648,23 @@ } + // wrap with properties object to prevent cross view mutation. final Properties properties = new Properties(); properties.putAll(map); - if (DEBUG) { + if (log.isTraceEnabled()) { - log.debug("Read properties: indexManager=" + indexManager - + ", namespace=" + namespace + " :: " + properties); + log.trace("Read properties: indexManager=" + indexManager + + ", namespace=" + namespace + ", timestamp=" + timestamp + + " :: " + properties); } return properties; } - + /** * Create a new view of the relation. * @@ -588,7 +725,7 @@ r.init(); - if(INFO) { + if(log.isInfoEnabled()) { log.info("new instance: "+r); @@ -614,7 +751,7 @@ * @param instance * The instance. */ - public T putInstance(T instance) { + public T putInstance(final T instance) { if (instance == null) throw new IllegalArgumentException(); @@ -623,7 +760,7 @@ final long timestamp = instance.getTimestamp(); - if (INFO) { + if (log.isInfoEnabled()) { log.info("namespace=" + namespace+", timestamp="+timestamp); @@ -634,11 +771,13 @@ try { - final T tmp = get(namespace, timestamp); + final NT nt = new NT(namespace, timestamp); + final T tmp = resourceCache.get(nt); + if (tmp != null) { - if(INFO) { + if(log.isInfoEnabled()) { log.info("Existing instance already in cache: "+tmp); @@ -648,10 +787,10 @@ } - put(instance); - - if (INFO) { + resourceCache.put(nt, instance); + if (log.isInfoEnabled()) { + log.info("Instance added to cache: " + instance); } @@ -665,15 +804,15 @@ } } - + /** - * Resources that hold hard references to local index objects MUST discarded - * during abort processing. Otherwise the same resource objects will be - * returned from the cache and buffered writes on the indices for those - * relations (if they are local index objects) will still be visible, this - * defeating the abort semantics. + * Resources that hold hard references to local index objects MUST be + * discarded during abort processing. Otherwise the same resource objects + * will be returned from the cache and buffered writes on the indices for + * those relations (if they are local index objects) will still be visible, + * thus defeating the abort semantics. */ - public void discard(ILocatableResource<T> instance) { + public void discard(final ILocatableResource<T> instance) { if (instance == null) throw new IllegalArgumentException(); @@ -682,9 +821,9 @@ final long timestamp = instance.getTimestamp(); - if (INFO) { + if (log.isInfoEnabled()) { - log.info("namespace=" + namespace+", timestamp="+timestamp); + log.info("namespace=" + namespace + ", timestamp=" + timestamp); } @@ -693,10 +832,16 @@ try { - final boolean found = clear(namespace, timestamp); - - if (INFO) { + final NT nt = new NT(namespace, timestamp); + /* + * Clear the resource cache, but we do not need to clear the + * property cache since it only retains immutable historical state. + */ + final boolean found = resourceCache.remove(nt) != null; + + if (log.isInfoEnabled()) { + log.info("instance=" + instance + ", found=" + found); } @@ -744,7 +889,7 @@ */ seeAlso.put(indexManager, null); - if (INFO) { + if (log.isInfoEnabled()) { log.info("size=" + seeAlso.size() + ", added indexManager=" + indexManager); Modified: branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/relation/locator/TestDefaultResourceLocator.java =================================================================== --- branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/relation/locator/TestDefaultResourceLocator.java 2011-03-03 18:38:51 UTC (rev 4266) +++ branches/QUADS_QUERY_BRANCH/bigdata/src/test/com/bigdata/relation/locator/TestDefaultResourceLocator.java 2011-03-03 21:27:02 UTC (rev 4267) @@ -28,13 +28,14 @@ package com.bigdata.relation.locator; +import java.lang.ref.WeakReference; +import java.nio.ByteBuffer; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import junit.framework.TestCase2; @@ -49,11 +50,11 @@ import com.bigdata.journal.Journal.Options; import com.bigdata.rdf.spo.ISPO; import com.bigdata.relation.AbstractRelation; -import com.bigdata.relation.accesspath.IAccessPath; +import com.bigdata.relation.AbstractResource; import com.bigdata.service.IBigdataFederation; import com.bigdata.striterator.IChunkedOrderedIterator; import com.bigdata.striterator.IKeyOrder; -import com.bigdata.util.concurrent.DaemonThreadFactory; +import com.bigdata.util.NT; /** * Test suite for location relations, etc. @@ -102,8 +103,6 @@ final Journal store = new Journal( properties ); - final ExecutorService executorService = Executors.newCachedThreadPool(DaemonThreadFactory.defaultThreadFactory()); - final String namespace = "test"; try { @@ -221,12 +220,239 @@ store.destroy(); - executorService.shutdownNow(); - } } + /** + * Unit test for property caching for locatable resources. + */ + public void test_propertyCache() { + + final Properties properties = getProperties(); + + final Journal store = new Journal( properties ); + + final String namespace = "test"; + + try { + + // write a small record onto the journal and force a commit. + { + final ByteBuffer b = ByteBuffer.allocate(4); + b.putInt(0); + b.flip(); + store.write(b); + assertNotSame(0L,store.commit()); + } + + // verify resource can not be located yet. + { + // resource does not exist in UNISOLATED view. + assertNull(store.getResourceLocator().locate(namespace, + ITx.UNISOLATED)); + + // resource does not exist at lastCommitTime. + assertNull(store.getResourceLocator().locate(namespace, + store.getLastCommitTime())); + + // resource does not exist at read-only tx. + { + final long tx = store.newTx(ITx.READ_COMMITTED); + try { + assertNull(store.getResourceLocator().locate(namespace, + store.getLastCommitTime())); + } finally { + store.abort(tx); + } + } + } + + // instantiate relation. + MockRelation mockRelation = new MockRelation(store, namespace, + ITx.UNISOLATED, properties); + + // verify resource still can not be located. + { + // resource does not exist in UNISOLATED view. + assertNull(store.getResourceLocator().locate(namespace, + ITx.UNISOLATED)); + + // resource does not exist at lastCommitTime. + assertNull(store.getResourceLocator().locate(namespace, + store.getLastCommitTime())); + + // resource does not exist at read-only tx. + { + final long tx = store.newTx(ITx.READ_COMMITTED); + try { + assertNull(store.getResourceLocator().locate(namespace, + store.getLastCommitTime())); + } finally { + store.abort(tx); + } + } + } + + // create the resource, which writes the properties into the GRS. + mockRelation.create(); + + /* + */ + { + + /* + * The UNISOLATED view of the resource should be locatable now + * since the writes on the global row store are unisolated. + */ + assertNotNull(store.getResourceLocator().locate(namespace, + ITx.UNISOLATED)); + + // a request for the unisolated view gives us the same instance. + assertTrue(store.getResourceLocator().locate(namespace, + ITx.UNISOLATED) == mockRelation); + + /* + * The read-committed view of the resource is also locatable. + */ + assertNotNull(store.getResourceLocator().locate(namespace, + ITx.READ_COMMITTED)); + + /* + * The read committed view is not the same instance as the + * unisolated view. + */ + assertTrue(((MockRelation) store.getResourceLocator().locate( + namespace, ITx.READ_COMMITTED)) != mockRelation); + + } + + // commit time immediately proceeding this commit. + final long priorCommitTime = store.getLastCommitTime(); + + // commit, noting the commit time. + final long lastCommitTime = store.commit(); + + if(log.isInfoEnabled()) { + log.info("priorCommitTime=" + priorCommitTime); + log.info("lastCommitTime =" + lastCommitTime); + } + + /* + * Now create a few transactions against the newly created resource + * and verify that the views returned for those transactions are + * distinct, but that they share the same set of default Properties + * (e.g., the propertyCache is working). + * + * @todo also test a read-historical read ! + */ + final long tx1 = store.newTx(store.getLastCommitTime()); // read-only tx + final long tx2 = store.newTx(store.getLastCommitTime()); // read-only tx + final long ts1 = store.getLastCommitTime() - 1; // historical read + try { + + assertTrue(tx1 != tx2); + assertTrue(ts1 != tx1); + assertTrue(ts1 != tx2); + + /* + * @todo There might not be enough commit latency to have + * lastCommitTime - 1 be GT priorCommitTime. If this happens + * either resolve issue 145 or add some latency into the test. + * + * @see http://sourceforge.net/apps/trac/bigdata/ticket/145 + */ + assertTrue(ts1 > priorCommitTime); + + // unisolated view. + final AbstractResource<?> view_un = (AbstractResource<?>) store + .getResourceLocator().locate(namespace, ITx.UNISOLATED); + assertNotNull(view_un); + + // tx1 view. + final AbstractResource<?> view_tx1 = (AbstractResource<?>) store + .getResourceLocator().locate(namespace, tx1); + assertNotNull(view_tx1); + + // tx2 view. + final AbstractResource<?> view_tx2 = (AbstractResource<?>) store + .getResourceLocator().locate(namespace, tx2); + assertNotNull(view_tx2); + + // all views are distinct. + assertTrue(view_un != view_tx1); + assertTrue(view_un != view_tx2); + assertTrue(view_tx1 != view_tx2); + + // each view has its correct timestamp. + assertEquals(ITx.UNISOLATED, view_un.getTimestamp()); + assertEquals(tx1, view_tx1.getTimestamp()); + assertEquals(tx2, view_tx2.getTimestamp()); + + // each view has its own Properties object. + final Properties p_un = view_un.getProperties(); + final Properties p_tx1 = view_tx1.getProperties(); + final Properties p_tx2 = view_tx2.getProperties(); + assertTrue(p_un != p_tx1); + assertTrue(p_un != p_tx2); + assertTrue(p_tx1 != p_tx2); + + /* + * Verify that the [propertyCache] is working. + * + * Note: Unfortunately, I have not been able to devise any means + * of testing the [propertyCache] without exposing that as a + * package private object. + */ + final DefaultResourceLocator<?> locator = (DefaultResourceLocator<?>) store + .getResourceLocator(); + + // Not cached for the UNISOLATED view (mutable views can not be + // cached). + assertNull(locator.propertyCache.get(new NT(namespace, + ITx.UNISOLATED))); + +// if (true) { +// final Iterator<Map.Entry<NT, WeakReference<Map<String, Object>>>> itr = locator.propertyCache +// .entryIterator(); +// while (itr.hasNext()) { +// final Map.Entry<NT, WeakReference<Map<String, Object>>> e = itr +// .next(); +// System.err.println(e.getKey() + " => " +// + e.getValue().get()); +// } +// } + + // Not cached for the actual tx ids or read-only timestamp. + assertNull(locator.propertyCache.get(new NT(namespace,tx1))); + assertNull(locator.propertyCache.get(new NT(namespace,tx2))); + assertNull(locator.propertyCache.get(new NT(namespace,ts1))); + + /* + * Cached for the last commit time, which should have been used + * to hand back the Properties for {tx1, tx2, ts1}. + */ + assertNotNull(locator.propertyCache.get(new NT(namespace, + lastCommitTime))); + + // nothing for the prior commit time. + assertNull(locator.propertyCache.get(new NT(namespace, + priorCommitTime))); + + } finally { + store.abort(tx1); + store.abort(tx2); + } + + } finally { + + store.destroy(); + + } + + } + + @SuppressWarnings("unchecked") private static class MockRelation extends AbstractRelation { static final private String indexName = "foo"; @@ -298,66 +524,41 @@ @Override public String getFQN(IKeyOrder keyOrder) { - // TODO Auto-generated method stub return null; } public long delete(IChunkedOrderedIterator itr) { - // TODO Auto-generated method stub return 0; } public long insert(IChunkedOrderedIterator itr) { - // TODO Auto-generated method stub return 0; } -// public IAccessPath getAccessPath(IPredicate predicate) { -// // TODO Auto-generated method stub -// return null; -// } - - public long getElementCount(boolean exact) { - // TODO Auto-generated method stub - return 0; - } - public Set getIndexNames() { - // TODO Auto-generated method stub return null; } - public IKeyOrder getPrimaryKeyOrder() { - // TODO Auto-generated method stub return null; } public Iterator getKeyOrders() { - // TODO Auto-generated method stub return null; } public IKeyOrder getKeyOrder(IPredicate p) { - // TODO Auto-generated method stub return null; } - -// public Object newElement(IPredicate predicate, IBindingSet bindingSet) { -// // TODO Auto-generated method stub -// return null; -// } public Object newElement(List a, IBindingSet bindingSet) { - // TODO Auto-generated method stub return null; } public Class<ISPO> getElementClass() { - return null; + } - } - } + } // class MockRelation } This was sent by the SourceForge.net collaborative development platform, the world's largest Open Source development site. |