You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@river.apache.org by pe...@apache.org on 2010/07/31 09:08:44 UTC

svn commit: r981013 - in /incubator/river/jtsk/trunk: ./ qa/src/com/sun/jini/qa/harness/ src/net/jini/security/ src/org/apache/river/imp/security/policy/se/

Author: peter_firmstone
Date: Sat Jul 31 07:08:43 2010
New Revision: 981013

URL: http://svn.apache.org/viewvc?rev=981013&view=rev
Log:
River-26 Changed the behaviour of ConcurrentPermissions to prevent the Enumeration<Permission> returned by elements() from throwing a ConcurrentModificationException, while the underlying associated PermissionCollection is being modified.  This behaviour was changed to enable UmbrellaGrantPermission's to be added to DynamicConcurrentPolicyProvider.  GrantPermission's PermissionCollection had to be changed to not add duplicate GrantPermission's, this was part of the UmbrellaGrantPermission functionality addition.

MultiReadPermissionCollection was changed to incorporate an elements cache to prevent the underlying PermissionCollection causing the Enumerator from throwing the ConcurrentModificationException when add was called concurrently.

MergedPolicyProvider had synchronization removed, the synchronization was added temporarily to assist in solving some earlier implementation concurrency bugs.

Modified:
    incubator/river/jtsk/trunk/   (props changed)
    incubator/river/jtsk/trunk/qa/src/com/sun/jini/qa/harness/MergedPolicyProvider.java
    incubator/river/jtsk/trunk/src/net/jini/security/GrantPermission.java
    incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/ConcurrentPermissions.java
    incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/DynamicConcurrentPolicyProvider.java
    incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/MultiReadPermissionCollection.java

Propchange: incubator/river/jtsk/trunk/
------------------------------------------------------------------------------
--- svn:ignore (original)
+++ svn:ignore Sat Jul 31 07:08:43 2010
@@ -4,8 +4,8 @@ build.properties
 configentry
 deps
 nbproject
-.*
 build
+.*
 qa_output.txt
 lib-dl
 dist

Modified: incubator/river/jtsk/trunk/qa/src/com/sun/jini/qa/harness/MergedPolicyProvider.java
URL: http://svn.apache.org/viewvc/incubator/river/jtsk/trunk/qa/src/com/sun/jini/qa/harness/MergedPolicyProvider.java?rev=981013&r1=981012&r2=981013&view=diff
==============================================================================
--- incubator/river/jtsk/trunk/qa/src/com/sun/jini/qa/harness/MergedPolicyProvider.java (original)
+++ incubator/river/jtsk/trunk/qa/src/com/sun/jini/qa/harness/MergedPolicyProvider.java Sat Jul 31 07:08:43 2010
@@ -102,7 +102,7 @@ public class MergedPolicyProvider extend
      *
      * @param source the <code>CodeSource</code>
      */
-    public synchronized PermissionCollection getPermissions(CodeSource source) {
+    public PermissionCollection getPermissions(CodeSource source) {
 	Iterator it = policies.iterator();
 	if (it.hasNext()) {
 	    PermissionCollection pc = 
@@ -131,7 +131,7 @@ public class MergedPolicyProvider extend
      *
      * @param domain the <code>ProtectionDomain</code>
      */
-    public synchronized PermissionCollection getPermissions(ProtectionDomain domain) {
+    public PermissionCollection getPermissions(ProtectionDomain domain) {
 	Iterator it = policies.iterator();
 	ArrayList list = new ArrayList();
 	if (it.hasNext()) {
@@ -190,7 +190,7 @@ public class MergedPolicyProvider extend
      * @param permission the <code>Permission</code> to check
      * @return true if the permission is granted
      */
-    public synchronized boolean implies(ProtectionDomain domain, Permission permission) {
+    public boolean implies(ProtectionDomain domain, Permission permission) {
 	Iterator it = policies.iterator();
 	while (it.hasNext()) {
 	    Policy p = (Policy) it.next();
@@ -205,7 +205,7 @@ public class MergedPolicyProvider extend
     /**
      * Refresh all of the underlying policies.
      */
-    public synchronized void refresh() {
+    public void refresh() {
 	System.out.println("In REFRESH");
 	Iterator it = policies.iterator();
 	while (it.hasNext()) {

Modified: incubator/river/jtsk/trunk/src/net/jini/security/GrantPermission.java
URL: http://svn.apache.org/viewvc/incubator/river/jtsk/trunk/src/net/jini/security/GrantPermission.java?rev=981013&r1=981012&r2=981013&view=diff
==============================================================================
--- incubator/river/jtsk/trunk/src/net/jini/security/GrantPermission.java (original)
+++ incubator/river/jtsk/trunk/src/net/jini/security/GrantPermission.java Sat Jul 31 07:08:43 2010
@@ -554,7 +554,7 @@ public final class GrantPermission exten
      * of permissions.
      */
     private static String constructName(Permission[] pa) {
-	StringBuffer sb = new StringBuffer();
+	StringBuffer sb = new StringBuffer(60);
 	for (int i = 0; i < pa.length; i++) {
 	    Permission p = pa[i];
 	    if (p instanceof UnresolvedPermission) {
@@ -763,7 +763,7 @@ public final class GrantPermission exten
 	    new ObjectStreamField("perms", List.class, true)
 	};
 
-	private List perms = new ArrayList();
+	private List perms = new ArrayList(40);
 	private Implier implier = new Implier();
 
 	public synchronized void add(Permission p) {
@@ -774,8 +774,10 @@ public final class GrantPermission exten
 		throw new SecurityException(
 		    "can't add to read-only PermissionCollection");
 	    }
-	    perms.add(p);
-	    implier.add((GrantPermission) p);
+	    if (!perms.contains(p)){
+		perms.add(p);
+		implier.add((GrantPermission) p);
+	    }
 	}
 	
 	public synchronized Enumeration elements() {

Modified: incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/ConcurrentPermissions.java
URL: http://svn.apache.org/viewvc/incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/ConcurrentPermissions.java?rev=981013&r1=981012&r2=981013&view=diff
==============================================================================
--- incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/ConcurrentPermissions.java (original)
+++ incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/ConcurrentPermissions.java Sat Jul 31 07:08:43 2010
@@ -27,41 +27,32 @@ import java.security.ProtectionDomain;
 import java.security.UnresolvedPermission;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.ConcurrentModificationException;
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 import java.util.Set;
+import java.util.Vector;
 import java.util.concurrent.ConcurrentHashMap;
 
 
 /**
- * Example Implementation alternative of Permissions implemented for concurrency, 
- * note this couldn't extend Permissions as it is declared final.
+ * ConcurrentPermission's is a replacement for java.security.Permissions, 
+ * it doesn't extend Permissions.
  * 
- * Note that if there is heavy contention for one Permission class
- * type, due to synchronization, concurrency will
- * suffer.  This is due to the original PermissionsCollection spec requiring
+ * If there is heavy contention for one Permission class
+ * type, concurrency may suffer due to internal synchronization.
+ * This is due to the original PermissionsCollection spec requiring
  * that all implementations do their own synchronization, this is a design
  * mistake, similar to Vector. 
  * 
- * This is an example class without optimisation, it will be slower
- * for single threaded applications and consume more memory.
- * 
- * It would also be possible to create an unlock method that permitted
- * adding or revoking permissions to this collection while it is referenced
- * from within a PermissionDomain, however that would break the contract of
- * setReadOnly().  This might make sense if implementing a SecurityManager or
- * it might not, it's just an observation that Permissions defined in policy
- * files, which are not dynamically granted, are not revokeable as a
- * Policy only augments the PermissionCollection associated with (referenced by)
- * a PermissionDomain.  However it would probably be best to extend
- * ProtectionDomain to alter this behaviour as it merges PermissionCollection's,
- * so you would end up with the Permissions implementation again, unless using the
- * constructor that sets the permissions as static, which is totally
- * contradictory.  So the best way to make a Permission revokeable is to 
- * grant it dynamically.
+ * ConcurrentPermission's defined behaviour for #elements() differs from
+ * PermissionCollection.  It is safe to alter ConcurrentPermissions while
+ * Enumerating through it's elements.  ConcurrentPermission's keeps a cache
+ * of elements, but makes no guarantees that new elements will be
+ * added during an Enumeration.
  * 
  * TODO: Serialization properly
  * @version 0.4 2009/11/10
@@ -119,12 +110,7 @@ implements Serializable {
             pc.add(permission);
             return;             
         } else {
-            PermissionCollection fresh = permission.newPermissionCollection();
-            if (fresh == null) {
-                fresh = new MultiReadPermissionCollection(permission);
-            } else {
-                fresh.add(permission);
-            }
+            PermissionCollection fresh = new MultiReadPermissionCollection(permission);   
             PermissionCollection existed = 
                     permsMap.putIfAbsent(permission.getClass(), fresh);
             if (existed != null) {
@@ -164,7 +150,11 @@ implements Serializable {
     /**
      * This Enumeration is not intended for concurrent access,
      * PermissionCollection's underlying state is protected by defensive copying, 
-     * it wont affect the thread safety of ConcurrentPermission.
+     * performed by MultiReadPermissionCollection, which creates a shared cache
+     * of the contained Permission's for each PermissionCollection, while holding a read lock
+     * this prevents the client having to deal with a ConcurrentModificationException.
+     * Thus allowing modification of the PermissionCollection while the
+     * contents are being Enumerated.
      * 
      * Any number of these Enumerations may be utilised , each accessed by 
      * a separate thread.
@@ -179,10 +169,13 @@ implements Serializable {
         ArrayList<PermissionCollection> elem = 
                 new ArrayList<PermissionCollection>(permsMap.size() 
                                     + unresolved.awaitingResolution() + 2);
+	// Unresolved Permission's are added first in case the are resolved in 
+	// the interim, meaning that they may also be present in resolved form
+	// also.  To do the reverse would risk some Permission's being absent.
         if (unresolved.awaitingResolution() > 0) {
             elem.add(unresolved);
         }
-        elem.addAll(permsMap.values());
+        elem.addAll(permsMap.values());	
         Iterator<PermissionCollection> perms = elem.iterator();
         return new PermissionEnumerator(perms);                 
     }
@@ -201,15 +194,15 @@ implements Serializable {
     }
     
     /*
-     * This Enumeration is not intended for concurrent access,
-     * PermissionCollection's underlying state is protected by defensive copying, 
-     * it wont affect the thread safety of ConcurrentPermission.
+     * This Enumeration is not intended for concurrent access, underlying
+     * PermissionCollection's need to be protected by MultiReadPermissionCollection's
+     * cache, so updates wont affect the thread safety of ConcurrentPermission.
      * 
      * Any number of these Enumerations may be utilised , each accessed by 
      * a separate thread.
      * 
      * @author Peter Firmstone
-     */
+     */   
     private final static class PermissionEnumerator implements Enumeration<Permission> {
         private final Iterator<PermissionCollection> epc;
         private volatile Enumeration<Permission> currentPermSet;
@@ -220,23 +213,15 @@ implements Serializable {
         }
 
         private Enumeration<Permission> getNextPermSet(){
-            Set<Permission> permissionSet = new HashSet<Permission>();
             if (epc.hasNext()){
                 PermissionCollection pc = epc.next();               
-                /* Local copy of the set containing a snapshot of 
-                 * references to Permission objects present at an instant in time,
-                 * we can Enumerate over, without contention or exception.  
-                 * We only take what we need as we need it, minimising memory.
-                 * Each object gets its own Enumeration.
+                /* We only take what we need, as we need it, minimising memory use.
+                 * Each underlying PermissionCollection adds its own Enumeration.
+		 * MultiReadPermissionCollection caches the elements so we
+		 * are protected from ConcurrentModificationException's
                  */
-                if ( pc instanceof Permissions){
-                    synchronized (pc){
-                        Enumeration<Permission> e = pc.elements();
-                        while (e.hasMoreElements()) {
-                            permissionSet.add(e.nextElement());
-                        }
-                    }
-                } else if ( pc instanceof PermissionPendingResolutionCollection ){
+                if ( pc instanceof PermissionPendingResolutionCollection ){
+		    Set<Permission> permissionSet = new HashSet<Permission>();
                     Enumeration<Permission> e = pc.elements();
                     while (e.hasMoreElements()) {
                         PermissionPendingResolution p = 
@@ -244,14 +229,15 @@ implements Serializable {
                         UnresolvedPermission up = p.asUnresolvedPermission();
                         permissionSet.add(up);
                     }
+		    return Collections.enumeration(permissionSet);
                 } else {
                     Enumeration<Permission> e = pc.elements();
-                    while (e.hasMoreElements()) {
-                        permissionSet.add(e.nextElement());
-                    }
+                    return e;
                 }
-            }
-            return Collections.enumeration(permissionSet);
+            } else {
+		Vector<Permission> empty = new Vector<Permission>(0);
+		return empty.elements();
+	    }
         }
 
         public boolean hasMoreElements() {        

Modified: incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/DynamicConcurrentPolicyProvider.java
URL: http://svn.apache.org/viewvc/incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/DynamicConcurrentPolicyProvider.java?rev=981013&r1=981012&r2=981013&view=diff
==============================================================================
--- incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/DynamicConcurrentPolicyProvider.java (original)
+++ incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/DynamicConcurrentPolicyProvider.java Sat Jul 31 07:08:43 2010
@@ -48,27 +48,55 @@ import org.apache.river.imp.util.Concurr
  * cost however, that of increased memory usage.</p>
  * 
  * <p>Due to the Java 2 Security system's static design, a Policy Provider
- * can only augment the policy files utilised, that is it can only relax security
+ * can only augment the policy files utilised, a Policy can only relax security
  * by granting additional permissions, this implementation adds an experimental 
  * feature for revoking permissions, however there are some caveats:</p>
  * 
- * <p>Firstly if the Policy.refresh() method is called, followed by the 
- * ProtectionDomain.toString() method, the ProtectionDomain
- * merge the permissions, from the policy with those in the ProtectionDomain, 
- * a ProtectionDomain cannot have Permissions
- * removed, only additional merged. </p>
+ * <p>Background: if ProtectionDomain.toString(), is called a ProtectionDomain will
+ * merge Permissions, from the policy with those in the ProtectionDomain,
+ * in a new private instance of Permissions, thus a ProtectionDomain cannot have 
+ * Permission's removed, only additional merged.  A ProtectionDomain must
+ * be created with the dynamic constructor otherwise it will never consult
+ * the policy.  The AccessController.checkPermission(Permission) method
+ * consults the current AccessControlContext, which contains all
+ * ProtectionDomain's on the current thread's stack, before consulting the
+ * AccessControllContext.checkPermission(Permission), it calls
+ * AccessControllContext.optimize() which  removes all duplicate ProtectionDomains
+ * in the ProtectionDomain array[]'s from the
+ * enclosing AccessContolContext for the execution domain and the nested
+ * AccessControlContext for the privileged domain (the privileged domain is
+ * an array of ProtectionDomain's on the stack since the last 
+ * AccessController.doPriveleged() call).  The optimize() method also calls
+ * the DomainCombiner, which, for example, gives the SubjectDomainCombiner the 
+ * opportunity to manipulate the ProtectionDomain's in the privileged array, in the
+ * SubjectDomainCombiner's case, it creates new copies of the ProtectionDomain's
+ * with new Principal[]'s injected.  The optimize() method returns a new
+ * optimized AccessControlContext.
+ * </p><p>
+ * Now the AccessController calls the new AccessControlContext.checkPermission(Permission),
+ * at this stage, each ProtectionDomain, if created with the dynamic constructor
+ * consults the Policy, calling Policy.implies(ProtectionDomain, Permission).
+ * </p><p>
+ * If any calls to the policy return false, the ProtectionDomain then checks its
+ * internal Permissions and if they return false, it returns false.  The first
+ * ProtectionDomain in the AccessControlContext to return false causes the 
+ * AccessController.checkPermission(Permission) to throw an AccessControlException
+ * </p><p>
+ * To optimise the time taken to check Permission's the ProtectionDomain's
+ * should either be static, which excludes the Policy, or dynamic with
+ * a null PermissionCollection in it's constructor, </p>
  * 
  * <p>So in order to prevent dynamic grants from finding
  * their way into a ProtectionDomain's private PermissionCollection,
  * one would have to ensure that no dynamically grantable permissions are 
- * returned via the methods:</p>
+ * returned via the method:</p>
  * <p>
- * getPermissions(Codesource source) or
- * getPermissions(ProtectionDomain domain)
+ * getPermissions(ProtectionDomain domain) and
+ * getPermissions(Codesource source) as a precaution.
  * </p>
  * <p>This is different to the behaviour of the existing Jini 2.0
  * DynamicPolicyProvider implementation where dynamically granted Permissions
- * are added.
+ * are added and can escape into the ProtectionDomain's private PermissionCollection.
  * 
  * However when a Policy is checked via implies(ProtectionDomain d, Permission p)
  * this implementation checks the dynamic grants
@@ -77,17 +105,12 @@ import org.apache.river.imp.util.Concurr
  * and if it returns dynamically granted permissions, then those permissions
  * cannot be revoked.</p>
  * <p>
- * It is thus reccommeded that Static policy files only be used for files
- * where the level of trust is relatively static.  This is the only implementation
- * where a dyanamic grant can be removed.  In the case of Proxy trust, a proxy
- * is no longer trusted when it has lost contact with it's Principal (server)
- * because the server cannot be asked if it trusts it's proxy and the proxy
- * cannot be given a thread of control to find it's server because it has
- * already attained too many Permissions.  In this new implementation it should
- * be possible to revoke AllPermission and grant Permissions dynamically as 
- * trust is gained.</p>
- * <p>
- * This may cause some undesireable side effects in existing programs.
+ * It is thus reccommeded that Static policy files only be used for setting
+ * up your priveleged code and use UmbrellaGrantPermission's and grant 
+ * all other Permission's using dynamic grants.  This minimises the double 
+ * checking of Permission, that occurs when a ProtectionDomain is constructed
+ * so it contains a default PermissionCollection that is not null.
+ *
  * </p><p>
  * To make the best utilisation of this Policy provider, set the System property:
  * </p>,<p>
@@ -110,12 +133,16 @@ public class DynamicConcurrentPolicyProv
 	    public ProtectionDomain run() { return Object.class.getProtectionDomain(); }
 	});
     
-    // A set of PolicyEntries constituting this Policy.
-    // PolicyEntry is lighter weight than PermissionCollection.
-    private final ReentrantReadWriteLock rwl;
-    private final ReadLock rl;
-    private final WriteLock wl;    
-    private final Set<PermissionGrant> dynamicGrants; // protected by rwl
+    /* reference update Protected by grantLock, this array reference must only 
+     * be copied or replaced, it must never be read directly or operated on 
+     * unless holding grantLock.
+     * Local methods must first copy the reference before using the array in
+     * loops etc in case the reference is updated.
+     */
+    private volatile PermissionGrant[] pGrants;
+    /* This lock protects adding and removal of PermissionGrant's*/
+    private final Object grantLock;
+    //private final Set<PermissionGrant> dynamicGrants; // protected by rwl
     private volatile Policy basePolicy; // effectively final looks after its own sync
     private final ConcurrentMap<ProtectionDomain, PermissionCollection> cache;
     private volatile boolean basePolicyIsDynamic; // Don't use cache if true.
@@ -134,10 +161,7 @@ public class DynamicConcurrentPolicyProv
     
     
     public DynamicConcurrentPolicyProvider(){
-        rwl = new ReentrantReadWriteLock();
-        rl = rwl.readLock();
-        wl = rwl.writeLock();
-        dynamicGrants = new HashSet<PermissionGrant>(30);
+	pGrants = new PermissionGrant[0];
         basePolicy = null;
         cache = new ConcurrentWeakIdentityMap<ProtectionDomain, PermissionCollection>();
         basePolicyIsDynamic = false;
@@ -149,6 +173,7 @@ public class DynamicConcurrentPolicyProv
         dwl = drwl.writeLock();
         denied = new HashSet<Denied>(30);
         checkDenied = false;
+	grantLock = new Object();
     }
     
     /**
@@ -307,17 +332,26 @@ public class DynamicConcurrentPolicyProv
                     ": Base policy is not dynamic and returned false" );
         }
         // If the base policy doesn't imply a Permission then we should check for dynamic grants
-        Collection<Permission> dynamicallyGrantedPermissions = new HashSet<Permission>();
-        try {
-            rl.lock();
-            Iterator<PermissionGrant> it = dynamicGrants.iterator();
-            while (it.hasNext()) {
-                PermissionGrant ge = it.next();
-                if ( ge.implies(domain)) {
-                    dynamicallyGrantedPermissions.addAll( Arrays.asList(ge.getPermissions()));
-                }
-            }               
-        } finally { rl.unlock(); }
+        Collection<Permission> dynamicallyGrantedPermissions = new HashSet<Permission>(pGrants.length);
+//        try {
+//            rl.lock();
+//            Iterator<PermissionGrant> it = dynamicGrants.iterator();
+//            while (it.hasNext()) {
+//                PermissionGrant ge = it.next();
+//                if ( ge.implies(domain)) {
+//                    dynamicallyGrantedPermissions.addAll( Arrays.asList(ge.getPermissions()));
+//                }
+//            }               
+//        } finally { rl.unlock(); }
+	PermissionGrant[] grantsRefCopy = pGrants; // In case the grants volatile reference is updated.
+	int l = grantsRefCopy.length;
+	for ( int i = 0; i < l; i++){
+	    if (grantsRefCopy[i].implies(domain)) {
+		dynamicallyGrantedPermissions.addAll( 
+			Arrays.asList(grantsRefCopy[i].getPermissions())
+			);
+	    }
+	}
         if (loggable) {
             logger.log(Level.FINEST, "Grants: " + dynamicallyGrantedPermissions.toString());
         }
@@ -341,6 +375,7 @@ public class DynamicConcurrentPolicyProv
             logger.log(Level.FINEST, "PermissionCollection: " + pc.toString());
         }
         // We have added dynamic grants, lets expand them
+	// But UmbrellaGrant's are to enable easy dynamic GrantPermission's?
         expandUmbrella(pc);
         return pc.implies(permission);
     }
@@ -355,16 +390,30 @@ public class DynamicConcurrentPolicyProv
         cache.clear();
         basePolicy.refresh();
         // Clean up any void grants.
-        try {
-            wl.lock();
-            Iterator<PermissionGrant> it = dynamicGrants.iterator();
-            while (it.hasNext()){
-                PermissionGrant pe = it.next();
-                if ( pe.isVoid()){
-                    it.remove();
-                }
-            }
-        } finally {wl.unlock();}
+	synchronized (grantLock) {
+	    // This lock doesn't stop reads to grants only other volatile reference updates.
+	    // Manipulating, alterations (writes) to the pGrants array is prohibited.
+	    int l = pGrants.length;
+	    ArrayList<PermissionGrant> grantHolder 
+		    = new ArrayList<PermissionGrant>(l);
+	    for ( int i = 0; i < l; i++ ){
+		if ( pGrants[i].isVoid()) continue;
+		grantHolder.add(pGrants[i]);
+	    }
+	    PermissionGrant[] remaining = new PermissionGrant[grantHolder.size()];
+	    pGrants = grantHolder.toArray(remaining); // Volatile reference update.
+	}
+	
+//        try {
+//            wl.lock();
+//            Iterator<PermissionGrant> it = dynamicGrants.iterator();
+//            while (it.hasNext()){
+//                PermissionGrant pe = it.next();
+//                if ( pe.isVoid()){
+//                    it.remove();
+//                }
+//            }
+//        } finally {wl.unlock();}
         ensureDependenciesResolved();
     }
 
@@ -406,10 +455,17 @@ public class DynamicConcurrentPolicyProv
         if (loggable){
             logger.log(Level.FINEST, "Granting: " + pe.toString());
         }
-        try {
-            wl.lock();
-            dynamicGrants.add(pe);           
-        } finally {wl.unlock();}
+	HashSet<PermissionGrant> holder = new HashSet<PermissionGrant>(pGrants.length +1 );
+	synchronized (grantLock){
+	    holder.addAll(Arrays.asList(pGrants));
+	    holder.add(pe);
+	    PermissionGrant [] updated = new PermissionGrant[holder.size()];
+	    pGrants = holder.toArray(updated);
+	}
+//        try {
+//            wl.lock();
+//            dynamicGrants.add(pe);           
+//        } finally {wl.unlock();}
     }
     
     // documentation inherited from DynamicPolicy.getGrants
@@ -424,20 +480,28 @@ public class DynamicConcurrentPolicyProv
 	    principals = principals.clone();
 	    checkNullElements(principals);
 	}
-        Collection<Permission> cperms = new HashSet<Permission>();
-        try {
-            rl.lock();
-            Iterator<PermissionGrant> it = dynamicGrants.iterator();
-            while (it.hasNext()) {
-                PermissionGrant ge = it.next();
-                // We want to capture
-                // all grants that may be granted by other means.
-                // such as ProtectionDomain, Certificates, CodeSource or Principals alone.
-                if ( ge.implies(loader, principals)) {
-                    cperms.addAll(Arrays.asList(ge.getPermissions()));
-                }     
-            }
-        } finally { rl.unlock(); }
+        Collection<Permission> cperms = new HashSet<Permission>(pGrants.length);
+//        try {
+//            rl.lock();
+//            Iterator<PermissionGrant> it = dynamicGrants.iterator();
+//            while (it.hasNext()) {
+//                PermissionGrant ge = it.next();
+//                // We want to capture
+//                // all grants that may be granted by other means.
+//                // such as ProtectionDomain, Certificates, CodeSource or Principals alone.
+//                if ( ge.implies(loader, principals)) {
+//                    cperms.addAll(Arrays.asList(ge.getPermissions()));
+//                }     
+//            }
+//        } finally { rl.unlock(); }
+	PermissionGrant [] grantsRefCopy = pGrants; // Interim updates not seen.
+	int l = grantsRefCopy.length;
+	for ( int i = 0; i < l; i++ ){
+	    if ( grantsRefCopy[i].implies(loader, principals) ){
+		cperms.addAll(Arrays.asList(grantsRefCopy[i].getPermissions()));
+	    }
+	}
+	
         Permission[] perms = cperms.toArray(new Permission[cperms.size()]);        
         return perms;
     }
@@ -469,10 +533,26 @@ public class DynamicConcurrentPolicyProv
             AccessController.checkPermission(new GrantPermission(perms));
         }
         // If we get to here, the caller has permission.
-        try {
-            wl.lock();
-            dynamicGrants.addAll(grants);
-        } finally {wl.unlock();}
+	// This is slightly naughty calling a pGrants method, however if it
+	// changes between now and gaining the lock, only the length of the
+	// HashSet is potentially not optimal, keeping the HashSet creation
+	// outside of the lock reduces the lock held duration.
+	HashSet<PermissionGrant> holder 
+		    = new HashSet<PermissionGrant>(grants.size() + pGrants.length);
+	    holder.addAll(grants);
+	synchronized (grantLock) {	    
+	    int l = pGrants.length;
+	    for ( int i = 0; i < l; i++ ){
+		if (pGrants[i].isVoid()) continue;
+		holder.add(pGrants[i]);
+	    }
+	    PermissionGrant[] updated = new PermissionGrant[holder.size()];
+	    pGrants = holder.toArray(updated);
+	}
+//        try {
+//            wl.lock();
+//            dynamicGrants.addAll(grants);
+//        } finally {wl.unlock();}
     }
 
     public void revoke(List<PermissionGrant> grants) {
@@ -483,11 +563,21 @@ public class DynamicConcurrentPolicyProv
             return;
         }
         AccessController.checkPermission(new RevokePermission());
-        try {
-            wl.lock();
-            dynamicGrants.removeAll(grants);
-        } finally {wl.unlock();}
-        cache.clear();
+	HashSet<PermissionGrant> holder = new HashSet<PermissionGrant>(pGrants.length);
+	synchronized (grantLock){
+	    int l = pGrants.length;
+	    for (int i = 0; i < l; i++){
+		if (pGrants[i].isVoid() || grants.contains(pGrants[i])) continue;
+		holder.add(pGrants[i]);
+	    }
+	    PermissionGrant[] updated = new PermissionGrant[holder.size()];
+	    pGrants = holder.toArray(updated);
+	}
+//        try {
+//            wl.lock();
+//            dynamicGrants.removeAll(grants);
+//        } finally {wl.unlock();}
+//        cache.clear();	
     }
 
     public List<PermissionGrant> getPermissionGrants() {
@@ -497,12 +587,17 @@ public class DynamicConcurrentPolicyProv
             return bp.getPermissionGrants();
         }
         ArrayList<PermissionGrant> grants;
-        try {
-            rl.lock();
-            grants = new ArrayList<PermissionGrant>(dynamicGrants.size());
-            grants.addAll(dynamicGrants);
-            return grants;
-        }finally {rl.unlock();}
+	PermissionGrant[] grantRefCopy = pGrants; // A local reference copy.
+	int l = grantRefCopy.length;
+	grants = new ArrayList<PermissionGrant>(l);
+	grants.addAll(Arrays.asList(grantRefCopy));
+	return grants;
+//        try {
+//            rl.lock();
+//            grants = new ArrayList<PermissionGrant>(dynamicGrants.size());
+//            grants.addAll(dynamicGrants);
+//            return grants;
+//        }finally {rl.unlock();}
     }
 
     public void add(List<Denied> denials) {

Modified: incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/MultiReadPermissionCollection.java
URL: http://svn.apache.org/viewvc/incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/MultiReadPermissionCollection.java?rev=981013&r1=981012&r2=981013&view=diff
==============================================================================
--- incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/MultiReadPermissionCollection.java (original)
+++ incubator/river/jtsk/trunk/src/org/apache/river/imp/security/policy/se/MultiReadPermissionCollection.java Sat Jul 31 07:08:43 2010
@@ -38,8 +38,16 @@ import java.util.logging.Logger;
 
 /**
  * MultiReadPermissionCollection is a wrapper class that enables mutliple
- * reads and RevokablePermissionCollection support.  It only supports
- * a homogenous class PermissionCollection.
+ * reads and synchronized writes, but only supports homogenous PermissionCollection's.
+ * 
+ * MultiReadPermissionCollection maintains a shared cache of underlying 
+ * Permission elements, this is to prevent an Enumeration from receiving a
+ * ConcurrentModificationException, sharing reduces the memory footprint.
+ * 
+ * The cache is replaced whenever the underlying PermissionCollection is
+ * modified.  The cache itself is never mutated, it's reference is simply
+ * changed to a new cache.  Old Enumeration's will refer to stale
+ * cache collections.
  * 
  * TODO Serialization Correctly
  * @version 0.2 2009/11/14
@@ -54,6 +62,12 @@ public final class MultiReadPermissionCo
     private final transient Lock wl;
     private boolean readOnly; // all access protected by rwl
     private Permission[] permissions; //never instantiate for ide code completion
+    // Read only copy to prevent publication of internal PermissionCollection
+    // state.  For generation of Enumeration<Permission>.
+    // It is volatile because other read locks may update the cache reference
+    // together if it is initially null, only the write lock sets the reference
+    // back to null.
+    private transient volatile ArrayList<Permission> cache;
 
     MultiReadPermissionCollection(Permission p){
         permCl = newPermissionCollection(p);
@@ -61,6 +75,7 @@ public final class MultiReadPermissionCo
         rl = rwl.readLock();
         wl = rwl.writeLock();
         readOnly = false;
+	cache = null;
     }
     
     @Override
@@ -116,6 +131,7 @@ public final class MultiReadPermissionCo
         wl.lock();
         try {
             permCl.add(permission);
+	    cache = null;
         }
         finally {wl.unlock();}
     }
@@ -126,11 +142,31 @@ public final class MultiReadPermissionCo
         try {return permCl.implies(permission);}
         finally {rl.unlock();}
     }
-
+    /* The Enumeration returned uses a cached copy backing ArrayList<Permission>
+     * of elements, to prevent a ConcurrentModificationException when add is 
+     * called, while enumerating. 
+     * The cached copy is never modified, no attempt is made to update it with
+     * any intervening add's.
+     */ 
     @Override
     public Enumeration<Permission> elements() {
         rl.lock();
-        try {return permCl.elements();}
+        try {
+	    ArrayList<Permission> localCache = cache; // Copy the Reference.
+	    // this reference is now stale, an intervening write may set cache
+	    // back to null.
+	    if (localCache == null){
+		localCache = new ArrayList<Permission>(40);
+		Enumeration<Permission> pIt = permCl.elements();
+		while (pIt.hasMoreElements()){
+		    localCache.add(pIt.nextElement());
+		}
+	    }
+	    // The worst that can happen is that more than one is generated or 
+	    // something gets missed, consider an AtomicReference.
+	    cache = localCache; 
+	    return Collections.enumeration(localCache);
+	}
         finally {rl.unlock();}
     }
     
@@ -142,6 +178,7 @@ public final class MultiReadPermissionCo
         if (pc == null){
             pc = new PermissionHash();
         }
+	pc.add(permission);
         return pc;                    
     }   
     
@@ -226,10 +263,8 @@ public final class MultiReadPermissionCo
         }
 
         @Override
-        public Enumeration<Permission> elements() {
-            synchronized (this){
-                return Collections.enumeration(permSet);
-            }
+        public Enumeration<Permission> elements() {          
+                return Collections.enumeration(permSet);          
         }      
     }    
 }