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 da...@apache.org on 2008/09/01 18:04:29 UTC

svn commit: r691007 - in /db/derby/code/trunk/java: engine/org/apache/derby/catalog/ engine/org/apache/derby/iapi/sql/ engine/org/apache/derby/iapi/sql/depend/ engine/org/apache/derby/iapi/sql/dictionary/ engine/org/apache/derby/impl/sql/ engine/org/ap...

Author: dag
Date: Mon Sep  1 09:04:29 2008
New Revision: 691007

URL: http://svn.apache.org/viewvc?rev=691007&view=rev
Log:
DERBY-3223 SQL roles: make use of privileges granted to roles in actual privilege checking

Patch derby-3223-invalidate-activations-2, which changes the way prepared statements
are invalidated to a more efficient approach which invalidates the activation, thereby
forcing a rechecking of privileges when the current role changes.

Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/catalog/Dependable.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/Activation.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/depend/DependencyManager.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementColumnPermission.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementRoutinePermission.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementTablePermission.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ViewDescriptor.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericActivationHolder.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericPreparedStatement.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/depend/BasicDependencyManager.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DropRoleConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RevokeRoleConstantAction.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/SetRoleConstantAction.java
    db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RolesConferredPrivilegesTest.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/catalog/Dependable.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/catalog/Dependable.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/catalog/Dependable.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/catalog/Dependable.java Mon Sep  1 09:04:29 2008
@@ -51,6 +51,7 @@
 	public static final String HEAP							= "Heap";
 	public static final String INDEX						= "Index";
 	public static final String PREPARED_STATEMENT 			= "PreparedStatement";
+	public static final String ACTIVATION                   = "Activation";
 	public static final String FILE                         = "File";
 	public static final String STORED_PREPARED_STATEMENT	= "StoredPreparedStatement";
 	public static final String TABLE						= "Table";

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/Activation.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/Activation.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/Activation.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/Activation.java Mon Sep  1 09:04:29 2008
@@ -25,6 +25,7 @@
 
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 import org.apache.derby.iapi.sql.conn.SQLSessionContext;
+import org.apache.derby.iapi.sql.depend.Dependent;
 
 import org.apache.derby.iapi.sql.dictionary.IndexRowGenerator;
 import org.apache.derby.iapi.sql.dictionary.TableDescriptor;
@@ -71,7 +72,7 @@
  *
  */
 
-public interface Activation
+public interface Activation extends Dependent
 {
 	/**
 	 * Resets the activation to the "pre-execution" state -

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/depend/DependencyManager.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/depend/DependencyManager.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/depend/DependencyManager.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/depend/DependencyManager.java Mon Sep  1 09:04:29 2008
@@ -326,11 +326,16 @@
 	public static final int REVOKE_PRIVILEGE_RESTRICT = 45;
 	public static final int DROP_COLUMN_RESTRICT = 46;
 
-	// Revoke action when a granted role is revoked. A revoke
-	// statement causes the dependents to drop.
+	// Revoke action when a granted role is dropped/revoked. A revoke statement
+	// causes the dependents to drop.
 	public static final int REVOKE_ROLE = 47;
 
-    /**
+	// Action when the current role is changed in a session. Used to force
+	// rechecking of privileges for prepared statements that depend on the
+	// current role for privileges by recreating the activation.
+	public static final int RECHECK_PRIVILEGES = 48;
+
+	/**
      * Extensions to this interface may use action codes > MAX_ACTION_CODE without fear of
      * clashing with action codes in this base interface.
      */

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ConstraintDescriptor.java Mon Sep  1 09:04:29 2008
@@ -560,6 +560,8 @@
 		    case DependencyManager.REVOKE_PRIVILEGE:
 		    case DependencyManager.REVOKE_ROLE:
 		    case DependencyManager.INTERNAL_RECOMPILE_REQUEST:
+				// Only used by Activations
+		    case DependencyManager.RECHECK_PRIVILEGES:
 				break;
 
 			/*
@@ -646,7 +648,8 @@
 			(action != DependencyManager.SET_CONSTRAINTS_ENABLE) &&
 			(action != DependencyManager.SET_TRIGGERS_ENABLE) &&
 			(action != DependencyManager.SET_TRIGGERS_DISABLE) &&
-			(action != DependencyManager.INTERNAL_RECOMPILE_REQUEST)
+			(action != DependencyManager.INTERNAL_RECOMPILE_REQUEST) &&
+			(action != DependencyManager.RECHECK_PRIVILEGES)
 		   )
 		{
 			/* 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementColumnPermission.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementColumnPermission.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementColumnPermission.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementColumnPermission.java Mon Sep  1 09:04:29 2008
@@ -29,6 +29,8 @@
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 import org.apache.derby.iapi.sql.Activation;
 import org.apache.derby.iapi.sql.execute.ExecPreparedStatement;
+import org.apache.derby.iapi.sql.depend.DependencyManager;
+import org.apache.derby.iapi.services.context.ContextManager;
 
 /**
  * This class describes a column permission used (required) by a statement.
@@ -209,20 +211,21 @@
 			}
 		} else {
 			// We found and successfully applied a role to resolve the
-			// remaining required permissions.
+			// (remaining) required permissions.
 			//
-			// So add a dependency on the role (qua provider), so that
-			// if role is no longer available to the current user
-			// (e.g. grant to user is revoked, role is dropped,
-			// another role has been set), or it is impacted by
-			// revoked permissions or other roles granted to it, we
-			// are able to invalidate the the ps.
-			//
-			// FIXME: Rather invalidate Activation so other
-			// sessions sharing the same ps are not impacted!!
-			dd.getDependencyManager().
-				addDependency(ps, dd.getRoleDefinitionDescriptor(role),
-							  lcc.getContextManager());
+			// Also add a dependency on the role (qua provider), so
+			// that if role is no longer available to the current
+			// user (e.g. grant is revoked, role is dropped,
+			// another role has been set), we are able to
+			// invalidate the ps or activation (the latter is used
+			// if the current role changes).
+			DependencyManager dm = dd.getDependencyManager();
+			RoleGrantDescriptor rgd =
+				dd.getRoleDefinitionDescriptor(role);
+			ContextManager cm = lcc.getContextManager();
+
+			dm.addDependency(ps, rgd, cm);
+			dm.addDependency(activation, rgd, cm);
 		}
 
 	} // end of check

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementRoutinePermission.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementRoutinePermission.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementRoutinePermission.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementRoutinePermission.java Mon Sep  1 09:04:29 2008
@@ -30,7 +30,8 @@
 import org.apache.derby.iapi.store.access.TransactionController;
 import org.apache.derby.iapi.sql.Activation;
 import org.apache.derby.iapi.sql.execute.ExecPreparedStatement;
-
+import org.apache.derby.iapi.sql.depend.DependencyManager;
+import org.apache.derby.iapi.services.context.ContextManager;
 /**
  * This class describes a routine execute permission
  * required by a statement.
@@ -131,17 +132,16 @@
 			}
 
 			if (resolved /* using a role*/) {
-				// Also add a dependency on the role (qua provider),
-				// so that if role is no longer available to the
-				// current user (e.g. grant is revoked, role is
-				// dropped, another role has been set), we are able to
-				// invalidate the the ps.
-				//
-				// FIXME: Rather invalidate Activation so other
-				// sessions sharing the same ps are not impacted!!
-				dd.getDependencyManager().
-					addDependency(ps, dd.getRoleDefinitionDescriptor(role),
-								  lcc.getContextManager());
+				// Also add a dependency on the role (qua provider), so that if
+				// role is no longer available to the current user (e.g. grant
+				// is revoked, role is dropped, another role has been set), we
+				// are able to invalidate the ps or activation (the latter is
+				// used if the current role changes).
+				DependencyManager dm = dd.getDependencyManager();
+				RoleGrantDescriptor rgd = dd.getRoleDefinitionDescriptor(role);
+				ContextManager cm = lcc.getContextManager();
+				dm.addDependency(ps, rgd, cm);
+				dm.addDependency(activation, rgd, cm);
 			}
 		}
 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementTablePermission.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementTablePermission.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementTablePermission.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/StatementTablePermission.java Mon Sep  1 09:04:29 2008
@@ -28,6 +28,8 @@
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.sql.Activation;
 import org.apache.derby.iapi.sql.execute.ExecPreparedStatement;
+import org.apache.derby.iapi.sql.depend.DependencyManager;
+import org.apache.derby.iapi.services.context.ContextManager;
 
 /**
  * This class describes a table permission required by a statement.
@@ -213,18 +215,19 @@
 					}
 
 					if (result) {
-						// Also add a dependency on the role (qua provider),
-						// so that if role is no longer available to the
-						// current user (e.g. grant is revoked, role is
-						// dropped, another role has been set), we are able to
-						// invalidate the the ps.
-						//
-						// FIXME: Rather invalidate Activation so other
-						// sessions sharing the same ps are not impacted!!
-						dd.getDependencyManager().
-							addDependency(ps,
-										  dd.getRoleDefinitionDescriptor(role),
-										  lcc.getContextManager());
+						// Also add a dependency on the role (qua provider), so
+						// that if role is no longer available to the current
+						// user (e.g. grant is revoked, role is dropped,
+						// another role has been set), we are able to
+						// invalidate the ps or activation (the latter is used
+						// if the current role changes).
+						DependencyManager dm = dd.getDependencyManager();
+						RoleGrantDescriptor rgd =
+							dd.getRoleDefinitionDescriptor(role);
+						ContextManager cm = lcc.getContextManager();
+
+						dm.addDependency(ps, rgd, cm);
+						dm.addDependency(activation, rgd, cm);
 					}
 				}
 			}

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ViewDescriptor.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ViewDescriptor.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ViewDescriptor.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/sql/dictionary/ViewDescriptor.java Mon Sep  1 09:04:29 2008
@@ -280,6 +280,10 @@
 			// ViewDescriptor drop itself.
 			case DependencyManager.REVOKE_ROLE:
 
+			// Only used by Activations
+		    case DependencyManager.RECHECK_PRIVILEGES:
+
+				break;
 				// When REVOKE_PRIVILEGE gets sent to a
 				// TablePermsDescriptor we must also send
 				// INTERNAL_RECOMPILE_REQUEST to its Dependents which
@@ -344,6 +348,9 @@
 			case DependencyManager.DROP_STATISTICS:
 			case DependencyManager.TRUNCATE_TABLE:
 
+				// Only used by Activations
+			case DependencyManager.RECHECK_PRIVILEGES:
+
 				// When REVOKE_PRIVILEGE gets sent to a
 				// TablePermsDescriptor we must also send
 				// INTERNAL_RECOMPILE_REQUEST to its Dependents which
@@ -353,12 +360,13 @@
 		    case DependencyManager.INTERNAL_RECOMPILE_REQUEST:
 				break;
 
-			//When REVOKE_PRIVILEGE gets sent (this happens for privilege 
-			//types SELECT, UPDATE, DELETE, INSERT, REFERENCES, TRIGGER), we  
-			//make the ViewDescriptor drop itself. 
+				// When REVOKE_PRIVILEGE gets sent (this happens for privilege
+				// types SELECT, UPDATE, DELETE, INSERT, REFERENCES, TRIGGER),
+				// we make the ViewDescriptor drop itself. REVOKE_ROLE also
+				// drops the dependent view.
 		    case DependencyManager.REVOKE_PRIVILEGE:
 		    case DependencyManager.DROP_COLUMN:
-		    case DependencyManager.REVOKE_ROLE:
+			case DependencyManager.REVOKE_ROLE:
 				drop(lcc, 
 						getDataDictionary().getTableDescriptor(uuid).getSchemaDescriptor(),
 						getDataDictionary().getTableDescriptor(uuid));

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericActivationHolder.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericActivationHolder.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericActivationHolder.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericActivationHolder.java Mon Sep  1 09:04:29 2008
@@ -21,6 +21,10 @@
 
 package org.apache.derby.impl.sql;
 
+import	org.apache.derby.catalog.Dependable;
+import	org.apache.derby.catalog.DependableFinder;
+import org.apache.derby.catalog.UUID;
+
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
 import org.apache.derby.iapi.sql.conn.SQLSessionContext;
 
@@ -34,6 +38,9 @@
 
 import org.apache.derby.impl.sql.execute.BaseActivation;
 
+import org.apache.derby.iapi.sql.depend.DependencyManager;
+import org.apache.derby.iapi.sql.depend.Provider;
+
 import org.apache.derby.iapi.types.DataTypeDescriptor;
 import org.apache.derby.iapi.sql.ParameterValueSet;
 import org.apache.derby.iapi.sql.ResultSet;
@@ -97,7 +104,6 @@
 	GeneratedClass			gc;
 	DataTypeDescriptor[]	paramTypes;
 	private final LanguageConnectionContext lcc;
-
 	/**
 	 * Constructor for an ActivationHolder
 	 *
@@ -252,30 +258,39 @@
 		*/
 		// synchronized (ps)
 		{
-			/* Has the activation class changed? */
-			if (gc != ps.getActivationClass())
+			/* Has the activation class changed or has the activation been
+			 * invalidated? */
+			if (gc != ps.getActivationClass() || !ac.isValid())
 			{
 
                 GeneratedClass newGC;
 
-				// ensure the statement is valid by rePreparing it.
-                // DERBY-3260: If someone else reprepares the statement at the
-                // same time as we do, there's a window between the calls to
-                // rePrepare() and getActivationClass() when the activation
-                // class can be set to null, leading to NullPointerException
-                // being thrown later. Therefore, synchronize on ps to close
-                // the window.
-                synchronized (ps) {
-                    ps.rePrepare(getLanguageConnectionContext());
-                    newGC = ps.getActivationClass();
-                }
+				if (gc != ps.getActivationClass()) {
+					// ensure the statement is valid by rePreparing it.
+					// DERBY-3260: If someone else reprepares the statement at
+					// the same time as we do, there's a window between the
+					// calls to rePrepare() and getActivationClass() when the
+					// activation class can be set to null, leading to
+					// NullPointerException being thrown later. Therefore,
+					// synchronize on ps to close the window.
+					synchronized (ps) {
+						ps.rePrepare(getLanguageConnectionContext());
+						newGC = ps.getActivationClass();
+					}
+				} else {
+					// Reuse the generated class, we just want a new activation
+					// since the old is no longer valid.
+					newGC = gc;
+				}
+
 
 				/*
-				** If we get here, it means the PreparedStatement has been
-				** recompiled.  Get a new Activation and check whether the
-				** parameters are compatible.  If so, transfer the parameters
-				** from the old Activation to the new one, and make that the
-				** current Activation.  If not, throw an exception.
+				** If we get here, it means the Activation has been invalidated
+				** or the PreparedStatement has been recompiled.  Get a new
+				** Activation and check whether the parameters are compatible.
+				** If so, transfer the parameters from the old Activation to
+				** the new one, and make that the current Activation.  If not,
+				** throw an exception.
 				*/
 				BaseActivation		newAC = (BaseActivation) newGC.newInstance(lcc);
 
@@ -578,8 +593,127 @@
 	}
 
 
-	/* Class implementation */
+	/* Dependable interface implementation */
 
+	/**
+	 * @see Dependable#getDependableFinder
+	 */
+	public DependableFinder getDependableFinder()
+	{
+		// Vacuous implementation to make class concrete, only needed for
+		// BaseActivation
+		if (SanityManager.DEBUG) {
+			SanityManager.NOTREACHED();
+		}
+
+		return null;
+	}
+
+
+	/**
+	 * @see Dependable#getObjectName
+	 */
+	public String getObjectName()
+	{
+		// Vacuous implementation to make class concrete, only needed for
+		// BaseActivation
+		if (SanityManager.DEBUG) {
+			SanityManager.NOTREACHED();
+		}
+
+		return null;
+	}
+
+
+
+	/**
+	 * @see Dependable#getObjectID
+	 */
+	public UUID getObjectID()
+	{
+		// Vacuous implementation to make class concrete, only needed for
+		// BaseActivation
+		if (SanityManager.DEBUG) {
+			SanityManager.NOTREACHED();
+		}
+
+		return null;
+	}
+
+
+	/**
+	 * @see Dependable#getClassType
+	 */
+	public String getClassType()
+	{
+		// Vacuous implementation to make class concrete, only needed for
+		// BaseActivation
+		if (SanityManager.DEBUG) {
+			SanityManager.NOTREACHED();
+		}
+
+		return null;
+	}
+
+
+	/**
+	 * @see Dependable#isPersistent
+	 */
+	public boolean isPersistent()
+	{
+		// Vacuous implementation to make class concrete, only needed for
+		// BaseActivation
+		if (SanityManager.DEBUG) {
+			SanityManager.NOTREACHED();
+		}
+
+		return false;
+	}
+
+
+	/* Dependent interface implementation */
+
+	/**
+	 * @see Dependent#isValid
+	 */
+	public boolean isValid() {
+		// Vacuous implementation to make class concrete, only needed for
+		// BaseActivation
+		if (SanityManager.DEBUG) {
+			SanityManager.NOTREACHED();
+		}
+
+		return false;
+	}
+
+	/**
+	 * @see Dependent#makeInvalid
+	 */
+	public void makeInvalid(int action,
+							LanguageConnectionContext lcc)
+			throws StandardException {
+		// Vacuous implementation to make class concrete, only needed for
+		// BaseActivation
+		if (SanityManager.DEBUG) {
+			SanityManager.NOTREACHED();
+		}
+	}
+
+	/**
+	 * @see Dependent#prepareToInvalidate
+	 */
+	public void prepareToInvalidate(Provider p, int action,
+							 LanguageConnectionContext lcc)
+			throws StandardException {
+		// Vacuous implementation to make class concrete, only needed for
+		// BaseActivation
+		if (SanityManager.DEBUG) {
+			SanityManager.NOTREACHED();
+		}
+	}
+
+
+	/* Class implementation */
 
 	/**
 	 * Mark the activation as unused.  

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericPreparedStatement.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericPreparedStatement.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericPreparedStatement.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/GenericPreparedStatement.java Mon Sep  1 09:04:29 2008
@@ -699,6 +699,8 @@
 		switch (action) {
 		case DependencyManager.CHANGED_CURSOR:
 		case DependencyManager.CREATE_INDEX:
+			// Used by activations only:
+		case DependencyManager.RECHECK_PRIVILEGES:
 			return;
 		}
 
@@ -722,7 +724,12 @@
 	{
 
 		boolean alreadyInvalid;
-		
+
+		switch (action) {
+		case DependencyManager.RECHECK_PRIVILEGES:
+			return;
+		}
+
 		synchronized (this) {
 
 			if (compilingStatement)

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/depend/BasicDependencyManager.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/depend/BasicDependencyManager.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/depend/BasicDependencyManager.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/depend/BasicDependencyManager.java Mon Sep  1 09:04:29 2008
@@ -881,6 +881,9 @@
 		    case REVOKE_ROLE:
 				return "REVOKE ROLE";
 
+		    case RECHECK_PRIVILEGES:
+				return "RECHECK PRIVILEGES";
+
 			default:
 				if (SanityManager.DEBUG)
 				{

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/BaseActivation.java Mon Sep  1 09:04:29 2008
@@ -29,6 +29,13 @@
 import java.util.Hashtable;
 import java.util.Vector;
 
+import	org.apache.derby.catalog.Dependable;
+import	org.apache.derby.catalog.DependableFinder;
+import org.apache.derby.catalog.UUID;
+import org.apache.derby.iapi.services.uuid.UUIDFactory;
+import org.apache.derby.iapi.services.monitor.Monitor;
+import org.apache.derby.iapi.sql.depend.Provider;
+
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.jdbc.ConnectionContext;
 import org.apache.derby.iapi.reference.Property;
@@ -142,6 +149,30 @@
 	private String[] autoGeneratedKeysColumnNames ;
 
 	/**
+	 * By setting isValid to false, we can force a new activation to be used
+	 * even if the prepared statement is still valid. This is used when
+	 * modifying the current role for a session, which may (if the statement
+	 * relies on privileges obtained via the current role) require rechecking
+	 * of privileges. The checking normally only happens the first time the
+	 * prepared statement is used in a session, when the activation is
+	 * constructed. Forcing creation of a new activation achieves the purpose
+	 * of getting the check performed over again and is cheaper than
+	 * invalidating the prepared statement itself. Also, the latter would
+	 * impact other sessions, forcing them to recreate their activations.
+	 */
+	private boolean isValid;
+
+	/**
+	 * For dependency tracking
+	 */
+	protected String UUIDString;
+
+	/**
+	 * For dependency tracking
+	 */
+	protected UUID   UUIDValue;
+
+	/**
 	 * The 'callActivation' of an activation of a statement executing in
 	 * the root connection is null.
 	 *
@@ -215,6 +246,15 @@
 		
 		// add this activation to the pool for the connection.
 		lcc.addActivation(this);
+
+		isValid = true;
+
+		/* Get the UUID for this activation */
+		UUIDFactory uuidFactory =
+			Monitor.getMonitor().getUUIDFactory();
+
+		UUIDValue = uuidFactory.createUUID();
+		UUIDString = UUIDValue.toString();
 	}
 
 
@@ -1686,4 +1726,104 @@
         }
         return reUse;
     } // end of getDB2Length
+
+
+	/* Dependable interface implementation */
+
+	/**
+	 * @see Dependable#getDependableFinder
+	 */
+	public DependableFinder getDependableFinder()
+	{
+	    return null;
+	}
+
+
+	/**
+	 * @see Dependable#getObjectName
+	 */
+	public String getObjectName()
+	{
+		return UUIDString;
+	}
+
+
+	/**
+	 * @see Dependable#getObjectID
+	 */
+	public UUID getObjectID()
+	{
+		return UUIDValue;
+	}
+
+
+	/**
+	 * @see Dependable#getClassType
+	 */
+	public String getClassType()
+	{
+		return Dependable.ACTIVATION;
+	}
+
+
+	/**
+	 * @see Dependable#isPersistent
+	 */
+	public boolean isPersistent()
+	{
+		/* activations are not persistent */
+		return false;
+	}
+
+
+	/* Dependent interface implementation */
+
+	/**
+	 * @see Dependent#isValid
+	 */
+	public boolean isValid() {
+		return isValid;
+	}
+
+	/**
+	 * @see Dependent#makeInvalid
+	 */
+	public void makeInvalid(int action,
+							LanguageConnectionContext lcc)
+			throws StandardException {
+
+		switch (action) {
+		case DependencyManager.RECHECK_PRIVILEGES:
+			// Make ourselves invalid.
+			isValid = false;
+			/* Clear out the old dependencies on this activation as we
+			 * will die shortly.
+			 */
+			DependencyManager dm =
+				lcc.getDataDictionary().getDependencyManager();
+			dm.clearDependencies(lcc, this);
+
+			break;
+		case DependencyManager.REVOKE_ROLE:
+			// Used by persistent objects (views, triggers, constraints)
+			break;
+		case DependencyManager.INTERNAL_RECOMPILE_REQUEST:
+			// Used to recompile prepared statements
+			break;
+		default:
+			if (SanityManager.DEBUG) {
+				SanityManager.DEBUG_PRINT("BaseActivation", "action=" + action);
+				SanityManager.NOTREACHED();
+			}
+		}
+
+	}
+
+	/**
+	 * @see Dependent#prepareToInvalidate
+	 */
+	public void prepareToInvalidate(Provider p, int action,
+							 LanguageConnectionContext lcc)
+			throws StandardException {
+	}
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DropRoleConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DropRoleConstantAction.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DropRoleConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/DropRoleConstantAction.java Mon Sep  1 09:04:29 2008
@@ -76,7 +76,8 @@
     /**
      * This is the guts of the Execution-time logic for DROP ROLE.
      *
-     * @see org.apache.derby.iapi.sql.execute.ConstantAction#executeConstantAction
+     * @see org.apache.derby.iapi.sql.
+     *      execute.ConstantAction#executeConstantAction
      */
     public void executeConstantAction( Activation activation )
         throws StandardException
@@ -105,13 +106,11 @@
         }
 
         // When a role is dropped, for every role in its grantee closure, we
-        // call two invalidate actions.  REVOKE_ROLE and
-        // INTERNAL_RECOMPILE_REQUEST.  The latter is used to force
-        // recompilation of dependent prepared statements, the former to drop
-        // dependent objects (constraints, triggers and views).  Note that
-        // until DERBY-1632 is fixed, we risk dropping objects not really
-        // dependent on this role, but one some other role just because it
-        // inherits from this one. See also RevokeRoleConstantAction.
+        // call the REVOKE_ROLE action. It is used to invalidate dependent
+        // objects (constraints, triggers and views).  Note that until
+        // DERBY-1632 is fixed, we risk dropping objects not really dependent
+        // on this role, but one some other role just because it inherits from
+        // this one. See also RevokeRoleConstantAction.
         RoleClosureIterator rci =
             dd.createRoleClosureIterator
             (activation.getTransactionController(),
@@ -123,9 +122,6 @@
 
             dd.getDependencyManager().invalidateFor
                 (r, DependencyManager.REVOKE_ROLE, lcc);
-
-            dd.getDependencyManager().invalidateFor
-                (r, DependencyManager.INTERNAL_RECOMPILE_REQUEST, lcc);
         }
 
         rdDef.drop(lcc);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RevokeRoleConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RevokeRoleConstantAction.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RevokeRoleConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/RevokeRoleConstantAction.java Mon Sep  1 09:04:29 2008
@@ -65,7 +65,8 @@
     /**
      * This is the guts of the Execution-time logic for REVOKE role.
      *
-     * @see org.apache.derby.iapi.sql.execute.ConstantAction#executeConstantAction
+     * @see org.apache.derby.iapi.sql.execute.
+     *      ConstantAction#executeConstantAction
      */
     public void executeConstantAction(Activation activation)
             throws StandardException {
@@ -134,17 +135,15 @@
                     // NOTE: Never called yet, withAdminOption not yet
                     // implemented.
 
+                    if (SanityManager.DEBUG) {
+                        SanityManager.NOTREACHED();
+                    }
+
                     // revoke only the ADMIN OPTION from grantee
                     //
                     if (rd.isWithAdminOption()) {
-                        // Remove old descriptor and add a new one
-                        // without admin option.
-
-                        if (SanityManager.DEBUG) {
-                            SanityManager.NOTREACHED();
-                        }
-
-                        // Do invalidation.
+                        // Invalidate and remove old descriptor and add a new
+                        // one without admin option.
                         //
                         // RoleClosureIterator rci =
                         //     dd.createRoleClosureIterator
@@ -157,11 +156,6 @@
                         //
                         //   dd.getDependencyManager().invalidateFor
                         //       (rdDef, DependencyManager.REVOKE_ROLE, lcc);
-                        //
-                        //   dd.getDependencyManager().invalidateFor
-                        //       (rdDef,
-                        //        DependencyManager.INTERNAL_RECOMPILE_REQUEST,
-                        //        lcc);
                         // }
                         //
                         // rd.drop(lcc);
@@ -181,14 +175,12 @@
                     // Normal revoke of role from grantee.
                     //
                     // When a role is revoked, for every role in its grantee
-                    // closure, we call two invalidate actions.  REVOKE_ROLE
-                    // and INTERNAL_RECOMPILE_REQUEST.  The latter is used to
-                    // force recompilation of dependent prepared statements,
-                    // the former to drop dependent objects (constraints,
-                    // triggers and views).  Note that until DERBY-1632 is
-                    // fixed, we risk dropping objects not really dependent on
-                    // this role, but one some other role just because it
-                    // inherits from this one.
+                    // closure, we call the REVOKE_ROLE action. It is used to
+                    // invalidate dependent objects (constraints, triggers and
+                    // views).  Note that until DERBY-1632 is fixed, we risk
+                    // dropping objects not really dependent on this role, but
+                    // one some other role just because it inherits from this
+                    // one. See also DropRoleConstantAction.
                     RoleClosureIterator rci =
                         dd.createRoleClosureIterator
                         (activation.getTransactionController(),
@@ -200,11 +192,6 @@
 
                         dd.getDependencyManager().invalidateFor
                             (rdDef, DependencyManager.REVOKE_ROLE, lcc);
-
-                        dd.getDependencyManager().invalidateFor
-                            (rdDef,
-                             DependencyManager.INTERNAL_RECOMPILE_REQUEST,
-                             lcc);
                     }
 
                     rd.drop(lcc);

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/SetRoleConstantAction.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/SetRoleConstantAction.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/SetRoleConstantAction.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/sql/execute/SetRoleConstantAction.java Mon Sep  1 09:04:29 2008
@@ -128,7 +128,7 @@
                 if (rdDef != null) {
                     dd.getDependencyManager().invalidateFor(
                         rdDef,
-                        DependencyManager.INTERNAL_RECOMPILE_REQUEST,
+                        DependencyManager.RECHECK_PRIVILEGES,
                         lcc);
                 } // else: old role else no longer exists, so ignore.
             }

Modified: db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RolesConferredPrivilegesTest.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RolesConferredPrivilegesTest.java?rev=691007&r1=691006&r2=691007&view=diff
==============================================================================
--- db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RolesConferredPrivilegesTest.java (original)
+++ db/derby/code/trunk/java/testing/org/apache/derbyTesting/functionTests/tests/lang/RolesConferredPrivilegesTest.java Mon Sep  1 09:04:29 2008
@@ -1123,17 +1123,19 @@
                     assertPsWorks(true, ps);
 
                     /*
-                     * Setting another role should invalidate the ps
+                     * Setting another role should make the ps fail, since we
+                     * no longer have the privilege.
                      */
                     setRole(c, "none");
                     assertPsWorks(false, ps);
+                    // set it back:
+                    setRole(c, "h");
+                    assertPsWorks(true, ps);
 
                     /*
                      * Remove privileges from role, and the execute should
                      * fail.
                      */
-                    setRole(c, "h");
-                    assertPsWorks(true, ps);
                     doGrantRevoke(REVOKE, "test_dbo", privilegeStmts[i][0],
                                   grantToThisRole[r]);
 
@@ -1142,7 +1144,8 @@
                                   grantToThisRole[r]);
 
                     /*
-                     * Revoking the role should also invalidate the ps
+                     * Revoking the role should also make the ps fail, since we
+                     * no longer have the privilege.
                      */
                     setRole(c, "h");
                     assertPsWorks(true, ps);
@@ -1191,7 +1194,7 @@
         }
 
         /*
-         * Dropping a role should also invalidate a dependent ps.
+         * Dropping a role should also cause a dependent ps fail.
          *
          * (We do this test outside the loop above for simplicity of
          * reestablish role graph after the drop..)
@@ -1224,6 +1227,140 @@
 
 
     /**
+     * Test behavior for when there are open result sets on prepared statements
+     * that require privileges obtained via the current role and something
+     * changes in the middle of accessing the result set. We should be able to
+     * finish using the result set.
+     */
+    public void testOpenRs() throws SQLException {
+        Connection dboConn = getConnection();
+        Statement s = dboConn.createStatement();
+
+        Connection c = openUserConnection("DonaldDuck");
+        Statement cStmt = c.createStatement();
+        ResultSet rs = null;
+        String select = "select * from s1.t1";
+
+        PreparedStatement ps = dboConn.prepareStatement(
+            "insert into s1.t1 values (?,?,?)");
+        for (int i=0; i < 5; i++) {
+            ps.setInt(1, i);
+            ps.setInt(2, i);
+            ps.setInt(3, i);
+            ps.execute();
+        }
+
+
+        /*
+         * Select privilege revoked
+         */
+        // Auto-commit on
+        doGrantRevoke(GRANT, "test_dbo", g_s, "h");
+        s.execute("grant h to DonaldDuck");
+        setRole(c, "h");
+        rs = cStmt.executeQuery(select);
+
+        rs.next();
+        // Now remove privilege in middle of rs reading
+        doGrantRevoke(REVOKE, "test_dbo", g_s, "h");
+
+        // check that we can read the next row
+        rs.next();
+        rs.close();
+
+        // Auto-commit off
+        c.setAutoCommit(false);
+        doGrantRevoke(GRANT, "test_dbo", g_s, "h");
+        setRole(c, "h");
+        rs = cStmt.executeQuery(select);
+
+        rs.next();
+        c.commit();
+        // Now remove privilege in middle of rs reading
+        doGrantRevoke(REVOKE, "test_dbo", g_s, "h");
+
+        // check that we can read the next row
+        rs.next();
+        rs.close();
+        c.setAutoCommit(true);
+
+        /*
+         * Role privilege revoked
+         */
+        // Auto-commit on
+        doGrantRevoke(GRANT, "test_dbo", g_s, "h");
+        s.execute("grant h to DonaldDuck");
+        setRole(c, "h");
+        rs = cStmt.executeQuery(select);
+
+        rs.next();
+        // Now remove privilege in middle of rs reading
+        s.execute("revoke h from DonaldDuck");
+
+        // check that we can read the next row
+        rs.next();
+        rs.close();
+
+        // Auto-commit off
+        c.setAutoCommit(false);
+        s.execute("grant h to DonaldDuck");
+        setRole(c, "h");
+        rs = cStmt.executeQuery(select);
+
+        rs.next();
+        c.commit();
+        // Now remove privilege in middle of rs reading
+        s.execute("revoke h from DonaldDuck");
+
+        // check that we can read the next row
+        rs.next();
+        rs.close();
+        c.setAutoCommit(true);
+        doGrantRevoke(REVOKE, "test_dbo", g_s, "h");
+
+
+        /*
+         * Current role changed
+         */
+        // Auto-commit on
+        doGrantRevoke(GRANT, "test_dbo", g_s, "h");
+        s.execute("grant h to DonaldDuck");
+        setRole(c, "h");
+        c.setAutoCommit(true);
+        rs = cStmt.executeQuery(select);
+
+        rs.next();
+        // Now change role in middle of rs reading
+        setRole(c, "none");
+
+        // check that we can read the next row
+        rs.next();
+        rs.close();
+
+        // Auto-commit off
+        c.setAutoCommit(false);
+        setRole(c, "h");
+        rs = cStmt.executeQuery(select);
+
+        rs.next();
+        // Now remove privilege in middle of rs reading
+        c.commit();
+        setRole(c, "none");
+
+        // check that we can read the next row
+        rs.next();
+        rs.close();
+        c.setAutoCommit(true);
+        doGrantRevoke(REVOKE, "test_dbo", g_s, "h");
+
+        // clean up
+        s.executeUpdate("delete from s1.t1");
+        c.close();
+        dboConn.close();
+    }
+
+
+    /**
      * stored function: s1.f1
      */
     public static int s1f1() {
@@ -1274,14 +1411,14 @@
             (hasPrivilege, c, schema, table, columns);
         assertSelectPrivilege
             (hasPrivilege, c, schema, table, null);
-        assertDeletePrivilege
-            (hasPrivilege, c, schema, table);
         assertInsertPrivilege
             (hasPrivilege, c, schema, table, null);
         assertUpdatePrivilege
             (hasPrivilege, c, schema, table, columns);
         assertUpdatePrivilege
             (hasPrivilege, c, schema, table, null);
+        assertDeletePrivilege
+            (hasPrivilege, c, schema, table);
         assertReferencesPrivilege
             (hasPrivilege, c, schema, table, columns);
         assertReferencesPrivilege
@@ -1906,13 +2043,7 @@
             ResultSet rs = ps.getResultSet();
             if (rs != null) {
                 rs.next();
-                // NOTE: If we don't close the rs, invalidation of the prepared
-                // statement via set role will fail, due to
-                // verifyNoOpenResultSets called fromprepareToInvalidate. Hence
-                // setRole will fail. BUG or not? I think this behavior is
-                // OK.. OR, we could force close the rs.. Actually when we move
-                // to invalidate Activation instead it would not matter....
-                //        rs.close();
+                rs.close();
             }
             ps.getConnection().rollback();
             ps.getConnection().setAutoCommit(true);