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/11/12 02:27:03 UTC

svn commit: r1034233 - in /activemq/activemq-apollo/trunk: apollo-bdb/src/main/scala/org/apache/activemq/apollo/store/bdb/ apollo-bdb/src/test/scala/org/apache/activemq/apollo/store/bdb/dto/ apollo-broker/src/main/scala/org/apache/activemq/apollo/broke...

Author: chirino
Date: Fri Nov 12 01:27:02 2010
New Revision: 1034233

URL: http://svn.apache.org/viewvc?rev=1034233&view=rev
Log:
All configuration DTO fields are now reference fields and not defaulted to values.  This allows the XML to be written back out /w the same data it was load from.  The previous approach to defaulting data values was causing the written XML balloon with more data.

Added:
    activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/OptionSupport.scala
Modified:
    activemq/activemq-apollo/trunk/apollo-bdb/src/main/scala/org/apache/activemq/apollo/store/bdb/BDBStore.scala
    activemq/activemq-apollo/trunk/apollo-bdb/src/test/scala/org/apache/activemq/apollo/store/bdb/dto/XmlCodecTest.java
    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/Connector.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala
    activemq/activemq-apollo/trunk/apollo-cassandra/src/main/scala/org/apache/activemq/apollo/store/cassandra/CassandraStore.scala
    activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Run.scala
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/CassandraStoreDTO.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/HawtDBStoreDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ServiceDTO.java
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/StoreDTO.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-hawtdb/src/main/scala/org/apache/activemq/apollo/store/hawtdb/HawtDBClient.scala
    activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/scala/org/apache/activemq/apollo/store/hawtdb/HawtDBStore.scala
    activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/ServletContextListener.scala

Modified: activemq/activemq-apollo/trunk/apollo-bdb/src/main/scala/org/apache/activemq/apollo/store/bdb/BDBStore.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-bdb/src/main/scala/org/apache/activemq/apollo/store/bdb/BDBStore.scala?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-bdb/src/main/scala/org/apache/activemq/apollo/store/bdb/BDBStore.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-bdb/src/main/scala/org/apache/activemq/apollo/store/bdb/BDBStore.scala Fri Nov 12 01:27:02 2010
@@ -27,6 +27,7 @@ import org.apache.activemq.apollo.util._
 import ReporterLevel._
 import org.fusesource.hawtdispatch.ListEventAggregator
 import org.apache.activemq.apollo.dto.{StoreStatusDTO, IntMetricDTO, TimeMetricDTO, StoreDTO}
+import org.apache.activemq.apollo.util.OptionSupport._
 
 /**
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
@@ -73,7 +74,7 @@ class BDBStore extends DelayingStoreSupp
 
   override def toString = "bdb store"
 
-  def flush_delay = config.flush_delay
+  def flush_delay = config.flush_delay.getOrElse(100)
   
   protected def get_next_msg_key = next_msg_key.getAndIncrement
 

Modified: activemq/activemq-apollo/trunk/apollo-bdb/src/test/scala/org/apache/activemq/apollo/store/bdb/dto/XmlCodecTest.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-bdb/src/test/scala/org/apache/activemq/apollo/store/bdb/dto/XmlCodecTest.java?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-bdb/src/test/scala/org/apache/activemq/apollo/store/bdb/dto/XmlCodecTest.java (original)
+++ activemq/activemq-apollo/trunk/apollo-bdb/src/test/scala/org/apache/activemq/apollo/store/bdb/dto/XmlCodecTest.java Fri Nov 12 01:27:02 2010
@@ -43,7 +43,6 @@ public class XmlCodecTest {
         assertNotNull(dto);
         VirtualHostDTO host = dto.virtual_hosts.get(0);
         assertEquals("vh-local", host.id);
-        assertEquals(true, host.enabled);
         assertEquals("localhost", host.host_names.get(0));
 
         assertNotNull( host.store );

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=1034233&r1=1034232&r2=1034233&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 Nov 12 01:27:02 2010
@@ -116,7 +116,6 @@ object Broker extends Log {
   def defaultConfig() = {
     val rc = new BrokerDTO
     rc.id = "default"
-    rc.enabled = true
     rc.notes = "A default configuration"
     rc.virtual_hosts.add(VirtualHost.defaultConfig)
     rc.connectors.add(Connector.defaultConfig)

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connector.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connector.scala?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connector.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connector.scala Fri Nov 12 01:27:02 2010
@@ -42,7 +42,6 @@ object Connector extends Log {
   def defaultConfig() = {
     val rc = new ConnectorDTO
     rc.id = "default"
-    rc.enabled = true
     rc.advertise = "tcp://localhost:61613"
     rc.bind = "tcp://0.0.0.0:61613"
     rc.protocol = "multi"

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=1034233&r1=1034232&r2=1034233&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 Nov 12 01:27:02 2010
@@ -30,6 +30,7 @@ import ReporterLevel._
 import org.fusesource.hawtbuf.{Buffer, AsciiBuffer}
 import collection.JavaConversions
 import java.util.concurrent.atomic.AtomicLong
+import org.apache.activemq.apollo.util.OptionSupport._
 
 /**
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
@@ -48,7 +49,6 @@ object VirtualHost extends Log {
   def defaultConfig() = {
     val rc = new VirtualHostDTO
     rc.id = "default"
-    rc.enabled = true
     rc.host_names.add("localhost")
     rc.store = null
     rc
@@ -147,7 +147,7 @@ class VirtualHost(val broker: Broker, va
           getKeyDone.run
         }
 
-        if( config.purge_on_startup ) {
+        if( config.purge_on_startup.getOrElse(false) ) {
           storeStartupDone.name = "store purge"
           store.purge {
             storeStartupDone.run

Modified: activemq/activemq-apollo/trunk/apollo-cassandra/src/main/scala/org/apache/activemq/apollo/store/cassandra/CassandraStore.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cassandra/src/main/scala/org/apache/activemq/apollo/store/cassandra/CassandraStore.scala?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cassandra/src/main/scala/org/apache/activemq/apollo/store/cassandra/CassandraStore.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-cassandra/src/main/scala/org/apache/activemq/apollo/store/cassandra/CassandraStore.scala Fri Nov 12 01:27:02 2010
@@ -30,6 +30,7 @@ import org.apache.activemq.apollo.dto._
 import org.apache.activemq.apollo.store._
 import org.apache.activemq.apollo.util._
 import ReporterLevel._
+import org.apache.activemq.apollo.util.OptionSupport._
 
 object CassandraStore extends Log {
 
@@ -68,7 +69,7 @@ class CassandraStore extends DelayingSto
   var config:CassandraStoreDTO = defaultConfig
   var blocking:ExecutorService = null
 
-  def flush_delay = config.flush_delay
+  def flush_delay = config.flush_delay.getOrElse(100)
 
   override def toString = "cassandra store"
 
@@ -104,7 +105,6 @@ class CassandraStore extends DelayingSto
     }
   }
 
-
   protected def _start(onCompleted: Runnable) = {
     blocking = Executors.newFixedThreadPool(20, new ThreadFactory(){
       def newThread(r: Runnable) = {
@@ -113,7 +113,7 @@ class CassandraStore extends DelayingSto
         rc
       }
     })
-    client.schema = Schema(config.keyspace)
+    client.schema = Schema(config.keyspace.getOrElse("ActiveMQ"))
 
     // TODO: move some of this parsing code into validation too.
     val HostPort = """([^:]+)(:(\d+))?""".r

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=1034233&r1=1034232&r2=1034233&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 Nov 12 01:27:02 2010
@@ -32,8 +32,9 @@ import org.apache.commons.logging.LogFac
 import org.apache.activemq.apollo.broker.{BrokerRegistry, Broker, ConfigStore, FileConfigStore}
 import org.fusesource.hawtdispatch._
 import Helper._
-import org.apache.activemq.apollo.util.{FileSupport, Logging, ServiceControl}
-import FileSupport._
+import org.apache.activemq.apollo.util.{Logging, ServiceControl}
+import org.apache.activemq.apollo.util.FileSupport._
+import org.apache.activemq.apollo.util.OptionSupport._
 
 /**
  * The apollo create command
@@ -95,7 +96,7 @@ class Run extends Action with Logging {
           store.listBrokers.foreach { id=>
             store.getBroker(id, true).foreach{ config=>
               // Only start the broker up if it's enabled..
-              if( config.enabled ) {
+              if( config.enabled.getOrElse(true) ) {
                 info("Starting broker '%s'...".format(config.id));
                 val broker = new Broker()
                 broker.config = config

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/CassandraStoreDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/CassandraStoreDTO.java?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/CassandraStoreDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/CassandraStoreDTO.java Fri Nov 12 01:27:02 2010
@@ -30,7 +30,7 @@ public class CassandraStoreDTO extends S
 
     @JsonProperty
     @XmlAttribute
-    public String keyspace="ActiveMQ";
+    public String keyspace;
 
     @JsonProperty
     @XmlElement(required=true)

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=1034233&r1=1034232&r2=1034233&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 Nov 12 01:27:02 2010
@@ -44,7 +44,7 @@ public class DestinationDTO {
      * a queue subscriptions is created, it will act like if a durable
      * subscription was created on the topic. 
      */
-    public boolean unified = false;
+    public Boolean unified;
 
     
 

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/HawtDBStoreDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/HawtDBStoreDTO.java?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/HawtDBStoreDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/HawtDBStoreDTO.java Fri Nov 12 01:27:02 2010
@@ -38,25 +38,25 @@ public class HawtDBStoreDTO extends Stor
     public File archive_directory;
 
 	@XmlAttribute(name="index-flush-interval")
-	public long index_flush_interval = 5 * 1000L;
+	public Long index_flush_interval;
 
 	@XmlAttribute(name="cleanup-interval")
-	public long cleanup_interval = 30 * 1000L;
+	public Long cleanup_interval;
 
 	@XmlAttribute(name="journal-log-size")
-	public int journal_log_size = 1024*1024*64;
+	public Integer journal_log_size;
 
     @XmlAttribute(name="journal-batch-size")
-    public int journal_batch_size = 1024*256;
+    public Integer journal_batch_size;
 
     @XmlAttribute(name="index-cache-size")
-    public int index_cache_size = 5000;
+    public Integer index_cache_size;
 
     @XmlAttribute(name="index-page-size")
-    public short index_page_size = 512;
+    public Short index_page_size;
 
     @XmlAttribute(name="fail-if-locked")
-    public boolean fail_if_locked = false;
+    public Boolean fail_if_locked;
 
 
 }

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ServiceDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ServiceDTO.java?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ServiceDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ServiceDTO.java Fri Nov 12 01:27:02 2010
@@ -34,7 +34,7 @@ public class ServiceDTO<ID> extends Stri
      * Should this service be running?
      */
     @XmlAttribute
-    public boolean enabled = true;
+    public Boolean enabled;
 
 
 }
\ No newline at end of file

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/StoreDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/StoreDTO.java?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/StoreDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/StoreDTO.java Fri Nov 12 01:27:02 2010
@@ -39,7 +39,7 @@ public abstract class StoreDTO {
      * would negate the operation.
      */
     @XmlAttribute(name="flush-delay", required=false)
-    public long flush_delay = 100;
+    public Long flush_delay;
 
 
 }

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=1034233&r1=1034232&r2=1034233&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 Nov 12 01:27:02 2010
@@ -41,13 +41,13 @@ public class VirtualHostDTO extends Serv
      */
     @JsonProperty("auto_create_queues")
     @XmlAttribute(name="auto-create-queues")
-    public boolean auto_create_queues = true;
+    public Boolean auto_create_queues;
 
     /**
      * Should queues be purged on startup?
      */
     @XmlAttribute(name="purge-on-startup")
-    public boolean purge_on_startup = false;
+    public Boolean purge_on_startup;
 
     /**
      * Holds the configuration for the destinations.

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/test/java/org/apache/activemq/apollo/dto/XmlCodecTest.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/test/java/org/apache/activemq/apollo/dto/XmlCodecTest.java?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/test/java/org/apache/activemq/apollo/dto/XmlCodecTest.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/test/java/org/apache/activemq/apollo/dto/XmlCodecTest.java Fri Nov 12 01:27:02 2010
@@ -38,9 +38,7 @@ public class XmlCodecTest {
         BrokerDTO dto = XmlCodec.unmarshalBrokerDTO(resource("simple.xml"));
         assertNotNull(dto);
         assertEquals("default", dto.id);
-        assertEquals(true, dto.enabled);
         assertEquals("vh-local", dto.virtual_hosts.get(0).id);
-        assertEquals(true, dto.virtual_hosts.get(0).enabled);
         assertEquals("localhost", dto.virtual_hosts.get(0).host_names.get(0));
         assertEquals("example.com", dto.virtual_hosts.get(0).host_names.get(1));
     }
@@ -50,18 +48,15 @@ public class XmlCodecTest {
     public void marshalling() throws Exception {
         BrokerDTO broker = new BrokerDTO();
         broker.id = "default";
-        broker.enabled = true;
 
         VirtualHostDTO host = new VirtualHostDTO();
         host.id = "vh-local";
-        host.enabled = true;
         host.host_names.add("localhost");
         host.host_names.add("example.com");
         broker.virtual_hosts.add(host);
 
         ConnectorDTO connector = new ConnectorDTO();
         connector.id = "port-61616";
-        connector.enabled = true;
         connector.bind = "tcp://0.0.0.0:61616";
         broker.connectors.add(connector);
         broker.basedir = "./activemq-data/default";

Modified: activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/scala/org/apache/activemq/apollo/store/hawtdb/HawtDBClient.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/scala/org/apache/activemq/apollo/store/hawtdb/HawtDBClient.scala?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/scala/org/apache/activemq/apollo/store/hawtdb/HawtDBClient.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/scala/org/apache/activemq/apollo/store/hawtdb/HawtDBClient.scala Fri Nov 12 01:27:02 2010
@@ -37,6 +37,8 @@ import java.util.concurrent.TimeUnit
 import org.fusesource.hawtdb.api._
 import org.apache.activemq.apollo.store._
 import org.apache.activemq.apollo.util._
+import OptionSupport._
+
 
 object HawtDBClient extends Log {
   val BEGIN = -1
@@ -90,17 +92,17 @@ class HawtDBClient(hawtDBStore: HawtDBSt
   //
   /////////////////////////////////////////////////////////////////////
 
-  private def directory = config.directory
-
-  private def journalMaxFileLength = config.journal_log_size
-
-  private def checkpointInterval = config.index_flush_interval
-
-  private def cleanupInterval = config.cleanup_interval
 
-  private def failIfDatabaseIsLocked = config.fail_if_locked
+  def directory = config.directory
+  def journal_log_size = config.journal_log_size.getOrElse(1024*1024*64)
+  def journal_batch_size = config.journal_batch_size.getOrElse(1024*256)
+  def index_flush_interval = config.index_flush_interval.getOrElse(5L * 1000L)
+  def cleanup_interval = config.cleanup_interval.getOrElse(30 * 1000L)
+  def fail_if_locked = config.fail_if_locked.getOrElse(false)
+  def index_page_size = config.index_page_size.getOrElse(512.toShort)
+  def index_cache_size = config.index_cache_size.getOrElse(5000)
 
-  private def indexFile = indexFileFactory.getTxPageFile()
+  private def index_file = indexFileFactory.getTxPageFile()
 
 
   /////////////////////////////////////////////////////////////////////
@@ -114,7 +116,7 @@ class HawtDBClient(hawtDBStore: HawtDBSt
   def lock(func: => Unit) {
     val lockFileName = new File(directory, "lock")
     lockFile = new LockFile(lockFileName, true)
-    if (failIfDatabaseIsLocked) {
+    if (fail_if_locked) {
       lockFile.lock()
       func
     } else {
@@ -144,8 +146,8 @@ class HawtDBClient(hawtDBStore: HawtDBSt
 
       journal = new Journal()
       journal.setDirectory(directory)
-      journal.setMaxFileLength(config.journal_log_size)
-      journal.setMaxWriteBatchSize(config.journal_batch_size);
+      journal.setMaxFileLength(journal_log_size)
+      journal.setMaxWriteBatchSize(journal_batch_size);
       journal.setChecksum(true);
       journal.setListener( new JournalListener{
         def synced(writes: Array[JournalListener.Write]) = {
@@ -172,8 +174,8 @@ class HawtDBClient(hawtDBStore: HawtDBSt
       indexFileFactory.setDrainOnClose(false)
       indexFileFactory.setSync(true)
       indexFileFactory.setUseWorkerThread(true)
-      indexFileFactory.setPageSize(config.index_page_size)
-      indexFileFactory.setCacheSize(config.index_cache_size);
+      indexFileFactory.setPageSize(index_page_size)
+      indexFileFactory.setCacheSize(index_cache_size);
 
       indexFileFactory.open
 
@@ -202,7 +204,7 @@ class HawtDBClient(hawtDBStore: HawtDBSt
       }
 
       if( initialized ) {
-        indexFile.flush()
+        index_file.flush()
       }
 
       recover(onComplete)
@@ -909,7 +911,7 @@ class HawtDBClient(hawtDBStore: HawtDBSt
 
   def flush() = {
     val start = System.currentTimeMillis()
-    indexFile.flush
+    index_file.flush
     val end = System.currentTimeMillis()
     if (end - start > 1000) {
       warn("Index flush latency: %,.3f seconds", ((end - start) / 1000.0f))
@@ -1049,7 +1051,7 @@ class HawtDBClient(hawtDBStore: HawtDBSt
   }
 
   private def withTx[T](func: (Transaction) => T): T = {
-    val tx = indexFile.tx
+    val tx = index_file.tx
     var ok = false
     try {
       val rc = func(tx)

Modified: activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/scala/org/apache/activemq/apollo/store/hawtdb/HawtDBStore.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/scala/org/apache/activemq/apollo/store/hawtdb/HawtDBStore.scala?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/scala/org/apache/activemq/apollo/store/hawtdb/HawtDBStore.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-hawtdb/src/main/scala/org/apache/activemq/apollo/store/hawtdb/HawtDBStore.scala Fri Nov 12 01:27:02 2010
@@ -28,6 +28,7 @@ import org.apache.activemq.apollo.store.
 import org.apache.activemq.apollo.util._
 import ReporterLevel._
 import org.fusesource.hawtdispatch.{DispatchQueue, BaseRetained, ListEventAggregator}
+import org.apache.activemq.apollo.util.OptionSupport._
 
 object HawtDBStore extends Log {
   val DATABASE_LOCKED_WAIT_DELAY = 10 * 1000;
@@ -74,7 +75,7 @@ class HawtDBStore extends DelayingStoreS
 
   override def toString = "hawtdb store"
 
-  def flush_delay = config.flush_delay
+  def flush_delay = config.flush_delay.getOrElse(100)
   
   protected def get_next_msg_key = next_msg_key.getAndIncrement
 
@@ -133,7 +134,7 @@ class HawtDBStore extends DelayingStoreS
         }
       }
     }
-    dispatchQueue.dispatchAfter(config.index_flush_interval, TimeUnit.MILLISECONDS, ^ {try_flush})
+    dispatchQueue.dispatchAfter(client.index_flush_interval, TimeUnit.MILLISECONDS, ^ {try_flush})
   }
 
   def scheduleCleanup(version:Int): Unit = {
@@ -145,7 +146,7 @@ class HawtDBStore extends DelayingStoreS
         }
       }
     }
-    dispatchQueue.dispatchAfter(config.cleanup_interval, TimeUnit.MILLISECONDS, ^ {try_cleanup})
+    dispatchQueue.dispatchAfter(client.cleanup_interval, TimeUnit.MILLISECONDS, ^ {try_cleanup})
   }
 
   protected def _stop(onCompleted: Runnable) = {

Added: activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/OptionSupport.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/OptionSupport.scala?rev=1034233&view=auto
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/OptionSupport.scala (added)
+++ activemq/activemq-apollo/trunk/apollo-util/src/main/scala/org/apache/activemq/apollo/util/OptionSupport.scala Fri Nov 12 01:27:02 2010
@@ -0,0 +1,68 @@
+/**
+ * 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.util
+import java.{lang=>jl}
+
+/**
+ * <p>
+ * </p>
+ *
+ * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
+ */
+object OptionSupport {
+
+  implicit def o(value:jl.Boolean):Option[Boolean] = value match {
+    case null => None
+    case x => Some(x.booleanValue)
+  }
+
+  implicit def o(value:jl.Character):Option[Char] = value match {
+    case null => None
+    case x => Some(x.charValue)
+  }
+
+  implicit def o(value:jl.Short):Option[Short] = value match {
+    case null => None
+    case x => Some(x.shortValue)
+  }
+
+  implicit def o(value:jl.Integer):Option[Int] = value match {
+    case null => None
+    case x => Some(x.intValue)
+  }
+
+  implicit def o(value:jl.Long):Option[Long] = value match {
+    case null => None
+    case x => Some(x.longValue)
+  }
+
+  implicit def o(value:jl.Double):Option[Double] = value match {
+    case null => None
+    case x => Some(x.doubleValue)
+  }
+
+  implicit def o(value:jl.Float):Option[Float] = value match {
+    case null => None
+    case x => Some(x.floatValue)
+  }
+
+  implicit def o[T](value:T):Option[T] = value match {
+    case null => None
+    case x => Some(x)
+  }
+
+}
\ No newline at end of file

Modified: activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/ServletContextListener.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/ServletContextListener.scala?rev=1034233&r1=1034232&r2=1034233&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/ServletContextListener.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/ServletContextListener.scala Fri Nov 12 01:27:02 2010
@@ -21,6 +21,7 @@ import org.apache.activemq.apollo.util._
 import javax.servlet.{ServletContextListener, ServletContextEvent}
 import org.apache.activemq.apollo.broker.{FileConfigStore, ConfigStore, BrokerRegistry, Broker}
 import org.fusesource.hawtdispatch._
+import org.apache.activemq.apollo.util.OptionSupport._
 
 /**
  * A servlet context listener which handles starting the
@@ -43,7 +44,7 @@ class ApolloListener extends ServletCont
 
               println("Config store contained broker: "+config.id);
               // Only start the broker up if it's enabled..
-              if( config.enabled ) {
+              if( config.enabled.getOrElse(true) ) {
 
                 println("starting broker: "+config.id);
                 val broker = new Broker()