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/07/07 06:18:02 UTC

svn commit: r961198 - in /activemq/sandbox/activemq-apollo-actor: activemq-broker/src/main/scala/org/apache/activemq/apollo/broker/ activemq-hawtdb/src/main/proto/ activemq-hawtdb/src/main/resources/META-INF/services/org/apache/activemq/apollo/ activem...

Author: chirino
Date: Wed Jul  7 04:18:01 2010
New Revision: 961198

URL: http://svn.apache.org/viewvc?rev=961198&view=rev
Log:
Changing the idea of a direct record store to a simpler MemoryPool, main thing we want to do is be able to work with lots of non-gced memory after all.

Added:
    activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/resources/META-INF/services/org/apache/activemq/apollo/
    activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/resources/META-INF/services/org/apache/activemq/apollo/memory-pools
    activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBMemoryPoolSPI.scala
    activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/target/generated-sources/proto/org/apache/activemq/broker/store/hawtdb/HawtDBMemoryPool.scala
      - copied, changed from r961197, activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/DirectRecord.java
    activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/resources/org/
    activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/resources/org/apache/
    activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/resources/org/apache/activemq/
    activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/resources/org/apache/activemq/apollo/
    activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/resources/org/apache/activemq/apollo/MemoryPoolFactory.scala
    activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/scala/org/apache/activemq/apollo/MemoryPool.scala
      - copied, changed from r961197, activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/DirectRecord.java
Removed:
    activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/DirectRecord.java
Modified:
    activemq/sandbox/activemq-apollo-actor/activemq-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala
    activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/proto/data.proto
    activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBClient.scala
    activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBStore.scala
    activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/Helpers.scala
    activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/Store.scala

Modified: activemq/sandbox/activemq-apollo-actor/activemq-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala?rev=961198&r1=961197&r2=961198&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala (original)
+++ activemq/sandbox/activemq-apollo-actor/activemq-broker/src/main/scala/org/apache/activemq/apollo/broker/VirtualHost.scala Wed Jul  7 04:18:01 2010
@@ -102,6 +102,7 @@ class VirtualHost(val broker: Broker, va
   }
 
   var store:Store = null
+  var memory_pool:MemoryPool = null
   var transactionManager:TransactionManagerX = new TransactionManagerX
   var protocols = Map[AsciiBuffer, WireFormat]()
   val queue_id_counter = new LongCounter

Modified: activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/proto/data.proto
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/proto/data.proto?rev=961198&r1=961197&r2=961198&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/proto/data.proto (original)
+++ activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/proto/data.proto Wed Jul  7 04:18:01 2010
@@ -34,9 +34,6 @@ enum Type {
   PUT_MAP_ENTRY = 32;
   REMOVE_MAP_ENTRY = 33;
 
-  ADD_DIRECT = 40;
-  REMOVE_DIRECT = 41;
-
   ADD_SUBSCRIPTION = 50;
   REMOVE_SUBSCRIPTION = 51;
 
@@ -132,20 +129,6 @@ message RemoveMapEntry {
 }
 
 ///////////////////////////////////////////////////////////////
-// Direct buffer related operations.
-///////////////////////////////////////////////////////////////
-message AddDirect {
-  required int64 directKey=1;
-  required int32 size=2;
-  required int32 page=3;
-}
-
-message RemoveDirect {
-  required int64 directKey=1;
-}
-
-
-///////////////////////////////////////////////////////////////
 // Records Stored used in the Indexes
 ///////////////////////////////////////////////////////////////
 message DatabaseRootRecord {
@@ -161,7 +144,6 @@ message DatabaseRootRecord {
   optional fixed32 queueIndexPage=53;
   optional fixed32 subscriptionIndexPage=54;
   optional fixed32 mapIndexPage=55;
-  optional fixed32 directIndexPage=56;
 
 }
 

Added: activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/resources/META-INF/services/org/apache/activemq/apollo/memory-pools
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/resources/META-INF/services/org/apache/activemq/apollo/memory-pools?rev=961198&view=auto
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/resources/META-INF/services/org/apache/activemq/apollo/memory-pools (added)
+++ activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/resources/META-INF/services/org/apache/activemq/apollo/memory-pools Wed Jul  7 04:18:01 2010
@@ -0,0 +1,17 @@
+## ---------------------------------------------------------------------------
+## 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.
+## ---------------------------------------------------------------------------
+org.apache.activemq.broker.store.hawtdb.HawtDBStoreSPI
\ No newline at end of file

Modified: activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBClient.scala
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBClient.scala?rev=961198&r1=961197&r2=961198&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBClient.scala (original)
+++ activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBClient.scala Wed Jul  7 04:18:01 2010
@@ -251,60 +251,6 @@ class HawtDBClient(hawtDBStore: HawtDBSt
 
   val last_direct_key = new AtomicLong
 
-  def createDirectRecord(size: Int)(callback: (DirectRecord) => Unit) = {
-
-    val page_count: Int = directFile.pages(size)
-    val page = directFile.allocator.alloc(page_count)
-    val buffer = directFile.slice(SliceType.READ_WRITE, page, page_count)
-    val key = last_direct_key.incrementAndGet
-
-    val pb = new AddDirect.Bean
-    pb.setDirectKey(key)
-    pb.setSize(size)
-    pb.setPage(page)
-    _store(pb, null)
-
-    val record = new DirectRecord
-    record.key = key
-    record.size = size
-    record.buffer = buffer
-    callback(record)
-  }
-
-  def openDirectRecord(key: Long)(callback: (Option[DirectRecord]) => Unit) = {
-    val result = withTx { tx =>
-      val helper = new TxHelper(tx)
-      import helper._
-      val pb:AddDirect.Getter = directIndex.get(key)
-      if( pb!=null ) {
-        val page_count: Int = directFile.pages(pb.getSize)
-        val buffer = directFile.slice(SliceType.READ, pb.getPage, page_count)
-
-        val record = new DirectRecord
-        record.key = key
-        record.size = pb.getSize
-        record.buffer = buffer
-
-        Some(record)
-      } else {
-        None
-      }
-    }
-    callback(result)
-  }
-
-
-  def closeDirectRecord(record: DirectRecord) = {
-    directFile.unslice(record.buffer)
-  }
-
-  def removeDirectRecord(key: Long)(callback: (Boolean) => Unit) = {
-    val update = new RemoveDirect.Bean
-    update.setDirectKey(key)
-    _store(update, null)
-  }
-
-
   def addQueue(record: QueueRecord, callback:Runnable) = {
     val update = new AddQueue.Bean()
     update.setKey(record.key)
@@ -994,18 +940,6 @@ class HawtDBClient(hawtDBStore: HawtDBSt
       case x: PutMapEntry.Getter =>
       case x: RemoveMapEntry.Getter =>
 
-      case x: AddDirect.Getter =>
-
-        directIndex.put(x.key, x.freeze)
-
-      case x: RemoveDirect.Getter =>
-
-        val record:AddDirect.Getter = directIndex.remove(x.getDirectKey)
-        if( record!=null ) {
-          val page_count: Int = directFile.pages(record.getSize)
-          directFile.allocator.free(record.getPage, page_count)
-        }
-
     }
   }
 

Added: activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBMemoryPoolSPI.scala
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBMemoryPoolSPI.scala?rev=961198&view=auto
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBMemoryPoolSPI.scala (added)
+++ activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBMemoryPoolSPI.scala Wed Jul  7 04:18:01 2010
@@ -0,0 +1,53 @@
+/**
+ * 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.broker.store.hawtdb
+
+import org.apache.activemq.apollo.store.StoreFactory
+import org.apache.activemq.apollo.dto.{HawtDBStoreDTO, StoreDTO}
+import org.apache.activemq.apollo.broker.{Reporting, ReporterLevel, Reporter}
+import ReporterLevel._
+
+/**
+ * <p>
+ * Hook to use a HawtDBStore when a HawtDBStoreDTO is
+ * used in a broker configuration.
+ * </p>
+ * <p>
+ * This class is discovered using the following resource file:
+ * <code>META-INF/services/org.apache.activemq.apollo/stores</code>
+ * </p>
+ * 
+ * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
+ */
+class HawtDBMemoryPoolSPI extends StoreFactory.SPI {
+
+  def create(config: StoreDTO) = {
+    if( config.isInstanceOf[HawtDBStoreDTO]) {
+      new HawtDBStore
+    } else {
+      null
+    }
+  }
+
+   def validate(config: StoreDTO, reporter:Reporter):ReporterLevel = {
+     if( config.isInstanceOf[HawtDBStoreDTO]) {
+       HawtDBStore.validate(config.asInstanceOf[HawtDBStoreDTO], reporter)
+     } else {
+       null
+     }
+   }
+}
\ No newline at end of file

Modified: activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBStore.scala
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBStore.scala?rev=961198&r1=961197&r2=961198&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBStore.scala (original)
+++ activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/HawtDBStore.scala Wed Jul  7 04:18:01 2010
@@ -29,7 +29,7 @@ import ReporterLevel._
 import java.util.concurrent._
 import org.apache.activemq.apollo.store._
 import org.apache.activemq.apollo.util.{IntMetricCounter, TimeCounter, IntCounter}
-import org.apache.activemq.broker.store.{DirectRecordStore, StoreUOW, Store}
+import org.apache.activemq.broker.store.{StoreUOW, Store}
 
 object HawtDBStore extends Log {
   val DATABASE_LOCKED_WAIT_DELAY = 10 * 1000;
@@ -58,7 +58,7 @@ object HawtDBStore extends Log {
 /**
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-class HawtDBStore extends Store with DirectRecordStore with BaseService with DispatchLogging {
+class HawtDBStore extends Store with BaseService with DispatchLogging {
 
   import HawtDBStore._
   override protected def log = HawtDBStore
@@ -125,36 +125,6 @@ class HawtDBStore extends Store with Dir
 
   /////////////////////////////////////////////////////////////////////
   //
-  // Implementation of the DirectRecordStore interface
-  //
-  /////////////////////////////////////////////////////////////////////
-
-  def createDirectRecord(size: Int)(callback: (DirectRecord) => Unit) = {
-    executor_pool {
-      client.createDirectRecord(size)(callback)
-    }
-  }
-
-  def openDirectRecord(key: Long)(callback: (Option[DirectRecord]) => Unit) = {
-    executor_pool {
-      client.openDirectRecord(key)(callback)
-    }
-  }
-
-  def closeDirectRecord(record: DirectRecord) = {
-    executor_pool {
-      client.closeDirectRecord(record)
-    }
-  }
-
-  def removeDirectRecord(key: Long)(callback: (Boolean) => Unit) = {
-    executor_pool {
-      client.removeDirectRecord(key)(callback)
-    }
-  }
-
-  /////////////////////////////////////////////////////////////////////
-  //
   // Implementation of the Store interface
   //
   /////////////////////////////////////////////////////////////////////

Modified: activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/Helpers.scala
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/Helpers.scala?rev=961198&r1=961197&r2=961198&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/Helpers.scala (original)
+++ activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/src/main/scala/org/apache/activemq/broker/store/hawtdb/Helpers.scala Wed Jul  7 04:18:01 2010
@@ -24,7 +24,7 @@ import java.io.{IOException, DataInput, 
 import org.fusesource.hawtdb.internal.journal.{LocationCodec, Location}
 import org.fusesource.hawtdb.api._
 import org.fusesource.hawtbuf.proto.{MessageBuffer, PBMessage}
-import org.apache.activemq.apollo.store.{DirectRecord, MessageRecord, QueueRecord, QueueEntryRecord}
+import org.apache.activemq.apollo.store.{MessageRecord, QueueRecord, QueueEntryRecord}
 
 /**
  * <p>
@@ -147,13 +147,6 @@ object Helpers {
     pb
   }
 
-  implicit def toDirectRecord(pb: AddDirect.Getter): DirectRecord = {
-    val rc = new DirectRecord
-    rc.key = pb.getDirectKey
-    rc.size = pb.getSize
-    rc
-  }
-  
   implicit def toLocation(value: Long): Location = {
     val temp = new Buffer(8)
     val editor = temp.bigEndianEditor

Copied: activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/target/generated-sources/proto/org/apache/activemq/broker/store/hawtdb/HawtDBMemoryPool.scala (from r961197, activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/DirectRecord.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/target/generated-sources/proto/org/apache/activemq/broker/store/hawtdb/HawtDBMemoryPool.scala?p2=activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/target/generated-sources/proto/org/apache/activemq/broker/store/hawtdb/HawtDBMemoryPool.scala&p1=activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/DirectRecord.java&r1=961197&r2=961198&rev=961198&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/DirectRecord.java (original)
+++ activemq/sandbox/activemq-apollo-actor/activemq-hawtdb/target/generated-sources/proto/org/apache/activemq/broker/store/hawtdb/HawtDBMemoryPool.scala Wed Jul  7 04:18:01 2010
@@ -14,22 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.activemq.apollo.store;
-
-import org.fusesource.hawtbuf.AsciiBuffer;
-import org.fusesource.hawtbuf.Buffer;
-
-import java.nio.ByteBuffer;
+package org.apache.activemq.broker.store.hawtdb
 
 /**
- * A memory mapped direct buffer associated with a key
+ * <p>
+ * </p>
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-public class DirectRecord {
-
-    public long key = -1;
-    public int size = 0;
-    public ByteBuffer buffer;
-
-}
\ No newline at end of file
+class HawtDBMemoryPool
\ No newline at end of file

Modified: activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/Store.scala
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/Store.scala?rev=961198&r1=961197&r2=961198&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/Store.scala (original)
+++ activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/Store.scala Wed Jul  7 04:18:01 2010
@@ -111,18 +111,4 @@ trait Store extends ServiceTrait {
  * Optional interface that stores can implement to give protocols direct access to the file system
  * for them to be able to do
  */
-trait DirectRecordStore {
 
-  def createDirectRecord(size:Int)(callback:(DirectRecord)=>Unit):Unit
-
-  def openDirectRecord(key:Long)(callback:(Option[DirectRecord])=>Unit):Unit
-
-  def closeDirectRecord(record:DirectRecord):Unit 
-
-  def removeDirectRecord(record:DirectRecord)(callback:(Boolean)=>Unit):Unit = {
-    removeDirectRecord(record.key)( callback )
-  }
-  def removeDirectRecord(key:Long)(callback:(Boolean)=>Unit):Unit
-
-
-}

Added: activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/resources/org/apache/activemq/apollo/MemoryPoolFactory.scala
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/resources/org/apache/activemq/apollo/MemoryPoolFactory.scala?rev=961198&view=auto
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/resources/org/apache/activemq/apollo/MemoryPoolFactory.scala (added)
+++ activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/resources/org/apache/activemq/apollo/MemoryPoolFactory.scala Wed Jul  7 04:18:01 2010
@@ -0,0 +1,39 @@
+package org.apache.activemq.apollo
+
+resSPI ::= SPI
+    } catch {
+      case e:Throwable =>
+        e.printStackTrace
+    }
+  }
+
+  def create(config:StoreDTO):Store = {
+    if( config == null ) {
+      return null
+    }
+    storesSPI.foreach { spi=>
+      val rc = spi.create(config)
+      if( rc!=null ) {
+        return rc
+      }
+    }
+    throw new IllegalArgumentException("Uknonwn store type: "+config.getClass)
+  }
+
+
+  def validate(config: StoreDTO, reporter:Reporter):ReporterLevel = {
+    if( config == null ) {
+      return INFO
+    } else {
+      storesSPI.foreach { spi=>
+        val rc = spi.validate(config, reporter)
+        if( rc!=null ) {
+          return rc
+        }
+      }
+    }
+    reporter.report(ERROR, "Uknonwn store type: "+config.getClass)
+    ERROR
+  }
+
+}
\ No newline at end of file

Copied: activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/scala/org/apache/activemq/apollo/MemoryPool.scala (from r961197, activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/DirectRecord.java)
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/scala/org/apache/activemq/apollo/MemoryPool.scala?p2=activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/scala/org/apache/activemq/apollo/MemoryPool.scala&p1=activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/DirectRecord.java&r1=961197&r2=961198&rev=961198&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/activemq-store/src/main/scala/org/apache/activemq/apollo/store/DirectRecord.java (original)
+++ activemq/sandbox/activemq-apollo-actor/activemq-util/src/main/scala/org/apache/activemq/apollo/MemoryPool.scala Wed Jul  7 04:18:01 2010
@@ -14,22 +14,22 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.activemq.apollo.store;
+package org.apache.activemq.apollo.broker
 
-import org.fusesource.hawtbuf.AsciiBuffer;
-import org.fusesource.hawtbuf.Buffer;
+import java.nio.ByteBuffer
 
-import java.nio.ByteBuffer;
+trait MemoryAllocation {
+  def size:Int
+  def buffer:ByteBuffer
+}
 
 /**
- * A memory mapped direct buffer associated with a key
+ * <p>
+ * </p>
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-public class DirectRecord {
-
-    public long key = -1;
-    public int size = 0;
-    public ByteBuffer buffer;
-
+trait MemoryPool {
+  def alloc(size:Int):MemoryAllocation
+  def free(MemoryAllocation)
 }
\ No newline at end of file