You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by ch...@apache.org on 2011/08/27 21:14:03 UTC

svn commit: r1162424 - in /activemq/activemq-apollo/trunk: apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/ apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/ apollo-cli/src/main/scala/org/apache/activemq/apollo/cli...

Author: chirino
Date: Sat Aug 27 19:14:03 2011
New Revision: 1162424

URL: http://svn.apache.org/viewvc?rev=1162424&view=rev
Log:
Making the xml config for access rules a little more readable.  Log and ignore any invalid rule configurations.  Document how to configure the new access rules.
re: https://issues.apache.org/jira/browse/APLO-56

Modified:
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Broker.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/Authorizer.scala
    activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AccessRuleDTO.java
    activemq/activemq-apollo/trunk/apollo-stomp/src/test/resources/apollo-stomp-secure.xml
    activemq/activemq-apollo/trunk/apollo-stomp/src/test/resources/apollo-stomp-ssl-secure.xml
    activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Broker.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Broker.scala?rev=1162424&r1=1162423&r2=1162424&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Broker.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Broker.scala Sat Aug 27 19:14:03 2011
@@ -364,7 +364,7 @@ class Broker() extends BaseService with 
 
     if (config.authentication != null && config.authentication.enabled.getOrElse(true)) {
       authenticator = new JaasAuthenticator(config.authentication, security_log)
-      authorizer=Authorizer(config.access_rules.toList, authenticator.acl_principal_kinds)
+      authorizer=Authorizer(this)
     } else {
       authenticator = null
       authorizer=Authorizer()

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala?rev=1162424&r1=1162423&r2=1162424&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala Sat Aug 27 19:14:03 2011
@@ -156,7 +156,7 @@ class VirtualHost(val broker: Broker, va
     }
     if( authenticator!=null ) {
       val rules = config.access_rules.toList ::: broker.config.access_rules.toList
-      authorizer = Authorizer(rules, authenticator.acl_principal_kinds)
+      authorizer = Authorizer(broker, this)
     } else {
       authorizer = Authorizer()
     }

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/Authorizer.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/Authorizer.scala?rev=1162424&r1=1162423&r2=1162424&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/Authorizer.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/Authorizer.scala Sat Aug 27 19:14:03 2011
@@ -1,25 +1,57 @@
 package org.apache.activemq.apollo.broker.security
 
 import java.lang.Boolean
-import collection.mutable.{ListBuffer, HashMap}
-import org.apache.activemq.apollo.broker.LocalRouter
-import org.apache.activemq.apollo.dto.{QueueDTO, AccessRuleDTO}
-import org.apache.activemq.apollo.util.path.PathParser._
+import org.apache.activemq.apollo.dto.AccessRuleDTO
 import java.util.regex.Pattern
-import org.apache.activemq.apollo.broker.security.Authorizer.ResourceMatcher
 import java.util.concurrent.atomic.AtomicLong
+import org.apache.activemq.apollo.broker._
+import org.apache.activemq.apollo.util.Log
 
 object SecuredResource {
   case class SecurityRules(version:Long, rules: Seq[(String,SecurityContext)=>Option[Boolean]])
 
-  sealed trait ResourceKind
-  object BrokerKind extends ResourceKind
-  object VirtualHostKind extends ResourceKind
-  object ConnectorKind extends ResourceKind
-  object QueueKind extends ResourceKind
-  object TopicKind extends ResourceKind
-  object DurableSubKind extends ResourceKind
-  object OtherKind extends ResourceKind
+  val ADMIN   = "admin"
+  val MONITOR = "monitor"
+  val CONFIG  = "config"
+  val CONNECT = "connect"
+  val CREATE  = "create"
+  val DESTROY = "destroy"
+  val SEND    = "send"
+  val RECEIVE = "receive"
+  val CONSUME = "consume"
+
+  sealed trait ResourceKind {
+    val id:String
+    def actions:Set[String]
+  }
+  object BrokerKind extends ResourceKind {
+    val id = "broker"
+    val actions = Set(ADMIN, MONITOR, CONFIG)
+  }
+  object VirtualHostKind extends ResourceKind{
+    val id = "virtual-host"
+    val actions = Set(ADMIN, MONITOR, CONFIG, CONNECT)
+  }
+  object ConnectorKind extends ResourceKind{
+    val id = "connector"
+    val actions = Set(ADMIN, MONITOR, CONFIG, CONNECT)
+  }
+  object TopicKind extends ResourceKind{
+    val id = "topic"
+    val actions = Set(ADMIN, MONITOR, CONFIG, CREATE, DESTROY, SEND, RECEIVE)
+  }
+  object QueueKind extends ResourceKind{
+    val id = "queue"
+    val actions = Set(ADMIN, MONITOR, CONFIG, CREATE, DESTROY, SEND, RECEIVE, CONSUME)
+  }
+  object DurableSubKind extends ResourceKind{
+    val id = "dsub"
+    val actions = Set(ADMIN, MONITOR, CONFIG, CREATE, DESTROY, SEND, RECEIVE, CONSUME)
+  }
+  object OtherKind extends ResourceKind{
+    val id = "other"
+    val actions = Set[String]()
+  }
 }
 import SecuredResource._
 
@@ -49,14 +81,35 @@ object Authorizer {
     def can(ctx: SecurityContext, action: String, resource: SecuredResource) = true
   }
 
-  def apply(config:Seq[AccessRuleDTO], default_principal_kinds:Set[String]):Authorizer = {
-    new RulesAuthorizer(version_counter.incrementAndGet(), config.map(ResourceMatcher(_, default_principal_kinds)))
-  }
+  def apply(broker:Broker, host:VirtualHost=null):Authorizer = {
+    import collection.JavaConversions._
 
-  case class ResourceMatcher(rule:AccessRuleDTO, default_principal_kinds:Set[String]) {
+    val rules = if( host==null ) {
+      val pk = broker.authenticator.acl_principal_kinds
+      broker.config.access_rules.toList.flatMap(parse(broker.console_log, _,pk))
+    } else {
+      val pk = host.authenticator.acl_principal_kinds
+      host.config.access_rules.toList.flatMap(parse(host.console_log, _,pk, host)) :::
+      broker.config.access_rules.toList.flatMap(parse(broker.console_log, _,pk))
+    }
+    new RulesAuthorizer(version_counter.incrementAndGet(), rules.toArray )
+  }
 
+  def parse(log:Log, rule:AccessRuleDTO, default_principal_kinds:Set[String], host:VirtualHost=null):Option[ResourceMatcher] ={
+    import log._
     var resource_matchers = List[(SecuredResource)=>Boolean]()
 
+    val actions = Option(rule.action).map(_.trim().toLowerCase).getOrElse("*") match {
+      case "*" => null
+      case action =>
+        val rc = action.split("\\s").map(_.trim()).toSet
+        // Not all actions can apply to all resource types.
+        resource_matchers ::= ((resource:SecuredResource) => {
+          !(resource.resource_kind.actions & rc).isEmpty
+        })
+        rc
+    }
+
     for(id_regex <- Option(rule.id_regex)) {
       val reg_ex = Pattern.compile(id_regex)
       resource_matchers ::= ((resource:SecuredResource) => {
@@ -67,7 +120,7 @@ object Authorizer {
     Option(rule.id).getOrElse("*") match {
       case "*" =>
       case id =>
-        if(rule.kind == "queue" || rule.kind == "topic") {
+        if(rule.kind == QueueKind.id || rule.kind == TopicKind.id ) {
           val filter = LocalRouter.destination_parser.decode_filter(id)
           resource_matchers ::= ((resource:SecuredResource) => {
             filter.matches(LocalRouter.destination_parser.decode_path(resource.id))
@@ -81,16 +134,37 @@ object Authorizer {
 
     Option(rule.kind).map(_.trim().toLowerCase).getOrElse("*") match {
       case "*" =>
+        if( host!=null) {
+          resource_matchers ::= ((resource:SecuredResource) => {
+            resource.resource_kind match {
+              case BrokerKind=> false
+              case ConnectorKind=> false
+              case _ => true
+            }
+          })
+        }
       case kind =>
-        val kinds = (kind.split(",").map(_.trim()).map{ v=>
+        val kinds = (kind.split("\\s").map(_.trim()).map{ v=>
           val kind:ResourceKind = v match {
-            case "broker"=>BrokerKind
-            case "virtual-host"=>VirtualHostKind
-            case "connector"=>ConnectorKind
-            case "queue"=>QueueKind
-            case "topic"=>TopicKind
-            case "dsub"=>DurableSubKind
-            case _ => OtherKind
+            case BrokerKind.id =>
+              if(host!=null) {
+                warn("Ignoring invalid access rule. kind='broker' can only be configured at the broker level: "+rule)
+                return None
+              }
+              BrokerKind
+            case ConnectorKind.id =>
+              if(host!=null) {
+                warn("Ignoring invalid access rule. kind='connector' can only be configured at the broker level: "+rule)
+                return None
+              }
+              ConnectorKind
+            case VirtualHostKind.id =>VirtualHostKind
+            case QueueKind.id =>QueueKind
+            case TopicKind.id =>TopicKind
+            case DurableSubKind.id =>DurableSubKind
+            case _ =>
+              warn("Ignoring invalid access rule. Unknown kind '"+v+"' "+rule)
+              return None
           }
           kind
         }).toSet
@@ -99,75 +173,90 @@ object Authorizer {
         })
     }
 
-    def resource_matches(resource:SecuredResource):Boolean = {
-      // Looking for a matcher that does not match so we can
-      // fail the match quickly.
-      !resource_matchers.find(_(resource)==false).isDefined
-    }
-
-    var action_matchers = List[(String, SecurityContext)=>Boolean]()
-
     val principal_kinds = Option(rule.principal_kind).map(_.trim().toLowerCase).getOrElse(null) match {
       case null => Some(default_principal_kinds)
       case "*" => None
-      case principal_kind => Some(principal_kind.split(",").map(_.trim()).toSet)
+      case principal_kind => Some(principal_kind.split("\\s").map(_.trim()).toSet)
     }
 
-    Option(rule.principal).map(_.trim().toLowerCase).getOrElse("+") match {
-      case "*" =>
-      case "+" =>
-        // user has to have at least one of the principle kinds
-        action_matchers ::= ((action:String, ctx:SecurityContext) => {
-          principal_kinds match {
-            case Some(principal_kinds)=>
-              ctx.principles.find(p=> principal_kinds.contains(p.getClass.getName) ).isDefined
-            case None =>
-              !ctx.principles.isEmpty
-          }
-        })
-
-      case principal =>
-        val principals = if(rule.separator!=null) {
-          principal.split(Pattern.quote(rule.separator)).map(_.trim()).toSet
-        } else {
-          Set(principal)
-        }
-        action_matchers ::= ((action:String, ctx:SecurityContext) => {
-          principal_kinds match {
-            case Some(principal_kinds)=>
-              ctx.principles.find{ p=>
-                val km = principal_kinds.contains(p.getClass.getName)
-                val nm = principals.contains(p.getName)
-                km && nm
-              }.isDefined
-            case None =>
-              ctx.principles.find(p=> principals.contains(p.getName) ).isDefined
+    def parse_principals(value:String): Option[(SecurityContext)=>Boolean] = {
+      Option(value).map(_.trim() match {
+        case "*" =>
+          ((ctx:SecurityContext) => { true })
+        case "+" =>
+          // user has to have at least one of the principle kinds
+          ((ctx:SecurityContext) => {
+            principal_kinds match {
+              case Some(principal_kinds)=>
+                ctx.principles.find(p=> principal_kinds.contains(p.getClass.getName) ).isDefined
+              case None =>
+                !ctx.principles.isEmpty
+            }
+          })
+        case principal =>
+          val principals = if(rule.separator!=null) {
+            principal.split(Pattern.quote(rule.separator)).map(_.trim()).toSet
+          } else {
+            Set(principal)
           }
-        })
+          ((ctx:SecurityContext) => {
+            principal_kinds match {
+              case Some(principal_kinds)=>
+                ctx.principles.find{ p=>
+                  val km = principal_kinds.contains(p.getClass.getName)
+                  val nm = principals.contains(p.getName)
+                  km && nm
+                }.isDefined
+              case None =>
+                ctx.principles.find(p=> principals.contains(p.getName) ).isDefined
+            }
+          })
+      })
     }
 
-    Option(rule.action).map(_.trim().toLowerCase).getOrElse("*") match {
-      case "*" =>
-      case action =>
-        val actions = Set(action.split(",").map(_.trim()): _* )
-        action_matchers ::= ((action:String, ctx:SecurityContext) => {
-          actions.contains(action)
-        })
+    val allow = parse_principals(rule.allow)
+    val deny = parse_principals(rule.deny)
+
+    if( allow.isEmpty && deny.isEmpty ) {
+      warn("Ignoring invalid access rule. Either the 'allow' or 'deny' attribute must be declared.")
+      return None
     }
 
-    val deny = Option(rule.deny).map(_.booleanValue()).getOrElse(false)
+    Some(ResourceMatcher(resource_matchers, actions, allow, deny))
+  }
+
+  case class ResourceMatcher(
+    resource_matchers:List[(SecuredResource)=>Boolean],
+    actions:Set[String],
+    allow:Option[(SecurityContext)=>Boolean],
+    deny:Option[(SecurityContext)=>Boolean]
+  ) {
+
+    def resource_matches(resource:SecuredResource):Boolean = {
+      // Looking for a matcher that does not match so we can
+      // fail the match quickly.
+      !resource_matchers.find(_(resource)==false).isDefined
+    }
 
     def action_matches(action:String, ctx:SecurityContext):Option[Boolean] = {
-      for(matcher <- action_matchers) {
-        if ( !matcher(action, ctx) ) {
-          return None
+      if(actions!=null && !actions.contains(action)) {
+        return None
+      }
+      for(matcher <- deny) {
+        if ( matcher(ctx) ) {
+          return Some(false)
+        }
+      }
+      for(matcher <- allow) {
+        if ( matcher(ctx) ) {
+          return Some(true)
         }
       }
-      return Some(!deny)
+      return None
     }
   }
 
-  case class RulesAuthorizer(version:Long, config:Seq[ResourceMatcher]) extends Authorizer {
+  case class RulesAuthorizer(version:Long, config:Array[ResourceMatcher]) extends Authorizer {
 
     def can(ctx:SecurityContext, action:String, resource:SecuredResource):Boolean = {
       if (ctx==null) {

Modified: activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala?rev=1162424&r1=1162423&r2=1162424&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala Sat Aug 27 19:14:03 2011
@@ -51,20 +51,17 @@ class Create extends Action {
   val home: String = System.getProperty("apollo.home")
 
   var broker_security_config =
-  """<!-- used to secure the web admin interface -->
+  """
   <authentication domain="apollo"/>
-  <acl>
-    <admin allow="admins"/>
-    <config allow="admins"/>
-  </acl>
+  <!-- Give admins full access -->
+  <access_rule allow="admins" action="*"/>
+  <access_rule allow="*" action="connect" kind="connector"/>
   """
+
   var host_security_config =
     """<!-- Uncomment to disable security for the virtual host -->
     <!-- <authentication enabled="false"/> -->
-    <acl>
-      <admin allow="admins"/>
-      <connect allow="admins"/>
-    </acl>
+    <access_rule allow="users" action="connect create destroy send receive consume"/>
     """
 
   var create_login_config = true

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AccessRuleDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AccessRuleDTO.java?rev=1162424&r1=1162423&r2=1162424&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AccessRuleDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AccessRuleDTO.java Sat Aug 27 19:14:03 2011
@@ -32,12 +32,6 @@ import javax.xml.bind.annotation.XmlRoot
 public class AccessRuleDTO {
 
     /**
-     * Is this a negative rule which denies access.  If not set, defaults to false.
-     */
-    @XmlAttribute
-    public Boolean deny;
-
-    /**
      * The class name of the JAAS principle that this rule will mach against.  If not set
      * the this defaults to the default principal kinds configured on the broker or virtual host.
      * If set to "*" then it matches all principal classes.
@@ -46,14 +40,20 @@ public class AccessRuleDTO {
     public String principal_kind;
 
     /**
-     * The principal which we are matching against.  If set to "+" then it matches all principals
+     * The principal which are allowed access to the action.  If set to "+" then it matches all principals
      * but requires at at least one.  If set to "*" the it matches all principals and even matches
      * the case where there are no principals associated with the subject.
-     *
-     * Defaults to "+" if not set.
      */
     @XmlAttribute
-    public String principal;
+    public String allow;
+
+    /**
+     * The principal which are denied access to the action  If set to "+" then it matches all principals
+     * but requires at at least one.  If set to "*" the it matches all principals and even matches
+     * the case where there are no principals associated with the subject.
+     */
+    @XmlAttribute
+    public String deny;
 
     /**
      * If the separator is set, then the principal field will be interpreted as a list of
@@ -96,24 +96,30 @@ public class AccessRuleDTO {
 
         AccessRuleDTO that = (AccessRuleDTO) o;
 
-        if (action != null ? !action.equals(that.action) : that.action != null) return false;
-        if (deny != null ? !deny.equals(that.deny) : that.deny != null) return false;
+        if (action != null ? !action.equals(that.action) : that.action != null)
+            return false;
+        if (allow != null ? !allow.equals(that.allow) : that.allow != null)
+            return false;
+        if (deny != null ? !deny.equals(that.deny) : that.deny != null)
+            return false;
         if (id != null ? !id.equals(that.id) : that.id != null) return false;
-        if (id_regex != null ? !id_regex.equals(that.id_regex) : that.id_regex != null) return false;
-        if (kind != null ? !kind.equals(that.kind) : that.kind != null) return false;
-        if (principal != null ? !principal.equals(that.principal) : that.principal != null) return false;
+        if (id_regex != null ? !id_regex.equals(that.id_regex) : that.id_regex != null)
+            return false;
+        if (kind != null ? !kind.equals(that.kind) : that.kind != null)
+            return false;
         if (principal_kind != null ? !principal_kind.equals(that.principal_kind) : that.principal_kind != null)
             return false;
-        if (separator != null ? !separator.equals(that.separator) : that.separator != null) return false;
+        if (separator != null ? !separator.equals(that.separator) : that.separator != null)
+            return false;
 
         return true;
     }
 
     @Override
     public int hashCode() {
-        int result = deny != null ? deny.hashCode() : 0;
-        result = 31 * result + (principal_kind != null ? principal_kind.hashCode() : 0);
-        result = 31 * result + (principal != null ? principal.hashCode() : 0);
+        int result = principal_kind != null ? principal_kind.hashCode() : 0;
+        result = 31 * result + (allow != null ? allow.hashCode() : 0);
+        result = 31 * result + (deny != null ? deny.hashCode() : 0);
         result = 31 * result + (separator != null ? separator.hashCode() : 0);
         result = 31 * result + (action != null ? action.hashCode() : 0);
         result = 31 * result + (kind != null ? kind.hashCode() : 0);
@@ -121,4 +127,26 @@ public class AccessRuleDTO {
         result = 31 * result + (id_regex != null ? id_regex.hashCode() : 0);
         return result;
     }
+
+    private static String attr(String name, Object value) {
+        if(value!=null) {
+            return " "+name+"='" + value +"'";
+        } else {
+            return "";
+        }
+
+    }
+    @Override
+    public String toString() {
+        return "<access_rule" +
+            attr("allow",allow)+
+            attr("deny",deny)+
+            attr("principal_kind",principal_kind)+
+            attr("separator",separator)+
+            attr("action",action)+
+            attr("kind",kind)+
+            attr("id",kind)+
+            attr("id_regex",id_regex)+
+            "/>";
+    }
 }

Modified: activemq/activemq-apollo/trunk/apollo-stomp/src/test/resources/apollo-stomp-secure.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-stomp/src/test/resources/apollo-stomp-secure.xml?rev=1162424&r1=1162423&r2=1162424&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-stomp/src/test/resources/apollo-stomp-secure.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-stomp/src/test/resources/apollo-stomp-secure.xml Sat Aug 27 19:14:03 2011
@@ -22,20 +22,17 @@
   <virtual_host id="default" purge_on_startup="true">
     <host_name>localhost</host_name>
 
-    <access_rule principal="connect_group" action="connect"/>
-
-    <access_rule principal="can_send_create_queue" kind="queue" action="send,create"/>
-    <access_rule principal="can_send_queue"        kind="queue" action="send"/>
-    <access_rule principal="can_receive_queue"     kind="queue" action="receive"/>
-    <access_rule principal="can_consume_queue"     kind="queue" action="consume"/>
-
-    <access_rule principal="can_send_create_topic" kind="topic" action="send,create"/>
-    <access_rule principal="can_send_topic"        kind="topic" action="send"/>
-    <access_rule principal="can_recieve_topic"     kind="topic" action="receive"/>
-
-    <access_rule principal="can_consume_create_ds" kind="dsub" action="consume,create"/>
-    <access_rule principal="can_consume_ds"        kind="dsub" action="consume"/>
-    <access_rule principal="can_recieve_topic"     kind="dsub" action="receive"/>
+    <access_rule allow="connect_group" action="connect"/>
+    <access_rule allow="can_send_create_queue" kind="queue" action="send create"/>
+    <access_rule allow="can_send_queue"        kind="queue" action="send"/>
+    <access_rule allow="can_receive_queue"     kind="queue" action="receive"/>
+    <access_rule allow="can_consume_queue"     kind="queue" action="consume"/>
+    <access_rule allow="can_send_create_topic" kind="topic" action="send create"/>
+    <access_rule allow="can_send_topic"        kind="topic" action="send"/>
+    <access_rule allow="can_recieve_topic"     kind="topic" action="receive"/>
+    <access_rule allow="can_consume_create_ds" kind="dsub"  action="consume create"/>
+    <access_rule allow="can_consume_ds"        kind="dsub"  action="consume"/>
+    <access_rule allow="can_recieve_topic"     kind="dsub"  action="receive"/>
 
   </virtual_host>
 

Modified: activemq/activemq-apollo/trunk/apollo-stomp/src/test/resources/apollo-stomp-ssl-secure.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-stomp/src/test/resources/apollo-stomp-ssl-secure.xml?rev=1162424&r1=1162423&r2=1162424&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-stomp/src/test/resources/apollo-stomp-ssl-secure.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-stomp/src/test/resources/apollo-stomp-ssl-secure.xml Sat Aug 27 19:14:03 2011
@@ -23,7 +23,7 @@
   <virtual_host id="default" purge_on_startup="true">
     <host_name>localhost</host_name>
 
-    <access_rule principal="connect_group" action="connect"/>
+    <access_rule allow="connect_group" action="connect"/>
 
   </virtual_host>
 

Modified: activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md?rev=1162424&r1=1162423&r2=1162424&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md (original)
+++ activemq/activemq-apollo/trunk/apollo-website/src/documentation/user-manual.md Sat Aug 27 19:14:03 2011
@@ -587,136 +587,108 @@ Example of customizing the principal kin
 #### Authorization
 
 User authorization to broker resources is accomplished by configuring an
-access control list using an `acl` element on the `broker`, `connector`,
-`virtual_host`, `topic`, `dsub`, or `queue` resources. The
-acl defines which principals are allowed or denied access to perform actions
-against the resources. An example of `acl` is shown below:
+access control rules using a `access_rule` elements in the `broker` or
+`virtual_host` elements. The rules defines which principals are allowed or
+denied access to perform actions against server resources. An example list of
+rule is shown below:
 
 {pygmentize:: xml}
-<acl>
-  <send deny="guest"/>
-  <send allow="*"/>
-  <receive allow="app1"/>
-</acl>
+<broker>
+  <access_rule deny="guest" action="send"/>
+  <access_rule allow="*"    action="send"/>
+  <access_rule allow="app1" action="receive"/>
+</broker>
 {pygmentize}
 
-If a configuration resource does not have an `acl` element defined within
-it, then the resource allows access if the containing resource would allow
-access to the action.  If the action is not defined in the containing
-resource then it allows anyone to access. The `acl`
-is made up of a list of authorization rule entries. Each entry defines
-that action the rule applies to and if the rule is allowing or denying
-access to a user principal. 
-
-Users can have many principals of many different kinds associated with
-them. The rules will only match up against principals of type
-`org.apache.activemq.jaas.GroupPrincipal` since that is the default
-setting of the `acl_principal_kind` of the `authentication` domain.
-
-If you want the rule to match against more/different kinds of principals,
-you should update the `authentication` element's configuration or you
-explicitly state the kind you want to match against in your rule
-definition. Example:
 
-{pygmentize:: xml}
-<acl>
-  <send deny="chirino" kind="org.apache.activemq.jaas.UserPrincipal"/>
-  <send allow="*"/>
-</acl>
-{pygmentize}
+The `allow` and `deny` attributes define the principals which are allowed or
+denied access. If set to "+" then it matches all principals but requires at
+at least one. If set to "*" the it matches all principals and even matches
+the case where there are no principals associated with the subject.
 
-#### Wildcards
+Either `allow` or `deny` must be defined. You can optionally define one or
+more of the following attributes to further narrow down when the rule matches
+an authorization check:
 
-Wild cards can be used in the `deny`, `allow`, and `kind` attributes to match 
-multiple values.  Two types of wildcards are supported:
+* `separator`: If set, then the `allow` and `deny` fields will be interpreted
+  to be a list of principles separated by the separator value.
 
-> `*` : Matches any value on zero or more principles. 
-> `+` : Matches any value on one or more principles.
+* `principal_kind`: A space separated list of class names of which will be
+  matched against the principle type. If set to `*` then it matches all
+  principal classes. Defaults to the default principal kinds configured on
+  the broker or virtual host.
 
-Examples of using the `*` wild card:
+* `action`: A space separated list of actions that will match the rule.
+  Example 'create,destroy'. You can use `*` to match all actions.  Defaults
+  to `*`.
 
-{pygmentize:: xml}
-<acl>
-  <connect allow="*"/>
-</acl>
-{pygmentize}
+* `kind`: A space separated list of broker resource types that will match
+  this rule. You can use `*` to match all key. Example values 'broker,queue'.
+  Defaults to `*`.
 
-The previous example allows anyone to connect even if the subject they 
-authenticated with has no principles associated with it.
+* `id`: The identifier of the resource that will match this rule. You can use
+  `*` to match all resources. Defaults to `*`
 
-Examples of using the `+` wild card:
+* `id_regex`: A regular expression to be applied against the id of the
+  resource.
 
-{pygmentize:: xml}
-<acl>
-  <connect allow="+" kind="org.apache.activemq.jaas.UserPrincipal"/>
-</acl>
-{pygmentize}
+If no access rules match an authorization check then access is denied. 
 
-The previous example allows an `UserPrincipal` principal to connect. It
-would reject the connection if subject that has no `UserPrincipals`.
+##### Ordering
 
-You can also use the wildcard on the kind attribute.  When the wild
-card is used on the kind attribute, then `*` acts like the `+` wild 
-card and only matches for one or more principles.
-
-For example:
+The order in which rules are defined are significant. The first entry that
+matches determines if he will have access. For example, lets say a user is
+groups 'blue' and 'red', and you are matching against the following 
+rules:
 
 {pygmentize:: xml}
-<acl>
-  <connect allow="Hiram" kind="*"/>
-</acl>
+<access_rule deny="blue" action="send"/>
+<access_rule allow="red" action="send"/>
 {pygmentize}
 
-The previous example allows a subject with at least one `hiram` to 
-principal connect.  The principal can be of any type..
-
-#### Ordering
-
-The order in which rule entries are defined are significant when the user
-matches multiple entries. The first entry the user matches determines if he
-will have access to the action. For example, lets say a user is groups
-'blue' and 'red', and you are matching against an ACL list defined as:
+Then the user would not be allowed to send since the deny rule was
+matched first. If the order in the ACL list were reversed, like
+so:
 
 {pygmentize:: xml}
-<acl>
-  <send deny="blue"/>
-  <send allow="red"/>
-</acl>
+<access_rule allow="red" action="send"/>
+<access_rule deny="blue" action="send"/>
 {pygmentize}
 
-Then the user would not be allowed to send since `<send deny="blue"/>` was
-defined first. If the order in the ACL list were reversed, like
-so:
+Then the user would be allowed access to the resource since the allow
+rule matched first.  When a single rule defines both `allow` and
+`deny` attributes and they both match then the action is denied.
 
 {pygmentize:: xml}
-<acl>
-  <send allow="red"/>
-  <send deny="blue"/>
-</acl>
+<access_rule deny="blue" allow="red" action="send"/>
 {pygmentize}
 
-Then the user would be allowed access to the resource since the first rule
-which matches the user is `<send allow="red"/>`.
+#### Resource Actions
+
+You can configure the `action` attribute of an access rules with
+one or more of the following values:
+
+* `admin` : use of the administrative web interface
+* `monitor` : read only use of the administrative web interface
+* `config` : use of the administrative web interface to access and change the
+  broker configuration.
+* `connect` : allows connections to the connector or virtual host
+* `create` : allows creation
+* `destroy` : allows destruction
+* `send` : allows the user to send to the destination
+* `receive` : allows the user to send to do non-destructive reads from the
+  destination
+* `consume` : allows the user to do destructive reads against a destination
+* `*` : All actions
+
+#### Resource Kinds
+
+You can configure the `kind` attribute of an access rules with one or more of
+the following values: `broker`, `connector`, `virtual_host`, `topic`,
+`queue`, `dsub`, or `*`. `*` matches all resource kinds.
 
-The type of resource being secured determines the types of actions that
-can be secured by the acl rule entries. Here is listing of which actions
-can be secured on which resources:
-
-* `broker`, `virtual_host`, `topic`, `queue`, and `dsub`
-  * `admin` : use of the administrative web interface
-  * `monitor` : read only use of the administrative web interface
-  * `config` : use of the administrative web interface to access and change the broker
-     configuration.
-* `connector` and `virtual_host`
-  * `connect` : allows connections to the connector or virtual host
-* `topic`, `queue` and `dsub`
-  * `create` : allows creation
-  * `destroy` : allows destruction
-  * `send` : allows the user to send to the destination
-  * `receive` : allows the user to send to do non_destructive read 
-    from the destination
-* `queue` and `dsub`
-  * `consume` : allows the user to do destructive reads against the queue.
+The `broker` and `connector` kinds can only be configured in rules
+defined in the `broker` element.
 
 #### Encrypting Passwords in the Configuration