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 2008/08/25 17:54:50 UTC

svn commit: r688766 - in /db/derby/code/branches/10.4/java: engine/org/apache/derby/iapi/services/locks/ engine/org/apache/derby/iapi/sql/dictionary/ engine/org/apache/derby/iapi/store/access/ engine/org/apache/derby/iapi/store/raw/ engine/org/apache/d...

Author: kahatlen
Date: Mon Aug 25 08:54:49 2008
New Revision: 688766

URL: http://svn.apache.org/viewvc?rev=688766&view=rev
Log:
DERBY-3693: Deadlocks accessing DB metadata

Merged fix from trunk (revisions 688274 and 688756).

Added:
    db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/LockOwner.java
      - copied, changed from r688274, db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/LockOwner.java
Modified:
    db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/LockFactory.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/sql/dictionary/SPSDescriptor.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/store/raw/Transaction.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/AbstractPool.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/ConcurrentLockSet.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockSet.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockSpace.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockTable.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/reflect/UpdateLoader.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
    db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java
    db/derby/code/branches/10.4/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java
    db/derby/code/branches/10.4/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/DatabaseMetaDataTest.java

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/CompatibilitySpace.java Mon Aug 25 08:54:49 2008
@@ -41,5 +41,5 @@
      * @return object representing the owner of the compatibility space, or
      * <code>null</code> if no owner has been specified.
      */
-    Object getOwner();
+    LockOwner getOwner();
 }

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/LockFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/LockFactory.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/LockFactory.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/LockFactory.java Mon Aug 25 08:54:49 2008
@@ -46,7 +46,7 @@
 	 * transaction object). Might be <code>null</code>.
 	 * @return an object which represents a compatibility space
 	 */
-	public CompatibilitySpace createCompatibilitySpace(Object owner);
+	public CompatibilitySpace createCompatibilitySpace(LockOwner owner);
 
 	/**
 		Lock an object within a compatibility space

Copied: db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/LockOwner.java (from r688274, db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/LockOwner.java)
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/LockOwner.java?p2=db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/LockOwner.java&p1=db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/LockOwner.java&r1=688274&r2=688766&rev=688766&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/locks/LockOwner.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/services/locks/LockOwner.java Mon Aug 25 08:54:49 2008
@@ -29,6 +29,24 @@
      * cannot be granted at once, even if {@code C_LockFactory.TIMED_WAIT}
      * was specified in the lock request.
      *
+     * <p>
+     *
+     * Normally, this method should return {@code false}, but in some very
+     * special cases it could be appropriate to return {@code true}. One
+     * example is when a stored prepared statement (SPS) is compiled and stored
+     * in a system table. In order to prevent exclusive locks in the system
+     * table from being held until the transaction that triggered the
+     * compilation is finished, the SPS will be compiled in a nested
+     * transaction that is committed and releases all locks upon completion.
+     * There is however a risk that the transaction that triggered the
+     * compilation is holding locks that the nested transaction needs, in
+     * which case the nested transaction will time out. The timeout will be
+     * detected by the calling code, and the operation will be retried in the
+     * parent transaction. To avoid long waits in the cases where the nested
+     * transaction runs into a lock conflict with its parent, the nested
+     * transaction's {@code LockOwner} instance could return {@code true} and
+     * thereby making it possible to detect lock conflicts instantly.
+     *
      * @return {@code true} if timed waits should time out immediately,
      * {@code false} otherwise
      */

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/sql/dictionary/SPSDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/sql/dictionary/SPSDescriptor.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/sql/dictionary/SPSDescriptor.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/sql/dictionary/SPSDescriptor.java Mon Aug 25 08:54:49 2008
@@ -692,6 +692,14 @@
 				try
 				{
 					nestedTC = lcc.getTransactionCompile().startNestedUserTransaction(false);
+                    // DERBY-3693: The nested transaction may run into a lock
+                    // conflict with its parent transaction, in which case we
+                    // don't want to wait for a timeout. If a lock timeout is
+                    // detected while we're executing the nested transaction,
+                    // we ignore the error and retry in the user transaction.
+                    // When retrying in the user transaction, we'll wait for
+                    // locks if necessary.
+                    nestedTC.setNoLockWait(true);
 				}
 				catch (StandardException se)
 				{

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/store/access/TransactionController.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/store/access/TransactionController.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/store/access/TransactionController.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/store/access/TransactionController.java Mon Aug 25 08:54:49 2008
@@ -1482,6 +1482,22 @@
     public CompatibilitySpace getLockSpace();
 
     /**
+     * Tell this transaction whether it should time out immediately if a lock
+     * cannot be granted without waiting. This mechanism can for instance be
+     * used if an operation is first attempted in a nested transaction to
+     * reduce the lifetime of locks held in the system tables (like when
+     * a stored prepared statement is compiled and stored). In such a case,
+     * the caller must catch timeout exceptions and retry the operation in the
+     * main transaction if a lock timeout occurs.
+     *
+     * @param noWait if {@code true} never wait for a lock in this transaction,
+     * but time out immediately
+     * @see org.apache.derby.iapi.services.locks.LockOwner#noWait()
+     * @see org.apache.derby.iapi.store.raw.Transaction#setNoLockWait(boolean)
+     */
+    public void setNoLockWait(boolean noWait);
+
+    /**
      * Return static information about the conglomerate to be included in a
      * a compiled plan.
      * <p>

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/store/raw/Transaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/store/raw/Transaction.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/store/raw/Transaction.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/iapi/store/raw/Transaction.java Mon Aug 25 08:54:49 2008
@@ -59,6 +59,20 @@
      **/
     CompatibilitySpace getCompatibilitySpace();
 
+    /**
+     * Tell this transaction whether it should time out immediately if a lock
+     * cannot be granted without waiting. This could be used in a nested
+     * transaction to prevent long waits if there is a lock conflict between
+     * the nested transaction and its parent. If it is used this way, the
+     * calling code should catch timeout exceptions from the nested transaction
+     * and retry the operation (without disabling waiting) in the parent
+     * transaction.
+     *
+     * @param noWait if {@code true} never wait for a lock in this transaction,
+     * but time out immediately
+     * @see org.apache.derby.iapi.services.locks.LockOwner#noWait()
+     */
+    void setNoLockWait(boolean noWait);
 
 	/**
 		Called after the transaction has been attached to an Access Manger

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/AbstractPool.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/AbstractPool.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/AbstractPool.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/AbstractPool.java Mon Aug 25 08:54:49 2008
@@ -39,6 +39,8 @@
 import java.io.Serializable;
 import java.util.Dictionary;
 import java.util.Enumeration;
+import org.apache.derby.iapi.reference.SQLState;
+import org.apache.derby.iapi.services.locks.LockOwner;
 
 /**
  * An abstract implementation of LockFactory that allows different
@@ -120,8 +122,16 @@
 		// See if NO_WAIT was passed in and the lock could not be granted.
 		if (lock == null) {
 			if (SanityManager.DEBUG) {
-				SanityManager.ASSERT(timeout == C_LockFactory.NO_WAIT, "timeout not NO_WAIT");
+                SanityManager.ASSERT(noLockWait(timeout, compatibilitySpace),
+                                     "timeout not NO_WAIT");
 			}
+
+            // If this is a timed wait, we should behave as if we timed out
+            // and throw a LOCK_TIMEOUT.
+            if (timeout == C_LockFactory.TIMED_WAIT) {
+                throw StandardException.newException(SQLState.LOCK_TIMEOUT);
+            }
+
 			return false;
 		}
 
@@ -145,7 +155,7 @@
 	 * @param owner the owner of the compatibility space
 	 * @return an object which represents a compatibility space
 	 */
-	public CompatibilitySpace createCompatibilitySpace(Object owner) {
+	public CompatibilitySpace createCompatibilitySpace(LockOwner owner) {
 		return new LockSpace(owner);
 	}
 
@@ -284,8 +294,27 @@
 										  Lockable ref, Object qualifier,
 										  int timeout)
 		throws StandardException {
-		return lockTable.zeroDurationLockObject(
+		boolean success = lockTable.zeroDurationLockObject(
 			compatibilitySpace, ref, qualifier, timeout);
+
+        if (!success) {
+
+            // zeroDurationLockObject should only return false if we have
+            // requested that we shouldn't wait for locks. Otherwise, an
+            // exception should have been thrown.
+            if (SanityManager.DEBUG) {
+                SanityManager.ASSERT(noLockWait(timeout, compatibilitySpace),
+                                     "Should have timed out");
+            }
+
+            // If this is a timed wait, we should behave as if we timed out and
+            // throw LOCK_TIMEOUT.
+            if (timeout == C_LockFactory.TIMED_WAIT) {
+                throw StandardException.newException(SQLState.LOCK_TIMEOUT);
+            }
+        }
+
+        return success;
 	}
 
 	public boolean isLockHeld(CompatibilitySpace compatibilitySpace,
@@ -307,6 +336,26 @@
 		((LockSpace) compatibilitySpace).clearLimit(group);
 	}
 
+    /**
+     * Check if we should not wait for locks, given the specified timeout and
+     * compatibility space. If the timeout is {@code C_LockFactory.NO_WAIT} or
+     * the {@code LockOwner} has the {@code noWait} flag set, we shouldn't
+     * wait for locks.
+     *
+     * @param timeout the specified timeout
+     * @param compat the compatibility space
+     * @return {@code true} if we shouldn't wait for locks, {@code false}
+     * otherwise
+     */
+    static boolean noLockWait(int timeout, CompatibilitySpace compat) {
+        if (timeout == C_LockFactory.NO_WAIT) {
+            return true;
+        } else {
+            LockOwner owner = compat.getOwner();
+            return owner != null && owner.noWait();
+        }
+    }
+
 //EXCLUDE-START-lockdiag- 
 
 	/**

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/ConcurrentLockSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/ConcurrentLockSet.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/ConcurrentLockSet.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/ConcurrentLockSet.java Mon Aug 25 08:54:49 2008
@@ -355,7 +355,7 @@
 				return lockItem;
 			}
 
-			if (timeout == C_LockFactory.NO_WAIT) {
+			if (AbstractPool.noLockWait(timeout, compatibilitySpace)) {
 
     			// remove all trace of lock
     			control.giveUpWait(lockItem, this);
@@ -792,22 +792,7 @@
     }
 
     /**
-     * Lock an object and release the lock immediately. Equivalent to
-     * <pre>
-     * Lock lock = lockTable.lockObject(space, ref, qualifier, timeout);
-     * lockTable.unlock(lock, 1);
-     * </pre>
-     * except that the implementation is more efficient.
-     *
-     * @param space the compatibility space
-     * @param ref a reference to the locked object
-     * @param qualifier qualifier of the lock
-     * @param timeout maximum time to wait in milliseconds
-     * (<code>LockFactory.NO_WAIT</code> means don't wait)
-     * @return <code>true</code> if the object was locked, or
-     * <code>false</code>if the timeout was <code>NO_WAIT</code> and the lock
-     * couldn't be obtained immediately
-     * @exception StandardException if the lock could not be obtained
+     * {@inheritDoc}
      */
     public boolean zeroDurationLockObject(
         CompatibilitySpace space, Lockable ref, Object qualifier, int timeout)
@@ -860,7 +845,7 @@
             }
 
             // can't be granted and are not willing to wait.
-            if (timeout == C_LockFactory.NO_WAIT) {
+            if (AbstractPool.noLockWait(timeout, space)) {
                 return false;
             }
         } finally {

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockSet.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockSet.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockSet.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockSet.java Mon Aug 25 08:54:49 2008
@@ -183,7 +183,7 @@
 				return lockItem;
 			}
 
-			if (timeout == C_LockFactory.NO_WAIT) {
+			if (AbstractPool.noLockWait(timeout, compatibilitySpace)) {
 
     			// remove all trace of lock
     			control.giveUpWait(lockItem, this);
@@ -579,22 +579,7 @@
     }
 
     /**
-     * Lock an object and release the lock immediately. Equivalent to
-     * <pre>
-     * Lock lock = lockTable.lockObject(space, ref, qualifier, timeout);
-     * lockTable.unlock(lock, 1);
-     * </pre>
-     * except that the implementation is more efficient.
-     *
-     * @param space the compatibility space
-     * @param ref a reference to the locked object
-     * @param qualifier qualifier of the lock
-     * @param timeout maximum time to wait in milliseconds
-     * (<code>LockFactory.NO_WAIT</code> means don't wait)
-     * @return <code>true</code> if the object was locked, or
-     * <code>false</code>if the timeout was <code>NO_WAIT</code> and the lock
-     * couldn't be obtained immediately
-     * @exception StandardException if the lock could not be obtained
+     * {@inheritDoc}
      */
     public boolean zeroDurationLockObject(
         CompatibilitySpace space, Lockable ref, Object qualifier, int timeout)
@@ -641,7 +626,7 @@
             }
 
             // can't be granted and are not willing to wait.
-            if (timeout == C_LockFactory.NO_WAIT) {
+            if (AbstractPool.noLockWait(timeout, space)) {
                 return false;
             }
         }

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockSpace.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockSpace.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockSpace.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockSpace.java Mon Aug 25 08:54:49 2008
@@ -32,6 +32,7 @@
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.Iterator;
+import org.apache.derby.iapi.services.locks.LockOwner;
 
 /**
 
@@ -53,7 +54,7 @@
 	/** Map from group references to groups of locks. */
 	private final HashMap groups;
 	/** Reference to the owner of this compatibility space. */
-	private final Object owner;
+	private final LockOwner owner;
 
 	private HashMap spareGroups[] = new HashMap[3];
 
@@ -68,7 +69,7 @@
 	 *
 	 * @param owner an object representing the owner of the compatibility space
 	 */
-	LockSpace(Object owner) {
+	LockSpace(LockOwner owner) {
 		groups = new HashMap();
 		this.owner = owner;
 	}
@@ -78,7 +79,7 @@
 	 *
 	 * @return the owner of the compatibility space
 	 */
-	public Object getOwner() {
+	public LockOwner getOwner() {
 		return owner;
 	}
 

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockTable.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockTable.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockTable.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/locks/LockTable.java Mon Aug 25 08:54:49 2008
@@ -39,9 +39,10 @@
      * @param ref the object to lock
      * @param qualifier qualifier of the lock
      * @param timeout maximum time to wait in milliseconds
-     * (<code>LockFactory.NO_WAIT</code> means don't wait)
-     * @return a reference to the lock, or <code>null</code> if the timeout was
-     * <code>NO_WAIT</code> and the lock couldn't be obtained immediately
+     * ({@code C_LockFactory.NO_WAIT} means don't wait)
+     * @return a reference to the lock, or <code>null</code> if the lock
+     * couldn't be obtained immediately and the timeout was {@code NO_WAIT}
+     * or {@code LockOwner} had the {@code noWait} flag set
      * @exception StandardException if the lock could not be obtained
      */
     Lock lockObject(CompatibilitySpace compatibilitySpace, Lockable ref,
@@ -102,10 +103,10 @@
      * @param ref a reference to the locked object
      * @param qualifier qualifier of the lock
      * @param timeout maximum time to wait in milliseconds
-     * (<code>LockFactory.NO_WAIT</code> means don't wait)
+     * ({@code C_LockFactory.NO_WAIT} means don't wait)
      * @return <code>true</code> if the object was locked, or
-     * <code>false</code>if the timeout was <code>NO_WAIT</code> and the lock
-     * couldn't be obtained immediately
+     * {@code false} if the object couldn't be locked immediately and timeout
+     * was {@code NO_WAIT} or {@code LockOwner} had the {@code noWait} flag set
      * @exception StandardException if the lock could not be obtained
      */
     boolean zeroDurationLockObject(CompatibilitySpace space, Lockable ref,

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/reflect/UpdateLoader.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/reflect/UpdateLoader.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/reflect/UpdateLoader.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/services/reflect/UpdateLoader.java Mon Aug 25 08:54:49 2008
@@ -23,7 +23,6 @@
 
 import org.apache.derby.iapi.services.context.ContextService;
 import org.apache.derby.iapi.services.monitor.Monitor;
-import org.apache.derby.iapi.services.monitor.Monitor;
 import org.apache.derby.iapi.services.stream.HeaderPrintWriter;
 import org.apache.derby.iapi.util.IdUtil;
 import org.apache.derby.iapi.error.StandardException;
@@ -46,6 +45,7 @@
 import org.apache.derby.iapi.reference.Module;
 import org.apache.derby.iapi.services.i18n.MessageService;
 import org.apache.derby.iapi.services.locks.CompatibilitySpace;
+import org.apache.derby.iapi.services.locks.LockOwner;
 
 /**
  * UpdateLoader implements then functionality of
@@ -62,7 +62,7 @@
  * UpdateLoader will then try to load the class from each of the jars
  * in order of derby.database.classpath using the jar's installed JarLoader.
  */
-final class UpdateLoader {
+final class UpdateLoader implements LockOwner {
     
     /**
      * List of packages that Derby will not support being loaded
@@ -404,6 +404,16 @@
 		}
 		return jarReader;
 	}
+
+    /**
+     * Tell the lock manager that we don't want timed waits to time out
+     * immediately.
+     *
+     * @return {@code false}
+     */
+    public boolean noWait() {
+        return false;
+    }
 }
 
 

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/store/access/RAMTransaction.java Mon Aug 25 08:54:49 2008
@@ -2436,6 +2436,19 @@
 	}
 
     /**
+     * {@inheritDoc}
+     *
+     * <p>
+     *
+     * For now, this only works if the transaction has its own compatibility
+     * space. If it has inherited the compatibility space from its parent,
+     * the request will be ignored (or cause a failure in debug builds).
+     */
+    public void setNoLockWait(boolean noWait) {
+        rawtran.setNoLockWait(noWait);
+    }
+
+    /**
      * Get string id of the transaction.
      * <p>
      * This transaction "name" will be the same id which is returned in

Modified: db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java (original)
+++ db/derby/code/branches/10.4/java/engine/org/apache/derby/impl/store/raw/xact/Xact.java Mon Aug 25 08:54:49 2008
@@ -58,8 +58,6 @@
 
 import org.apache.derby.iapi.services.property.PersistentSet;
 
-import org.apache.derby.catalog.UUID;
-
 import java.util.Stack;
 import java.util.Enumeration;
 import java.util.Properties;
@@ -76,11 +74,9 @@
 import org.apache.derby.iapi.services.property.PropertyUtil;
 import org.apache.derby.iapi.reference.Property;
 
-import org.apache.derby.impl.store.raw.log.LogToFile;
-
 import org.apache.derby.iapi.services.io.LimitObjectInput;
 
-import org.apache.derby.iapi.services.context.ContextService;
+import org.apache.derby.iapi.services.locks.LockOwner;
 
 /**
 
@@ -98,7 +94,7 @@
 	@see Transaction
 
 */
-public class Xact extends RawTransaction implements Limit  {
+public class Xact extends RawTransaction implements Limit, LockOwner {
 
 	/*
 	** Static Fields
@@ -252,6 +248,11 @@
 	// backup copy.
 	private boolean backupBlocked;
 
+    /**
+     * Tells if lock requests should time out immediately if they cannot be
+     * granted without waiting.
+     */
+    private boolean dontWaitForLocks;
 
 	/*
 	** Constructor
@@ -571,6 +572,35 @@
         return(this.compatibilitySpace);
     }
 
+    /**
+     * Tells whether lock requests should time out immediately if they can't
+     * be granted without waiting. Only works if this object is the owner of
+     * the compatibility space used in the request.
+     *
+     * @return whether waiting for locks should time out immediately
+     */
+    public boolean noWait() {
+        return dontWaitForLocks;
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * <p>
+     *
+     * This only works if this transaction is the owner of the compatibility
+     * space used in the request. If this transaction has inherited the
+     * compatibility space from its parent, the call to this method has no
+     * effect (except in debug builds, where an error will be raised).
+     */
+    public void setNoLockWait(boolean noWait) {
+        if (SanityManager.DEBUG) {
+            SanityManager.ASSERT(compatibilitySpace.getOwner() == this,
+                    "Trying to set no-wait mode on transaction that " +
+                    "shares compatibility space with its parent");
+        }
+        dontWaitForLocks = noWait;
+    }
 
 	/**
 		get the short (internal to raw store) transaction id that is unique

Modified: db/derby/code/branches/10.4/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java (original)
+++ db/derby/code/branches/10.4/java/storeless/org/apache/derby/impl/storeless/NoOpTransaction.java Mon Aug 25 08:54:49 2008
@@ -422,4 +422,8 @@
         return(null);
     }
 
+    public void setNoLockWait(boolean noWait) {
+        // TODO Auto-generated method stub
+    }
+
 }

Modified: db/derby/code/branches/10.4/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/DatabaseMetaDataTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/branches/10.4/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/DatabaseMetaDataTest.java?rev=688766&r1=688765&r2=688766&view=diff
==============================================================================
--- db/derby/code/branches/10.4/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/DatabaseMetaDataTest.java (original)
+++ db/derby/code/branches/10.4/java/testing/org/apache/derbyTesting/functionTests/tests/jdbcapi/DatabaseMetaDataTest.java Mon Aug 25 08:54:49 2008
@@ -44,6 +44,7 @@
 import java.util.List;
 import java.util.Locale;
 //import java.util.Map;
+import java.util.Properties;
 import java.util.Random;
 import java.util.StringTokenizer;
 
@@ -204,6 +205,23 @@
                 // test from waiting one minute
                 DatabasePropertyTestSetup.setLockTimeouts(
                     new DatabaseMetaDataTest("initialCompilationTest"), 2, 4)));
+
+        // Test for DERBY-3693 needs a fresh database to ensure that the size
+        // of SYSTABLES is so small that creating a relatively small number of
+        // tables will cause the query plan for getTables() to be invalidated.
+        // Also, set a high lock timeout explicitly so that we can check that
+        // an internal timeout followed by a retry didn't happen, and set
+        // derby.language.stalePlanCheckInterval to a low value so that the
+        // invalidation happens earlier.
+        Properties props = new Properties();
+        props.setProperty("derby.locks.waitTimeout", "90");
+        props.setProperty("derby.language.stalePlanCheckInterval", "5");
+        suite.addTest(
+            TestConfiguration.singleUseDatabaseDecorator(
+                new DatabasePropertyTestSetup(
+                    new DatabaseMetaDataTest("recompileTimeoutTest"),
+                    props, true)));
+
         return suite;
     }
     
@@ -255,6 +273,44 @@
     }
 
     /**
+     * Tests that we don't get an internal timeout when a meta-data statement
+     * is recompiled because the size of the tables it queries has changed
+     * (DERBY-3693). The test must be run on a fresh database, to ensure that
+     * SYSTABLES initially has a relatively small number of records. The lock
+     * timeout must be high (more than 60 seconds) to enable us to see the
+     * difference between an internal lock timeout and slow execution.
+     * derby.language.stalePlanCheckInterval should be set to 5 (the lowest
+     * possible value) so that we don't have to wait long for the query plan
+     * to be invalidated.
+     */
+    public void recompileTimeoutTest() throws SQLException {
+        DatabaseMetaData dmd = getDMD();
+
+        // Make sure getTables() is initially compiled while SYSTABLES is small
+        JDBC.assertDrainResults(dmd.getTables(null, "%", "%", null));
+
+        // Grow SYSTABLES
+        Statement s = createStatement();
+        for (int i = 0; i < 20; i++) {
+            s.executeUpdate("create table t" + i + "(x int)");
+        }
+
+        // Execute getTables() derby.language.stalePlanCheckInterval times so
+        // that its plan is invalidated. Before DERBY-3693 was fixed, the
+        // recompilation after the invalidation would get an internal timeout
+        // and take very long time to complete.
+        for (int i = 0; i < 5; i++) {
+            long time = System.currentTimeMillis();
+            JDBC.assertDrainResults(dmd.getTables(null, "%", "T0", null));
+            time = System.currentTimeMillis() - time;
+            if (time > 60000) {
+                fail("getTables() took a very long time, possibly because " +
+                     "of an internal timeout. i=" + i + ", time=" + time);
+            }
+        }
+    }
+
+    /**
      * Test the methods that indicate if a feature
      * is supported or not. Methods start with
      * 'support'. See secton 7.3 in JDBC 3.0 specification.