You are viewing a plain text version of this content. The canonical link for it is here.
Posted to derby-commits@db.apache.org by ka...@apache.org on 2007/03/01 09:09:30 UTC

svn commit: r513222 - in /db/derby/code/trunk/java: engine/org/apache/derby/diag/ engine/org/apache/derby/iapi/services/loader/ engine/org/apache/derby/iapi/services/locks/ engine/org/apache/derby/iapi/store/access/ engine/org/apache/derby/iapi/store/r...

Author: kahatlen
Date: Thu Mar  1 00:09:28 2007
New Revision: 513222

URL: http://svn.apache.org/viewvc?view=rev&rev=513222
Log:
DERBY-2328: Reduce monitor contention in SinglePool

Removed the Hashtable in SinglePool and made each transaction have a
direct pointer to its lock space.

Description of the changes:
  - New method in LockFactory: createCompatibilitySpace()
  - New (almost empty) interface CompatibilitySpace to be returned
    from createCompatibilitySpace()
  - LockSpace implements CompatibilitySpace and is returned by
    SinglePool.createCompatibilitySpace()
  - SinglePool no longer extends Hashtable. Now it casts the supplied
    compatibility space object to LockSpace and uses it directly.
  - Modified signatures of all methods/variables that used
    compatibility spaces (Object -> CompatibilitySpace)
  - Modified code that assumed compatibility space objects were
    transaction objects (virtual lock table and code that locked an
    object with the transaction as lock group)
  - Code that tested space1.equals(space2) was changed to
    (space1 == space2)

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/diag/LockTable.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/loader/ClassFactoryContext.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/Latch.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/Limit.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/LockFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Transaction.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/db/StoreClassFactoryContext.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/ActiveLock.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Control.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/D_LockControl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Deadlock.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Lock.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockControl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockSet.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockSpace.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/SinglePool.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/reflect/UpdateLoader.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/PropertyConglomerate.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainer.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java
    db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/services/T_LockFactory.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/services/T_User.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_TWC.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_Util.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/diag/LockTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/diag/LockTable.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/diag/LockTable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/diag/LockTable.java Thu Mar  1 00:09:28 2007
@@ -300,7 +300,9 @@
 		}
 
 		attributes.put(VirtualLockTable.LOCKOBJ, lock);
-		attributes.put(VirtualLockTable.XACTID, lock.getCompatabilitySpace().toString());
+		Object owner = lock.getCompatabilitySpace().getOwner();
+		attributes.put(VirtualLockTable.XACTID,
+					   (owner == null) ? "<null>" : owner.toString());
 		attributes.put(VirtualLockTable.LOCKMODE, lock_type.toString());
 
 		attributes.put(VirtualLockTable.LOCKCOUNT, Integer.toString(lockCount));

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/loader/ClassFactoryContext.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/loader/ClassFactoryContext.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/loader/ClassFactoryContext.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/loader/ClassFactoryContext.java Thu Mar  1 00:09:28 2007
@@ -23,6 +23,7 @@
 
 import org.apache.derby.iapi.services.context.ContextImpl;
 import org.apache.derby.iapi.services.context.ContextManager;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.property.PersistentSet;
 import org.apache.derby.iapi.error.ExceptionSeverity;
 import org.apache.derby.iapi.error.StandardException;
@@ -55,7 +56,7 @@
      * Used when the classpath changes or a database
      * jar file is installed, removed or replaced.
      */
-	public abstract Object getLockSpace() throws StandardException;
+    public abstract CompatibilitySpace getLockSpace() throws StandardException;
 
     /**
      * Get the set of properties stored with this service.

Added: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java?view=auto&rev=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java Thu Mar  1 00:09:28 2007
@@ -0,0 +1,45 @@
+/*
+
+   Derby - Class org.apache.derby.iapi.services.locks.CompatibilitySpace
+
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to you under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+
+*/
+
+package org.apache.derby.iapi.services.locks;
+
+/**
+ * <p> This interface must be implemented by objects returned from
+ * <code>LockFactory.createCompatibilitySpace()</code>. </p>
+ *
+ * <p> A <code>CompatibilitySpace</code> can have an owner (for instance a
+ * transaction). Currently, the owner is used by the virtual lock table to find
+ * out which transaction a lock belongs to. Some parts of the code also use the
+ * owner as a group object which guarantees that the lock is released on a
+ * commit or an abort. The owner has no special meaning to the lock manager and
+ * can be any object, including <code>null</code>. </p>
+ *
+ * @see LockFactory#createCompatibilitySpace
+ */
+public interface CompatibilitySpace {
+    /**
+     * Gets an object representing the owner of the compatibility space.
+     *
+     * @return object representing the owner of the compatibility space, or
+     * <code>null</code> if no owner has been specified.
+     */
+    Object getOwner();
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/Latch.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/Latch.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/Latch.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/Latch.java Thu Mar  1 00:09:28 2007
@@ -27,9 +27,9 @@
 public interface Latch {
 
 	/**	
-		Get the compatability space the latch is held in.
+		Get the compatibility space the latch is held in.
 	*/
-	public Object getCompatabilitySpace();
+	public CompatibilitySpace getCompatabilitySpace();
 
 	/**
 		Gte the object the latch is held on.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/Limit.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/Limit.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/Limit.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/Limit.java Thu Mar  1 00:09:28 2007
@@ -37,7 +37,7 @@
 	/**
 		Called by the lock factory when a limit has been reached.
 
-		@param compatabilitySpace lock space the limit was set for
+		@param compatibilitySpace lock space the limit was set for
 		@param group lock group the limit was set for
 		@param limit the limit's setting
 		@param lockList the list of Lockable's in the group
@@ -45,8 +45,8 @@
 
         @exception StandardException Standard Cloudscape error policy.
 	*/
-	public void reached(Object compatabilitySpace, Object group, int limit,
-		Enumeration lockList, int lockCount)
+	public void reached(CompatibilitySpace compatibilitySpace, Object group,
+						int limit, Enumeration lockList, int lockCount)
 		throws StandardException;
 
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/LockFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/LockFactory.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/LockFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/LockFactory.java Thu Mar  1 00:09:28 2007
@@ -38,26 +38,35 @@
 public interface LockFactory extends PropertySetCallback {
 
 	/**
-		Lock an object within a compatability space
+	 * Create an object which can be used as a compatibility space. A
+	 * compatibility space object can only be used in the
+	 * <code>LockFactory</code> that created it.
+	 *
+	 * @param owner the owner of the compatibility space (typically a
+	 * transaction object). Might be <code>null</code>.
+	 * @return an object which represents a compatibility space
+	 */
+	public CompatibilitySpace createCompatibilitySpace(Object owner);
+
+	/**
+		Lock an object within a compatibility space
 		and associate the lock with a group object,
 		waits up to timeout milli-seconds for the object to become unlocked. A 
         timeout of 0 means do not wait for the lock to be unlocked.
 		Note the actual time waited is approximate.
 		<P>
 		A compatibility space in an space where lock requests are assumed to be
-        compatabile and granted by the lock manager if the trio
-        {compatabilitySpace, ref, qualifier} are equal (i.e. reference equality
-        for qualifier, equals() method
-		for compatabilitySpace and ref ). A typical reference to use for the compatability
-		space is a reference to an object representing a transaction.
+        compatible and granted by the lock manager if the trio
+        {compatibilitySpace, ref, qualifier} are equal (i.e. reference equality
+        for qualifier and compatibilitySpace, equals() method for ref).
 		Granted by the lock manager means that the Lockable object may or may 
         not be queried to see if the request is compatible.
 		<BR>
-		A compatability space is not assumed to be owned by a single thread.
+		A compatibility space is not assumed to be owned by a single thread.
 	
 
 
-		@param compatabilitySpace object defining compatability space (by value)
+		@param compatibilitySpace object defining compatibility space
 		@param group handle of group, must be private to a thread.
 		@param ref reference to object to be locked
 		@param qualifier A qualification of the request.
@@ -74,38 +83,45 @@
 		@exception StandardException Standard Cloudscape error policy.
 
 	*/
-	public boolean lockObject(Object compatabilitySpace, Object group, Lockable ref, Object qualifier, int timeout)
+	public boolean lockObject(CompatibilitySpace compatibilitySpace,
+							  Object group, Lockable ref, Object qualifier,
+							  int timeout)
 		throws StandardException;
 
 	/**
-		Unlock a single lock on a single object held within this compatability space
-		that was locked with the supplied qualifier.
+		Unlock a single lock on a single object held within this compatibility
+		space and locked with the supplied qualifier.
 
-		@param compatabilitySpace object defining compatability space (by value)
+		@param compatibilitySpace object defining compatibility space
 		@param group handle of group.
 		@param ref Reference to object to be unlocked.
 		@param qualifier qualifier of lock to be unlocked
 
 		@return number of locks released (one or zero).
 	*/
-	public int unlock(Object compatabilitySpace, Object group, Lockable ref, Object qualifier);
+	public int unlock(CompatibilitySpace compatibilitySpace, Object group,
+					  Lockable ref, Object qualifier);
 
 	/**
 		Unlock all locks in a group. 
 
+		@param compatibilitySpace object defining compatibility space
 		@param group handle of group that objects were locked with.
 	*/
-	public void unlockGroup(Object compatabilitySpace, Object group);
+	public void unlockGroup(CompatibilitySpace compatibilitySpace,
+							Object group);
 
 	/**
 		Unlock all locks on a group that match the passed in value.
 	*/
-	public void unlockGroup(Object compatabilitySpace, Object group, Matchable key);
+	public void unlockGroup(CompatibilitySpace compatibilitySpace,
+							Object group, Matchable key);
 
 	/**
 		Transfer a set of locks from one group to another.
 	*/
-	public void transfer(Object compatabilitySpace, Object oldGroup, Object newGroup);
+	public void transfer(CompatibilitySpace compatibilitySpace,
+						 Object oldGroup, Object newGroup);
 
 	/**
 		Returns true if locks held by anyone are blocking anyone else
@@ -113,21 +129,22 @@
 	public boolean anyoneBlocked();
 
 	/**
-		Return true if locks are held in this compatability space and
+		Return true if locks are held in this compatibility space and
 		 this group.
 
 		@param group handle of group that objects were locked with.
 
 	*/
-	public boolean areLocksHeld(Object compatabilitySpace, Object group);
+	public boolean areLocksHeld(CompatibilitySpace compatibilitySpace,
+								Object group);
 
 	/**
-		Return true if locks are held in this compatability space.
+		Return true if locks are held in this compatibility space.
 	*/
-	public boolean areLocksHeld(Object compatabilitySpace);
+	public boolean areLocksHeld(CompatibilitySpace compatibilitySpace);
 
 	/**
-		Lock an object with zero duration within a compatability space,
+		Lock an object with zero duration within a compatibility space,
 		waits up to timeout milli-seconds for the object to become unlocked. A 
         timeout of 0 means do not wait for the lock to be unlocked.
 		Note the actual time waited is approximate.
@@ -135,19 +152,17 @@
 		Zero duration means the lock is released as soon as it is obtained.
 		<P>
 		A compatibility space in an space where lock requests are assumed to be
-        compatabile and granted by the lock manager if the trio
-        {compatabilitySpace, ref, qualifier} are equal (i.e. reference equality
-        for qualifier, equals() method
-		for compatabilitySpace and ref ). A typical reference to use for the compatability
-		space is a reference to an object representing a transaction.
+        compatible and granted by the lock manager if the trio
+        {compatibilitySpace, ref, qualifier} are equal (i.e. reference equality
+        for qualifier and compatibilitySpace, equals() method for ref).
 		Granted by the lock manager means that the Lockable object may or may 
         not be queried to see if the request is compatible.
 		<BR>
-		A compatability space is not assumed to be owned by a single thread.
+		A compatibility space is not assumed to be owned by a single thread.
 	
 
 
-		@param compatabilitySpace object defining compatability space (by value)
+		@param compatibilitySpace object defining compatibility space
 		@param ref reference to object to be locked
 		@param qualifier A qualification of the request.
 		@param timeout the maximum time to wait in milliseconds, LockFactory.NO_WAIT means don't wait.
@@ -163,13 +178,16 @@
 		@exception StandardException Standard Cloudscape error policy.
 
 	*/
-	public boolean zeroDurationlockObject(Object compatabilitySpace, Lockable ref, Object qualifier, int timeout)
+	public boolean zeroDurationlockObject(CompatibilitySpace compatibilitySpace,
+										  Lockable ref, Object qualifier,
+										  int timeout)
 		throws StandardException;
 
 	/**
 		Check to see if a specific lock is held.
 	*/
-	public boolean isLockHeld(Object compatabilitySpace, Object group, Lockable ref, Object qualifier);
+	public boolean isLockHeld(CompatibilitySpace compatibilitySpace,
+							  Object group, Lockable ref, Object qualifier);
 
 	/**
 		Install a limit that is called when the size of the group exceeds
@@ -187,12 +205,13 @@
 		Only one limit may be in place for a group at any time.
 		@see Limit
 	*/
-	public void setLimit(Object compatabilitySpace, Object group, int limit, Limit callback);
+	public void setLimit(CompatibilitySpace compatibilitySpace, Object group,
+						 int limit, Limit callback);
 
 	/**
 		Clear a limit set by setLimit.
 	*/
-	public void clearLimit(Object compatabilitySpace, Object group); 
+	public void clearLimit(CompatibilitySpace compatibilitySpace, Object group);
 
 	/**
 		Make a virtual lock table for diagnostics.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/TransactionController.java Thu Mar  1 00:09:28 2007
@@ -26,6 +26,7 @@
 import java.io.Serializable;
 
 import org.apache.derby.iapi.services.context.ContextManager;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.property.PersistentSet;
 import org.apache.derby.iapi.services.io.Storable;
 
@@ -1419,12 +1420,13 @@
 	*/
 	public FileResource getFileHandler();
 
-	/**
-		Return an object that when used as the compatability space *and*
-		group for a lock request, guarantees that the lock will be removed
-		on a commit or an abort.
-	*/
-	public Object getLockObject();
+    /**
+     * Return an object that when used as the compatibility space for a lock
+     * request, <strong>and</strong> the group object is the one returned by a
+     * call to <code>getOwner()</code> on that object, guarantees that the lock
+     * will be removed on a commit or an abort.
+     */
+    public CompatibilitySpace getLockSpace();
 
     /**
      * Return static information about the conglomerate to be included in a

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java Thu Mar  1 00:09:28 2007
@@ -23,6 +23,7 @@
 
 import org.apache.derby.iapi.services.daemon.DaemonService;
 import org.apache.derby.iapi.services.context.ContextManager;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.LockFactory;
 
 import org.apache.derby.iapi.services.property.PersistentSet;
@@ -611,7 +612,7 @@
 	*/
 
 	public Transaction startNestedReadOnlyUserTransaction(
-    Object         compatibilitySpace,
+    CompatibilitySpace compatibilitySpace,
     ContextManager contextMgr,
     String         transName)
         throws StandardException;

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Transaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Transaction.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Transaction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/Transaction.java Thu Mar  1 00:09:28 2007
@@ -22,6 +22,7 @@
 package org.apache.derby.iapi.store.raw;
 
 import org.apache.derby.iapi.services.daemon.Serviceable;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.store.raw.log.LogInstant;
 import org.apache.derby.iapi.store.access.FileResource;
@@ -55,7 +56,7 @@
      *
 	 * @return The compatibility space of the transaction.
      **/
-    Object getCompatibilitySpace();
+    CompatibilitySpace getCompatibilitySpace();
 
 
 	/**

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/xact/TransactionFactory.java Thu Mar  1 00:09:28 2007
@@ -24,6 +24,7 @@
 import org.apache.derby.iapi.services.context.ContextManager;
 import org.apache.derby.iapi.services.daemon.Serviceable;
 import org.apache.derby.iapi.services.io.Formatable;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.LockFactory;
 
 import org.apache.derby.iapi.store.access.TransactionInfo;
@@ -106,7 +107,7 @@
 	*/
 	public RawTransaction startNestedReadOnlyUserTransaction(
     RawStoreFactory rsf,
-    Object          compatibilitySpace,
+    CompatibilitySpace compatibilitySpace,
     ContextManager  contextMgr,
     String          transName)
         throws StandardException;

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/db/StoreClassFactoryContext.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/db/StoreClassFactoryContext.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/db/StoreClassFactoryContext.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/db/StoreClassFactoryContext.java Thu Mar  1 00:09:28 2007
@@ -24,6 +24,7 @@
 import org.apache.derby.iapi.services.loader.ClassFactoryContext;
 import org.apache.derby.iapi.services.loader.ClassFactory;
 import org.apache.derby.iapi.services.loader.JarReader;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.property.PersistentSet;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.store.access.AccessFactory;
@@ -42,11 +43,12 @@
 		this.jarReader = jarReader;
 	}
 
-	public Object getLockSpace() throws StandardException {
+	public CompatibilitySpace getLockSpace() throws StandardException {
 		if (store == null)
 			return null;
-		return store.getTransaction(getContextManager()).getLockObject();	
+		return store.getTransaction(getContextManager()).getLockSpace();
 	}
+
 	public PersistentSet getPersistentSet() throws StandardException {
 		if (store == null)
 			return null;

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/ActiveLock.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/ActiveLock.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/ActiveLock.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/ActiveLock.java Thu Mar  1 00:09:28 2007
@@ -21,6 +21,7 @@
 
 package org.apache.derby.impl.services.locks;
 
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.Lockable;
 import org.apache.derby.iapi.services.locks.C_LockFactory;
 
@@ -55,7 +56,7 @@
 	/**
 		If true then this lock can be granted even if
 		it is not the first lock request on the wait queue.
-		This can occur if the compatability space already holds
+		This can occur if the compatibility space already holds
 		a lock on the object.
 	*/
 	protected boolean canSkip;
@@ -66,7 +67,8 @@
 
 		MT - single thread required
 	*/
-	protected ActiveLock(Object space, Lockable ref, Object qualifier) {
+	protected ActiveLock(CompatibilitySpace space, Lockable ref,
+						 Object qualifier) {
 		super(space, ref, qualifier);
 	}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Control.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Control.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Control.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Control.java Thu Mar  1 00:09:28 2007
@@ -21,6 +21,7 @@
 
 package org.apache.derby.impl.services.locks;
 
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.Lockable;
 import org.apache.derby.iapi.services.locks.Latch;
 import java.util.List;
@@ -31,7 +32,8 @@
 
 	public LockControl getLockControl();
 
-	public Lock getLock(Object compatabilitySpace, Object qualifier);
+	public Lock getLock(CompatibilitySpace compatibilitySpace,
+						Object qualifier);
 
 //EXCLUDE-START-lockdiag- 
 	/**
@@ -55,7 +57,9 @@
 
 	public List getWaiting();
 
-	public boolean isGrantable(boolean otherWaiters, Object  compatabilitySpace, Object  qualifier);
+	public boolean isGrantable(boolean otherWaiters,
+							   CompatibilitySpace compatibilitySpace,
+							   Object qualifier);
 
 
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/D_LockControl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/D_LockControl.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/D_LockControl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/D_LockControl.java Thu Mar  1 00:09:28 2007
@@ -25,6 +25,7 @@
 import org.apache.derby.iapi.services.diag.DiagnosticUtil;
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.services.sanity.SanityManager;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.Lockable;
 
 import java.util.Properties;
@@ -116,40 +117,45 @@
 	** Debugging routines
 	*/
 
-	static void debugLock(String type, Object compatabilitySpace, Object group, Lockable ref, Object qualifier, int timeout) {
+	static void debugLock(String type, CompatibilitySpace compatibilitySpace,
+						  Object group, Lockable ref, Object qualifier,
+						  int timeout) {
 
 		if (SanityManager.DEBUG) {
 
 			SanityManager.DEBUG(Constants.LOCK_TRACE, type +
                 debugLockString(
-                    compatabilitySpace, group, ref, qualifier, timeout));
+                    compatibilitySpace, group, ref, qualifier, timeout));
 		}
 	}
-	static void debugLock(String type, Object compatabilitySpace, Object group) {
+	static void debugLock(String type, CompatibilitySpace compatibilitySpace,
+						  Object group) {
 
 		if (SanityManager.DEBUG) {
 
 			SanityManager.DEBUG(Constants.LOCK_TRACE, type +
-					debugLockString(compatabilitySpace, group));
+					debugLockString(compatibilitySpace, group));
 		}
 	}
-	static void debugLock(String type, Object compatabilitySpace, Object group, Lockable ref) {
+	static void debugLock(String type, CompatibilitySpace compatibilitySpace,
+						  Object group, Lockable ref) {
 
 		if (SanityManager.DEBUG) {
 
 			SanityManager.DEBUG(Constants.LOCK_TRACE, type +
-					debugLockString(compatabilitySpace, group, ref));
+					debugLockString(compatibilitySpace, group, ref));
 		}
 	}
 
 
-	static String debugLockString(Object compatabilitySpace, Object group) {
+	static String debugLockString(CompatibilitySpace compatibilitySpace,
+								  Object group) {
 
 		if (SanityManager.DEBUG) {
 
 			StringBuffer sb = new StringBuffer("");
 
-			debugAppendObject(sb, " CompatabilitySpace=", compatabilitySpace);
+			debugAppendObject(sb, " CompatibilitySpace=", compatibilitySpace);
 			debugAppendObject(sb, " Group=", group);
 
 			debugAddThreadInfo(sb);
@@ -161,14 +167,15 @@
 		}
 	}
 
-	static String debugLockString(Object compatabilitySpace, Object group, Lockable ref) {
+	static String debugLockString(CompatibilitySpace compatibilitySpace,
+								  Object group, Lockable ref) {
 
 		if (SanityManager.DEBUG) {
 
 			StringBuffer sb = new StringBuffer("");
 
 			debugAppendObject(sb, " Lockable ", ref);
-			debugAppendObject(sb, " CompatabilitySpace=", compatabilitySpace);
+			debugAppendObject(sb, " CompatibilitySpace=", compatibilitySpace);
 			debugAppendObject(sb, " Group=", group);
 
 			debugAddThreadInfo(sb);
@@ -181,7 +188,9 @@
 	}
 
 
-	static String debugLockString(Object compatabilitySpace, Object group, Lockable ref, Object qualifier, int timeout) {
+	static String debugLockString(CompatibilitySpace compatibilitySpace,
+								  Object group, Lockable ref,
+								  Object qualifier, int timeout) {
 
 		if (SanityManager.DEBUG) {
 
@@ -189,7 +198,7 @@
 
 			debugAppendObject(sb, " Lockable ", ref);
 			debugAppendObject(sb, " Qualifier=", qualifier);
-			debugAppendObject(sb, " CompatabilitySpace=", compatabilitySpace);
+			debugAppendObject(sb, " CompatibilitySpace=", compatibilitySpace);
 			debugAppendObject(sb, " Group=", group);
 
 			if (timeout >= 0) {

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Deadlock.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Deadlock.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Deadlock.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Deadlock.java Thu Mar  1 00:09:28 2007
@@ -208,13 +208,7 @@
 				break;
 			}
 
-			LockSpace ls = (LockSpace) factory.get(space);
-			if (ls == null) {
-				// space only holds latches, pick as victim
-				victim = space;
-				break;
-			}
-
+			LockSpace ls = (LockSpace) space;
 			int spaceCount = ls.deadlockCount(minLockCount);
 
 			if (spaceCount <= minLockCount) {

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Lock.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Lock.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Lock.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/Lock.java Thu Mar  1 00:09:28 2007
@@ -21,6 +21,7 @@
 
 package org.apache.derby.impl.services.locks;
 
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.Lockable;
 import org.apache.derby.iapi.services.locks.Latch;
 
@@ -41,7 +42,7 @@
 		Compatibility space the object is locked in.
 		MT - immutable - reference only
 	*/
-	private final Object	space;
+	private final CompatibilitySpace space;
 
 	/**
 		Object being locked.
@@ -56,7 +57,7 @@
 
 	int count;
 
-	protected Lock(Object space, Lockable ref, Object qualifier) {
+	protected Lock(CompatibilitySpace space, Lockable ref, Object qualifier) {
 		super();
 		this.space = space;
 		this.ref = ref;
@@ -73,11 +74,11 @@
 	}
 
 	/**
-		Return the compatability space this lock is held in
+		Return the compatibility space this lock is held in.
 
 		MT - Thread safe
 	*/
-	public final Object getCompatabilitySpace() {
+	public final CompatibilitySpace getCompatabilitySpace() {
 		return space;
 	}
 
@@ -148,7 +149,8 @@
 		if (other instanceof Lock) {
 			Lock ol = (Lock) other;
 
-			return (space.equals(ol.space)) && ref.equals(ol.ref) && (qualifier == ol.qualifier);
+			return (space == ol.space) && ref.equals(ol.ref)
+				&& (qualifier == ol.qualifier);
 		}
 
 		return false;
@@ -162,9 +164,11 @@
 		return new LockControl(this, ref);
 	}
 
-	public Lock getLock(Object compatabilitySpace, Object qualifier) {
-		if (space.equals(compatabilitySpace) && (this.qualifier == qualifier))
+	public Lock getLock(CompatibilitySpace compatibilitySpace,
+						Object qualifier) {
+		if ((space == compatibilitySpace) && (this.qualifier == qualifier)) {
 			return this;
+		}
 		return null;
 	}
 
@@ -216,11 +220,13 @@
 		return null;
 	}
 
-    public boolean isGrantable(boolean noWaitersBeforeMe, Object compatabilitySpace, Object  requestQualifier)
+    public boolean isGrantable(boolean noWaitersBeforeMe,
+                               CompatibilitySpace compatibilitySpace,
+                               Object requestQualifier)
     {
-        boolean sameSpace = space.equals(compatabilitySpace);
-		if (sameSpace && ref.lockerAlwaysCompatible())
+		if ((space == compatibilitySpace) && ref.lockerAlwaysCompatible()) {
 			return true;
+		}
 
 		return ref.requestCompatible(requestQualifier, this.qualifier);
 	}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockControl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockControl.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockControl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockControl.java Thu Mar  1 00:09:28 2007
@@ -21,6 +21,7 @@
 
 package org.apache.derby.impl.services.locks;
 
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.Lockable;
 import org.apache.derby.iapi.services.locks.Latch;
 
@@ -192,7 +193,7 @@
 
     public boolean isGrantable(
     boolean noWaitersBeforeMe,
-    Object  compatabilitySpace, 
+    CompatibilitySpace compatibilitySpace,
     Object  qualifier)
     {
 		if (isUnlocked())
@@ -215,7 +216,7 @@
 				Lock gl = firstGrant == null ? (Lock) lgranted.get(index) : firstGrant;
 
                 boolean sameSpace = 
-                    (gl.getCompatabilitySpace().equals(compatabilitySpace));
+                    (gl.getCompatabilitySpace() == compatibilitySpace);
 
                 if (sameSpace && selfCompatible) 
                 {
@@ -267,7 +268,8 @@
 		1) The Lockable has just been unlocked, 
 	*/
 
-	public Lock addLock(LockSet ls, Object compatabilitySpace, Object qualifier) {
+	public Lock addLock(LockSet ls, CompatibilitySpace compatibilitySpace,
+						Object qualifier) {
 
 		if (SanityManager.DEBUG) {
 
@@ -303,7 +305,8 @@
 				Lock gl = firstGrant == null ? (Lock) granted.get(index) : firstGrant;
 
 
-				boolean sameSpace = (gl.getCompatabilitySpace().equals(compatabilitySpace));
+				boolean sameSpace =
+					(gl.getCompatabilitySpace() == compatibilitySpace);
 
 				// if it's one of our locks and we are always compatible with 
                 // our own locks then yes, we can be granted.
@@ -357,14 +360,15 @@
 		}
 
 		if (grantLock) {
-			lockItem = new Lock(compatabilitySpace, lref, qualifier);
+			lockItem = new Lock(compatibilitySpace, lref, qualifier);
 			grant(lockItem);
 			return lockItem;
 		}
 		
-		ActiveLock waitingLock = new ActiveLock(compatabilitySpace, lref, qualifier);
+		ActiveLock waitingLock =
+			new ActiveLock(compatibilitySpace, lref, qualifier);
 
-		// If the object is already locked by this compatability space
+		// If the object is already locked by this compatibility space
 		// then this lock can be granted by skipping other waiters.
 		if (spaceHasALock) {
 			waitingLock.canSkip = true;
@@ -571,7 +575,8 @@
 	/**
 		Find a granted lock matching this space and qualifier
 	*/
-	public final Lock getLock(Object compatabilitySpace, Object qualifier) {
+	public final Lock getLock(CompatibilitySpace compatibilitySpace,
+							  Object qualifier) {
 
 		if (isUnlocked())
 			return null;
@@ -585,7 +590,7 @@
 
 			Lock gl = firstGrant == null ? (Lock) lgranted.get(index) : firstGrant;
 
-            if (!gl.getCompatabilitySpace().equals(compatabilitySpace))
+            if (gl.getCompatabilitySpace() != compatibilitySpace)
 				continue;
 
 			if (gl.getQualifier() == qualifier)

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockSet.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockSet.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockSet.java Thu Mar  1 00:09:28 2007
@@ -21,6 +21,7 @@
 
 package org.apache.derby.impl.services.locks;
 
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.Latch;
 import org.apache.derby.iapi.services.locks.Lockable;
 import org.apache.derby.iapi.services.locks.C_LockFactory;
@@ -118,7 +119,7 @@
 	/**
 	 *	Lock an object within a specific compatibility space.
 	 *
-	 *	@param	compatabilitySpace Compatibility space.
+	 *	@param	compatibilitySpace Compatibility space.
 	 *	@param	ref Lockable reference.
 	 *	@param	qualifier Qualifier.
 	 *	@param	timeout Timeout in milli-seconds
@@ -128,7 +129,7 @@
 	 *	@exception	StandardException Standard Cloudscape policy.
 
 	*/
-	public Lock lockObject(Object compatabilitySpace, Lockable ref,
+	public Lock lockObject(CompatibilitySpace compatibilitySpace, Lockable ref,
 						   Object qualifier, int timeout)
 		throws StandardException
 	{		
@@ -154,7 +155,7 @@
 			if (gc == null) {
 
 				// object is not locked, can be granted
-				Lock gl = new Lock(compatabilitySpace, ref, qualifier);
+				Lock gl = new Lock(compatibilitySpace, ref, qualifier);
 
 				gl.grant();
 
@@ -181,7 +182,7 @@
                 }
 			}
 
-			lockItem = control.addLock(this, compatabilitySpace, qualifier);
+			lockItem = control.addLock(this, compatibilitySpace, qualifier);
 
 			if (lockItem.getCount() != 0) {
 				return lockItem;
@@ -300,7 +301,7 @@
 
                         if (control.isGrantable(
                                 control.firstWaiter() == waitingLock,
-                                compatabilitySpace, 
+                                compatibilitySpace,
                                 qualifier)) {
 
                             // Yes, we are granted, put us on the granted queue.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockSpace.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockSpace.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockSpace.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/LockSpace.java Thu Mar  1 00:09:28 2007
@@ -21,6 +21,7 @@
 
 package org.apache.derby.impl.services.locks;
 
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.Lockable;
 import org.apache.derby.iapi.services.locks.Limit;
 
@@ -29,26 +30,30 @@
 import org.apache.derby.iapi.error.StandardException;
 
 import java.util.Enumeration;
-import java.util.Dictionary;
 import java.util.HashMap;
 import java.util.Iterator;
 
 /**
 
 	A LockSpace represents the complete set of locks held within
-	a single compatability space, broken into groups of locks.
+	a single compatibility space, broken into groups of locks.
 
-    A LockSpace contains a hashtable keyed by the group reference,
+    A LockSpace contains a HashMap keyed by the group reference,
     the data for each key is a HashMap of Lock's.
 
+    <p> A <code>LockSpace</code> can have an owner (for instance a
+    transaction). Currently, the owner is used by the virtual lock table to
+    find out which transaction a lock belongs to. Some parts of the code also
+    use the owner as a group object which guarantees that the lock is released
+    on a commit or an abort. The owner has no special meaning to the lock
+    manager and can be any object, including <code>null</code>. </p>
 */
-final class LockSpace {
+final class LockSpace implements CompatibilitySpace {
 
 	/** Map from group references to groups of locks. */
 	private final HashMap groups;
-	private final Object compatSpace;
-	// the object I live in
-	private final Dictionary holder;
+	/** Reference to the owner of this compatibility space. */
+	private final Object owner;
 
 	private HashMap spareGroups[] = new HashMap[3];
 
@@ -58,10 +63,23 @@
 	private int    nextLimitCall;
 	private Limit  callback;
 
-	LockSpace(Dictionary holder, Object compatSpace) {
-		this.compatSpace = compatSpace;
-		this.holder = holder;
+	/**
+	 * Creates a new <code>LockSpace</code> instance.
+	 *
+	 * @param owner an object representing the owner of the compatibility space
+	 */
+	LockSpace(Object owner) {
 		groups = new HashMap();
+		this.owner = owner;
+	}
+
+	/**
+	 * Get the object representing the owner of the compatibility space.
+	 *
+	 * @return the owner of the compatibility space
+	 */
+	public Object getOwner() {
+		return owner;
 	}
 
 	/**
@@ -96,7 +114,7 @@
 		if (groupSize > nextLimitCall) {
 
 			inLimit = true;
-			callback.reached(compatSpace, group, limit,
+			callback.reached(this, group, limit,
 				new LockList(java.util.Collections.enumeration(dl.keySet())), groupSize);
 			inLimit = false;
 
@@ -128,10 +146,9 @@
 			lset.unlock((Lock) list.next(), 0);
 		}
 
-		if ((callbackGroup == null) && groups.isEmpty())
-			holder.remove(compatSpace);
-		else if (group.equals(callbackGroup))
+		if ((callbackGroup != null) && group.equals(callbackGroup)) {
 			nextLimitCall = limit;
+		}
 
 		saveGroup(dl);
 	}
@@ -187,11 +204,9 @@
 		if (allUnlocked) {
 			groups.remove(group);
 			saveGroup(dl);
-			if ((callbackGroup == null) && groups.isEmpty())
-				holder.remove(compatSpace);
-			else if (group.equals(callbackGroup))
+			if ((callbackGroup != null) && group.equals(callbackGroup)) {
 				nextLimitCall = limit;
-
+			}
 		}
 	}
 
@@ -262,7 +277,7 @@
 			if (control == null)
 				return 0;
 
-			Lock setLock = control.getLock(compatSpace, qualifier);
+			Lock setLock = control.getLock(this, qualifier);
 			if (setLock == null)
 				return 0;
 
@@ -279,11 +294,9 @@
 			if (dl.isEmpty()) {
 				groups.remove(group);
 				saveGroup(dl);
-				if ((callbackGroup == null) && groups.isEmpty())
-					holder.remove(compatSpace);
-				else if (group.equals(callbackGroup))
+				if ((callbackGroup != null) && group.equals(callbackGroup)) {
 					nextLimitCall = limit;
-
+				}
 			}
 
 			return 1;
@@ -317,7 +330,7 @@
 		if (dl == null)
 			return false;
 
-		Object heldLock = dl.get(new Lock(compatSpace, ref, qualifier));
+		Object heldLock = dl.get(new Lock(this, ref, qualifier));
 		return (heldLock != null);
 	}
 
@@ -331,14 +344,10 @@
 		Clear a limit set by setLimit.
 	*/
 	synchronized void clearLimit(Object group) {
-
 		if (group.equals(callbackGroup)) {
 			callbackGroup = null;
 			nextLimitCall = limit = Integer.MAX_VALUE;
 			callback = null;
-
-			if (groups.isEmpty())
-				holder.remove(compatSpace);
 		}
 	}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/SinglePool.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/SinglePool.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/SinglePool.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/locks/SinglePool.java Thu Mar  1 00:09:28 2007
@@ -21,6 +21,7 @@
 
 package org.apache.derby.impl.services.locks;
 
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.LockFactory;
 import org.apache.derby.iapi.services.locks.C_LockFactory;
 import org.apache.derby.iapi.services.locks.Lockable;
@@ -49,8 +50,7 @@
 	MT - Mutable - Container Object : Thread Aware
 */
 
-public class SinglePool extends Hashtable
-	implements  LockFactory
+public final class SinglePool implements LockFactory
 {
 	/**
 		The complete set of locks in the system
@@ -61,26 +61,6 @@
 	protected final LockSet			lockTable;
 
 	/**
-	    This is now in this object, it now extends Hashtable.
-		A hash table of all compatability spaces. Key is the object 
-        representing the compatability space, value is a LockSpace object. 
-        Addition and removal from the Hashtable is performed under the 
-        Hashtable's monitor. This requires holding this monitor while making 
-        calls to the thread safe methods of LockSpace. This is to ensure
-        that it is guaranteed that a LockSpace is only removed when it is 
-        empty and no-one is in the process of adding to it. No deadlocks are 
-        possible because the spaces reference is not visible outside this 
-        class and the LockSpace class does not call back into this class.
-
-		<BR>
-		MT - immutable - content dynamic : Java synchronized(spaces)
-
-		This class creates a LockSet and LockSpaces, both classes are thread 
-        safe.
-		
-	*/
-
-	/**
 		True if all deadlocks errors should be logged.
 	*/
 	int deadlockMonitor;
@@ -103,7 +83,8 @@
 
 		@see LockFactory#lockObject
 	*/
-	protected Lock lockAnObject(Object compatabilitySpace, Object group,
+	protected Lock lockAnObject(CompatibilitySpace compatibilitySpace,
+								Object group,
 								Lockable ref, Object qualifier, int timeout)
 			throws StandardException
 	{
@@ -111,7 +92,7 @@
 			if (SanityManager.DEBUG_ON(Constants.LOCK_TRACE)) {
 
 				D_LockControl.debugLock("Lock Request before Grant: ", 
-                    compatabilitySpace, group, ref, qualifier, timeout);
+                    compatibilitySpace, group, ref, qualifier, timeout);
 
                 if (SanityManager.DEBUG_ON(Constants.LOCK_STACK_TRACE))
                 {
@@ -127,7 +108,7 @@
 		}
 
 		Lock lock = 
-            lockTable.lockObject(compatabilitySpace, ref, qualifier, timeout);
+            lockTable.lockObject(compatibilitySpace, ref, qualifier, timeout);
 
 		// See if NO_WAIT was passed in and the lock could not be granted.
 		if (lock == null) {
@@ -141,31 +122,27 @@
 			if (SanityManager.DEBUG_ON(Constants.LOCK_TRACE)) {
 				D_LockControl.debugLock(
                     "Lock Request Granted: ", 
-                    compatabilitySpace, group, ref, qualifier, timeout);
+                    compatibilitySpace, group, ref, qualifier, timeout);
 			}
 		}
 
-		// find the space and atomically add lock to required group
-		synchronized (this) {
-
-			LockSpace ls = (LockSpace) get(compatabilitySpace);
-			if (ls == null)	{
-				 ls = new LockSpace(this, compatabilitySpace);
-				 put(compatabilitySpace, ls);
-			}
-
-			// we hold the spaces monitor while adding the lock to close 
-            // the window between finding the LockSpace and adding a lock 
-            // to it, thus ensuring the LockSpace is not removed from the 
-            // spaces Hashtable underneath us.
-
-			ls.addLock(group, lock);
-		}
+		((LockSpace) compatibilitySpace).addLock(group, lock);
 
 		return lock;
 	}
 
 	/**
+	 * Create an object which can be used as a compatibility space within this
+	 * lock manager.
+	 *
+	 * @param owner the owner of the compatibility space
+	 * @return an object which represents a compatibility space
+	 */
+	public CompatibilitySpace createCompatibilitySpace(Object owner) {
+		return new LockSpace(owner);
+	}
+
+	/**
 		Lock a specific object
 
 		<BR>
@@ -175,11 +152,13 @@
 
 		@see LockFactory#lockObject
 	*/
-	public boolean lockObject(Object compatabilitySpace, Object group, Lockable ref, Object qualifier, int timeout)
+	public boolean lockObject(CompatibilitySpace compatibilitySpace,
+							  Object group, Lockable ref, Object qualifier,
+							  int timeout)
 		throws StandardException {
 
 		Lock lock =
-			lockAnObject(compatabilitySpace, group, ref, qualifier, timeout);
+			lockAnObject(compatibilitySpace, group, ref, qualifier, timeout);
 
 		return lock != null;
 	}
@@ -193,20 +172,19 @@
 		@see LockFactory#unlock
 	*/
 
-	public int unlock(Object compatabilitySpace, Object group, Lockable ref, Object qualifier)
+	public int unlock(CompatibilitySpace compatibilitySpace, Object group,
+					  Lockable ref, Object qualifier)
 	{
 		if (SanityManager.DEBUG) {
 			if (SanityManager.DEBUG_ON(Constants.LOCK_TRACE)) {
 				D_LockControl.debugLock("Lock Unlock: ", 
-                    compatabilitySpace, group, ref, qualifier, -1);
+                    compatibilitySpace, group, ref, qualifier, -1);
 			}
 		}
 
-		LockSpace ls = (LockSpace) get(compatabilitySpace);
-		if (ls == null)
-			return 0;
-
-		int count = ls.unlockReference(lockTable, ref, qualifier, group);
+		int count =
+			((LockSpace) compatibilitySpace).unlockReference(
+				lockTable, ref, qualifier, group);
 
 		if (SanityManager.DEBUG) {
 			SanityManager.ASSERT(
@@ -227,36 +205,30 @@
 
 		@see LockFactory#unlockGroup
 	*/
-	public void unlockGroup(Object compatabilitySpace, Object group) {
+	public void unlockGroup(CompatibilitySpace compatibilitySpace,
+							Object group) {
 
 		if (SanityManager.DEBUG) {
 			if (SanityManager.DEBUG_ON(Constants.LOCK_TRACE)) {
-				D_LockControl.debugLock("Lock Unlock Group: ", compatabilitySpace, group);
+				D_LockControl.debugLock("Lock Unlock Group: ",
+										compatibilitySpace, group);
 			}
 		}
 
-		LockSpace ls = (LockSpace) get(compatabilitySpace);
-		if (ls == null)
-			return;
-
-		ls.unlockGroup(lockTable, group);
+		((LockSpace) compatibilitySpace).unlockGroup(lockTable, group);
 	}
 
-	public void unlockGroup(Object compatabilitySpace, Object group, Matchable key) {
+	public void unlockGroup(CompatibilitySpace compatibilitySpace, Object group,
+							Matchable key) {
 
 		if (SanityManager.DEBUG) {
 			if (SanityManager.DEBUG_ON(Constants.LOCK_TRACE)) {
-				D_LockControl.debugLock("Lock Unlock Group: ", compatabilitySpace, group);
+				D_LockControl.debugLock("Lock Unlock Group: ",
+										compatibilitySpace, group);
 			}
 		}
 
-		LockSpace ls = (LockSpace) get(compatabilitySpace);
-		if (ls == null)
-			return;
-
-		ls.unlockGroup(lockTable, group, key);
-
-
+		((LockSpace) compatibilitySpace).unlockGroup(lockTable, group, key);
 	}
 
 	/**
@@ -267,14 +239,15 @@
 
 		@see LockFactory#transfer
 	*/
-	public void transfer(Object compatabilitySpace, Object oldGroup, Object newGroup) {
+	public void transfer(CompatibilitySpace compatibilitySpace, Object oldGroup,
+						 Object newGroup) {
 
 		if (SanityManager.DEBUG) {
 			if (SanityManager.DEBUG_ON(Constants.LOCK_TRACE)) {
 				StringBuffer sb = new StringBuffer("Lock Transfer:");
 
 				D_LockControl.debugAppendObject(
-                    sb, " CompatabilitySpace=", compatabilitySpace);
+                    sb, " CompatibilitySpace=", compatibilitySpace);
 				D_LockControl.debugAppendObject(sb, " Old Group=", oldGroup);
 				D_LockControl.debugAppendObject(sb, " New Group=", newGroup);
 
@@ -284,16 +257,7 @@
 			}
 		}
 
-		LockSpace ls = (LockSpace) get(compatabilitySpace);
-		if (ls == null)
-			return;
-
-		// there is a window where someone could remove the LockSpace from the
-        // spaces Hashtable, since we do not hold the spaces' monitor. This is
-        // Ok as the LockSpace will have no locks and this method 
-        // will correctly do nothing.
-
-		ls.transfer(oldGroup, newGroup);
+		((LockSpace) compatibilitySpace).transfer(oldGroup, newGroup);
 	}
 
 	/**
@@ -313,18 +277,9 @@
 
 		@see LockFactory#areLocksHeld
 	*/
-	public boolean areLocksHeld(Object compatabilitySpace, Object group) {
-
-		LockSpace ls = (LockSpace) get(compatabilitySpace);
-		if (ls == null)
-			return false;
-
-		// there is a window where someone could remove the LockSpace from the 
-        // spaces Hashtable, since we do not hold the spaces' monitor. This is 
-        // Ok as the LockSpace will have no locks and this method will 
-        // correctly return false.
-
-		return ls.areLocksHeld(group);
+	public boolean areLocksHeld(CompatibilitySpace compatibilitySpace,
+								Object group) {
+		return ((LockSpace) compatibilitySpace).areLocksHeld(group);
 	}
 
 	/**
@@ -335,21 +290,20 @@
 
 		@see LockFactory#areLocksHeld
 	*/
-	public boolean areLocksHeld(Object compatabilitySpace) {
-		LockSpace ls = (LockSpace) get(compatabilitySpace);
-		if (ls == null)
-			return false;
-		return ls.areLocksHeld();
+	public boolean areLocksHeld(CompatibilitySpace compatibilitySpace) {
+		return ((LockSpace) compatibilitySpace).areLocksHeld();
 	}
 
-	public boolean zeroDurationlockObject(Object compatabilitySpace, Lockable ref, Object qualifier, int timeout)
+	public boolean zeroDurationlockObject(CompatibilitySpace compatibilitySpace,
+										  Lockable ref, Object qualifier,
+										  int timeout)
 		throws StandardException {
 
 		if (SanityManager.DEBUG) {
 			if (SanityManager.DEBUG_ON(Constants.LOCK_TRACE)) {
 
 				D_LockControl.debugLock("Zero Duration Lock Request before Grant: ", 
-                    compatabilitySpace, (Object) null, ref, qualifier, timeout);
+                    compatibilitySpace, (Object) null, ref, qualifier, timeout);
 
                 if (SanityManager.DEBUG_ON(Constants.LOCK_STACK_TRACE))
                 {
@@ -383,7 +337,7 @@
 			// we can also grant this request now, as skipping
 			// over the waiters won't block them as we release
 			// the lock rightway.
-			if (control.isGrantable(true, compatabilitySpace, qualifier))
+			if (control.isGrantable(true, compatibilitySpace, qualifier))
 				return true;
 
 			// can't be granted and are not willing to wait.
@@ -392,13 +346,13 @@
 		}
 
 		Lock lock = 
-            lockTable.lockObject(compatabilitySpace, ref, qualifier, timeout);
+            lockTable.lockObject(compatibilitySpace, ref, qualifier, timeout);
 
 		if (SanityManager.DEBUG) {
 			if (SanityManager.DEBUG_ON(Constants.LOCK_TRACE)) {
 				D_LockControl.debugLock(
                     "Zero Lock Request Granted: ", 
-                    compatabilitySpace, (Object) null, ref, qualifier, timeout);
+                    compatibilitySpace, (Object) null, ref, qualifier, timeout);
 			}
 		}
 
@@ -408,36 +362,23 @@
 		return true;
 	}
 
-	public boolean isLockHeld(Object compatabilitySpace, Object group, Lockable ref, Object qualifier) {
-
-		LockSpace ls = (LockSpace) get(compatabilitySpace);
-		if (ls == null)
-			return false;
-
-		return ls.isLockHeld(group, ref, qualifier);
+	public boolean isLockHeld(CompatibilitySpace compatibilitySpace,
+							  Object group, Lockable ref, Object qualifier) {
+		return ((LockSpace) compatibilitySpace).isLockHeld(
+			group, ref, qualifier);
 	}
 
-	public synchronized void setLimit(Object compatabilitySpace, Object group, int limit, Limit callback) {
-
-		LockSpace ls = (LockSpace) get(compatabilitySpace);
-		if (ls == null)	{
-			 ls = new LockSpace(this, compatabilitySpace);
-			 put(compatabilitySpace, ls);
-		}
-
-		ls.setLimit(group, limit, callback);
-		
+	public void setLimit(CompatibilitySpace compatibilitySpace,
+						 Object group, int limit, Limit callback) {
+		((LockSpace) compatibilitySpace).setLimit(group, limit, callback);
 	}
 
 	/**
 		Clear a limit set by setLimit.
 	*/
-	public void clearLimit(Object compatabilitySpace, Object group) {
-		LockSpace ls = (LockSpace) get(compatabilitySpace);
-		if (ls == null)
-			return;
-
-		ls.clearLimit(group);
+	public void clearLimit(CompatibilitySpace compatibilitySpace, Object group)
+	{
+		((LockSpace) compatibilitySpace).clearLimit(group);
 	}
 
 //EXCLUDE-START-lockdiag- 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/reflect/UpdateLoader.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/reflect/UpdateLoader.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/reflect/UpdateLoader.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/reflect/UpdateLoader.java Thu Mar  1 00:09:28 2007
@@ -45,6 +45,7 @@
 import org.apache.derby.iapi.reference.MessageId;
 import org.apache.derby.iapi.reference.Module;
 import org.apache.derby.iapi.services.i18n.MessageService;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 
 class UpdateLoader {
 
@@ -58,6 +59,7 @@
 	private int version;
     private boolean normalizeToUpper;
 	private DatabaseClasses parent;
+	private final CompatibilitySpace compat;
 
 	private boolean needReload;
 	private JarReader jarReader;
@@ -68,6 +70,7 @@
         this.normalizeToUpper = normalizeToUpper;
 		this.parent = parent;
 		lf = (LockFactory) Monitor.getServiceModule(parent, Module.LockFactory);
+		compat = lf.createCompatibilitySpace(this);
 
 		if (verbose) {
 			vs = Monitor.getStream();
@@ -160,7 +163,7 @@
 			throw new ClassNotFoundException(MessageService.getTextMessage(MessageId.CM_CLASS_LOAD_EXCEPTION, className, jl == null ? null : jl.getJarName(), se));
 		} finally {
 			if (unlockLoader) {
-				lf.unlock(this, this, classLoaderLock, ShExQual.SH);
+				lf.unlock(compat, this, classLoaderLock, ShExQual.SH);
 			}
 		}
 	}
@@ -207,7 +210,7 @@
 			return null;
 		} finally {
 			if (unlockLoader) {
-				lf.unlock(this, this, classLoaderLock, ShExQual.SH);
+				lf.unlock(compat, this, classLoaderLock, ShExQual.SH);
 			}
 		}
 	}
@@ -254,17 +257,20 @@
 		// engine, in which case tc will be null. In that case
 		// we lock the class loader only for the duration of
 		// the loadClass().
-		Object lockSpace = null;
+		CompatibilitySpace lockSpace = null;
 		
 		if (cfc != null) {
 			lockSpace = cfc.getLockSpace();
 		}
 		if (lockSpace == null)
-			lockSpace = this;
+			lockSpace = compat;
 
-		lf.lockObject(lockSpace, lockSpace, classLoaderLock, qualifier, C_LockFactory.TIMED_WAIT);
+		Object lockGroup = lockSpace.getOwner();
 
-		return (lockSpace == this);
+		lf.lockObject(lockSpace, lockGroup, classLoaderLock, qualifier,
+					  C_LockFactory.TIMED_WAIT);
+
+		return (lockGroup == this);
 	}
 
 	Class checkLoaded(String className, boolean resolve) {

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/catalog/DataDictionaryImpl.java Thu Mar  1 00:09:28 2007
@@ -129,6 +129,7 @@
 import org.apache.derby.catalog.types.RoutineAliasInfo;
 
 import org.apache.derby.iapi.services.io.FormatableBitSet;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.ShExLockable;
 import org.apache.derby.iapi.services.locks.ShExQual;
 import org.apache.derby.iapi.util.StringUtil;
@@ -812,7 +813,7 @@
                 try
                 {
                     lockFactory.zeroDurationlockObject(
-                        lcc.getTransactionExecute().getLockObject(),
+                        lcc.getTransactionExecute().getLockSpace(),
                         cacheCoordinator,
                         ShExQual.SH,
                         C_LockFactory.WAIT_FOREVER);
@@ -874,10 +875,12 @@
                             // waiting (while holding the dataDictionary 
                             // synchronization).
 
+                            CompatibilitySpace space =
+                                lcc.getTransactionExecute().getLockSpace();
+
                             lockGranted = 
                                 lockFactory.lockObject(
-                                    lcc.getTransactionExecute().getLockObject(),
-                                    lcc.getTransactionExecute().getLockObject(),
+                                    space, space.getOwner(),
                                     cacheCoordinator,
                                     ShExQual.SH,
                                     C_LockFactory.NO_WAIT);
@@ -939,10 +942,12 @@
 					*/
 					if ((lcc.getStatementContext() != null) && lcc.getStatementContext().inUse())
 					{
-						int unlockCount = lockFactory.unlock(lcc.getTransactionExecute().getLockObject(),
-										lcc.getTransactionExecute().getLockObject(),
-										cacheCoordinator,
-										ShExQual.SH);
+						CompatibilitySpace space =
+							lcc.getTransactionExecute().getLockSpace();
+						int unlockCount =
+							lockFactory.unlock(
+								space, space.getOwner(),
+								cacheCoordinator, ShExQual.SH);
 						if (SanityManager.DEBUG)
 						{
 							if (unlockCount != 1)
@@ -1028,7 +1033,7 @@
                     // up the stack.
 
                     lockFactory.zeroDurationlockObject(
-                        lcc.getTransactionExecute().getLockObject(),
+                        lcc.getTransactionExecute().getLockSpace(),
                         cacheCoordinator,
                         ShExQual.EX,
                         C_LockFactory.TIMED_WAIT);
@@ -1066,7 +1071,7 @@
 
                         boolean lockGranted = 
                             lockFactory.zeroDurationlockObject(
-                                lcc.getTransactionExecute().getLockObject(),
+                                lcc.getTransactionExecute().getLockSpace(),
                                 cacheCoordinator,
                                 ShExQual.EX,
                                 C_LockFactory.NO_WAIT);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/PropertyConglomerate.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/PropertyConglomerate.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/PropertyConglomerate.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/PropertyConglomerate.java Thu Mar  1 00:09:28 2007
@@ -28,6 +28,7 @@
 import org.apache.derby.iapi.types.UserType;
 import org.apache.derby.iapi.services.io.FormatableBitSet;
 import org.apache.derby.iapi.services.io.FormatableHashtable; 
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.ShExLockable;
 import org.apache.derby.iapi.services.locks.ShExQual;
 import org.apache.derby.iapi.services.locks.C_LockFactory;
@@ -756,8 +757,10 @@
 		// it. Thus readers see the old uncommited values. When this
 		// thread releases its exclusive lock the cached is cleared
 		// and the next reader will re-populate the cache.
-		Object csGroup = tc.getLockObject();
-		lf.lockObject(csGroup, csGroup, cachedLock, ShExQual.EX, C_LockFactory.TIMED_WAIT);
+		CompatibilitySpace cs = tc.getLockSpace();
+		Object csGroup = cs.getOwner();
+		lf.lockObject(cs, csGroup, cachedLock, ShExQual.EX,
+					  C_LockFactory.TIMED_WAIT);
 	}
 
 	/**
@@ -766,8 +769,9 @@
 	  */
 	private boolean iHoldTheUpdateLock(TransactionController tc) throws StandardException
 	{
-		Object csGroup = tc.getLockObject();
-		return lf.isLockHeld(csGroup, csGroup, cachedLock, ShExQual.EX);
+		CompatibilitySpace cs = tc.getLockSpace();
+		Object csGroup = cs.getOwner();
+		return lf.isLockHeld(cs, csGroup, cachedLock, ShExQual.EX);
 	}
 }
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java Thu Mar  1 00:09:28 2007
@@ -34,6 +34,7 @@
 import org.apache.derby.iapi.services.io.Storable;
 
 import org.apache.derby.iapi.services.daemon.Serviceable;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.monitor.Monitor;
 import org.apache.derby.iapi.services.sanity.SanityManager;
 import org.apache.derby.iapi.error.StandardException;
@@ -2303,15 +2304,15 @@
         // above the access context, which is required for
         // error handling assumptions to be correct.
         //
-        // Note that the compatibility space for the nested transaction
-        // is "this", thus the new transaction shares the compatibility space
+        // Note that the nested transaction inherits the compatibility space
+        // from "this", thus the new transaction shares the compatibility space
         // of the current transaction.
         
 
         Transaction child_rawtran = 
             ((readOnly) ?
                 accessmanager.getRawStore().startNestedReadOnlyUserTransaction(
-                    this.getLockObject(), cm, 
+                    getLockSpace(), cm,
                     AccessFactoryGlobals.NESTED_READONLY_USER_TRANS) :
                 accessmanager.getRawStore().startNestedUpdateUserTransaction(
                     cm, AccessFactoryGlobals.NESTED_UPDATE_USER_TRANS));
@@ -2401,12 +2402,14 @@
 		return rawtran.getFileHandler();
 	}
 
-	/**
-		Return an object that when used as the compatability space *and*
-		group for a lock request, guarantees that the lock will be removed
-		on a commit or an abort.
-	*/
-	public Object getLockObject()
+    /**
+     * Return an object that when used as the compatibility space,
+     * <strong>and</strong> the object returned when calling
+     * <code>getOwner()</code> on that object is used as group for a lock
+     * request, guarantees that the lock will be removed on a commit or an
+     * abort.
+     */
+    public CompatibilitySpace getLockSpace()
     {
 		return rawtran.getCompatibilitySpace();
 	}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java Thu Mar  1 00:09:28 2007
@@ -28,6 +28,7 @@
 import org.apache.derby.iapi.services.crypto.CipherFactoryBuilder;
 import org.apache.derby.iapi.services.crypto.CipherFactory;
 import org.apache.derby.iapi.services.crypto.CipherProvider;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.LockFactory;
 import org.apache.derby.iapi.services.monitor.Monitor;
 import org.apache.derby.iapi.services.monitor.ModuleControl;
@@ -400,7 +401,7 @@
 	}
 
 	public Transaction startNestedReadOnlyUserTransaction(
-    Object          compatibilitySpace,
+    CompatibilitySpace compatibilitySpace,
     ContextManager  contextMgr,
     String          transName)
         throws StandardException

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainer.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainer.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainer.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseContainer.java Thu Mar  1 00:09:28 2007
@@ -22,6 +22,7 @@
 package org.apache.derby.impl.store.raw.data;
 
 import org.apache.derby.iapi.reference.SQLState;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.Lockable;
 import org.apache.derby.iapi.services.locks.Latch;
 import org.apache.derby.iapi.services.locks.C_LockFactory;
@@ -212,9 +213,10 @@
                     new Long(getContainerId()));
         }
 
+		CompatibilitySpace cs = ntt.getCompatibilitySpace();
 		// Latch this container, the commit will release the latch
 		ntt.getLockFactory().lockObject(
-                ntt, ntt, this, null, C_LockFactory.WAIT_FOREVER);
+                cs, ntt, this, null, C_LockFactory.WAIT_FOREVER);
 
 		try
 		{
@@ -302,8 +304,9 @@
         }
 
 		// Latch this container, the commit will release the latch
+		CompatibilitySpace cs = ntt.getCompatibilitySpace();
 		ntt.getLockFactory().lockObject(
-                ntt, ntt, this, null, C_LockFactory.WAIT_FOREVER);
+                cs, ntt, this, null, C_LockFactory.WAIT_FOREVER);
 
 		BasePage newPage = null;
 		try

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java Thu Mar  1 00:09:28 2007
@@ -70,6 +70,7 @@
 import org.apache.derby.iapi.services.sanity.SanityManager;
 import org.apache.derby.iapi.services.io.DynamicByteArrayOutputStream;
 import org.apache.derby.iapi.util.ByteArray;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.property.PropertyUtil;
 import org.apache.derby.iapi.reference.Property;
 
@@ -168,7 +169,7 @@
 	protected final XactFactory		xactFactory;
 	protected final DataFactory		dataFactory;
 	protected final LogFactory		logFactory;
-	protected final Object   		compatibilitySpace;
+	private final CompatibilitySpace compatibilitySpace;
 
 	// these fields remain fixedfor the lifetime
 	private LockingPolicy defaultLocking;
@@ -258,7 +259,7 @@
     LogFactory  logFactory, 
     DataFactory dataFactory,
     boolean     readOnly,
-    Object      compatibilitySpace) 
+    CompatibilitySpace compatibilitySpace)
     {
 
 		super();
@@ -268,8 +269,12 @@
 		this.dataFactory = dataFactory;
 		this.readOnly    = readOnly;
 
-		this.compatibilitySpace = 
-            (compatibilitySpace == null ? this : compatibilitySpace);
+		if (compatibilitySpace == null) {
+			this.compatibilitySpace =
+				getLockFactory().createCompatibilitySpace(this);
+		} else {
+			this.compatibilitySpace = compatibilitySpace;
+		}
 
  		if (SanityManager.DEBUG)
 		{
@@ -532,7 +537,8 @@
 			Property.DEFAULT_LOCKS_ESCALATION_THRESHOLD);
 
 
-		getLockFactory().setLimit(this, this, escalationThreshold, this);
+		getLockFactory().setLimit(
+			compatibilitySpace, this, escalationThreshold, this);
 
 	}
 
@@ -562,7 +568,7 @@
      *
 	 * @return The compatibility space of the transaction.
      **/
-    public Object getCompatibilitySpace()
+    public final CompatibilitySpace getCompatibilitySpace()
     {
         if (SanityManager.DEBUG)
         {
@@ -1123,7 +1129,7 @@
 				sanityCheck_xaclosed = true;
 		}
 
-		getLockFactory().clearLimit(this, this);
+		getLockFactory().clearLimit(compatibilitySpace, this);
 
 		if (SanityManager.DEBUG)
 		{
@@ -2381,8 +2387,8 @@
 	** Methods of Limit
 	*/
 
-	public void reached(Object compatabilitySpace, Object group, int limit,
-		Enumeration lockList, int lockCount)
+	public void reached(CompatibilitySpace compatibilitySpace, Object group,
+						int limit, Enumeration lockList, int lockCount)
 		throws StandardException {
 
 		// Count row locks by table

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/xact/XactFactory.java Thu Mar  1 00:09:28 2007
@@ -28,6 +28,7 @@
 import org.apache.derby.iapi.services.context.ContextManager;
 import org.apache.derby.iapi.services.daemon.DaemonService;
 import org.apache.derby.iapi.services.daemon.Serviceable;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.services.locks.LockFactory;
 import org.apache.derby.iapi.services.monitor.ModuleControl;
 import org.apache.derby.iapi.services.monitor.ModuleSupportable;
@@ -250,7 +251,7 @@
     RawStoreFactory rsf,
     ContextManager  cm,
     boolean         readOnly,
-    Object          compatibilitySpace,
+    CompatibilitySpace compatibilitySpace,
     String          xact_context_id,
     String          transName,
     boolean         excludeMe)
@@ -290,7 +291,7 @@
 
 	public RawTransaction startNestedReadOnlyUserTransaction(
     RawStoreFactory rsf,
-    Object          compatibilitySpace,
+    CompatibilitySpace compatibilitySpace,
     ContextManager  cm,
     String          transName)
         throws StandardException

Modified: db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java (original)
+++ db/derby/code/trunk/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java Thu Mar  1 00:09:28 2007
@@ -27,6 +27,7 @@
 import org.apache.derby.iapi.services.context.ContextManager;
 import org.apache.derby.iapi.services.io.FormatableBitSet;
 import org.apache.derby.iapi.services.io.Storable;
+import org.apache.derby.iapi.services.locks.CompatibilitySpace;
 import org.apache.derby.iapi.store.access.AccessFactory;
 import org.apache.derby.iapi.store.access.BackingStoreHashtable;
 import org.apache.derby.iapi.store.access.ColumnOrdering;
@@ -230,7 +231,7 @@
         return null;
     }
 
-    public Object getLockObject() {
+    public CompatibilitySpace getLockSpace() {
         // TODO Auto-generated method stub
         return null;
     }

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/services/T_LockFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/services/T_LockFactory.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/services/T_LockFactory.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/services/T_LockFactory.java Thu Mar  1 00:09:28 2007
@@ -120,7 +120,7 @@
 	*/
 	void S001() throws StandardException, T_Fail {
 
-		Object cs = new Object();	// create an object for the compatability space
+		CompatibilitySpace cs = lf.createCompatibilitySpace(null);
 		Object g0 = new Object();	// create an object for a lock group
 		Lockable l0 = new T_L1();		// simple lockable
 
@@ -197,7 +197,7 @@
 
 	void S002() throws StandardException, T_Fail {
 
-		Object cs = new Object();	// create an object for the compatability space
+		CompatibilitySpace cs = lf.createCompatibilitySpace(null);
 		Object g0 = new Object();	// create an object for a lock group
 		Object g1 = new Object();
 		Lockable l0 = new T_L1();		// simple lockable
@@ -264,7 +264,7 @@
 
 	void S003() throws StandardException, T_Fail {
 
-		Object cs = new Object();	// create an object for the compatability space
+		CompatibilitySpace cs = lf.createCompatibilitySpace(null);
 		Object g0 = new Object();	// create an object for a lock group
 		Object g1 = new Object();
 		Lockable l0 = new T_L1();		// simple lockable
@@ -322,7 +322,7 @@
 
 	void S004() throws StandardException, T_Fail {
 
-		Object cs = new Object();	// create an object for the compatability space
+		CompatibilitySpace cs = lf.createCompatibilitySpace(null);
 		Object g0 = new Object();	// create an object for a lock group
 		Object g1 = new Object();
 		Object g2 = new Object();
@@ -398,8 +398,8 @@
 	*/
 	void S005() throws StandardException, T_Fail {
 
-		Object cs0 = new Object();	// create an object for the compatability space
-		Object cs1 = new Object();	// create an object for the compatability space
+		CompatibilitySpace cs0 = lf.createCompatibilitySpace(null);
+		CompatibilitySpace cs1 = lf.createCompatibilitySpace(null);
 
 		Object g0 = new Object();	// create an object for a lock group
 		Object g1 = new Object();	// create an object for a lock group
@@ -469,7 +469,7 @@
 	*/
 	void S007() throws StandardException, T_Fail {
 
-		Object cs = new Object();	// create an object for the compatability space
+		CompatibilitySpace cs = lf.createCompatibilitySpace(null);
 		Object g0 = new Object();	// create an object for a lock group
 		Object g1 = new Object();	// create an object for a lock group
 		Lockable l0 = new T_L1();
@@ -693,7 +693,7 @@
 
 		@exception T_Fail	Number of locks is not as expected.
 	*/
-	void checkLockCount(Object cs, int expected) throws T_Fail {
+	void checkLockCount(CompatibilitySpace cs, int expected) throws T_Fail {
 		boolean expect = expected != 0;
 		boolean got = lf.areLocksHeld(cs);
 		if (got != expect)
@@ -706,7 +706,8 @@
 		@exception T_Fail	Number of locks is not as expected.
 	*/
 
-	void checkLockGroupCount(Object cs, Object group, int expected) throws T_Fail {
+	void checkLockGroupCount(CompatibilitySpace cs, Object group, int expected)
+			throws T_Fail {
 		boolean expect = expected != 0;
 		boolean got = lf.areLocksHeld(cs, group);
 		if (got != expect)

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/services/T_User.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/services/T_User.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/services/T_User.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/services/T_User.java Thu Mar  1 00:09:28 2007
@@ -69,7 +69,7 @@
 
 	private void T001() throws StandardException, T_Fail {
 
-		Object cs = new Object();	// create an object for the compatability space
+		CompatibilitySpace cs = lf.createCompatibilitySpace(null);
 		Integer g0 = new Integer(1);	// create an object for a lock group
 
 		// check we have no locks held
@@ -112,7 +112,7 @@
 
 	private void T002() throws StandardException, T_Fail {
 
-		Object cs = new Object();	// create an object for the compatability space
+		CompatibilitySpace cs = lf.createCompatibilitySpace(null);
 		Integer g0 = new Integer(1);	// create an object for a lock group
 
 		// check we have no locks held
@@ -137,7 +137,7 @@
 
 	private void T003() throws StandardException, T_Fail {
 
-		Object cs = new Object();	// create an object for the compatability space
+		CompatibilitySpace cs = lf.createCompatibilitySpace(null);
 		Integer g0 = new Integer(1);	// create an object for a lock group
 
 		// check we have no locks held
@@ -156,7 +156,7 @@
 	}
 	private void T004() throws StandardException, T_Fail {
 
-		Object cs = new Object();	// create an object for the compatability space
+		CompatibilitySpace cs = lf.createCompatibilitySpace(null);
 		Integer g0 = new Integer(1);	// create an object for a lock group
 
 		// check we have no locks held
@@ -185,7 +185,7 @@
 			throw T_Fail.testFailMsg("value corrupted in multi-user test, exapected " + value + ", got " + item.value);
 	}
 
-	void checkLockCount(Object cs, int expected) throws T_Fail {
+	void checkLockCount(CompatibilitySpace cs, int expected) throws T_Fail {
 		boolean expect = expected != 0;
 		boolean got = lf.areLocksHeld(cs);
 		if (got != expect)

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_AccessFactory.java Thu Mar  1 00:09:28 2007
@@ -2969,13 +2969,8 @@
                 ";current_xact_after_nest = "  + current_xact_after_nest);
         }
 
-        if ((tc.getLockObject() != child_tc.getLockObject()) ||
-            !(tc.getLockObject().equals(child_tc.getLockObject())))
-
-        {
-			throw T_Fail.testFailMsg(
-                "(nestedUserTransaction) getLockObject should return same object from botht these calls.");
-        }
+        T_Fail.T_ASSERT(tc.getLockSpace() == child_tc.getLockSpace(),
+                        "getLockSpace() returned different object for child.");
 
         // the locks of the nested transaction should not conflict, so this
         // open should work.

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_TWC.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_TWC.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_TWC.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_TWC.java Thu Mar  1 00:09:28 2007
@@ -77,7 +77,9 @@
 		switchTransactionContext();
 		try {
 		if (SanityManager.DEBUG)
-			SanityManager.ASSERT(!lf.areLocksHeld(tran));
+			SanityManager.ASSERT(
+				!lf.areLocksHeld(tran.getCompatibilitySpace()),
+				"Transaction holds locks.");
 		} finally {
 			resetContext();
 		}

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_Util.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_Util.java?view=diff&rev=513222&r1=513221&r2=513222
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_Util.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/unitTests/store/T_Util.java Thu Mar  1 00:09:28 2007
@@ -95,7 +95,7 @@
 	 * check that transaction does not hold any lock
 	 */
 	public void t_checkNullLockCount(Transaction t) throws T_Fail {
-		if (lFactory.areLocksHeld(t))
+		if (lFactory.areLocksHeld(t.getCompatibilitySpace()))
 			throw T_Fail.testFailMsg("Previous action did not clean up all locks.");
 	}