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 2010/12/10 06:30:34 UTC

svn commit: r1044209 [1/2] - in /activemq/activemq-apollo/trunk: ./ apollo-broker/ apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/ apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/ apollo-cli/ apollo-cli/src/main/...

Author: chirino
Date: Fri Dec 10 05:30:31 2010
New Revision: 1044209

URL: http://svn.apache.org/viewvc?rev=1044209&view=rev
Log:
fully modeled out the security configuration in apollo.xml
finished integration with jass
implemented authorizer based on the security config in the xml files
upgraded the jetty version
jetty now secures the admin interface so that only admin users can access it
updated create tool so it generates all the login config stuff.

Added:
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AclAuthorizer.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/EncryptionSupport.scala
      - copied, changed from r1043831, activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/FileLoginModule.scala
    activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/groups.properties
    activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/login.config
    activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/users.properties
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthenticationDTO.java
      - copied, changed from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerAclDTO.java
      - copied, changed from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthenticatorDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationAclDTO.java
      - copied, changed from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/PrincipalDTO.java
      - copied, changed from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueAclDTO.java
      - copied, changed from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostAclDTO.java
      - copied, changed from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/WebAdminDTO.java
      - copied, changed from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthorizerDTO.java
Removed:
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AuthenticatorFactory.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AuthorizerFactory.scala
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthenticatorDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthorizerDTO.java
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/web/resources/BrokerResource.index.scaml
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/web/resources/ConfigurationResource.index.scaml
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/web/resources/RootResource.index.scaml
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/web/resources/RuntimeResource.index.scaml
Modified:
    activemq/activemq-apollo/trunk/apollo-broker/pom.xml
    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/ConfigStore.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Router.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-broker/src/main/scala/org/apache/activemq/apollo/broker/security/JaasAuthenticator.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala
    activemq/activemq-apollo/trunk/apollo-cli/pom.xml
    activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/apollo-ssl.xml
    activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/apollo.xml
    activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala
    activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Run.scala
    activemq/activemq-apollo/trunk/apollo-distro/pom.xml
    activemq/activemq-apollo/trunk/apollo-distro/src/main/descriptors/common-bin.xml
    activemq/activemq-apollo/trunk/apollo-dto/pom.xml
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectorDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/test/java/org/apache/activemq/apollo/dto/XmlCodecTest.java
    activemq/activemq-apollo/trunk/apollo-dto/src/test/resources/org/apache/activemq/apollo/dto/simple.xml
    activemq/activemq-apollo/trunk/apollo-web/pom.xml
    activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/ApolloListener.scala
    activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/ConfigurationResource.scala
    activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/RootResource.scala
    activemq/activemq-apollo/trunk/apollo-web/src/main/webapp/WEB-INF/org/apache/activemq/apollo/dto/BrokerSummaryDTO.jade
    activemq/activemq-apollo/trunk/apollo-web/src/test/scala/org/apache/activemq/apollo/web/FileConfigStoreTest.scala
    activemq/activemq-apollo/trunk/apollo-website/src/main/webapp/documentation/getting-started.md
    activemq/activemq-apollo/trunk/pom.xml

Modified: activemq/activemq-apollo/trunk/apollo-broker/pom.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/pom.xml?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/pom.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/pom.xml Fri Dec 10 05:30:31 2010
@@ -61,6 +61,12 @@
       <artifactId>activemq-jaas</artifactId>
       <version>${activemq-version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.jasypt</groupId>
+      <artifactId>jasypt</artifactId>
+      <version>${jasypt-version}</version>
+    </dependency>
+
 
     <!-- Scala Support -->
     <dependency>

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=1044209&r1=1044208&r2=1044209&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 Fri Dec 10 05:30:31 2010
@@ -31,6 +31,7 @@ import org.apache.activemq.apollo.util._
 import ReporterLevel._
 import collection.mutable.LinkedHashMap
 import java.util.concurrent.{ThreadFactory, Executors, ConcurrentHashMap}
+import security.{AclAuthorizer, Authorizer, JaasAuthenticator, Authenticator}
 
 /**
  * <p>
@@ -203,6 +204,8 @@ class Broker() extends BaseService with 
     }
   }
 
+  var authenticator:Authenticator = _
+  var authorizer:Authorizer = _
 
   override def _start(onCompleted:Runnable) = {
 
@@ -213,6 +216,11 @@ class Broker() extends BaseService with 
         key_storage.config = config.key_storage
       }
 
+      if( config.authentication != null ) {
+        authenticator = new JaasAuthenticator(config.authentication.domain)
+        authorizer = new AclAuthorizer(config.authentication.kinds().toList)
+      }
+
       default_virtual_host = null
       for (c <- config.virtual_hosts) {
         val host = new VirtualHost(this, virtual_host_id_counter.incrementAndGet)

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/ConfigStore.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/ConfigStore.scala?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/ConfigStore.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/ConfigStore.scala Fri Dec 10 05:30:31 2010
@@ -25,12 +25,15 @@ import _root_.org.fusesource.hawtdispatc
 import java.util.concurrent.{TimeUnit, ExecutorService, Executors}
 import org.fusesource.hawtbuf.{ByteArrayInputStream, ByteArrayOutputStream}
 import javax.xml.bind.{Marshaller, JAXBContext}
-import java.io.{OutputStreamWriter, File}
+import security.EncryptionSupport
 import XmlCodec._
 import org.apache.activemq.apollo.util._
 import scala.util.continuations._
 import org.fusesource.hawtdispatch.DispatchQueue
 import java.util.Arrays
+import FileSupport._
+import java.util.Properties
+import java.io.{FileInputStream, OutputStreamWriter, File}
 
 object ConfigStore {
 
@@ -38,10 +41,6 @@ object ConfigStore {
 
   def apply():ConfigStore = store
 
-  def sync[T] (func: ConfigStore=>T):T = store.dispatchQueue.sync {
-    func(store)
-  }
-
   def update(value:ConfigStore) = store=value
 
 }
@@ -53,22 +52,22 @@ object ConfigStore {
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-trait ConfigStore extends Service {
+trait ConfigStore {
 
-  def listBrokers: List[String]
+  def load(eval:Boolean): BrokerDTO
 
-  def getBroker(id:String, eval:Boolean): Option[BrokerDTO]
+  def store(config:BrokerDTO): Boolean
 
-  def putBroker(config:BrokerDTO): Boolean
-
-  def removeBroker(id:String, rev:Int): Boolean
+  def can_write:Boolean
 
-  def dispatchQueue:DispatchQueue
+  def start:Unit
 
-  def can_write:Boolean
+  def stop:Unit
 
 }
 
+object FileConfigStore extends Log
+
 /**
  * <p>
  * A simple ConfigStore implementation which only support one broker configuration
@@ -77,20 +76,23 @@ trait ConfigStore extends Service {
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-class FileConfigStore extends ConfigStore with BaseService with Logging {
-  override protected def log: Log = FileConfigStore
+class FileConfigStore extends ConfigStore {
+  import FileConfigStore._
 
   object StoredBrokerModel {
     def apply(config:BrokerDTO) = {
       val data = marshall(config)
-      new StoredBrokerModel(config.id, config.rev, data, 0)
+      new StoredBrokerModel(config.rev, data, 0)
     }
   }
-  case class StoredBrokerModel(id:String, rev:Int, data:Array[Byte], lastModified:Long)
+  case class StoredBrokerModel(rev:Int, data:Array[Byte], lastModified:Long)
 
   var file:File = new File("activemq.xml")
+
   @volatile
   var latest:StoredBrokerModel = null
+  @volatile
+  var running = false
 
   val dispatchQueue = createQueue("config store")
 
@@ -98,23 +100,10 @@ class FileConfigStore extends ConfigStor
   // so... use an executor
   var ioWorker:ExecutorService = null
 
-  protected def _start(onCompleted:Runnable) = {
-    ioWorker = Executors.newSingleThreadExecutor
-    ioWorker.execute(^{
-      startup(onCompleted)
-    })
-  }
-
-  protected def _stop(onCompleted:Runnable) = {
-    ioWorker.submit(^{
-      onCompleted.run
-      ioWorker.shutdown
-    })
-  }
-
-  def can_write:Boolean = file.canWrite
 
-  def startup(onCompleted:Runnable) = {
+  def start = {
+    ioWorker = Executors.newSingleThreadExecutor
+    running = true
 
     file = file.getCanonicalFile;
     file.getParentFile.mkdir
@@ -140,7 +129,7 @@ class FileConfigStore extends ConfigStor
       } else {
         val x = read(rev, file)
         if ( can_write && !Arrays.equals(r.data, x.data) ) {
-          write(StoredBrokerModel(x.id, x.rev+1, x.data, x.lastModified))
+          write(x.copy(rev=x.rev+1))
         } else {
           x
         }
@@ -153,33 +142,24 @@ class FileConfigStore extends ConfigStor
       }
     }
 
-    dispatchQueue {
-      latest = last
-      schedualNextUpdateCheck
-      onCompleted.run
-    }
+    latest = last
+    schedualNextUpdateCheck
   }
 
-                  
-  def listBrokers = {
-    List(latest.id)
+  def stop = {
+    running = false
+    ioWorker.shutdown
   }
 
-
-  def getBroker(id:String, eval:Boolean) = {
-    if( latest.id == id ) {
-      Some(unmarshall(latest.data, eval))
-    } else {
-      None
-    }
+  def load(eval:Boolean) = {
+    unmarshall(latest.data, eval)
   }
 
-  def putBroker(config:BrokerDTO) = {
+  def can_write:Boolean = file.canWrite
+
+  def store(config:BrokerDTO) = {
     debug("storing broker model: %s ver %d", config.id, config.rev)
-    if( latest.id != config.id ) {
-      debug("this store can only update broker: "+latest.id)
-      false
-    } else if( latest.rev+1 != config.rev ) {
+    if( latest.rev+1 != config.rev ) {
       debug("update request does not match next revision: %d", latest.rev+1)
       false
     } else {
@@ -188,15 +168,10 @@ class FileConfigStore extends ConfigStor
     }
   }
 
-  def removeBroker(id:String, rev:Int) = {
-    // not supported.
-    false
-  }
-
   private def fileRev(rev:Int) = new File(file.getParent, file.getName+"."+rev)
 
   private def schedualNextUpdateCheck:Unit = dispatchQueue.after(1, TimeUnit.SECONDS) {
-    if( serviceState.isStarted ) {
+    if( running ) {
       val lastModified = latest.lastModified
       val latestData = latest.data
       val nextRev = latest.rev+1
@@ -208,13 +183,9 @@ class FileConfigStore extends ConfigStor
             if ( !Arrays.equals(latestData, config.data) ) {
               val c = unmarshall(config.data)
               c.rev = config.rev
-              dispatchQueue {
-                putBroker(c)
-              }
+              store(c)
             } else {
-              dispatchQueue {
-                latest =   StoredBrokerModel(latest.id, latest.rev, latest.data, l)
-              }
+              latest = latest.copy(lastModified = l)
             }
           }
           schedualNextUpdateCheck
@@ -239,19 +210,31 @@ class FileConfigStore extends ConfigStor
   private def read(rev:Int, file: File) ={
     val data = IOHelper.readBytes(file)
     val config = unmarshall(data) // validates the xml
-    StoredBrokerModel(config.id, rev, data, file.lastModified)
+    StoredBrokerModel(rev, data, file.lastModified)
   }
 
   private  def write(config:StoredBrokerModel) = {
     // write to the files..
     IOHelper.writeBinaryFile(file, config.data)
     IOHelper.writeBinaryFile(fileRev(config.rev), config.data)
-    StoredBrokerModel(config.id, config.rev, config.data, file.lastModified)
+    config.copy(lastModified = file.lastModified)
   }
 
+
   def unmarshall(in:Array[Byte], evalProps:Boolean=false) = {
     if (evalProps) {
-      unmarshalBrokerDTO(new ByteArrayInputStream(in), System.getProperties)
+
+      val props = new Properties(System.getProperties)
+      val prop_file = file.getParentFile / (file.getName + ".properties")
+      if( prop_file.exists() ) {
+        FileSupport.using(new FileInputStream(prop_file)) { is=>
+          val p = new Properties
+          p.load(new FileInputStream(prop_file))
+          props.putAll( EncryptionSupport.decrypt( p ))
+        }
+      }
+
+      unmarshalBrokerDTO(new ByteArrayInputStream(in), props)
     } else {
       unmarshalBrokerDTO(new ByteArrayInputStream(in))
     }
@@ -263,5 +246,3 @@ class FileConfigStore extends ConfigStor
     baos.toByteArray
   }
 }
-
-object FileConfigStore extends Log

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Router.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Router.scala?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Router.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Router.scala Fri Dec 10 05:30:31 2010
@@ -128,13 +128,7 @@ class Router(val host:VirtualHost) exten
 
   def _create_queue(id:Long, binding:Binding):Queue = {
 
-    val config = {
-      import collection.JavaConversions._
-      host.config.queues.find{ config=>
-        binding.matches(config)
-      }
-    }.getOrElse(new QueueDTO)
-
+    val config = host.queue_config(binding).getOrElse(new QueueDTO)
 
     var qid = id
     if( qid == -1 ) {
@@ -349,11 +343,7 @@ class RoutingNode(val router:Router, val
 
   import OptionSupport._
 
-  val config = {
-    import collection.JavaConversions._
-    import DestinationParser.default._
-    router.host.config.destinations.find( x=> parseFilter(ascii(x.path)).matches(name) ).getOrElse(DEFAULT_CONFIG)
-  }
+  val config = router.host.destination_config(name).getOrElse(DEFAULT_CONFIG)
 
   def unified = config.unified.getOrElse(false)
   def slow_consumer_policy = config.slow_consumer_policy.getOrElse("block")

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=1044209&r1=1044208&r2=1044209&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 Fri Dec 10 05:30:31 2010
@@ -21,7 +21,6 @@ import _root_.java.lang.{String}
 import _root_.scala.collection.JavaConversions._
 import org.fusesource.hawtdispatch._
 
-import org.apache.activemq.apollo.dto.{VirtualHostDTO}
 import java.util.concurrent.TimeUnit
 import org.apache.activemq.apollo.store.{Store, StoreFactory}
 import org.apache.activemq.apollo.util._
@@ -31,8 +30,9 @@ import org.fusesource.hawtbuf.{Buffer, A
 import collection.JavaConversions
 import java.util.concurrent.atomic.AtomicLong
 import org.apache.activemq.apollo.util.OptionSupport._
-import security.{Authenticator, Authorizer}
-import org.apache.activemq.apollo.util.path.PathParser
+import org.apache.activemq.apollo.util.path.{Path, PathParser}
+import org.apache.activemq.apollo.dto.{DestinationDTO, QueueDTO, BindingDTO, VirtualHostDTO}
+import security.{AclAuthorizer, JaasAuthenticator, Authenticator, Authorizer}
 
 /**
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
@@ -115,6 +115,15 @@ class VirtualHost(val broker: Broker, va
   override protected def _start(onCompleted:Runnable):Unit = {
 
     val tracker = new LoggingTracker("virtual host startup", dispatchQueue)
+
+    if( config.authentication != null ) {
+      authenticator = new JaasAuthenticator(config.authentication.domain)
+      authorizer = new AclAuthorizer(config.authentication.kinds().toList)
+    } else {
+      authenticator = broker.authenticator
+      authorizer = broker.authorizer
+    }
+
     store = StoreFactory.create(config.store)
 
     //    val memory_pool_config: String = null
@@ -253,4 +262,22 @@ class VirtualHost(val broker: Broker, va
     dispatchQueue.dispatchAfter(1, TimeUnit.SECONDS, ^{ if(serviceState.isStarted) { connectionRegroup } } )
   }
 
+  def destination_config(name:Path):Option[DestinationDTO] = {
+    import collection.JavaConversions._
+    import DestinationParser.default._
+    import AsciiBuffer._
+    config.destinations.find( x=> parseFilter(ascii(x.path)).matches(name) )
+  }
+
+  def queue_config(binding:Binding):Option[QueueDTO] = {
+    import collection.JavaConversions._
+    config.queues.find{ config=>
+      binding.matches(config)
+    }
+  }
+
+  def queue_config(dto:BindingDTO):Option[QueueDTO] = {
+    queue_config(BindingFactory.create(dto))
+  }
+
 }

Added: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AclAuthorizer.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AclAuthorizer.scala?rev=1044209&view=auto
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AclAuthorizer.scala (added)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/AclAuthorizer.scala Fri Dec 10 05:30:31 2010
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.activemq.apollo.broker.security
+
+import org.apache.activemq.apollo.broker.{Destination, VirtualHost, Broker}
+import scala.util.continuations._
+import org.apache.activemq.apollo.util.path.Path
+import org.apache.activemq.apollo.dto.{PrincipalDTO, QueueAclDTO, DestinationAclDTO, BindingDTO}
+
+/**
+ * <p>
+ * Authorizes based on the acl configuration found in
+ * the broker configuration model
+ * </p>
+ *
+ * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
+ */
+class AclAuthorizer(val default_kinds:List[String]) extends Authorizer {
+  import collection.JavaConversions._
+
+  var allow_deafult = true
+
+  private def sync[T](func: =>T): T @suspendable = shift { k: (T=>Unit) =>
+    k(func)
+  }
+
+  def is_in(ctx: SecurityContext, allowed:java.util.Set[PrincipalDTO]):Boolean = {
+    ctx.intersects(allowed.toSet, default_kinds)
+  }
+
+  def can_admin(ctx: SecurityContext, broker: Broker) = sync {
+    if( broker.config.acl!=null ) {
+      is_in(ctx, broker.config.acl.admins)
+    } else {
+      allow_deafult
+    }
+  }
+
+  def can_connect_to(ctx: SecurityContext, host: VirtualHost) = sync {
+    if( host.config.acl!=null ) {
+      is_in(ctx, host.config.acl.connects)
+    } else {
+      allow_deafult
+    }
+  }
+
+
+  private def for_dest(ctx: SecurityContext, host: VirtualHost, dest: Path)(func: DestinationAclDTO=>java.util.Set[PrincipalDTO]) = {
+    host.destination_config(dest).map { config=>
+      if( config.acl!=null ) {
+        is_in(ctx, func(config.acl))
+      } else {
+        allow_deafult
+      }
+    }.getOrElse(allow_deafult)
+  }
+
+  def can_send_to(ctx: SecurityContext, host: VirtualHost, dest: Path) = sync {
+    for_dest(ctx, host, dest)(_.sends)
+  }
+  def can_receive_from(ctx: SecurityContext, host: VirtualHost, dest: Path) = sync {
+    for_dest(ctx, host, dest)(_.receives)
+  }
+  def can_destroy(ctx: SecurityContext, host: VirtualHost, dest: Path) = sync  {
+    for_dest(ctx, host, dest)(_.destroys)
+  }
+  def can_create(ctx: SecurityContext, host: VirtualHost, dest: Path) = sync  {
+    for_dest(ctx, host, dest)(_.creates)
+  }
+
+  private def for_queue(ctx: SecurityContext, host: VirtualHost, dto: BindingDTO)(func: QueueAclDTO=>java.util.Set[PrincipalDTO]) = {
+    host.queue_config(dto).map { config=>
+      if( config.acl!=null ) {
+        is_in(ctx, func(config.acl))
+      } else {
+        allow_deafult
+      }
+    }.getOrElse(allow_deafult)
+  }
+
+  def can_send_to(ctx: SecurityContext, host: VirtualHost, dest: BindingDTO) = sync  {
+    for_queue(ctx, host, dest)(_.sends)
+  }
+
+  def can_receive_from(ctx: SecurityContext, host: VirtualHost, dest: BindingDTO) = sync  {
+    for_queue(ctx, host, dest)(_.receives)
+  }
+
+  def can_destroy(ctx: SecurityContext, host: VirtualHost, dest: BindingDTO) = sync  {
+    for_queue(ctx, host, dest)(_.destroys)
+  }
+
+  def can_create(ctx: SecurityContext, host: VirtualHost, dest: BindingDTO) = sync  {
+    for_queue(ctx, host, dest)(_.creates)
+  }
+
+  def can_consume_from(ctx: SecurityContext, host: VirtualHost, dest: BindingDTO) = sync  {
+    for_queue(ctx, host, dest)(_.consumes)
+  }
+
+}
\ No newline at end of file

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=1044209&r1=1044208&r2=1044209&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 Fri Dec 10 05:30:31 2010
@@ -15,6 +15,10 @@
  * limitations under the License.
  */
 package org.apache.activemq.apollo.broker.security
+import scala.util.continuations._
+import org.apache.activemq.apollo.dto.BindingDTO
+import org.apache.activemq.apollo.broker.{VirtualHost, Broker, Destination}
+import org.apache.activemq.apollo.util.path.Path
 
 /**
  * <p>
@@ -22,6 +26,61 @@ package org.apache.activemq.apollo.broke
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-class Authorizer {
+trait Authorizer {
+
+  /**
+   * @returns true if the user is an admin.
+   */
+  def can_admin(ctx:SecurityContext, broker:Broker):Boolean @suspendable
+
+  /**
+   * @returns true if the user is allowed to connect to the virtual host
+   */
+  def can_connect_to(ctx:SecurityContext, host:VirtualHost):Boolean @suspendable
+
+  /**
+   * @returns true if the user is allowed to send to the destination
+   */
+  def can_send_to(ctx:SecurityContext, host:VirtualHost, dest:Path):Boolean @suspendable
+
+  /**
+   * @returns true if the user is allowed to receive from the destination
+   */
+  def can_receive_from(ctx:SecurityContext, host:VirtualHost, dest:Path):Boolean @suspendable
+
+  /**
+   * @returns true if the user is allowed to create the destination
+   */
+  def can_create(ctx:SecurityContext, host:VirtualHost, dest:Path):Boolean @suspendable
+
+  /**
+   * @returns true if the user is allowed to destroy the destination
+   */
+  def can_destroy(ctx:SecurityContext, host:VirtualHost, dest:Path):Boolean @suspendable
+
+  /**
+   * @returns true if the user is allowed to send to the queue
+   */
+  def can_send_to(ctx:SecurityContext, host:VirtualHost, dest:BindingDTO):Boolean @suspendable
+
+  /**
+   * @returns true if the user is allowed to receive from the queue
+   */
+  def can_receive_from(ctx:SecurityContext, host:VirtualHost, dest:BindingDTO):Boolean @suspendable
+
+  /**
+   * @returns true if the user is allowed to consume from the queue
+   */
+  def can_consume_from(ctx:SecurityContext, host:VirtualHost, dest:BindingDTO):Boolean @suspendable
+
+  /**
+   * @returns true if the user is allowed to create the queue
+   */
+  def can_create(ctx:SecurityContext, host:VirtualHost, dest:BindingDTO):Boolean @suspendable
+
+  /**
+   * @returns true if the user is allowed to destroy the queue
+   */
+  def can_destroy(ctx:SecurityContext, host:VirtualHost, dest:BindingDTO):Boolean @suspendable
 
 }
\ No newline at end of file

Copied: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/EncryptionSupport.scala (from r1043831, activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala)
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/EncryptionSupport.scala?p2=activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/EncryptionSupport.scala&p1=activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala&r1=1043831&r2=1044209&rev=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/EncryptionSupport.scala Fri Dec 10 05:30:31 2010
@@ -16,12 +16,10 @@
  */
 package org.apache.activemq.apollo.broker.security
 
-import java.security.Principal
-import collection.mutable.HashSet
-import javax.security.auth.Subject
-import java.security.cert.X509Certificate
-import org.apache.activemq.apollo.util.OptionSupport._
-import org.apache.activemq.jaas.UserPrincipal
+import java.util.Properties
+import org.jasypt.properties.PropertyValueEncryptionUtils
+import org.jasypt.encryption.pbe.StandardPBEStringEncryptor
+import org.jasypt.encryption.pbe.config.EnvironmentStringPBEConfig
 
 /**
  * <p>
@@ -29,30 +27,29 @@ import org.apache.activemq.jaas.UserPrin
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-class SecurityContext {
+object EncryptionSupport {
 
-  var user:String = _
-  var password:String = _
-  var certificates = Array[X509Certificate]()
+  val encryptor = new StandardPBEStringEncryptor
+  encryptor.setConfig({
+    val config = new EnvironmentStringPBEConfig
+    config.setAlgorithm("PBEWithMD5AndDES")
+    config.setPasswordEnvName("APOLLO_ENCRYPTION_PASSWORD")
+    config
+  })
+
+  def decrypt(props:Properties):Properties = {
+
+    import collection.JavaConversions._
+    props.keySet.toArray.foreach{ k=>
+      val key = k.asInstanceOf[String]
+      var value = props.getProperty(key)
+      if (PropertyValueEncryptionUtils.isEncryptedValue(value)) {
+        value = PropertyValueEncryptionUtils.decrypt(value, encryptor);
+        props.setProperty(key, value)
+      }
+    }
+    props
 
-  /**
-   * A place for the authorization layer attach
-   * some authorization data (i.e. an authorization
-   * cache)
-   */
-  var authorization_data:AnyRef = _
-
-  var subject:Subject = _
-
-  def principles = subject.map(x=>collection.JavaConversions.asSet(x.getPrincipals).toSet).getOrElse(Set())
-
-  def intersects(other_priciples:Set[Principal] ) = {
-    !principles.intersect(other_priciples).isEmpty
-  }
-
-  def principle[T](kind:Class[T]):Option[T] ={
-    principles.find( x=> kind.isAssignableFrom(x.getClass) ).map(kind.cast(_))
   }
 
-  def user_principle = principle(classOf[UserPrincipal])
 }
\ No newline at end of file

Added: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/FileLoginModule.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/FileLoginModule.scala?rev=1044209&view=auto
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/FileLoginModule.scala (added)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/FileLoginModule.scala Fri Dec 10 05:30:31 2010
@@ -0,0 +1,167 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License") you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.activemq.apollo.broker.security
+
+import java.io.File
+import java.io.FileInputStream
+import java.io.IOException
+import java.security.Principal
+import java.util.Properties
+import javax.security.auth.Subject
+import javax.security.auth.callback.Callback
+import javax.security.auth.callback.CallbackHandler
+import javax.security.auth.callback.NameCallback
+import javax.security.auth.callback.PasswordCallback
+import javax.security.auth.callback.UnsupportedCallbackException
+import javax.security.auth.login.FailedLoginException
+import javax.security.auth.login.LoginException
+import javax.security.auth.spi.LoginModule
+
+import org.apache.activemq.jaas.GroupPrincipal
+import org.apache.activemq.jaas.UserPrincipal
+import java.{util => ju}
+import org.apache.activemq.apollo.util.{FileSupport, Log}
+import FileSupport._
+
+object FileLoginModule extends Log {
+  val LOGIN_CONFIG = "java.security.auth.login.config"
+  val USERS_FILE = "users_file"
+  val GROUPS_FILE = "groups_file"
+}
+
+/**
+ * <p>
+ * </p>
+ *
+ * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
+ */
+class FileLoginModule extends LoginModule {
+
+  import FileLoginModule._
+
+  private var subject: Subject = _
+  private var callback_handler: CallbackHandler = _
+
+  private var user_file: File = _
+  private var group_file: File = _
+
+  private val users = new Properties()
+  private val groups = new Properties()
+
+  private var user: String = _
+  private val principals = new ju.HashSet[Principal]()
+
+  def initialize(subject: Subject, callback_handler: CallbackHandler, shared_state: ju.Map[String, _], options: ju.Map[String, _]): Unit = {
+
+    this.subject = subject
+    this.callback_handler = callback_handler
+
+    val base_dir = if (System.getProperty(LOGIN_CONFIG) != null) {
+      new File(System.getProperty(LOGIN_CONFIG)).getParentFile()
+    } else {
+      new File(".")
+    }
+
+    user_file = new File(base_dir, options.get(USERS_FILE).asInstanceOf[String])
+    group_file = new File(base_dir, options.get(GROUPS_FILE).asInstanceOf[String])
+
+    debug("Initialized user_file=%s group_file=%s", user_file, group_file)
+  }
+
+  def login: Boolean = {
+    try {
+      users.clear()
+      using( new FileInputStream(user_file) ) { in=>
+        users.load(in)
+      }
+      EncryptionSupport.decrypt(users)
+    } catch {
+      case ioe: IOException => throw new LoginException("Unable to load user properties file " + user_file)
+    }
+
+    try {
+      groups.clear
+      using( new FileInputStream(group_file) ) { in=>
+        groups.load(in)
+      }
+    } catch {
+      case ioe: IOException => throw new LoginException("Unable to load group properties file " + group_file)
+    }
+
+    val callbacks = new Array[Callback](2)
+    callbacks(0) = new NameCallback("Username: ")
+    callbacks(1) = new PasswordCallback("Password: ", false)
+    try {
+      callback_handler.handle(callbacks)
+    } catch {
+      case ioe: IOException =>
+        throw new LoginException(ioe.getMessage())
+      case uce: UnsupportedCallbackException =>
+        throw new LoginException(uce.getMessage() + " not available to obtain information from user")
+    }
+
+    user = callbacks(0).asInstanceOf[NameCallback].getName()
+    var tmpPassword = callbacks(1).asInstanceOf[PasswordCallback].getPassword()
+    if (tmpPassword == null) {
+      tmpPassword = new Array[Char](0)
+    }
+    val password = users.getProperty(user)
+
+    if (password == null || !password.equals(new String(tmpPassword))) {
+      throw new FailedLoginException("Invalid user id or password")
+    }
+    debug("login %s", user)
+    true
+  }
+
+  def commit: Boolean = {
+    principals.add(new UserPrincipal(user))
+    val en = groups.keys()
+    while (en.hasMoreElements()) {
+      val name = en.nextElement().asInstanceOf[String]
+      val userList = (groups.getProperty(name) + "").split(",")
+      userList.foreach{
+        x =>
+          if (user == x) {
+            principals.add(new GroupPrincipal(name))
+          }
+      }
+    }
+
+    subject.getPrincipals().addAll(principals)
+
+    user = null
+    debug("commit")
+    return true
+  }
+
+  def abort: Boolean = {
+    user = null
+    debug("abort")
+    return true
+  }
+
+  def logout: Boolean = {
+    subject.getPrincipals().removeAll(principals)
+    principals.clear
+    user = null
+    debug("logout")
+    return true
+  }
+
+
+}

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/JaasAuthenticator.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/JaasAuthenticator.scala?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/JaasAuthenticator.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/JaasAuthenticator.scala Fri Dec 10 05:30:31 2010
@@ -36,9 +36,7 @@ import org.fusesource.hawtdispatch._
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
 
-class JaasAuthenticator extends Authenticator {
-
-  var jass_realm: String = _
+class JaasAuthenticator(val jass_realm: String) extends Authenticator {
 
   /*
    * The 'BLOCKABLE_THREAD_POOL ! { ... }' magic makes the code block

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala Fri Dec 10 05:30:31 2010
@@ -21,7 +21,8 @@ import collection.mutable.HashSet
 import javax.security.auth.Subject
 import java.security.cert.X509Certificate
 import org.apache.activemq.apollo.util.OptionSupport._
-import org.apache.activemq.jaas.UserPrincipal
+import org.apache.activemq.jaas.{GroupPrincipal, UserPrincipal}
+import org.apache.activemq.apollo.dto.PrincipalDTO
 
 /**
  * <p>
@@ -35,24 +36,35 @@ class SecurityContext {
   var password:String = _
   var certificates = Array[X509Certificate]()
 
-  /**
-   * A place for the authorization layer attach
-   * some authorization data (i.e. an authorization
-   * cache)
-   */
-  var authorization_data:AnyRef = _
+  private val principles = new HashSet[PrincipalDTO]()
 
-  var subject:Subject = _
+  private var _subject:Subject = _
 
-  def principles = subject.map(x=>collection.JavaConversions.asSet(x.getPrincipals).toSet).getOrElse(Set())
+  def subject = _subject
 
-  def intersects(other_priciples:Set[Principal] ) = {
-    !principles.intersect(other_priciples).isEmpty
+  def subject_= (value:Subject) {
+    _subject = value
+    principles.clear
+    if( value!=null ) {
+      import collection.JavaConversions._
+      value.getPrincipals.foreach { x=>
+        principles.add(new PrincipalDTO(x.getName, x.getClass.getName))
+      }
+    }
   }
 
-  def principle[T](kind:Class[T]):Option[T] ={
-    principles.find( x=> kind.isAssignableFrom(x.getClass) ).map(kind.cast(_))
+  def intersects(values:Set[PrincipalDTO], default_kinds:List[String]):Boolean = {
+    val (v1, v2) = values.partition(_.kind == null)
+    if( principles.intersect(v2).isEmpty ) {
+      return true
+    }
+    default_kinds.foreach { x=>
+      if( ! (v1.map(y=> new PrincipalDTO(y.name, x) ).intersect(v1).isEmpty) ) {
+        return true
+      }
+    }
+    false
   }
 
-  def user_principle = principle(classOf[UserPrincipal])
+
 }
\ No newline at end of file

Modified: activemq/activemq-apollo/trunk/apollo-cli/pom.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/pom.xml?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/pom.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-cli/pom.xml Fri Dec 10 05:30:31 2010
@@ -40,15 +40,8 @@
     </dependency>
 
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
-      <version>${jetty-version}</version>
-      <optional>true</optional>
-    </dependency>
-    
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
+      <groupId>org.eclipse.jetty.aggregate</groupId>
+      <artifactId>jetty-all-server</artifactId>
       <version>${jetty-version}</version>
       <optional>true</optional>
     </dependency>
@@ -59,6 +52,12 @@
       <version>${karaf-version}</version>
     </dependency>
 
+    <dependency>
+      <groupId>org.jasypt</groupId>
+      <artifactId>jasypt</artifactId>
+      <version>${jasypt-version}</version>
+    </dependency>
+
     <!-- Testing Dependencies -->
     <dependency>
       <groupId>org.scalatest</groupId>

Modified: activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/apollo-ssl.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/apollo-ssl.xml?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/apollo-ssl.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/apollo-ssl.xml Fri Dec 10 05:30:31 2010
@@ -14,10 +14,18 @@
   limitations under the License.
 -->
 <broker id="default" rev="1" xmlns="http://activemq.apache.org/schema/activemq/apollo">
+
   <notes>
     The default configuration with tls/ssl enabled.
   </notes>
 
+  <authentication domain="apollo"/>
+  <acl>
+    <admin name="admins"/>
+  </acl>
+
+  <web-admin host="127.0.0.1" port="8080"/>
+
   <virtual-host id="default" auto-create-queues="true" >
     <!--
       You should add all the host names that this virtual host is known as

Modified: activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/apollo.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/apollo.xml?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/apollo.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/apollo.xml Fri Dec 10 05:30:31 2010
@@ -13,11 +13,20 @@
   License for the specific language governing permissions and
   limitations under the License.
 -->
-<broker id="default" rev="1" xmlns="http://activemq.apache.org/schema/activemq/apollo">
+<broker id="default" rev="1"
+  xmlns="http://activemq.apache.org/schema/activemq/apollo">
+
   <notes>
     The default configuration.
   </notes>
 
+  <authentication domain="apollo"/>
+  <acl>
+    <admin name="admins"/>
+  </acl>
+
+  <web-admin host="127.0.0.1" port="8080"/>
+
   <virtual-host id="default" auto-create-queues="true" >
 
     <!--

Added: activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/groups.properties
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/groups.properties?rev=1044209&view=auto
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/groups.properties (added)
+++ activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/groups.properties Fri Dec 10 05:30:31 2010
@@ -0,0 +1,24 @@
+## ---------------------------------------------------------------------------
+## Licensed to the Apache Software Foundation (ASF) under one or more
+## contributor license agreements.  See the NOTICE file distributed with
+## this work for additional information regarding copyright ownership.
+## The ASF licenses this file to You under the Apache License, Version 2.0
+## (the "License"); you may not use this file except in compliance with
+## the License.  You may obtain a copy of the License at
+##
+## http://www.apache.org/licenses/LICENSE-2.0
+##
+## Unless required by applicable law or agreed to in writing, software
+## distributed under the License is distributed on an "AS IS" BASIS,
+## WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+## See the License for the specific language governing permissions and
+## limitations under the License.
+## ---------------------------------------------------------------------------
+
+#
+# Allows you to place multiple users in a group.
+# Example:
+#
+# power_users=admin,chirino
+#
+admins=admin
\ No newline at end of file

Added: activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/login.config
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/login.config?rev=1044209&view=auto
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/login.config (added)
+++ activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/login.config Fri Dec 10 05:30:31 2010
@@ -0,0 +1,33 @@
+// ---------------------------------------------------------------------------
+// Licensed to the Apache Software Foundation (ASF) under one or more
+// contributor license agreements.  See the NOTICE file distributed with
+// this work for additional information regarding copyright ownership.
+// The ASF licenses this file to You under the Apache License, Version 2.0
+// (the "License"); you may not use this file except in compliance with
+// the License.  You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+// ---------------------------------------------------------------------------
+apollo {
+
+  //
+  // Sets up simple file based security
+  //
+  org.apache.activemq.apollo.broker.security.FileLoginModule required
+    users_file="users.properties"
+    groups_file="groups.properties"
+    ;
+
+
+  //
+  // You could use any JAAS based login module too.
+  //
+  // com.sun.security.auth.module.UnixLoginModule optional;
+
+};
\ No newline at end of file

Added: activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/users.properties
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/users.properties?rev=1044209&view=auto
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/users.properties (added)
+++ activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/users.properties Fri Dec 10 05:30:31 2010
@@ -0,0 +1,26 @@
+## ---------------------------------------------------------------------------
+## Licensed to the Apache Software Foundation (ASF) under one or more
+## contributor license agreements.  See the NOTICE file distributed with
+## this work for additional information regarding copyright ownership.
+## The ASF licenses this file to You under the Apache License, Version 2.0
+## (the "License"); you may not use this file except in compliance with
+## the License.  You may obtain a copy of the License at
+##
+## http://www.apache.org/licenses/LICENSE-2.0
+##
+## Unless required by applicable law or agreed to in writing, software
+## distributed under the License is distributed on an "AS IS" BASIS,
+## WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+## See the License for the specific language governing permissions and
+## limitations under the License.
+## ---------------------------------------------------------------------------
+
+#
+# The list of users that can login.  This file supports both plain text or
+# encrypted passwords.  Here is an example what an encrypted password
+# would look like:
+#
+# admin=ENC(Cf3Jf3tM+UrSOoaKU50od5CuBa8rxjoL)
+#
+
+admin=password
\ No newline at end of file

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=1044209&r1=1044208&r2=1044209&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 Fri Dec 10 05:30:31 2010
@@ -62,6 +62,15 @@ class Create extends Action {
       var target = etc / "log4j.properties"
       write("etc/log4j.properties", target)
 
+      target = etc / "users.properties"
+      write("etc/users.properties", target)
+
+      target = etc / "groups.properties"
+      write("etc/groups.properties", target)
+
+      target = etc / "login.config"
+      write("etc/login.config", target)
+
       // Generate a keystore with a new key
       println("Generating ssl keystore...")
       val ssl = system(etc, Array(

Modified: activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Run.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Run.scala?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Run.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Run.scala Fri Dec 10 05:30:31 2010
@@ -23,19 +23,22 @@ import org.fusesource.jansi.Ansi
 import org.fusesource.jansi.Ansi.Color._
 import org.fusesource.jansi.Ansi.Attribute._
 
-import org.mortbay.jetty.Connector
-import org.mortbay.jetty.Handler
-import org.mortbay.jetty.Server
-import org.mortbay.jetty.nio.SelectChannelConnector
-import org.mortbay.jetty.webapp.WebAppContext
 import org.apache.commons.logging.LogFactory
 import org.apache.activemq.apollo.broker.{BrokerRegistry, Broker, ConfigStore, FileConfigStore}
 import org.fusesource.hawtdispatch._
 import Helper._
-import org.apache.activemq.apollo.util.{Logging, ServiceControl}
 import org.apache.activemq.apollo.util.FileSupport._
 import org.apache.activemq.apollo.util.OptionSupport._
 import org.apache.activemq.apollo.cli.Apollo
+import org.eclipse.jetty.server.{Connector, Handler, Server}
+import org.eclipse.jetty.security._
+import authentication.BasicAuthenticator
+import org.eclipse.jetty.webapp.WebAppContext
+import org.eclipse.jetty.server.nio.SelectChannelConnector
+import org.eclipse.jetty.plus.jaas.JAASLoginService
+import org.eclipse.jetty.server.handler.HandlerCollection
+import org.apache.activemq.apollo.util.{Logging, ServiceControl}
+import org.apache.activemq.apollo.dto.{WebAdminDTO, PrincipalDTO}
 
 /**
  * The apollo create command
@@ -43,18 +46,14 @@ import org.apache.activemq.apollo.cli.Ap
 @command(scope="apollo", name = "run", description = "runs the broker instance")
 class Run extends Action with Logging {
 
-  @option(name = "--port", description = "The port of the http based administration service")
-  var port: Int = 8080
-
-  @option(name = "--prefix", description = "The prefix path of the web application.")
-  var prefix: String = "/"
-
   @option(name = "--conf", description = "The Apollo configuration file.")
   var conf: File = _
 
   @option(name = "--tmp", description = "A temp directory.")
   var tmp: File = _
 
+  def x(value:AnyRef) = println(value)
+
   def execute(session: CommandSession):AnyRef = {
 
     try {
@@ -69,6 +68,11 @@ class Run extends Action with Logging {
         error("Configuration file'%s' does not exist.\n\nTry creating a broker instance using the 'apollo create' command.".format(conf));
       }
 
+      val login_config = conf.getParentFile / "login.config"
+      if( login_config.exists ) {
+        System.setProperty("java.security.auth.login.config", login_config.getCanonicalPath)
+      }
+
       val webapp = {
         val x = System.getProperty("apollo.webapp")
         if( x != null ) {
@@ -92,50 +96,89 @@ class Run extends Action with Logging {
       val store = new FileConfigStore
       store.file = conf
       ConfigStore() = store
-      store.start(^{
-        store.dispatchQueue {
-          store.listBrokers.foreach { id=>
-            store.getBroker(id, true).foreach{ config=>
-              // Only start the broker up if it's enabled..
-              if( config.enabled.getOrElse(true) ) {
-                debug("Starting broker '%s'", config.id);
-                val broker = new Broker()
-                broker.config = config
-                BrokerRegistry.add(config.id, broker)
-                broker.start(^{
-                  info("Broker '%s' started", config.id);
-                })
-              }
-            }
-          }
-        }
-      })
+      store.start
+      val config = store.load(true)
 
+      // Only start the broker up if it's enabled..
+      if( config.enabled.getOrElse(true) ) {
+        debug("Starting broker '%s'", config.id);
+        val broker = new Broker()
+        broker.config = config
+        BrokerRegistry.add(config.id, broker)
+        broker.start(^{
+          info("Broker '%s' started", config.id);
+        })
+      }
 
-      // Start up the admin interface...
-      debug("Starting administration interface");
-      var server = new Server
 
-      var connector = new SelectChannelConnector
-      connector.setPort(port)
-      connector.setServer(server)
+      val web_admin = config.web_admin.getOrElse(new WebAdminDTO)
+      if( web_admin.enabled.getOrElse(true) ) {
 
-      var app_context = new WebAppContext
-      app_context.setContextPath(prefix)
-      app_context.setWar(webapp.getCanonicalPath)
-      app_context.setServer(server)
-      app_context.setLogUrlOnStart(true)
+        val prefix = web_admin.prefix.getOrElse("/")
+        val port = web_admin.port.getOrElse(8080)
+        val host = web_admin.host.getOrElse("127.0.0.1")
 
-      app_context.setTempDirectory(tmp)
-      System.setProperty("scalate.workdir", (tmp / "scalate").getCanonicalPath )
+        // Start up the admin interface...
+        debug("Starting administration interface");
 
-      server.setHandlers(Array[Handler](app_context))
-      server.setConnectors(Array[Connector](connector))
-      server.start
+        System.setProperty("scalate.workdir", (tmp / "scalate").getCanonicalPath )
+
+        var connector = new SelectChannelConnector
+        connector.setHost(host)
+        connector.setPort(port)
+
+
+        def admin_app = {
+          var app_context = new WebAppContext
+          app_context.setContextPath(prefix)
+          app_context.setWar(webapp.getCanonicalPath)
+          app_context.setTempDirectory(tmp)
+          app_context
+        }
+
+        def secured(handler:Handler) = {
+          x(config)
+          if( config.authentication!=null && config.acl!=null ) {
+            import collection.JavaConversions._
+
+            val security_handler = new ConstraintSecurityHandler
+            val login_service = new JAASLoginService(config.authentication.domain)
+            val role_class_names:List[String] = config.authentication.kinds().toList
+
+            login_service.setRoleClassNames(role_class_names.toArray)
+            security_handler.setLoginService(login_service)
+            security_handler.setIdentityService(new DefaultIdentityService)
+            security_handler.setAuthenticator(new BasicAuthenticator)
+
+            val cm = new ConstraintMapping
+            val c = new org.eclipse.jetty.http.security.Constraint()
+            c.setName("BASIC")
+            val admins:Set[PrincipalDTO] = config.acl.admins.toSet
+            c.setRoles(admins.map(_.name).toArray)
+            c.setAuthenticate(true)
+            cm.setConstraint(c)
+            cm.setPathSpec("/*")
+            cm.setMethod("GET")
+            security_handler.addConstraintMapping(cm)
+
+            security_handler.setHandler(handler)
+            security_handler
+          } else {
+            handler
+          }
+        }
+
+        var server = new Server
+        server.setHandler(secured(admin_app))
+        server.setConnectors(Array[Connector](connector))
+        server.start
+
+        val localPort = connector.getLocalPort
+        def url = "http://"+host+":" + localPort + prefix
+        info("Administration interface available at: "+bold(url))
+
+      }
 
-      val localPort = connector.getLocalPort
-      def url = "http://localhost:" + localPort + prefix
-      info("Administration interface available at: "+bold(url))
 
       if(java.lang.Boolean.getBoolean("hawtdispatch.profile")) {
         monitor_hawtdispatch

Modified: activemq/activemq-apollo/trunk/apollo-distro/pom.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-distro/pom.xml?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-distro/pom.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-distro/pom.xml Fri Dec 10 05:30:31 2010
@@ -83,19 +83,17 @@
     </dependency>
 
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty.aggregate</groupId>
+      <artifactId>jetty-all-server</artifactId>
       <version>${jetty-version}</version>
       <optional>true</optional>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
-      <version>${jetty-version}</version>
-      <optional>true</optional>
+      <groupId>org.jasypt</groupId>
+      <artifactId>jasypt</artifactId>
+      <version>${jasypt-version}</version>
     </dependency>
 
-
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>

Modified: activemq/activemq-apollo/trunk/apollo-distro/src/main/descriptors/common-bin.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-distro/src/main/descriptors/common-bin.xml?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-distro/src/main/descriptors/common-bin.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-distro/src/main/descriptors/common-bin.xml Fri Dec 10 05:30:31 2010
@@ -32,15 +32,21 @@
         <include>org.fusesource.hawtdb:*</include>
         <include>org.codehaus.jackson:jackson-mapper-asl</include>
         <include>org.codehaus.jackson:jackson-core-asl</include>
+        
+        <!-- security stuff -->
+        <include>org.jasypt:jasypt</include>
+        <include>commons-lang:commons-lang</include>
+        <include>commons-codec:commons-codec</include>
+        <include>org.apache.activemq:activemq-jaas</include>
 
         <!-- for the webapp -->
-        <include>org.mortbay.jetty:*</include>
+        <include>org.eclipse.jetty.aggregate:jetty-all-server</include>
+        <include>javax.servlet:servlet-api</include>
         <include>com.sun.jersey:jersey-server</include>
         <include>com.sun.jersey:jersey-core</include>
         <include>asm:asm</include>
         <include>org.codehaus.jackson:jackson-jaxrs</include>
         <include>org.fusesource.scalate:scalate-core</include>
-        <!-- <include>org.scala-lang:scala-compiler</include> -->
         
         <!-- the logging apis used -->
         <include>org.slf4j:slf4j-api</include>

Modified: activemq/activemq-apollo/trunk/apollo-dto/pom.xml
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/pom.xml?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/pom.xml (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/pom.xml Fri Dec 10 05:30:31 2010
@@ -72,6 +72,7 @@
       <scope>test</scope>
       <version>${log4j-version}</version>
     </dependency>
+
   </dependencies>
 
   <build>

Copied: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthenticationDTO.java (from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java)
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthenticationDTO.java?p2=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthenticationDTO.java&p1=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java&r1=1043831&r2=1044209&rev=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthenticationDTO.java Fri Dec 10 05:30:31 2010
@@ -16,12 +16,14 @@
  */
 package org.apache.activemq.apollo.dto;
 
-import org.codehaus.jackson.annotate.JsonProperty;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
-import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlElement;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 
 /**
  * <p>
@@ -29,22 +31,23 @@ import javax.xml.bind.annotation.XmlRoot
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-@XmlRootElement(name="broker-summary")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class BrokerSummaryDTO extends StringIdDTO {
+public class AuthenticationDTO {
+
+    @XmlAttribute
+    public String domain;
+
+    @XmlElement(name="kind")
+    public List<String> kinds = new ArrayList<String>();
+
 
-    /**
-     * Is a running broker accessible via management API calls?
-     */
-    @JsonProperty
-	@XmlAttribute
-	public boolean manageable;
-
-    /**
-     * Is the broker configuration accessible via API calls? 
-     */
-    @JsonProperty
-	@XmlAttribute
-	public boolean configurable;
+    public List<String> kinds() {
+        if( kinds.isEmpty() ) {
+            ArrayList<String> rc = new ArrayList<String>();
+            rc.add("org.apache.activemq.jaas.GroupPrincipal");
+            return rc;
+        }
+        return kinds;
+    }
 
 }

Copied: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerAclDTO.java (from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthenticatorDTO.java)
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerAclDTO.java?p2=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerAclDTO.java&p1=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthenticatorDTO.java&r1=1043831&r2=1044209&rev=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/AuthenticatorDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerAclDTO.java Fri Dec 10 05:30:31 2010
@@ -16,17 +16,21 @@
  */
 package org.apache.activemq.apollo.dto;
 
-import org.codehaus.jackson.annotate.JsonTypeInfo;
-
-import javax.xml.bind.annotation.XmlType;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import java.util.*;
 
 /**
+ * <p>
+ * </p>
+ *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-@XmlType(name = "authenticator-type")
-//@XmlSeeAlso({xxx.class})
-@JsonTypeInfo(use=JsonTypeInfo.Id.CLASS, include=JsonTypeInfo.As.PROPERTY, property="@class")
-public abstract class AuthenticatorDTO {
+@XmlAccessorType(XmlAccessType.FIELD)
+public class BrokerAclDTO {
 
+    @XmlElement(name="admin")
+    public Set<PrincipalDTO> admins = new HashSet<PrincipalDTO>();
 
 }

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerDTO.java?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerDTO.java Fri Dec 10 05:30:31 2010
@@ -16,13 +16,10 @@
  */
 package org.apache.activemq.apollo.dto;
 
-import org.codehaus.jackson.annotate.JsonProperty;
-
+import javax.xml.bind.annotation.*;
 import java.util.ArrayList;
 import java.util.List;
 
-import javax.xml.bind.annotation.*;
-
 /**
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
@@ -33,7 +30,6 @@ public class BrokerDTO extends ServiceDT
     /**
      * Used to track config revisions.
      */
-    @JsonProperty
     @XmlAttribute
     public int rev;
 
@@ -46,7 +42,6 @@ public class BrokerDTO extends ServiceDT
     /**
      * Used to store any configuration notes.
      */
-    @JsonProperty
     @XmlElement
     public String notes;
 
@@ -70,4 +65,12 @@ public class BrokerDTO extends ServiceDT
     @XmlElementRef
     public KeyStorageDTO key_storage;
 
+    @XmlElement(name="acl")
+    public BrokerAclDTO acl;
+
+    @XmlElement(name="web-admin")
+    public WebAdminDTO web_admin;
+
+    @XmlElement(name="authentication")
+    public AuthenticationDTO authentication;
 }

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerStatusDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerStatusDTO.java?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerStatusDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerStatusDTO.java Fri Dec 10 05:30:31 2010
@@ -16,8 +16,6 @@
  */
 package org.apache.activemq.apollo.dto;
 
-import org.codehaus.jackson.annotate.JsonProperty;
-
 import javax.xml.bind.annotation.*;
 import java.util.ArrayList;
 import java.util.List;
@@ -59,7 +57,6 @@ public class BrokerStatusDTO extends Ser
     /**
      * The current running configuration of the object
      */
-    @JsonProperty
     @XmlElement
     public BrokerDTO config = null;
 

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java Fri Dec 10 05:30:31 2010
@@ -16,8 +16,6 @@
  */
 package org.apache.activemq.apollo.dto;
 
-import org.codehaus.jackson.annotate.JsonProperty;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlAttribute;
@@ -36,14 +34,12 @@ public class BrokerSummaryDTO extends St
     /**
      * Is a running broker accessible via management API calls?
      */
-    @JsonProperty
 	@XmlAttribute
 	public boolean manageable;
 
     /**
      * Is the broker configuration accessible via API calls? 
      */
-    @JsonProperty
 	@XmlAttribute
 	public boolean configurable;
 

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectorDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectorDTO.java?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectorDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectorDTO.java Fri Dec 10 05:30:31 2010
@@ -16,9 +16,10 @@
  */
 package org.apache.activemq.apollo.dto;
 
-import org.codehaus.jackson.annotate.JsonProperty;
-
-import javax.xml.bind.annotation.*;
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlRootElement;
 
 /**
  *
@@ -33,25 +34,21 @@ public class ConnectorDTO extends Servic
     /**
      * The transport uri which it will accept connections on.
      */
-    @JsonProperty
     @XmlAttribute
     public String bind;
 
     /**
      * The protocol that the transport will use.
      */
-    @JsonProperty
     @XmlAttribute
     public String protocol;
 
     /**
      * The uri which will be advertised for remote endpoints to connect to.
      */
-    @JsonProperty
     @XmlAttribute
     public String advertise;
 
-    @JsonProperty
     @XmlAttribute(name="connection-limit")
     public Integer connection_limit;
 

Copied: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationAclDTO.java (from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java)
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationAclDTO.java?p2=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationAclDTO.java&p1=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java&r1=1043831&r2=1044209&rev=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationAclDTO.java Fri Dec 10 05:30:31 2010
@@ -18,10 +18,8 @@ package org.apache.activemq.apollo.dto;
 
 import org.codehaus.jackson.annotate.JsonProperty;
 
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlAttribute;
-import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.*;
+import java.util.*;
 
 /**
  * <p>
@@ -29,22 +27,19 @@ import javax.xml.bind.annotation.XmlRoot
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-@XmlRootElement(name="broker-summary")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class BrokerSummaryDTO extends StringIdDTO {
+public class DestinationAclDTO {
 
-    /**
-     * Is a running broker accessible via management API calls?
-     */
-    @JsonProperty
-	@XmlAttribute
-	public boolean manageable;
-
-    /**
-     * Is the broker configuration accessible via API calls? 
-     */
-    @JsonProperty
-	@XmlAttribute
-	public boolean configurable;
+    @XmlElement(name="creates")
+    public Set<PrincipalDTO> creates = new HashSet<PrincipalDTO>();
+
+    @XmlElement(name="destroy")
+    public Set<PrincipalDTO> destroys = new HashSet<PrincipalDTO>();
+
+    @XmlElement(name="send")
+    public Set<PrincipalDTO> sends = new HashSet<PrincipalDTO>();
+
+    @XmlElement(name="receive")
+    public Set<PrincipalDTO> receives = new HashSet<PrincipalDTO>();
 
 }

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationDTO.java?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationDTO.java Fri Dec 10 05:30:31 2010
@@ -46,4 +46,7 @@ public class DestinationDTO extends Stri
     @XmlElement(name="slow-consumer-policy")
     public String slow_consumer_policy;
 
+    @XmlElement(name="acl")
+    public DestinationAclDTO acl;
+
 }

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationStatusDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationStatusDTO.java?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationStatusDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationStatusDTO.java Fri Dec 10 05:30:31 2010
@@ -16,8 +16,6 @@
  */
 package org.apache.activemq.apollo.dto;
 
-import org.codehaus.jackson.annotate.JsonProperty;
-
 import javax.xml.bind.annotation.*;
 import java.util.ArrayList;
 import java.util.List;
@@ -45,20 +43,17 @@ public class DestinationStatusDTO extend
      * Ids of all connections that are producing to the destination
      */
     @XmlElement(name="producer")
-    @JsonProperty("producers")
     public List<LinkDTO> producers = new ArrayList<LinkDTO>();
 
     /**
      * Ids of all connections that are consuming from the destination
      */
     @XmlElement(name="consumer")
-    @JsonProperty("consumers")
     public List<LinkDTO> consumers = new ArrayList<LinkDTO>();
 
     /**
      * Ids of all queues that are associated with the destination
      */
     @XmlElement(name="queue")
-    @JsonProperty("queues")
     public List<LongIdLabeledDTO> queues = new ArrayList<LongIdLabeledDTO>();
 }
\ No newline at end of file

Copied: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/PrincipalDTO.java (from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationStatusDTO.java)
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/PrincipalDTO.java?p2=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/PrincipalDTO.java&p1=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationStatusDTO.java&r1=1043831&r2=1044209&rev=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/DestinationStatusDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/PrincipalDTO.java Fri Dec 10 05:30:31 2010
@@ -16,11 +16,7 @@
  */
 package org.apache.activemq.apollo.dto;
 
-import org.codehaus.jackson.annotate.JsonProperty;
-
 import javax.xml.bind.annotation.*;
-import java.util.ArrayList;
-import java.util.List;
 
 /**
  * <p>
@@ -28,37 +24,47 @@ import java.util.List;
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-@XmlRootElement(name="destination-status")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class DestinationStatusDTO extends LongIdDTO {
+public class PrincipalDTO {
 
-    /**
-     * The destination name
-     */
-    @XmlAttribute
+    @XmlAttribute(required = true)
     public String name;
 
-    @XmlElement
-    public DestinationDTO config;
+    @XmlAttribute
+    public String kind;
+
+
+    public PrincipalDTO() {
+    }
+
+    public PrincipalDTO(String name) {
+        this.name = name;
+    }
+
+    public PrincipalDTO(String name, String kind) {
+        this.name = name;
+        this.kind = kind;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        PrincipalDTO that = (PrincipalDTO) o;
+
+        if (kind != null ? !kind.equals(that.kind) : that.kind != null) return false;
+        if (name != null ? !name.equals(that.name) : that.name != null) return false;
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = name != null ? name.hashCode() : 0;
+        result = 31 * result + (kind != null ? kind.hashCode() : 0);
+        return result;
+    }
+
 
-    /**
-     * Ids of all connections that are producing to the destination
-     */
-    @XmlElement(name="producer")
-    @JsonProperty("producers")
-    public List<LinkDTO> producers = new ArrayList<LinkDTO>();
-
-    /**
-     * Ids of all connections that are consuming from the destination
-     */
-    @XmlElement(name="consumer")
-    @JsonProperty("consumers")
-    public List<LinkDTO> consumers = new ArrayList<LinkDTO>();
-
-    /**
-     * Ids of all queues that are associated with the destination
-     */
-    @XmlElement(name="queue")
-    @JsonProperty("queues")
-    public List<LongIdLabeledDTO> queues = new ArrayList<LongIdLabeledDTO>();
-}
\ No newline at end of file
+}

Copied: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueAclDTO.java (from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java)
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueAclDTO.java?p2=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueAclDTO.java&p1=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java&r1=1043831&r2=1044209&rev=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueAclDTO.java Fri Dec 10 05:30:31 2010
@@ -16,12 +16,11 @@
  */
 package org.apache.activemq.apollo.dto;
 
-import org.codehaus.jackson.annotate.JsonProperty;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
+import java.util.*;
 
 /**
  * <p>
@@ -29,22 +28,11 @@ import javax.xml.bind.annotation.XmlRoot
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-@XmlRootElement(name="broker-summary")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class BrokerSummaryDTO extends StringIdDTO {
+public class QueueAclDTO extends DestinationAclDTO {
 
-    /**
-     * Is a running broker accessible via management API calls?
-     */
-    @JsonProperty
-	@XmlAttribute
-	public boolean manageable;
+    @XmlElement(name="consume")
+    public Set<PrincipalDTO> consumes = new HashSet<PrincipalDTO>();
 
-    /**
-     * Is the broker configuration accessible via API calls? 
-     */
-    @JsonProperty
-	@XmlAttribute
-	public boolean configurable;
 
 }

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDTO.java?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/QueueDTO.java Fri Dec 10 05:30:31 2010
@@ -96,4 +96,7 @@ public class QueueDTO {
     @XmlAttribute(name="flush-range-size")
     public Integer flush_range_size;
 
+    @XmlElement(name="acl")
+    public QueueAclDTO acl;
+
 }

Copied: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostAclDTO.java (from r1043831, activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java)
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostAclDTO.java?p2=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostAclDTO.java&p1=activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java&r1=1043831&r2=1044209&rev=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/BrokerSummaryDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostAclDTO.java Fri Dec 10 05:30:31 2010
@@ -16,12 +16,11 @@
  */
 package org.apache.activemq.apollo.dto;
 
-import org.codehaus.jackson.annotate.JsonProperty;
-
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
+import java.util.*;
 
 /**
  * <p>
@@ -29,22 +28,10 @@ import javax.xml.bind.annotation.XmlRoot
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-@XmlRootElement(name="broker-summary")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class BrokerSummaryDTO extends StringIdDTO {
-
-    /**
-     * Is a running broker accessible via management API calls?
-     */
-    @JsonProperty
-	@XmlAttribute
-	public boolean manageable;
+public class VirtualHostAclDTO {
 
-    /**
-     * Is the broker configuration accessible via API calls? 
-     */
-    @JsonProperty
-	@XmlAttribute
-	public boolean configurable;
+    @XmlElement(name="connect")
+    public Set<PrincipalDTO> connects = new HashSet<PrincipalDTO>();
 
 }

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostDTO.java?rev=1044209&r1=1044208&r2=1044209&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/VirtualHostDTO.java Fri Dec 10 05:30:31 2010
@@ -66,4 +66,10 @@ public class VirtualHostDTO extends Serv
     @XmlAttribute(name="regroup-connections")
     public Boolean regroup_connections;
 
+    @XmlElement(name="acl")
+    public VirtualHostAclDTO acl;
+
+    @XmlElement(name="authentication")
+    public AuthenticationDTO authentication;
+
 }