rewrote the persistent storage with support for unit-of-work and new multiverse stm
This commit is contained in:
parent
e4a4451533
commit
dfc08f559d
21 changed files with 1275 additions and 3835 deletions
9
LICENSE
9
LICENSE
|
|
@ -1,15 +1,20 @@
|
||||||
This software is licensed under the Apache 2 license, quoted below.
|
This software is licensed under the Apache 2 license, quoted below.
|
||||||
|
|
||||||
Copyright 2009 Scalable Solutions AB <http://scalablesolutions.se>
|
Copyright 2009 Scalable Solutions AB [http://scalablesolutions.se]
|
||||||
|
|
||||||
Licensed under the Apache License, Version 2.0 (the "License"); you may not
|
Licensed 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
|
use this file except in compliance with the License. You may obtain a copy of
|
||||||
the License at
|
the License at
|
||||||
|
|
||||||
http://www.apache.org/licenses/LICENSE-2.0
|
[http://www.apache.org/licenses/LICENSE-2.0]
|
||||||
|
|
||||||
Unless required by applicable law or agreed to in writing, software
|
Unless required by applicable law or agreed to in writing, software
|
||||||
distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||||
WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||||
License for the specific language governing permissions and limitations under
|
License for the specific language governing permissions and limitations under
|
||||||
the License.
|
the License.
|
||||||
|
|
||||||
|
---------------
|
||||||
|
|
||||||
|
Licenses for dependency projects can be found here:
|
||||||
|
[http://wiki.github.com/jboner/akka/licenses]
|
||||||
|
|
@ -62,6 +62,11 @@
|
||||||
<artifactId>javautils</artifactId>
|
<artifactId>javautils</artifactId>
|
||||||
<version>2.7.4-0.1</version>
|
<version>2.7.4-0.1</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.multiverse</groupId>
|
||||||
|
<artifactId>multiverse</artifactId>
|
||||||
|
<version>0.3</version>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
<!-- For Protocol/Serialization -->
|
<!-- For Protocol/Serialization -->
|
||||||
<dependency>
|
<dependency>
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,7 @@ import java.io.{ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, By
|
||||||
import reflect.{BeanProperty, Manifest}
|
import reflect.{BeanProperty, Manifest}
|
||||||
import sbinary.DefaultProtocol
|
import sbinary.DefaultProtocol
|
||||||
import org.codehaus.jackson.map.ObjectMapper
|
import org.codehaus.jackson.map.ObjectMapper
|
||||||
import sjson.json.{Serializer=>SJSONSerializer}
|
import sjson.json.{Serializer =>SJSONSerializer}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
|
|
|
||||||
|
|
@ -1,29 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2009 Scalable Solutions.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package se.scalablesolutions.akka.stm
|
|
||||||
|
|
||||||
import state.{Transactional, TransactionalMap}
|
|
||||||
import util.Helpers.ReadWriteLock
|
|
||||||
import scala.collection.immutable.HashSet
|
|
||||||
|
|
||||||
@serializable
|
|
||||||
class ChangeSet {
|
|
||||||
private val lock = new ReadWriteLock
|
|
||||||
|
|
||||||
private var transactionalItems: Set[Transactional] = new HashSet
|
|
||||||
private[akka] def +(item: Transactional) = lock.withWriteLock {
|
|
||||||
transactionalItems += item
|
|
||||||
}
|
|
||||||
private[akka] def items: List[Transactional] = lock.withReadLock {
|
|
||||||
transactionalItems.toList.asInstanceOf[List[Transactional]]
|
|
||||||
}
|
|
||||||
|
|
||||||
private[akka] def clear = lock.withWriteLock {
|
|
||||||
transactionalItems = new HashSet
|
|
||||||
}
|
|
||||||
|
|
||||||
// FIXME: add hashCode and equals - VERY IMPORTANT
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
@ -34,6 +34,8 @@
|
||||||
|
|
||||||
package se.scalablesolutions.akka.collection
|
package se.scalablesolutions.akka.collection
|
||||||
|
|
||||||
|
trait PersistentDataStructure
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A clean-room port of Rich Hickey's persistent hash trie implementation from
|
* A clean-room port of Rich Hickey's persistent hash trie implementation from
|
||||||
* Clojure (http://clojure.org). Originally presented as a mutable structure in
|
* Clojure (http://clojure.org). Originally presented as a mutable structure in
|
||||||
|
|
@ -43,7 +45,7 @@ package se.scalablesolutions.akka.collection
|
||||||
* @author Rich Hickey
|
* @author Rich Hickey
|
||||||
*/
|
*/
|
||||||
@serializable
|
@serializable
|
||||||
final class HashTrie[K, +V] private (root: Node[K, V]) extends Map[K, V] {
|
final class HashTrie[K, +V] private (root: Node[K, V]) extends Map[K, V] with PersistentDataStructure {
|
||||||
lazy val size = root.size
|
lazy val size = root.size
|
||||||
|
|
||||||
def this() = this(new EmptyNode[K])
|
def this() = this(new EmptyNode[K])
|
||||||
|
|
|
||||||
|
|
@ -12,6 +12,7 @@ import org.multiverse.api.{Transaction => MultiverseTransaction}
|
||||||
import org.multiverse.stms.alpha.AlphaStm
|
import org.multiverse.stms.alpha.AlphaStm
|
||||||
import org.multiverse.utils.GlobalStmInstance
|
import org.multiverse.utils.GlobalStmInstance
|
||||||
import org.multiverse.utils.TransactionThreadLocal._
|
import org.multiverse.utils.TransactionThreadLocal._
|
||||||
|
import org.multiverse.templates.{OrElseTemplate, AtomicTemplate}
|
||||||
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
|
|
||||||
|
|
@ -23,7 +24,34 @@ object Multiverse {
|
||||||
GlobalStmInstance.set(STM)
|
GlobalStmInstance.set(STM)
|
||||||
setThreadLocalTransaction(null)
|
setThreadLocalTransaction(null)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Example of Or-Else transaction management.
|
||||||
|
* <pre>
|
||||||
|
* import se.scalablesolutions.akka.stm.{Transaction => Tx}
|
||||||
|
* Tx.Or {
|
||||||
|
* .. // try to do something
|
||||||
|
* } Tx.Else {
|
||||||
|
* .. // if transaction clashes try do do something else to minimize contention
|
||||||
|
* }
|
||||||
|
* </pre>
|
||||||
|
*
|
||||||
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
|
*/
|
||||||
|
object Transaction {
|
||||||
|
|
||||||
|
// -- Monad --------------------------
|
||||||
|
|
||||||
|
|
||||||
|
// -- OrElse --------------------------
|
||||||
|
def Or[A](orBody: => A) = elseBody(orBody)
|
||||||
|
def elseBody[A](orBody: => A) = new {
|
||||||
|
def Else(elseBody: => A) = new OrElseTemplate[A] {
|
||||||
|
def run(t: MultiverseTransaction) = orBody
|
||||||
|
def orelserun(t: MultiverseTransaction) = elseBody
|
||||||
|
}.execute
|
||||||
|
}
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
|
|
@ -31,9 +59,7 @@ object Multiverse {
|
||||||
private[this] var _id = 0L
|
private[this] var _id = 0L
|
||||||
def id = _id
|
def id = _id
|
||||||
@volatile private[this] var status: TransactionStatus = TransactionStatus.New
|
@volatile private[this] var status: TransactionStatus = TransactionStatus.New
|
||||||
private[kernel] var transaction: MultiverseTransaction = _
|
private[akka] var transaction: MultiverseTransaction = _
|
||||||
|
|
||||||
private[this] val transactionalItems = new ChangeSet
|
|
||||||
|
|
||||||
private[this] var participants: List[String] = Nil
|
private[this] var participants: List[String] = Nil
|
||||||
private[this] var precommitted: List[String] = Nil
|
private[this] var precommitted: List[String] = Nil
|
||||||
|
|
@ -46,12 +72,11 @@ object Multiverse {
|
||||||
|
|
||||||
def register(transactional: Transactional) = synchronized {
|
def register(transactional: Transactional) = synchronized {
|
||||||
ensureIsActiveOrNew
|
ensureIsActiveOrNew
|
||||||
transactionalItems + transactional
|
|
||||||
}
|
}
|
||||||
|
|
||||||
def begin(participant: String) = synchronized {
|
def begin(participant: String) = synchronized {
|
||||||
ensureIsActiveOrNew
|
ensureIsActiveOrNew
|
||||||
transaction = Multiverse.STM.startUpdateTransaction
|
transaction = Multiverse.STM.startUpdateTransaction("akka")
|
||||||
_id = transaction.getReadVersion
|
_id = transaction.getReadVersion
|
||||||
log.debug("Creating a new transaction with id [%s]", _id)
|
log.debug("Creating a new transaction with id [%s]", _id)
|
||||||
|
|
||||||
|
|
@ -80,7 +105,6 @@ object Multiverse {
|
||||||
} else false
|
} else false
|
||||||
if (haveAllPreCommitted && transaction != null) {
|
if (haveAllPreCommitted && transaction != null) {
|
||||||
transaction.commit
|
transaction.commit
|
||||||
transactionalItems.items.foreach(_.commit)
|
|
||||||
status = TransactionStatus.Completed
|
status = TransactionStatus.Completed
|
||||||
reset
|
reset
|
||||||
true
|
true
|
||||||
|
|
@ -118,7 +142,6 @@ object Multiverse {
|
||||||
def isAborted = status == TransactionStatus.Aborted
|
def isAborted = status == TransactionStatus.Aborted
|
||||||
|
|
||||||
private def reset = {
|
private def reset = {
|
||||||
transactionalItems.clear
|
|
||||||
participants = Nil
|
participants = Nil
|
||||||
precommitted = Nil
|
precommitted = Nil
|
||||||
}
|
}
|
||||||
|
|
@ -133,7 +156,7 @@ object Multiverse {
|
||||||
throw new IllegalStateException("Expected ACTIVE or NEW transaction - current status [" + status + "]: " + toString)
|
throw new IllegalStateException("Expected ACTIVE or NEW transaction - current status [" + status + "]: " + toString)
|
||||||
|
|
||||||
// For reinitialize transaction after sending it over the wire
|
// For reinitialize transaction after sending it over the wire
|
||||||
private[kernel] def reinit = synchronized {
|
private[akka] def reinit = synchronized {
|
||||||
import net.lag.logging.{Logger, Level}
|
import net.lag.logging.{Logger, Level}
|
||||||
if (log == null) {
|
if (log == null) {
|
||||||
log = Logger.get(this.getClass.getName)
|
log = Logger.get(this.getClass.getName)
|
||||||
|
|
@ -155,31 +178,6 @@ object Multiverse {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
@serializable sealed abstract class TransactionStatus
|
@serializable sealed abstract class TransactionStatus
|
||||||
object TransactionStatus {
|
object TransactionStatus {
|
||||||
case object New extends TransactionStatus
|
case object New extends TransactionStatus
|
||||||
|
|
|
||||||
|
|
@ -50,7 +50,6 @@ trait TransactionManagement extends Logging {
|
||||||
activeTx = tx
|
activeTx = tx
|
||||||
threadBoundTx.set(tx)
|
threadBoundTx.set(tx)
|
||||||
setThreadLocalTransaction(tx.get.transaction)
|
setThreadLocalTransaction(tx.get.transaction)
|
||||||
println("------ Start: " + tx.get.transaction)
|
|
||||||
tx
|
tx
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -4,6 +4,7 @@
|
||||||
|
|
||||||
package se.scalablesolutions.akka.state
|
package se.scalablesolutions.akka.state
|
||||||
|
|
||||||
|
import org.multiverse.datastructures.refs.manual.Ref
|
||||||
import stm.TransactionManagement
|
import stm.TransactionManagement
|
||||||
import akka.collection._
|
import akka.collection._
|
||||||
|
|
||||||
|
|
@ -17,6 +18,14 @@ import scala.collection.mutable.{ArrayBuffer, HashMap}
|
||||||
* Example Scala usage:
|
* Example Scala usage:
|
||||||
* <pre>
|
* <pre>
|
||||||
* val myMap = TransactionalState.newMap
|
* val myMap = TransactionalState.newMap
|
||||||
|
* val myVector = TransactionalState.newVector
|
||||||
|
* val myRef = TransactionalState.newRef
|
||||||
|
* </pre>
|
||||||
|
* Or:
|
||||||
|
* <pre>
|
||||||
|
* val myMap = TransactionalMap()
|
||||||
|
* val myVector = TransactionalVector()
|
||||||
|
* val myRef = TransactionalRef()
|
||||||
* </pre>
|
* </pre>
|
||||||
*/
|
*/
|
||||||
object TransactionalState extends TransactionalState
|
object TransactionalState extends TransactionalState
|
||||||
|
|
@ -31,9 +40,9 @@ object TransactionalState extends TransactionalState
|
||||||
* </pre>
|
* </pre>
|
||||||
*/
|
*/
|
||||||
class TransactionalState {
|
class TransactionalState {
|
||||||
def newMap[K, V]: TransactionalMap[K, V] = new InMemoryTransactionalMap[K, V]
|
def newMap[K, V] = new TransactionalMap[K, V]
|
||||||
def newVector[T]: TransactionalVector[T] = new InMemoryTransactionalVector[T]
|
def newVector[T] = new TransactionalVector[T]
|
||||||
def newRef[T]: TransactionalRef[T] = new TransactionalRef[T]
|
def newRef[T] = new TransactionalRef[T]
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -43,203 +52,175 @@ class TransactionalState {
|
||||||
trait Transactional {
|
trait Transactional {
|
||||||
// FIXME: won't work across the cluster
|
// FIXME: won't work across the cluster
|
||||||
val uuid = Uuid.newUuid.toString
|
val uuid = Uuid.newUuid.toString
|
||||||
|
|
||||||
private[akka] def begin
|
|
||||||
private[akka] def commit
|
|
||||||
private[akka] def rollback
|
|
||||||
|
|
||||||
protected def verifyTransaction = {
|
|
||||||
val cflowTx = TransactionManagement.threadBoundTx.get
|
|
||||||
if (!cflowTx.isDefined) {
|
|
||||||
throw new IllegalStateException("Can't access transactional reference outside the scope of a transaction [" + this + "]")
|
|
||||||
} else {
|
|
||||||
cflowTx.get.register(this)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base trait for all state implementations (persistent or in-memory).
|
* Implements a transactional managed reference.
|
||||||
*
|
|
||||||
* FIXME: Create Java versions using pcollections
|
|
||||||
*
|
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
||||||
*/
|
|
||||||
trait TransactionalMap[K, V] extends Transactional with scala.collection.mutable.Map[K, V] {
|
|
||||||
override def hashCode: Int = System.identityHashCode(this);
|
|
||||||
override def equals(other: Any): Boolean = false
|
|
||||||
def remove(key: K)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Not thread-safe, but should only be using from within an Actor, e.g. one single thread at a time.
|
|
||||||
*
|
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
||||||
*/
|
|
||||||
class InMemoryTransactionalMap[K, V] extends TransactionalMap[K, V] {
|
|
||||||
protected[akka] var state = new HashTrie[K, V]
|
|
||||||
protected[akka] var snapshot = state
|
|
||||||
|
|
||||||
// ---- For Transactional ----
|
|
||||||
override def begin = snapshot = state
|
|
||||||
override def commit = snapshot = state
|
|
||||||
override def rollback = state = snapshot
|
|
||||||
|
|
||||||
// ---- Overriding scala.collection.mutable.Map behavior ----
|
|
||||||
override def contains(key: K): Boolean = {
|
|
||||||
verifyTransaction
|
|
||||||
state.contains(key)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def clear = {
|
|
||||||
verifyTransaction
|
|
||||||
state = new HashTrie[K, V]
|
|
||||||
}
|
|
||||||
|
|
||||||
override def size: Int = {
|
|
||||||
verifyTransaction
|
|
||||||
state.size
|
|
||||||
}
|
|
||||||
|
|
||||||
// ---- For scala.collection.mutable.Map ----
|
|
||||||
override def remove(key: K) = {
|
|
||||||
verifyTransaction
|
|
||||||
state = state - key
|
|
||||||
}
|
|
||||||
|
|
||||||
override def elements: Iterator[(K, V)] = {
|
|
||||||
// verifyTransaction
|
|
||||||
state.elements
|
|
||||||
}
|
|
||||||
|
|
||||||
override def get(key: K): Option[V] = {
|
|
||||||
verifyTransaction
|
|
||||||
state.get(key)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def put(key: K, value: V): Option[V] = {
|
|
||||||
verifyTransaction
|
|
||||||
val oldValue = state.get(key)
|
|
||||||
state = state.update(key, value)
|
|
||||||
oldValue
|
|
||||||
}
|
|
||||||
|
|
||||||
override def -=(key: K) = {
|
|
||||||
verifyTransaction
|
|
||||||
remove(key)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def update(key: K, value: V) = {
|
|
||||||
verifyTransaction
|
|
||||||
put(key, value)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Base for all transactional vector implementations.
|
|
||||||
*
|
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
||||||
*/
|
|
||||||
abstract class TransactionalVector[T] extends Transactional with RandomAccessSeq[T] {
|
|
||||||
override def hashCode: Int = System.identityHashCode(this);
|
|
||||||
override def equals(other: Any): Boolean = false
|
|
||||||
|
|
||||||
def add(elem: T)
|
|
||||||
|
|
||||||
def get(index: Int): T
|
|
||||||
|
|
||||||
def getRange(start: Int, count: Int): List[T]
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Implements an in-memory transactional vector.
|
|
||||||
*
|
|
||||||
* Not thread-safe, but should only be using from within an Actor, e.g. one single thread at a time.
|
|
||||||
*
|
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
||||||
*/
|
|
||||||
class InMemoryTransactionalVector[T] extends TransactionalVector[T] {
|
|
||||||
private[akka] var state: Vector[T] = EmptyVector
|
|
||||||
private[akka] var snapshot = state
|
|
||||||
|
|
||||||
def add(elem: T) = {
|
|
||||||
verifyTransaction
|
|
||||||
state = state + elem
|
|
||||||
}
|
|
||||||
|
|
||||||
def get(index: Int): T = {
|
|
||||||
verifyTransaction
|
|
||||||
state(index)
|
|
||||||
}
|
|
||||||
|
|
||||||
def getRange(start: Int, count: Int): List[T] = {
|
|
||||||
verifyTransaction
|
|
||||||
state.slice(start, count).toList.asInstanceOf[List[T]]
|
|
||||||
}
|
|
||||||
|
|
||||||
// ---- For Transactional ----
|
|
||||||
override def begin = snapshot = state
|
|
||||||
|
|
||||||
override def commit = snapshot = state
|
|
||||||
|
|
||||||
override def rollback = state = snapshot
|
|
||||||
|
|
||||||
// ---- For Seq ----
|
|
||||||
def length: Int = {
|
|
||||||
verifyTransaction
|
|
||||||
state.length
|
|
||||||
}
|
|
||||||
|
|
||||||
def apply(index: Int): T = {
|
|
||||||
verifyTransaction
|
|
||||||
state(index)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def elements: Iterator[T] = {
|
|
||||||
//verifyTransaction
|
|
||||||
state.elements
|
|
||||||
}
|
|
||||||
|
|
||||||
override def toList: List[T] = {
|
|
||||||
verifyTransaction
|
|
||||||
state.toList
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Implements a transactional reference.
|
|
||||||
*
|
|
||||||
* Not thread-safe, but should only be using from within an Actor, e.g. one single thread at a time.
|
|
||||||
*
|
*
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
class TransactionalRef[T] extends Transactional {
|
class TransactionalRef[T] extends Transactional {
|
||||||
private[akka] var ref: Option[T] = None
|
protected[this] var ref: Option[Ref[T]] = None
|
||||||
private[akka] var snapshot: Option[T] = None
|
|
||||||
|
def set(elem: T) = swap(elem)
|
||||||
override def begin = if (ref.isDefined) snapshot = Some(ref.get)
|
|
||||||
|
|
||||||
override def commit = if (ref.isDefined) snapshot = Some(ref.get)
|
|
||||||
|
|
||||||
override def rollback = if (snapshot.isDefined) ref = Some(snapshot.get)
|
|
||||||
|
|
||||||
def swap(elem: T) = {
|
def swap(elem: T) = {
|
||||||
verifyTransaction
|
synchronized { if (ref.isEmpty) ref = Some(new Ref[T]) }
|
||||||
ref = Some(elem)
|
ref.get.set(elem)
|
||||||
|
}
|
||||||
|
|
||||||
|
def get: Option[T] =
|
||||||
|
if (isEmpty) None
|
||||||
|
else Some(ref.get.get)
|
||||||
|
|
||||||
|
def getOrWait: T = {
|
||||||
|
synchronized { if (ref.isEmpty) ref = Some(new Ref[T]) }
|
||||||
|
ref.get.getOrAwait
|
||||||
|
}
|
||||||
|
|
||||||
|
def getOrElse(default: => T): T =
|
||||||
|
if (isEmpty) default
|
||||||
|
else ref.get.get
|
||||||
|
|
||||||
|
def isDefined: Boolean = ref.isDefined && !ref.get.isNull
|
||||||
|
|
||||||
|
def isEmpty: Boolean = !isDefined
|
||||||
|
}
|
||||||
|
|
||||||
|
object TransactionalRef {
|
||||||
|
def apply[T](elem: T) = {
|
||||||
|
if (elem == null) throw new IllegalArgumentException("Can't define TransactionalRef with a null initial value, needs to be a PersistentDataStructure")
|
||||||
|
val ref = new TransactionalRef[T]
|
||||||
|
ref.swap(elem)
|
||||||
|
ref
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Implements an in-memory transactional Map based on Clojure's PersistentMap.
|
||||||
|
*
|
||||||
|
* Not thread-safe, but should only be using from within an Actor, e.g. one single thread at a time.
|
||||||
|
*
|
||||||
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
|
*/
|
||||||
|
class TransactionalMap[K, V] extends Transactional with scala.collection.mutable.Map[K, V] {
|
||||||
|
protected[this] val ref = TransactionalRef[HashTrie[K, V]](new HashTrie[K, V])
|
||||||
|
|
||||||
|
def -=(key: K) = remove(key)
|
||||||
|
|
||||||
|
def +=(key: K, value: V) = put(key, value)
|
||||||
|
|
||||||
|
def remove(key: K) = ref.swap(ref.get.get - key)
|
||||||
|
|
||||||
|
def apply(key: K): Option[V] = get(key)
|
||||||
|
|
||||||
|
def get(key: K): Option[V] = ref.get.get.get(key)
|
||||||
|
|
||||||
|
override def put(key: K, value: V): Option[V] = {
|
||||||
|
val map = ref.get.get
|
||||||
|
val oldValue = map.get(key)
|
||||||
|
ref.swap(map.update(key, value))
|
||||||
|
oldValue
|
||||||
|
}
|
||||||
|
|
||||||
|
def update(key: K, value: V) = {
|
||||||
|
val map = ref.get.get
|
||||||
|
val oldValue = map.get(key)
|
||||||
|
ref.swap(map.update(key, value))
|
||||||
|
}
|
||||||
|
|
||||||
|
def elements: Iterator[(K, V)] = ref.get.get.elements
|
||||||
|
|
||||||
|
override def contains(key: K): Boolean = ref.get.get.contains(key)
|
||||||
|
|
||||||
|
override def clear = ref.swap(new HashTrie[K, V])
|
||||||
|
|
||||||
|
def size: Int = ref.get.get.size
|
||||||
|
|
||||||
|
override def hashCode: Int = System.identityHashCode(this);
|
||||||
|
|
||||||
|
override def equals(other: Any): Boolean =
|
||||||
|
other.isInstanceOf[TransactionalMap[_, _]] &&
|
||||||
|
other.hashCode == hashCode
|
||||||
|
}
|
||||||
|
|
||||||
|
object TransactionalMap {
|
||||||
|
def apply[T]() = new TransactionalMap
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Implements an in-memory transactional Vector based on Clojure's PersistentVector.
|
||||||
|
*
|
||||||
|
* Not thread-safe, but should only be using from within an Actor, e.g. one single thread at a time.
|
||||||
|
*
|
||||||
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
|
*/
|
||||||
|
class TransactionalVector[T] extends Transactional with RandomAccessSeq[T] {
|
||||||
|
private[this] val ref = TransactionalRef[Vector[T]](EmptyVector)
|
||||||
|
|
||||||
|
def clear = ref.swap(EmptyVector)
|
||||||
|
|
||||||
|
def +(elem: T) = add(elem)
|
||||||
|
|
||||||
|
def add(elem: T) = ref.swap(ref.get.get + elem)
|
||||||
|
|
||||||
|
def get(index: Int): T = ref.get.get.apply(index)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Removes the <i>tail</i> element of this vector.
|
||||||
|
*/
|
||||||
|
def pop = ref.swap(ref.get.get.pop)
|
||||||
|
|
||||||
|
def update(index: Int, elem: T) = ref.swap(ref.get.get.update(index, elem))
|
||||||
|
|
||||||
|
def length: Int = ref.get.get.length
|
||||||
|
|
||||||
|
def apply(index: Int): T = ref.get.get.apply(index)
|
||||||
|
|
||||||
|
override def hashCode: Int = System.identityHashCode(this);
|
||||||
|
|
||||||
|
override def equals(other: Any): Boolean =
|
||||||
|
other.isInstanceOf[TransactionalVector[_]] &&
|
||||||
|
other.hashCode == hashCode
|
||||||
|
}
|
||||||
|
|
||||||
|
object TransactionalVector {
|
||||||
|
def apply[T]() = new TransactionalVector
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
class TransactionalRef[T] private(elem: T) extends Transactional {
|
||||||
|
private[this] val ref = new Ref[T](elem)
|
||||||
|
|
||||||
|
def swap(elem: T) = {
|
||||||
|
println("----- setting ref: " + ref)
|
||||||
|
println("----- setting in thread: " + Thread.currentThread)
|
||||||
|
ref.set(elem)
|
||||||
}
|
}
|
||||||
|
|
||||||
def get: Option[T] = {
|
def get: Option[T] = {
|
||||||
verifyTransaction
|
if (ref.isNull) None
|
||||||
ref
|
else Some(ref.get)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def getOrWait: T = ref.getOrAwait
|
||||||
|
|
||||||
def getOrElse(default: => T): T = {
|
def getOrElse(default: => T): T = {
|
||||||
verifyTransaction
|
if (ref.isNull) default
|
||||||
ref.getOrElse(default)
|
else ref.get
|
||||||
}
|
}
|
||||||
|
|
||||||
def isDefined: Boolean = {
|
def isDefined: Boolean = !ref.isNull
|
||||||
verifyTransaction
|
|
||||||
ref.isDefined
|
def isEmpty: Boolean = ref.isNull
|
||||||
|
}
|
||||||
|
|
||||||
|
object TransactionalRef {
|
||||||
|
def apply[T](elem: T) = {
|
||||||
|
if (elem == null) throw new IllegalArgumentException("Can't define TransactionalRef with a null initial value")
|
||||||
|
new TransactionalRef[T](elem)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
*/
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -43,7 +43,8 @@ import Vector._
|
||||||
* @author Rich Hickey
|
* @author Rich Hickey
|
||||||
*/
|
*/
|
||||||
@serializable
|
@serializable
|
||||||
class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail: Array[AnyRef]) extends RandomAccessSeq[T] { outer =>
|
class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail: Array[AnyRef])
|
||||||
|
extends RandomAccessSeq[T] with PersistentDataStructure { outer =>
|
||||||
private val tailOff = length - tail.length
|
private val tailOff = length - tail.length
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
|
||||||
|
|
@ -26,6 +26,7 @@ case class FailureOneWay(key: String, value: String, failer: Actor)
|
||||||
class InMemStatefulActor extends Actor {
|
class InMemStatefulActor extends Actor {
|
||||||
timeout = 100000
|
timeout = 100000
|
||||||
makeTransactionRequired
|
makeTransactionRequired
|
||||||
|
//dispatcher = se.scalablesolutions.akka.reactor.Dispatchers.newThreadBasedDispatcher(this)
|
||||||
private val mapState = TransactionalState.newMap[String, String]
|
private val mapState = TransactionalState.newMap[String, String]
|
||||||
private val vectorState = TransactionalState.newVector[String]
|
private val vectorState = TransactionalState.newVector[String]
|
||||||
private val refState = TransactionalState.newRef[String]
|
private val refState = TransactionalState.newRef[String]
|
||||||
|
|
@ -86,7 +87,7 @@ class InMemFailerActor extends Actor {
|
||||||
}
|
}
|
||||||
|
|
||||||
class InMemoryActorSpec extends TestCase {
|
class InMemoryActorSpec extends TestCase {
|
||||||
/*
|
|
||||||
@Test
|
@Test
|
||||||
def testOneWayMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
|
def testOneWayMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess = {
|
||||||
val stateful = new InMemStatefulActor
|
val stateful = new InMemStatefulActor
|
||||||
|
|
@ -213,7 +214,7 @@ class InMemoryActorSpec extends TestCase {
|
||||||
Thread.sleep(100)
|
Thread.sleep(100)
|
||||||
assertEquals("init", (stateful !! GetRefState).get) // check that state is == init state
|
assertEquals("init", (stateful !! GetRefState).get) // check that state is == init state
|
||||||
}
|
}
|
||||||
*/
|
|
||||||
@Test
|
@Test
|
||||||
def testRefShouldRollbackStateForStatefulServerInCaseOfFailure = {
|
def testRefShouldRollbackStateForStatefulServerInCaseOfFailure = {
|
||||||
val stateful = new InMemStatefulActor
|
val stateful = new InMemStatefulActor
|
||||||
|
|
|
||||||
|
|
@ -95,15 +95,14 @@
|
||||||
<version>2.7.4-0.1</version>
|
<version>2.7.4-0.1</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<<<<<<< HEAD:kernel/pom.xml
|
|
||||||
<groupId>org.multiverse</groupId>
|
<groupId>org.multiverse</groupId>
|
||||||
<artifactId>multiverse</artifactId>
|
<artifactId>multiverse</artifactId>
|
||||||
<version>0.3</version>
|
<version>0.3</version>
|
||||||
=======
|
</dependency>
|
||||||
|
<dependency>
|
||||||
<groupId>com.rabbitmq</groupId>
|
<groupId>com.rabbitmq</groupId>
|
||||||
<artifactId>rabbitmq-client</artifactId>
|
<artifactId>rabbitmq-client</artifactId>
|
||||||
<version>0.9.1</version>
|
<version>0.9.1</version>
|
||||||
>>>>>>> master:akka-kernel/pom.xml
|
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<!-- For Protocol/Serialization -->
|
<!-- For Protocol/Serialization -->
|
||||||
|
|
|
||||||
|
|
@ -9,11 +9,14 @@ import java.io.{Flushable, Closeable}
|
||||||
import util.Logging
|
import util.Logging
|
||||||
import util.Helpers._
|
import util.Helpers._
|
||||||
import serialization.Serializer
|
import serialization.Serializer
|
||||||
import akka.Config.config
|
import Config.config
|
||||||
|
|
||||||
import org.apache.cassandra.db.ColumnFamily
|
import org.apache.cassandra.db.ColumnFamily
|
||||||
import org.apache.cassandra.service._
|
import org.apache.cassandra.service._
|
||||||
|
|
||||||
|
import org.apache.commons.pool._
|
||||||
|
import org.apache.commons.pool.impl._
|
||||||
|
|
||||||
import org.apache.thrift.transport._
|
import org.apache.thrift.transport._
|
||||||
import org.apache.thrift.protocol._
|
import org.apache.thrift.protocol._
|
||||||
|
|
||||||
|
|
@ -238,3 +241,89 @@ object Protocol {
|
||||||
object SimpleJSON extends Protocol(new TSimpleJSONProtocol.Factory)
|
object SimpleJSON extends Protocol(new TSimpleJSONProtocol.Factory)
|
||||||
object JSON extends Protocol(new TJSONProtocol.Factory)
|
object JSON extends Protocol(new TJSONProtocol.Factory)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
trait Pool[T] extends java.io.Closeable {
|
||||||
|
def borrowObject: T
|
||||||
|
def returnObject(t: T): Unit
|
||||||
|
def invalidateObject(t: T): Unit
|
||||||
|
def addObject: Unit
|
||||||
|
def getNumIdle: Int
|
||||||
|
def getNumActive: Int
|
||||||
|
def clear: Unit
|
||||||
|
def setFactory(factory: PoolItemFactory[T]): Unit
|
||||||
|
}
|
||||||
|
|
||||||
|
trait PoolFactory[T] {
|
||||||
|
def createPool: Pool[T]
|
||||||
|
}
|
||||||
|
|
||||||
|
trait PoolItemFactory[T] {
|
||||||
|
def makeObject: T
|
||||||
|
def destroyObject(t: T): Unit
|
||||||
|
def validateObject(t: T): Boolean
|
||||||
|
def activateObject(t: T): Unit
|
||||||
|
def passivateObject(t: T): Unit
|
||||||
|
}
|
||||||
|
|
||||||
|
trait PoolBridge[T, OP <: ObjectPool] extends Pool[T] {
|
||||||
|
val impl: OP
|
||||||
|
override def borrowObject: T = impl.borrowObject.asInstanceOf[T]
|
||||||
|
override def returnObject(t: T) = impl.returnObject(t)
|
||||||
|
override def invalidateObject(t: T) = impl.invalidateObject(t)
|
||||||
|
override def addObject = impl.addObject
|
||||||
|
override def getNumIdle: Int = impl.getNumIdle
|
||||||
|
override def getNumActive: Int = impl.getNumActive
|
||||||
|
override def clear: Unit = impl.clear
|
||||||
|
override def close: Unit = impl.close
|
||||||
|
override def setFactory(factory: PoolItemFactory[T]) = impl.setFactory(toPoolableObjectFactory(factory))
|
||||||
|
|
||||||
|
def toPoolableObjectFactory[T](pif: PoolItemFactory[T]) = new PoolableObjectFactory {
|
||||||
|
def makeObject: Object = pif.makeObject.asInstanceOf[Object]
|
||||||
|
def destroyObject(o: Object): Unit = pif.destroyObject(o.asInstanceOf[T])
|
||||||
|
def validateObject(o: Object): Boolean = pif.validateObject(o.asInstanceOf[T])
|
||||||
|
def activateObject(o: Object): Unit = pif.activateObject(o.asInstanceOf[T])
|
||||||
|
def passivateObject(o: Object): Unit = pif.passivateObject(o.asInstanceOf[T])
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object StackPool {
|
||||||
|
def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,StackObjectPool] {
|
||||||
|
val impl = new StackObjectPool(toPoolableObjectFactory(factory))
|
||||||
|
}
|
||||||
|
|
||||||
|
def apply[T](factory: PoolItemFactory[T], maxIdle: Int) = new PoolBridge[T,StackObjectPool] {
|
||||||
|
val impl = new StackObjectPool(toPoolableObjectFactory(factory),maxIdle)
|
||||||
|
}
|
||||||
|
|
||||||
|
def apply[T](factory: PoolItemFactory[T], maxIdle: Int, initIdleCapacity: Int) = new PoolBridge[T,StackObjectPool] {
|
||||||
|
val impl = new StackObjectPool(toPoolableObjectFactory(factory),maxIdle,initIdleCapacity)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object SoftRefPool {
|
||||||
|
def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,SoftReferenceObjectPool] {
|
||||||
|
val impl = new SoftReferenceObjectPool(toPoolableObjectFactory(factory))
|
||||||
|
}
|
||||||
|
|
||||||
|
def apply[T](factory: PoolItemFactory[T], initSize: Int) = new PoolBridge[T,SoftReferenceObjectPool] {
|
||||||
|
val impl = new SoftReferenceObjectPool(toPoolableObjectFactory(factory),initSize)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
trait TransportFactory[T <: TTransport] extends PoolItemFactory[T] {
|
||||||
|
def createTransport: T
|
||||||
|
def makeObject: T = createTransport
|
||||||
|
def destroyObject(transport: T): Unit = transport.close
|
||||||
|
def validateObject(transport: T) = transport.isOpen
|
||||||
|
def activateObject(transport: T): Unit = if( !transport.isOpen ) transport.open else ()
|
||||||
|
def passivateObject(transport: T): Unit = transport.flush
|
||||||
|
}
|
||||||
|
|
||||||
|
case class SocketProvider(val host: String, val port: Int) extends TransportFactory[TSocket] {
|
||||||
|
def createTransport = {
|
||||||
|
val t = new TSocket(host, port)
|
||||||
|
t.open
|
||||||
|
t
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -17,6 +17,8 @@ import org.apache.cassandra.service._
|
||||||
import org.apache.thrift.transport._
|
import org.apache.thrift.transport._
|
||||||
import org.apache.thrift.protocol._
|
import org.apache.thrift.protocol._
|
||||||
|
|
||||||
|
import scala.collection.mutable.ArrayBuffer
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
|
|
@ -62,11 +64,12 @@ object CassandraStorage extends MapStorage
|
||||||
StackPool(SocketProvider(CASSANDRA_SERVER_HOSTNAME, CASSANDRA_SERVER_PORT)),
|
StackPool(SocketProvider(CASSANDRA_SERVER_HOSTNAME, CASSANDRA_SERVER_PORT)),
|
||||||
protocol,
|
protocol,
|
||||||
CONSISTENCY_LEVEL)
|
CONSISTENCY_LEVEL)
|
||||||
|
|
||||||
// ===============================================================
|
// ===============================================================
|
||||||
// For Ref
|
// For Ref
|
||||||
// ===============================================================
|
// ===============================================================
|
||||||
|
|
||||||
override def insertRefStorageFor(name: String, element: AnyRef) = {
|
def insertRefStorageFor(name: String, element: AnyRef) = {
|
||||||
sessions.withSession {
|
sessions.withSession {
|
||||||
_ ++| (name,
|
_ ++| (name,
|
||||||
new ColumnPath(REF_COLUMN_PARENT.getColumn_family, null, REF_KEY),
|
new ColumnPath(REF_COLUMN_PARENT.getColumn_family, null, REF_KEY),
|
||||||
|
|
@ -76,7 +79,7 @@ object CassandraStorage extends MapStorage
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getRefStorageFor(name: String): Option[AnyRef] = {
|
def getRefStorageFor(name: String): Option[AnyRef] = {
|
||||||
try {
|
try {
|
||||||
val column: Option[Column] = sessions.withSession {
|
val column: Option[Column] = sessions.withSession {
|
||||||
_ | (name, new ColumnPath(REF_COLUMN_PARENT.getColumn_family, null, REF_KEY))
|
_ | (name, new ColumnPath(REF_COLUMN_PARENT.getColumn_family, null, REF_KEY))
|
||||||
|
|
@ -94,7 +97,7 @@ object CassandraStorage extends MapStorage
|
||||||
// For Vector
|
// For Vector
|
||||||
// ===============================================================
|
// ===============================================================
|
||||||
|
|
||||||
override def insertVectorStorageEntryFor(name: String, element: AnyRef) = {
|
def insertVectorStorageEntryFor(name: String, element: AnyRef) = {
|
||||||
sessions.withSession {
|
sessions.withSession {
|
||||||
_ ++| (name,
|
_ ++| (name,
|
||||||
new ColumnPath(VECTOR_COLUMN_PARENT.getColumn_family, null, intToBytes(getVectorStorageSizeFor(name))),
|
new ColumnPath(VECTOR_COLUMN_PARENT.getColumn_family, null, intToBytes(getVectorStorageSizeFor(name))),
|
||||||
|
|
@ -104,10 +107,20 @@ object CassandraStorage extends MapStorage
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
override def insertVectorStorageEntriesFor(name: String, elements: List[AnyRef]) = {
|
def insertVectorStorageEntriesFor(name: String, elements: List[AnyRef]) = {
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getVectorStorageEntryFor(name: String, index: Int): AnyRef = {
|
def updateVectorStorageEntryFor(name: String, index: Int, elem: AnyRef) = {
|
||||||
|
sessions.withSession {
|
||||||
|
_ ++| (name,
|
||||||
|
new ColumnPath(VECTOR_COLUMN_PARENT.getColumn_family, null, intToBytes(index)),
|
||||||
|
serializer.out(elem),
|
||||||
|
System.currentTimeMillis,
|
||||||
|
CONSISTENCY_LEVEL)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def getVectorStorageEntryFor(name: String, index: Int): AnyRef = {
|
||||||
val column: Option[Column] = sessions.withSession {
|
val column: Option[Column] = sessions.withSession {
|
||||||
_ | (name, new ColumnPath(VECTOR_COLUMN_PARENT.getColumn_family, null, intToBytes(index)))
|
_ | (name, new ColumnPath(VECTOR_COLUMN_PARENT.getColumn_family, null, intToBytes(index)))
|
||||||
}
|
}
|
||||||
|
|
@ -115,7 +128,7 @@ object CassandraStorage extends MapStorage
|
||||||
else throw new NoSuchElementException("No element for vector [" + name + "] and index [" + index + "]")
|
else throw new NoSuchElementException("No element for vector [" + name + "] and index [" + index + "]")
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[AnyRef] = {
|
def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): RandomAccessSeq[AnyRef] = {
|
||||||
val startBytes = if (start.isDefined) intToBytes(start.get) else null
|
val startBytes = if (start.isDefined) intToBytes(start.get) else null
|
||||||
val finishBytes = if (finish.isDefined) intToBytes(finish.get) else null
|
val finishBytes = if (finish.isDefined) intToBytes(finish.get) else null
|
||||||
val columns: List[Column] = sessions.withSession {
|
val columns: List[Column] = sessions.withSession {
|
||||||
|
|
@ -126,10 +139,12 @@ object CassandraStorage extends MapStorage
|
||||||
count,
|
count,
|
||||||
CONSISTENCY_LEVEL)
|
CONSISTENCY_LEVEL)
|
||||||
}
|
}
|
||||||
columns.map(column => serializer.in(column.value, None))
|
val buffer = new ArrayBuffer[AnyRef]
|
||||||
|
for (elem <- columns.map(column => serializer.in(column.value, None))) buffer.append(elem)
|
||||||
|
buffer
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getVectorStorageSizeFor(name: String): Int = {
|
def getVectorStorageSizeFor(name: String): Int = {
|
||||||
sessions.withSession {
|
sessions.withSession {
|
||||||
_ |# (name, VECTOR_COLUMN_PARENT)
|
_ |# (name, VECTOR_COLUMN_PARENT)
|
||||||
}
|
}
|
||||||
|
|
@ -139,7 +154,7 @@ object CassandraStorage extends MapStorage
|
||||||
// For Map
|
// For Map
|
||||||
// ===============================================================
|
// ===============================================================
|
||||||
|
|
||||||
override def insertMapStorageEntryFor(name: String, key: AnyRef, element: AnyRef) = {
|
def insertMapStorageEntryFor(name: String, key: AnyRef, element: AnyRef) = {
|
||||||
sessions.withSession {
|
sessions.withSession {
|
||||||
_ ++| (name,
|
_ ++| (name,
|
||||||
new ColumnPath(MAP_COLUMN_PARENT.getColumn_family, null, serializer.out(key)),
|
new ColumnPath(MAP_COLUMN_PARENT.getColumn_family, null, serializer.out(key)),
|
||||||
|
|
@ -149,7 +164,7 @@ object CassandraStorage extends MapStorage
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
override def insertMapStorageEntriesFor(name: String, entries: List[Tuple2[AnyRef, AnyRef]]) = {
|
def insertMapStorageEntriesFor(name: String, entries: List[Tuple2[AnyRef, AnyRef]]) = {
|
||||||
val cf2columns: java.util.Map[String, java.util.List[Column]] = new java.util.HashMap
|
val cf2columns: java.util.Map[String, java.util.List[Column]] = new java.util.HashMap
|
||||||
for (entry <- entries) {
|
for (entry <- entries) {
|
||||||
val columns: java.util.List[Column] = new java.util.ArrayList
|
val columns: java.util.List[Column] = new java.util.ArrayList
|
||||||
|
|
@ -161,7 +176,7 @@ object CassandraStorage extends MapStorage
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getMapStorageEntryFor(name: String, key: AnyRef): Option[AnyRef] = {
|
def getMapStorageEntryFor(name: String, key: AnyRef): Option[AnyRef] = {
|
||||||
try {
|
try {
|
||||||
val column: Option[Column] = sessions.withSession {
|
val column: Option[Column] = sessions.withSession {
|
||||||
_ | (name, new ColumnPath(MAP_COLUMN_PARENT.getColumn_family, null, serializer.out(key)))
|
_ | (name, new ColumnPath(MAP_COLUMN_PARENT.getColumn_family, null, serializer.out(key)))
|
||||||
|
|
@ -175,7 +190,7 @@ object CassandraStorage extends MapStorage
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getMapStorageFor(name: String): List[Tuple2[AnyRef, AnyRef]] = {
|
def getMapStorageFor(name: String): List[Tuple2[AnyRef, AnyRef]] = {
|
||||||
throw new UnsupportedOperationException
|
throw new UnsupportedOperationException
|
||||||
/*
|
/*
|
||||||
val columns = server.get_columns_since(name, MAP_COLUMN_FAMILY, -1)
|
val columns = server.get_columns_since(name, MAP_COLUMN_FAMILY, -1)
|
||||||
|
|
@ -187,15 +202,15 @@ object CassandraStorage extends MapStorage
|
||||||
*/
|
*/
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getMapStorageSizeFor(name: String): Int = {
|
def getMapStorageSizeFor(name: String): Int = {
|
||||||
sessions.withSession {
|
sessions.withSession {
|
||||||
_ |# (name, MAP_COLUMN_PARENT)
|
_ |# (name, MAP_COLUMN_PARENT)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
override def removeMapStorageFor(name: String): Unit = removeMapStorageFor(name, null)
|
def removeMapStorageFor(name: String): Unit = removeMapStorageFor(name, null)
|
||||||
|
|
||||||
override def removeMapStorageFor(name: String, key: AnyRef): Unit = {
|
def removeMapStorageFor(name: String, key: AnyRef): Unit = {
|
||||||
val keyBytes = if (key == null) null else serializer.out(key)
|
val keyBytes = if (key == null) null else serializer.out(key)
|
||||||
sessions.withSession {
|
sessions.withSession {
|
||||||
_ -- (name,
|
_ -- (name,
|
||||||
|
|
@ -205,7 +220,7 @@ object CassandraStorage extends MapStorage
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getMapStorageRangeFor(name: String, start: Option[AnyRef], finish: Option[AnyRef], count: Int):
|
def getMapStorageRangeFor(name: String, start: Option[AnyRef], finish: Option[AnyRef], count: Int):
|
||||||
List[Tuple2[AnyRef, AnyRef]] = {
|
List[Tuple2[AnyRef, AnyRef]] = {
|
||||||
val startBytes = if (start.isDefined) serializer.out(start.get) else null
|
val startBytes = if (start.isDefined) serializer.out(start.get) else null
|
||||||
val finishBytes = if (finish.isDefined) serializer.out(finish.get) else null
|
val finishBytes = if (finish.isDefined) serializer.out(finish.get) else null
|
||||||
|
|
|
||||||
|
|
@ -1,287 +1,261 @@
|
||||||
package se.scalablesolutions.akka.state
|
/**
|
||||||
|
* Copyright (C) 2009 Scalable Solutions.
|
||||||
import akka.util.Logging
|
*/
|
||||||
import serialization.{Serializer}
|
|
||||||
import akka.Config.config
|
package se.scalablesolutions.akka.state
|
||||||
import sjson.json.Serializer._
|
|
||||||
|
import akka.util.Logging
|
||||||
import com.mongodb._
|
import serialization.{Serializer}
|
||||||
|
import Config.config
|
||||||
import java.util.{Map=>JMap, List=>JList, ArrayList=>JArrayList}
|
import sjson.json.Serializer._
|
||||||
|
|
||||||
/**
|
import com.mongodb._
|
||||||
* A module for supporting MongoDB based persistence.
|
|
||||||
* <p/>
|
import scala.collection.mutable.ArrayBuffer
|
||||||
* The module offers functionality for:
|
|
||||||
* <li>Persistent Maps</li>
|
import java.util.{Map=>JMap, List=>JList, ArrayList=>JArrayList}
|
||||||
* <li>Persistent Vectors</li>
|
|
||||||
* <li>Persistent Refs</li>
|
/**
|
||||||
* <p/>
|
* A module for supporting MongoDB based persistence.
|
||||||
* @author <a href="http://debasishg.blogspot.com">Debasish Ghosh</a>
|
* <p/>
|
||||||
*/
|
* The module offers functionality for:
|
||||||
object MongoStorage extends MapStorage with VectorStorage with RefStorage with Logging {
|
* <li>Persistent Maps</li>
|
||||||
|
* <li>Persistent Vectors</li>
|
||||||
// enrich with null safe findOne
|
* <li>Persistent Refs</li>
|
||||||
class RichDBCollection(value: DBCollection) {
|
* <p/>
|
||||||
def findOneNS(o: DBObject): Option[DBObject] = {
|
* @author <a href="http://debasishg.blogspot.com">Debasish Ghosh</a>
|
||||||
value.findOne(o) match {
|
*/
|
||||||
case null => None
|
object MongoStorage extends MapStorage with VectorStorage with RefStorage with Logging {
|
||||||
case x => Some(x)
|
|
||||||
}
|
// enrich with null safe findOne
|
||||||
}
|
class RichDBCollection(value: DBCollection) {
|
||||||
}
|
def findOneNS(o: DBObject): Option[DBObject] = {
|
||||||
|
value.findOne(o) match {
|
||||||
implicit def enrichDBCollection(c: DBCollection) = new RichDBCollection(c)
|
case null => None
|
||||||
|
case x => Some(x)
|
||||||
val KEY = "key"
|
}
|
||||||
val VALUE = "value"
|
}
|
||||||
val COLLECTION = "akka_coll"
|
}
|
||||||
|
|
||||||
val MONGODB_SERVER_HOSTNAME = config.getString("akka.storage.mongodb.hostname", "127.0.0.1")
|
implicit def enrichDBCollection(c: DBCollection) = new RichDBCollection(c)
|
||||||
val MONGODB_SERVER_DBNAME = config.getString("akka.storage.mongodb.dbname", "testdb")
|
|
||||||
val MONGODB_SERVER_PORT = config.getInt("akka.storage.mongodb.port", 27017)
|
val KEY = "key"
|
||||||
|
val VALUE = "value"
|
||||||
val db = new Mongo(MONGODB_SERVER_HOSTNAME, MONGODB_SERVER_PORT, MONGODB_SERVER_DBNAME)
|
val COLLECTION = "akka_coll"
|
||||||
val coll = db.getCollection(COLLECTION)
|
|
||||||
|
val MONGODB_SERVER_HOSTNAME = config.getString("akka.storage.mongodb.hostname", "127.0.0.1")
|
||||||
// FIXME: make this pluggable
|
val MONGODB_SERVER_DBNAME = config.getString("akka.storage.mongodb.dbname", "testdb")
|
||||||
private[this] val serializer = SJSON
|
val MONGODB_SERVER_PORT = config.getInt("akka.storage.mongodb.port", 27017)
|
||||||
|
|
||||||
override def insertMapStorageEntryFor(name: String,
|
val db = new Mongo(MONGODB_SERVER_HOSTNAME, MONGODB_SERVER_PORT, MONGODB_SERVER_DBNAME)
|
||||||
key: AnyRef, value: AnyRef) {
|
val coll = db.getCollection(COLLECTION)
|
||||||
insertMapStorageEntriesFor(name, List((key, value)))
|
|
||||||
}
|
// FIXME: make this pluggable
|
||||||
|
private[this] val serializer = SJSON
|
||||||
override def insertMapStorageEntriesFor(name: String,
|
|
||||||
entries: List[Tuple2[AnyRef, AnyRef]]) {
|
def insertMapStorageEntryFor(name: String,
|
||||||
import java.util.{Map, HashMap}
|
key: AnyRef, value: AnyRef) {
|
||||||
|
insertMapStorageEntriesFor(name, List((key, value)))
|
||||||
val m: Map[AnyRef, AnyRef] = new HashMap
|
}
|
||||||
for ((k, v) <- entries) {
|
|
||||||
m.put(k, serializer.out(v))
|
def insertMapStorageEntriesFor(name: String,
|
||||||
}
|
entries: List[Tuple2[AnyRef, AnyRef]]) {
|
||||||
|
import java.util.{Map, HashMap}
|
||||||
nullSafeFindOne(name) match {
|
|
||||||
case None =>
|
val m: Map[AnyRef, AnyRef] = new HashMap
|
||||||
coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, m))
|
for ((k, v) <- entries) m.put(k, serializer.out(v))
|
||||||
case Some(dbo) => {
|
|
||||||
// collate the maps
|
nullSafeFindOne(name) match {
|
||||||
val o = dbo.get(VALUE).asInstanceOf[Map[AnyRef, AnyRef]]
|
case None =>
|
||||||
o.putAll(m)
|
coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, m))
|
||||||
|
case Some(dbo) => {
|
||||||
// remove existing reference
|
// collate the maps
|
||||||
removeMapStorageFor(name)
|
val o = dbo.get(VALUE).asInstanceOf[Map[AnyRef, AnyRef]]
|
||||||
// and insert
|
o.putAll(m)
|
||||||
coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, o))
|
|
||||||
}
|
// remove existing reference
|
||||||
}
|
removeMapStorageFor(name)
|
||||||
}
|
// and insert
|
||||||
|
coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, o))
|
||||||
override def removeMapStorageFor(name: String) = {
|
}
|
||||||
val q = new BasicDBObject
|
}
|
||||||
q.put(KEY, name)
|
}
|
||||||
coll.remove(q)
|
|
||||||
}
|
def removeMapStorageFor(name: String) = {
|
||||||
|
val q = new BasicDBObject
|
||||||
override def removeMapStorageFor(name: String, key: AnyRef) = {
|
q.put(KEY, name)
|
||||||
nullSafeFindOne(name) match {
|
coll.remove(q)
|
||||||
case None =>
|
}
|
||||||
case Some(dbo) => {
|
|
||||||
val orig = dbo.get(VALUE).asInstanceOf[DBObject].toMap
|
def removeMapStorageFor(name: String, key: AnyRef) = {
|
||||||
orig.remove(key.asInstanceOf[String])
|
nullSafeFindOne(name) match {
|
||||||
|
case None =>
|
||||||
// remove existing reference
|
case Some(dbo) => {
|
||||||
removeMapStorageFor(name)
|
val orig = dbo.get(VALUE).asInstanceOf[DBObject].toMap
|
||||||
// and insert
|
orig.remove(key.asInstanceOf[String])
|
||||||
coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, orig))
|
|
||||||
}
|
// remove existing reference
|
||||||
}
|
removeMapStorageFor(name)
|
||||||
}
|
// and insert
|
||||||
|
coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, orig))
|
||||||
override def getMapStorageEntryFor(name: String,
|
}
|
||||||
key: AnyRef): Option[AnyRef] = {
|
}
|
||||||
getValueForKey(name, key.asInstanceOf[String])
|
}
|
||||||
}
|
|
||||||
|
def getMapStorageEntryFor(name: String, key: AnyRef): Option[AnyRef] = {
|
||||||
override def getMapStorageSizeFor(name: String): Int = {
|
getValueForKey(name, key.asInstanceOf[String])
|
||||||
nullSafeFindOne(name) match {
|
}
|
||||||
case None => 0
|
|
||||||
case Some(dbo) =>
|
def getMapStorageSizeFor(name: String): Int = {
|
||||||
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]].keySet.size
|
nullSafeFindOne(name) match {
|
||||||
}
|
case None => 0
|
||||||
}
|
case Some(dbo) =>
|
||||||
|
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]].keySet.size
|
||||||
override def getMapStorageFor(name: String): List[Tuple2[AnyRef, AnyRef]] = {
|
}
|
||||||
val m =
|
}
|
||||||
nullSafeFindOne(name) match {
|
|
||||||
case None =>
|
def getMapStorageFor(name: String): List[Tuple2[AnyRef, AnyRef]] = {
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
val m = nullSafeFindOne(name) match {
|
||||||
case Some(dbo) =>
|
case None =>
|
||||||
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
|
throw new Predef.NoSuchElementException(name + " not present")
|
||||||
}
|
case Some(dbo) =>
|
||||||
val n =
|
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
|
||||||
List(m.keySet.toArray: _*).asInstanceOf[List[String]]
|
}
|
||||||
val vals =
|
val n = List(m.keySet.toArray: _*).asInstanceOf[List[String]]
|
||||||
for(s <- n)
|
val vals = for(s <- n) yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]]))
|
||||||
yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]]))
|
vals.asInstanceOf[List[Tuple2[String, AnyRef]]]
|
||||||
vals.asInstanceOf[List[Tuple2[String, AnyRef]]]
|
}
|
||||||
}
|
|
||||||
|
def getMapStorageRangeFor(name: String, start: Option[AnyRef],
|
||||||
override def getMapStorageRangeFor(name: String, start: Option[AnyRef],
|
finish: Option[AnyRef],
|
||||||
finish: Option[AnyRef],
|
count: Int): List[Tuple2[AnyRef, AnyRef]] = {
|
||||||
count: Int): List[Tuple2[AnyRef, AnyRef]] = {
|
val m = nullSafeFindOne(name) match {
|
||||||
val m =
|
case None =>
|
||||||
nullSafeFindOne(name) match {
|
throw new Predef.NoSuchElementException(name + " not present")
|
||||||
case None =>
|
case Some(dbo) =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
|
||||||
case Some(dbo) =>
|
}
|
||||||
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
|
|
||||||
}
|
/**
|
||||||
|
* <tt>count</tt> is the max number of results to return. Start with
|
||||||
/**
|
* <tt>start</tt> or 0 (if <tt>start</tt> is not defined) and go until
|
||||||
* <tt>count</tt> is the max number of results to return. Start with
|
* you hit <tt>finish</tt> or <tt>count</tt>.
|
||||||
* <tt>start</tt> or 0 (if <tt>start</tt> is not defined) and go until
|
*/
|
||||||
* you hit <tt>finish</tt> or <tt>count</tt>.
|
val s = if (start.isDefined) start.get.asInstanceOf[Int] else 0
|
||||||
*/
|
val cnt =
|
||||||
val s = if (start.isDefined) start.get.asInstanceOf[Int] else 0
|
if (finish.isDefined) {
|
||||||
val cnt =
|
val f = finish.get.asInstanceOf[Int]
|
||||||
if (finish.isDefined) {
|
if (f >= s) Math.min(count, (f - s)) else count
|
||||||
val f = finish.get.asInstanceOf[Int]
|
}
|
||||||
if (f >= s) Math.min(count, (f - s)) else count
|
else count
|
||||||
}
|
|
||||||
else count
|
val n = List(m.keySet.toArray: _*).asInstanceOf[List[String]].sort((e1, e2) => (e1 compareTo e2) < 0).slice(s, s + cnt)
|
||||||
|
val vals = for(s <- n) yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]]))
|
||||||
val n =
|
vals.asInstanceOf[List[Tuple2[String, AnyRef]]]
|
||||||
List(m.keySet.toArray: _*).asInstanceOf[List[String]].sort((e1, e2) => (e1 compareTo e2) < 0).slice(s, s + cnt)
|
}
|
||||||
val vals =
|
|
||||||
for(s <- n)
|
private def getValueForKey(name: String, key: String): Option[AnyRef] = {
|
||||||
yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]]))
|
try {
|
||||||
vals.asInstanceOf[List[Tuple2[String, AnyRef]]]
|
nullSafeFindOne(name) match {
|
||||||
}
|
case None => None
|
||||||
|
case Some(dbo) =>
|
||||||
private def getValueForKey(name: String, key: String): Option[AnyRef] = {
|
Some(serializer.in[AnyRef](dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]].get(key).asInstanceOf[Array[Byte]]))
|
||||||
try {
|
}
|
||||||
nullSafeFindOne(name) match {
|
} catch {
|
||||||
case None => None
|
case e => throw new Predef.NoSuchElementException(e.getMessage)
|
||||||
case Some(dbo) =>
|
}
|
||||||
Some(serializer.in[AnyRef](
|
}
|
||||||
dbo.get(VALUE)
|
|
||||||
.asInstanceOf[JMap[String, AnyRef]]
|
def updateVectorStorageEntryFor(name: String, index: Int, elem: AnyRef) = throw new UnsupportedOperationException("The updateVectorStorageEntryFor method is not yet implemented for MongoDB")
|
||||||
.get(key).asInstanceOf[Array[Byte]]))
|
|
||||||
}
|
def insertVectorStorageEntriesFor(name: String, elements: List[AnyRef]) = {
|
||||||
} catch {
|
val q = new BasicDBObject
|
||||||
case e =>
|
q.put(KEY, name)
|
||||||
throw new Predef.NoSuchElementException(e.getMessage)
|
|
||||||
}
|
val currentList =
|
||||||
}
|
coll.findOneNS(q) match {
|
||||||
|
case None =>
|
||||||
override def insertVectorStorageEntriesFor(name: String, elements: List[AnyRef]) = {
|
new JArrayList[AnyRef]
|
||||||
val q = new BasicDBObject
|
case Some(dbo) =>
|
||||||
q.put(KEY, name)
|
dbo.get(VALUE).asInstanceOf[JArrayList[AnyRef]]
|
||||||
|
}
|
||||||
val currentList =
|
if (!currentList.isEmpty) {
|
||||||
coll.findOneNS(q) match {
|
// record exists
|
||||||
case None =>
|
// remove before adding
|
||||||
new JArrayList[AnyRef]
|
coll.remove(q)
|
||||||
case Some(dbo) =>
|
}
|
||||||
dbo.get(VALUE).asInstanceOf[JArrayList[AnyRef]]
|
|
||||||
}
|
// add to the current list
|
||||||
if (!currentList.isEmpty) {
|
elements.map(serializer.out(_)).foreach(currentList.add(_))
|
||||||
// record exists
|
coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, currentList))
|
||||||
// remove before adding
|
}
|
||||||
coll.remove(q)
|
|
||||||
}
|
def insertVectorStorageEntryFor(name: String, element: AnyRef) = insertVectorStorageEntriesFor(name, List(element))
|
||||||
|
|
||||||
// add to the current list
|
def getVectorStorageEntryFor(name: String, index: Int): AnyRef = {
|
||||||
elements.map(serializer.out(_)).foreach(currentList.add(_))
|
try {
|
||||||
|
val o = nullSafeFindOne(name) match {
|
||||||
coll.insert(
|
case None =>
|
||||||
new BasicDBObject()
|
throw new Predef.NoSuchElementException(name + " not present")
|
||||||
.append(KEY, name)
|
case Some(dbo) =>
|
||||||
.append(VALUE, currentList)
|
dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
|
||||||
)
|
}
|
||||||
}
|
serializer.in[AnyRef](o.get(index).asInstanceOf[Array[Byte]])
|
||||||
|
} catch {
|
||||||
override def insertVectorStorageEntryFor(name: String, element: AnyRef) = {
|
case e => throw new Predef.NoSuchElementException(e.getMessage)
|
||||||
insertVectorStorageEntriesFor(name, List(element))
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getVectorStorageEntryFor(name: String, index: Int): AnyRef = {
|
def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): RandomAccessSeq[AnyRef] = {
|
||||||
try {
|
try {
|
||||||
val o =
|
val o = nullSafeFindOne(name) match {
|
||||||
nullSafeFindOne(name) match {
|
case None =>
|
||||||
case None =>
|
throw new Predef.NoSuchElementException(name + " not present")
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
case Some(dbo) =>
|
||||||
|
dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
|
||||||
case Some(dbo) =>
|
}
|
||||||
dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
|
|
||||||
}
|
// pick the subrange and make a Scala list
|
||||||
serializer.in[AnyRef](
|
val l = List(o.subList(start.get, start.get + count).toArray: _*)
|
||||||
o.get(index).asInstanceOf[Array[Byte]])
|
val buffer = new ArrayBuffer[AnyRef]
|
||||||
} catch {
|
for (elem <- l.map(e => serializer.in[AnyRef](e.asInstanceOf[Array[Byte]]))) buffer.append(elem)
|
||||||
case e =>
|
buffer
|
||||||
throw new Predef.NoSuchElementException(e.getMessage)
|
} catch {
|
||||||
}
|
case e => throw new Predef.NoSuchElementException(e.getMessage)
|
||||||
}
|
}
|
||||||
|
}
|
||||||
override def getVectorStorageRangeFor(name: String,
|
|
||||||
start: Option[Int], finish: Option[Int], count: Int): List[AnyRef] = {
|
def getVectorStorageSizeFor(name: String): Int = {
|
||||||
try {
|
nullSafeFindOne(name) match {
|
||||||
val o =
|
case None => 0
|
||||||
nullSafeFindOne(name) match {
|
case Some(dbo) =>
|
||||||
case None =>
|
dbo.get(VALUE).asInstanceOf[JList[AnyRef]].size
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
}
|
||||||
|
}
|
||||||
case Some(dbo) =>
|
|
||||||
dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
|
private def nullSafeFindOne(name: String): Option[DBObject] = {
|
||||||
}
|
val o = new BasicDBObject
|
||||||
|
o.put(KEY, name)
|
||||||
// pick the subrange and make a Scala list
|
coll.findOneNS(o)
|
||||||
val l =
|
}
|
||||||
List(o.subList(start.get, start.get + count).toArray: _*)
|
|
||||||
|
def insertRefStorageFor(name: String, element: AnyRef) = {
|
||||||
for(e <- l)
|
nullSafeFindOne(name) match {
|
||||||
yield serializer.in[AnyRef](e.asInstanceOf[Array[Byte]])
|
case None =>
|
||||||
} catch {
|
case Some(dbo) => {
|
||||||
case e =>
|
val q = new BasicDBObject
|
||||||
throw new Predef.NoSuchElementException(e.getMessage)
|
q.put(KEY, name)
|
||||||
}
|
coll.remove(q)
|
||||||
}
|
}
|
||||||
|
}
|
||||||
override def getVectorStorageSizeFor(name: String): Int = {
|
coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, serializer.out(element)))
|
||||||
nullSafeFindOne(name) match {
|
}
|
||||||
case None => 0
|
|
||||||
case Some(dbo) =>
|
def getRefStorageFor(name: String): Option[AnyRef] = {
|
||||||
dbo.get(VALUE).asInstanceOf[JList[AnyRef]].size
|
nullSafeFindOne(name) match {
|
||||||
}
|
case None => None
|
||||||
}
|
case Some(dbo) =>
|
||||||
|
Some(serializer.in[AnyRef](dbo.get(VALUE).asInstanceOf[Array[Byte]]))
|
||||||
private def nullSafeFindOne(name: String): Option[DBObject] = {
|
}
|
||||||
val o = new BasicDBObject
|
}
|
||||||
o.put(KEY, name)
|
}
|
||||||
coll.findOneNS(o)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def insertRefStorageFor(name: String, element: AnyRef) = {
|
|
||||||
nullSafeFindOne(name) match {
|
|
||||||
case None =>
|
|
||||||
case Some(dbo) => {
|
|
||||||
val q = new BasicDBObject
|
|
||||||
q.put(KEY, name)
|
|
||||||
coll.remove(q)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
coll.insert(
|
|
||||||
new BasicDBObject()
|
|
||||||
.append(KEY, name)
|
|
||||||
.append(VALUE, serializer.out(element)))
|
|
||||||
}
|
|
||||||
|
|
||||||
override def getRefStorageFor(name: String): Option[AnyRef] = {
|
|
||||||
nullSafeFindOne(name) match {
|
|
||||||
case None => None
|
|
||||||
case Some(dbo) =>
|
|
||||||
Some(serializer.in[AnyRef](dbo.get(VALUE).asInstanceOf[Array[Byte]]))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
|
||||||
|
|
@ -38,23 +38,23 @@ object PersistentState extends PersistentState
|
||||||
* </pre>
|
* </pre>
|
||||||
*/
|
*/
|
||||||
class PersistentState {
|
class PersistentState {
|
||||||
def newMap(config: PersistentStorageConfig): TransactionalMap[AnyRef, AnyRef] = config match {
|
def newMap(config: PersistentStorageConfig): PersistentMap[AnyRef, AnyRef] = config match {
|
||||||
case CassandraStorageConfig() => new CassandraPersistentTransactionalMap
|
case CassandraStorageConfig() => new CassandraPersistentMap
|
||||||
case MongoStorageConfig() => new MongoPersistentTransactionalMap
|
case MongoStorageConfig() => new MongoPersistentMap
|
||||||
case TerracottaStorageConfig() => throw new UnsupportedOperationException
|
case TerracottaStorageConfig() => throw new UnsupportedOperationException
|
||||||
case TokyoCabinetStorageConfig() => throw new UnsupportedOperationException
|
case TokyoCabinetStorageConfig() => throw new UnsupportedOperationException
|
||||||
}
|
}
|
||||||
|
|
||||||
def newVector(config: PersistentStorageConfig): TransactionalVector[AnyRef] = config match {
|
def newVector(config: PersistentStorageConfig): PersistentVector[AnyRef] = config match {
|
||||||
case CassandraStorageConfig() => new CassandraPersistentTransactionalVector
|
case CassandraStorageConfig() => new CassandraPersistentVector
|
||||||
case MongoStorageConfig() => new MongoPersistentTransactionalVector
|
case MongoStorageConfig() => new MongoPersistentVector
|
||||||
case TerracottaStorageConfig() => throw new UnsupportedOperationException
|
case TerracottaStorageConfig() => throw new UnsupportedOperationException
|
||||||
case TokyoCabinetStorageConfig() => throw new UnsupportedOperationException
|
case TokyoCabinetStorageConfig() => throw new UnsupportedOperationException
|
||||||
}
|
}
|
||||||
|
|
||||||
def newRef(config: PersistentStorageConfig): TransactionalRef[AnyRef] = config match {
|
def newRef(config: PersistentStorageConfig): PersistentRef[AnyRef] = config match {
|
||||||
case CassandraStorageConfig() => new CassandraPersistentTransactionalRef
|
case CassandraStorageConfig() => new CassandraPersistentRef
|
||||||
case MongoStorageConfig() => new MongoPersistentTransactionalRef
|
case MongoStorageConfig() => new MongoPersistentRef
|
||||||
case TerracottaStorageConfig() => throw new UnsupportedOperationException
|
case TerracottaStorageConfig() => throw new UnsupportedOperationException
|
||||||
case TokyoCabinetStorageConfig() => throw new UnsupportedOperationException
|
case TokyoCabinetStorageConfig() => throw new UnsupportedOperationException
|
||||||
}
|
}
|
||||||
|
|
@ -68,32 +68,10 @@ class PersistentState {
|
||||||
*
|
*
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
abstract class PersistentTransactionalMap[K, V] extends TransactionalMap[K, V] {
|
trait PersistentMap[K, V] extends Transactional with scala.collection.mutable.Map[K, V]
|
||||||
|
|
||||||
// FIXME: need to handle remove in another changeSet
|
|
||||||
protected[akka] val changeSet = new HashMap[K, V]
|
|
||||||
|
|
||||||
def getRange(start: Option[AnyRef], count: Int)
|
|
||||||
|
|
||||||
// ---- For Transactional ----
|
|
||||||
override def begin = {}
|
|
||||||
|
|
||||||
override def rollback = changeSet.clear
|
|
||||||
|
|
||||||
// ---- For scala.collection.mutable.Map ----
|
|
||||||
override def put(key: K, value: V): Option[V] = {
|
|
||||||
verifyTransaction
|
|
||||||
changeSet += key -> value
|
|
||||||
None // always return None to speed up writes (else need to go to DB to get
|
|
||||||
}
|
|
||||||
|
|
||||||
override def -=(key: K) = remove(key)
|
|
||||||
|
|
||||||
override def update(key: K, value: V) = put(key, value)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implementation of <tt>PersistentTransactionalMap</tt> for every concrete
|
* Implementation of <tt>PersistentMap</tt> for every concrete
|
||||||
* storage will have the same workflow. This abstracts the workflow.
|
* storage will have the same workflow. This abstracts the workflow.
|
||||||
*
|
*
|
||||||
* Subclasses just need to provide the actual concrete instance for the
|
* Subclasses just need to provide the actual concrete instance for the
|
||||||
|
|
@ -101,86 +79,65 @@ abstract class PersistentTransactionalMap[K, V] extends TransactionalMap[K, V] {
|
||||||
*
|
*
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
abstract class TemplatePersistentTransactionalMap extends PersistentTransactionalMap[AnyRef, AnyRef] {
|
trait TemplatePersistentMap extends PersistentMap[AnyRef, AnyRef] {
|
||||||
|
protected val newAndUpdatedEntries = TransactionalState.newMap[AnyRef, AnyRef]
|
||||||
|
protected val removedEntries = TransactionalState.newMap[AnyRef, AnyRef]
|
||||||
|
protected val shouldClearOnCommit = TransactionalRef[Boolean](false)
|
||||||
|
|
||||||
// to be concretized in subclasses
|
// to be concretized in subclasses
|
||||||
val storage: MapStorage
|
val storage: MapStorage
|
||||||
|
|
||||||
override def remove(key: AnyRef) = {
|
def commit = {
|
||||||
verifyTransaction
|
storage.removeMapStorageFor(uuid, removedEntries.toList)
|
||||||
if (changeSet.contains(key)) changeSet -= key
|
storage.insertMapStorageEntriesFor(uuid, newAndUpdatedEntries.toList)
|
||||||
else storage.removeMapStorageFor(uuid, key)
|
if (shouldClearOnCommit.isDefined & shouldClearOnCommit.get.get) storage.removeMapStorageFor(uuid)
|
||||||
|
newAndUpdatedEntries.clear
|
||||||
|
removedEntries.clear
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getRange(start: Option[AnyRef], count: Int) =
|
def -=(key: AnyRef) = remove(key)
|
||||||
getRange(start, None, count)
|
|
||||||
|
|
||||||
def getRange(start: Option[AnyRef], finish: Option[AnyRef], count: Int) = {
|
def +=(key: AnyRef, value: AnyRef) = put(key, value)
|
||||||
verifyTransaction
|
|
||||||
try {
|
override def put(key: AnyRef, value: AnyRef): Option[AnyRef] = newAndUpdatedEntries.put(key, value)
|
||||||
|
|
||||||
|
override def update(key: AnyRef, value: AnyRef) = newAndUpdatedEntries.update(key, value)
|
||||||
|
|
||||||
|
def remove(key: AnyRef) = removedEntries.remove(key)
|
||||||
|
|
||||||
|
def slice(start: Option[AnyRef], count: Int): List[Tuple2[AnyRef, AnyRef]] = slice(start, None, count)
|
||||||
|
|
||||||
|
def slice(start: Option[AnyRef], finish: Option[AnyRef], count: Int): List[Tuple2[AnyRef, AnyRef]] = try {
|
||||||
storage.getMapStorageRangeFor(uuid, start, finish, count)
|
storage.getMapStorageRangeFor(uuid, start, finish, count)
|
||||||
} catch {
|
} catch { case e: Exception => Nil }
|
||||||
case e: Exception => Nil
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// ---- For Transactional ----
|
override def clear = shouldClearOnCommit.swap(true)
|
||||||
override def commit = {
|
|
||||||
storage.insertMapStorageEntriesFor(uuid, changeSet.toList)
|
|
||||||
changeSet.clear
|
|
||||||
}
|
|
||||||
|
|
||||||
// ---- Overriding scala.collection.mutable.Map behavior ----
|
override def contains(key: AnyRef): Boolean = try {
|
||||||
override def clear = {
|
newAndUpdatedEntries.contains(key) ||
|
||||||
verifyTransaction
|
|
||||||
try {
|
|
||||||
storage.removeMapStorageFor(uuid)
|
|
||||||
} catch {
|
|
||||||
case e: Exception => {}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
override def contains(key: AnyRef): Boolean = {
|
|
||||||
try {
|
|
||||||
verifyTransaction
|
|
||||||
storage.getMapStorageEntryFor(uuid, key).isDefined
|
storage.getMapStorageEntryFor(uuid, key).isDefined
|
||||||
} catch {
|
} catch { case e: Exception => false }
|
||||||
case e: Exception => false
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
override def size: Int = {
|
override def size: Int = try {
|
||||||
verifyTransaction
|
storage.getMapStorageSizeFor(uuid) + newAndUpdatedEntries.size
|
||||||
try {
|
} catch { case e: Exception => 0 }
|
||||||
storage.getMapStorageSizeFor(uuid)
|
|
||||||
} catch {
|
|
||||||
case e: Exception => 0
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// ---- For scala.collection.mutable.Map ----
|
|
||||||
override def get(key: AnyRef): Option[AnyRef] = {
|
override def get(key: AnyRef): Option[AnyRef] = {
|
||||||
verifyTransaction
|
if (newAndUpdatedEntries.contains(key)) newAndUpdatedEntries.get(key)
|
||||||
// if (changeSet.contains(key)) changeSet.get(key)
|
else try {
|
||||||
// else {
|
|
||||||
val result = try {
|
|
||||||
storage.getMapStorageEntryFor(uuid, key)
|
storage.getMapStorageEntryFor(uuid, key)
|
||||||
} catch {
|
} catch { case e: Exception => None }
|
||||||
case e: Exception => None
|
|
||||||
}
|
|
||||||
result
|
|
||||||
//}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
override def elements: Iterator[Tuple2[AnyRef, AnyRef]] = {
|
override def elements: Iterator[Tuple2[AnyRef, AnyRef]] = {
|
||||||
//verifyTransaction
|
|
||||||
new Iterator[Tuple2[AnyRef, AnyRef]] {
|
new Iterator[Tuple2[AnyRef, AnyRef]] {
|
||||||
private val originalList: List[Tuple2[AnyRef, AnyRef]] = try {
|
private val originalList: List[Tuple2[AnyRef, AnyRef]] = try {
|
||||||
storage.getMapStorageFor(uuid)
|
storage.getMapStorageFor(uuid)
|
||||||
} catch {
|
} catch {
|
||||||
case e: Throwable => Nil
|
case e: Throwable => Nil
|
||||||
}
|
}
|
||||||
private var elements = originalList.reverse
|
// FIXME how to deal with updated entries, these should be replaced in the originalList not just added
|
||||||
|
private var elements = newAndUpdatedEntries.toList ::: originalList.reverse
|
||||||
override def next: Tuple2[AnyRef, AnyRef]= synchronized {
|
override def next: Tuple2[AnyRef, AnyRef]= synchronized {
|
||||||
val element = elements.head
|
val element = elements.head
|
||||||
elements = elements.tail
|
elements = elements.tail
|
||||||
|
|
@ -197,7 +154,7 @@ abstract class TemplatePersistentTransactionalMap extends PersistentTransactiona
|
||||||
*
|
*
|
||||||
* @author <a href="http://debasishg.blogspot.com">Debasish Ghosh</a>
|
* @author <a href="http://debasishg.blogspot.com">Debasish Ghosh</a>
|
||||||
*/
|
*/
|
||||||
class CassandraPersistentTransactionalMap extends TemplatePersistentTransactionalMap {
|
class CassandraPersistentMap extends TemplatePersistentMap {
|
||||||
val storage = CassandraStorage
|
val storage = CassandraStorage
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -206,7 +163,7 @@ class CassandraPersistentTransactionalMap extends TemplatePersistentTransactiona
|
||||||
*
|
*
|
||||||
* @author <a href="http://debasishg.blogspot.com">Debasish Ghosh</a>
|
* @author <a href="http://debasishg.blogspot.com">Debasish Ghosh</a>
|
||||||
*/
|
*/
|
||||||
class MongoPersistentTransactionalMap extends TemplatePersistentTransactionalMap {
|
class MongoPersistentMap extends TemplatePersistentMap {
|
||||||
val storage = MongoStorage
|
val storage = MongoStorage
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -218,69 +175,62 @@ class MongoPersistentTransactionalMap extends TemplatePersistentTransactionalMap
|
||||||
*
|
*
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
abstract class PersistentTransactionalVector[T] extends TransactionalVector[T] {
|
trait PersistentVector[T] extends Transactional with RandomAccessSeq[T]
|
||||||
|
|
||||||
// FIXME: need to handle remove in another changeSet
|
|
||||||
protected[akka] val changeSet = new ArrayBuffer[T]
|
|
||||||
|
|
||||||
// ---- For Transactional ----
|
|
||||||
override def begin = {}
|
|
||||||
|
|
||||||
override def rollback = changeSet.clear
|
|
||||||
|
|
||||||
// ---- For TransactionalVector ----
|
|
||||||
override def add(value: T) = {
|
|
||||||
verifyTransaction
|
|
||||||
changeSet += value
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implements a template for a concrete persistent transactional vector based storage.
|
* Implements a template for a concrete persistent transactional vector based storage.
|
||||||
*
|
*
|
||||||
* @author <a href="http://debasishg.blogspot.com">Debasish Ghosh</a>
|
* @author <a href="http://debasishg.blogspot.com">Debasish Ghosh</a>
|
||||||
*/
|
*/
|
||||||
abstract class TemplatePersistentTransactionalVector extends PersistentTransactionalVector[AnyRef] {
|
trait TemplatePersistentVector extends PersistentVector[AnyRef] {
|
||||||
|
protected val newElems = TransactionalState.newVector[AnyRef]
|
||||||
|
protected val updatedElems = TransactionalState.newMap[Int, AnyRef]
|
||||||
|
protected val removedElems = TransactionalState.newVector[AnyRef]
|
||||||
|
protected val shouldClearOnCommit = TransactionalRef[Boolean](false)
|
||||||
|
|
||||||
val storage: VectorStorage
|
val storage: VectorStorage
|
||||||
|
|
||||||
// ---- For TransactionalVector ----
|
def commit = {
|
||||||
override def get(index: Int): AnyRef = {
|
// FIXME: should use batch function once the bug is resolved
|
||||||
verifyTransaction
|
for (element <- newElems) storage.insertVectorStorageEntryFor(uuid, element)
|
||||||
if (changeSet.size > index) changeSet(index)
|
for (entry <- updatedElems) storage.updateVectorStorageEntryFor(uuid, entry._1, entry._2)
|
||||||
|
newElems.clear
|
||||||
|
updatedElems.clear
|
||||||
|
}
|
||||||
|
|
||||||
|
def apply(index: Int): AnyRef = get(index)
|
||||||
|
|
||||||
|
def get(index: Int): AnyRef = {
|
||||||
|
if (newElems.size > index) newElems(index)
|
||||||
else storage.getVectorStorageEntryFor(uuid, index)
|
else storage.getVectorStorageEntryFor(uuid, index)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def getRange(start: Int, count: Int): List[AnyRef] =
|
override def slice(start: Int, count: Int): RandomAccessSeq[AnyRef] = slice(Some(start), None, count)
|
||||||
getRange(Some(start), None, count)
|
|
||||||
|
|
||||||
def getRange(start: Option[Int], finish: Option[Int], count: Int): List[AnyRef] = {
|
def slice(start: Option[Int], finish: Option[Int], count: Int): RandomAccessSeq[AnyRef] =
|
||||||
verifyTransaction
|
|
||||||
storage.getVectorStorageRangeFor(uuid, start, finish, count)
|
storage.getVectorStorageRangeFor(uuid, start, finish, count)
|
||||||
}
|
|
||||||
|
|
||||||
override def length: Int = {
|
/**
|
||||||
verifyTransaction
|
* Removes the <i>tail</i> element of this vector.
|
||||||
storage.getVectorStorageSizeFor(uuid)
|
*/
|
||||||
}
|
// FIXME
|
||||||
|
def pop: AnyRef = throw new UnsupportedOperationException("need to implement persistent vector pop")
|
||||||
|
|
||||||
override def apply(index: Int): AnyRef = get(index)
|
// FIXME
|
||||||
|
def update(index: Int, newElem: AnyRef) = storage.updateVectorStorageEntryFor(uuid, index, newElem)
|
||||||
|
|
||||||
override def first: AnyRef = get(0)
|
override def first: AnyRef = get(0)
|
||||||
|
|
||||||
override def last: AnyRef = {
|
override def last: AnyRef = {
|
||||||
verifyTransaction
|
if (newElems.length != 0) newElems.last
|
||||||
val l = length
|
else {
|
||||||
if (l == 0) throw new NoSuchElementException("Vector is empty")
|
val len = length
|
||||||
get(length - 1)
|
if (len == 0) throw new NoSuchElementException("Vector is empty")
|
||||||
|
get(len - 1)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// ---- For Transactional ----
|
def length: Int = storage.getVectorStorageSizeFor(uuid) + newElems.length
|
||||||
override def commit = {
|
|
||||||
// FIXME: should use batch function once the bug is resolved
|
|
||||||
for (element <- changeSet) storage.insertVectorStorageEntryFor(uuid, element)
|
|
||||||
changeSet.clear
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -288,7 +238,7 @@ abstract class TemplatePersistentTransactionalVector extends PersistentTransacti
|
||||||
*
|
*
|
||||||
* @author <a href="http://debasishg.blogspot.com">Debaissh Ghosh</a>
|
* @author <a href="http://debasishg.blogspot.com">Debaissh Ghosh</a>
|
||||||
*/
|
*/
|
||||||
class CassandraPersistentTransactionalVector extends TemplatePersistentTransactionalVector {
|
class CassandraPersistentVector extends TemplatePersistentVector {
|
||||||
val storage = CassandraStorage
|
val storage = CassandraStorage
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -297,38 +247,37 @@ class CassandraPersistentTransactionalVector extends TemplatePersistentTransacti
|
||||||
*
|
*
|
||||||
* @author <a href="http://debasishg.blogspot.com">Debaissh Ghosh</a>
|
* @author <a href="http://debasishg.blogspot.com">Debaissh Ghosh</a>
|
||||||
*/
|
*/
|
||||||
class MongoPersistentTransactionalVector extends TemplatePersistentTransactionalVector {
|
class MongoPersistentVector extends TemplatePersistentVector {
|
||||||
val storage = MongoStorage
|
val storage = MongoStorage
|
||||||
}
|
}
|
||||||
|
|
||||||
abstract class TemplatePersistentTransactionalRef extends TransactionalRef[AnyRef] {
|
trait PersistentRef[T] extends Transactional
|
||||||
|
|
||||||
|
trait TemplatePersistentRef extends PersistentRef[AnyRef] {
|
||||||
|
protected val ref = new TransactionalRef[AnyRef]
|
||||||
|
|
||||||
val storage: RefStorage
|
val storage: RefStorage
|
||||||
|
|
||||||
override def commit = if (ref.isDefined) {
|
def commit = if (ref.isDefined) {
|
||||||
storage.insertRefStorageFor(uuid, ref.get)
|
storage.insertRefStorageFor(uuid, ref.get)
|
||||||
ref = None
|
ref.swap(null)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def rollback = ref = None
|
def get: Option[AnyRef] = if (ref.isDefined) ref.get else storage.getRefStorageFor(uuid)
|
||||||
|
|
||||||
override def get: Option[AnyRef] = {
|
def isDefined: Boolean = ref.isDefined || storage.getRefStorageFor(uuid).isDefined
|
||||||
verifyTransaction
|
|
||||||
storage.getRefStorageFor(uuid)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def isDefined: Boolean = get.isDefined
|
def getOrElse(default: => AnyRef): AnyRef = {
|
||||||
|
val current = get
|
||||||
override def getOrElse(default: => AnyRef): AnyRef = {
|
if (current.isDefined) current
|
||||||
val ref = get
|
|
||||||
if (ref.isDefined) ref
|
|
||||||
else default
|
else default
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class CassandraPersistentTransactionalRef extends TemplatePersistentTransactionalRef {
|
class CassandraPersistentRef extends TemplatePersistentRef {
|
||||||
val storage = CassandraStorage
|
val storage = CassandraStorage
|
||||||
}
|
}
|
||||||
|
|
||||||
class MongoPersistentTransactionalRef extends TemplatePersistentTransactionalRef {
|
class MongoPersistentRef extends TemplatePersistentRef {
|
||||||
val storage = MongoStorage
|
val storage = MongoStorage
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,96 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2009 Scalable Solutions.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package se.scalablesolutions.akka.state
|
|
||||||
|
|
||||||
import org.apache.commons.pool._
|
|
||||||
import org.apache.commons.pool.impl._
|
|
||||||
|
|
||||||
import org.apache.thrift.transport._
|
|
||||||
|
|
||||||
trait Pool[T] extends java.io.Closeable {
|
|
||||||
def borrowObject: T
|
|
||||||
def returnObject(t: T): Unit
|
|
||||||
def invalidateObject(t: T): Unit
|
|
||||||
def addObject: Unit
|
|
||||||
def getNumIdle: Int
|
|
||||||
def getNumActive: Int
|
|
||||||
def clear: Unit
|
|
||||||
def setFactory(factory: PoolItemFactory[T]): Unit
|
|
||||||
}
|
|
||||||
|
|
||||||
trait PoolFactory[T] {
|
|
||||||
def createPool: Pool[T]
|
|
||||||
}
|
|
||||||
|
|
||||||
trait PoolItemFactory[T] {
|
|
||||||
def makeObject: T
|
|
||||||
def destroyObject(t: T): Unit
|
|
||||||
def validateObject(t: T): Boolean
|
|
||||||
def activateObject(t: T): Unit
|
|
||||||
def passivateObject(t: T): Unit
|
|
||||||
}
|
|
||||||
|
|
||||||
trait PoolBridge[T, OP <: ObjectPool] extends Pool[T] {
|
|
||||||
val impl: OP
|
|
||||||
override def borrowObject: T = impl.borrowObject.asInstanceOf[T]
|
|
||||||
override def returnObject(t: T) = impl.returnObject(t)
|
|
||||||
override def invalidateObject(t: T) = impl.invalidateObject(t)
|
|
||||||
override def addObject = impl.addObject
|
|
||||||
override def getNumIdle: Int = impl.getNumIdle
|
|
||||||
override def getNumActive: Int = impl.getNumActive
|
|
||||||
override def clear: Unit = impl.clear
|
|
||||||
override def close: Unit = impl.close
|
|
||||||
override def setFactory(factory: PoolItemFactory[T]) = impl.setFactory(toPoolableObjectFactory(factory))
|
|
||||||
|
|
||||||
def toPoolableObjectFactory[T](pif: PoolItemFactory[T]) = new PoolableObjectFactory {
|
|
||||||
def makeObject: Object = pif.makeObject.asInstanceOf[Object]
|
|
||||||
def destroyObject(o: Object): Unit = pif.destroyObject(o.asInstanceOf[T])
|
|
||||||
def validateObject(o: Object): Boolean = pif.validateObject(o.asInstanceOf[T])
|
|
||||||
def activateObject(o: Object): Unit = pif.activateObject(o.asInstanceOf[T])
|
|
||||||
def passivateObject(o: Object): Unit = pif.passivateObject(o.asInstanceOf[T])
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
object StackPool {
|
|
||||||
def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,StackObjectPool] {
|
|
||||||
val impl = new StackObjectPool(toPoolableObjectFactory(factory))
|
|
||||||
}
|
|
||||||
|
|
||||||
def apply[T](factory: PoolItemFactory[T], maxIdle: Int) = new PoolBridge[T,StackObjectPool] {
|
|
||||||
val impl = new StackObjectPool(toPoolableObjectFactory(factory),maxIdle)
|
|
||||||
}
|
|
||||||
|
|
||||||
def apply[T](factory: PoolItemFactory[T], maxIdle: Int, initIdleCapacity: Int) = new PoolBridge[T,StackObjectPool] {
|
|
||||||
val impl = new StackObjectPool(toPoolableObjectFactory(factory),maxIdle,initIdleCapacity)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
object SoftRefPool {
|
|
||||||
def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,SoftReferenceObjectPool] {
|
|
||||||
val impl = new SoftReferenceObjectPool(toPoolableObjectFactory(factory))
|
|
||||||
}
|
|
||||||
|
|
||||||
def apply[T](factory: PoolItemFactory[T], initSize: Int) = new PoolBridge[T,SoftReferenceObjectPool] {
|
|
||||||
val impl = new SoftReferenceObjectPool(toPoolableObjectFactory(factory),initSize)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
trait TransportFactory[T <: TTransport] extends PoolItemFactory[T] {
|
|
||||||
def createTransport: T
|
|
||||||
def makeObject: T = createTransport
|
|
||||||
def destroyObject(transport: T): Unit = transport.close
|
|
||||||
def validateObject(transport: T) = transport.isOpen
|
|
||||||
def activateObject(transport: T): Unit = if( !transport.isOpen ) transport.open else ()
|
|
||||||
def passivateObject(transport: T): Unit = transport.flush
|
|
||||||
}
|
|
||||||
|
|
||||||
case class SocketProvider(val host: String, val port: Int) extends TransportFactory[TSocket] {
|
|
||||||
def createTransport = {
|
|
||||||
val t = new TSocket(host, port)
|
|
||||||
t.open
|
|
||||||
t
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
@ -1,3 +1,7 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009 Scalable Solutions.
|
||||||
|
*/
|
||||||
|
|
||||||
package se.scalablesolutions.akka.state
|
package se.scalablesolutions.akka.state
|
||||||
|
|
||||||
// abstracts persistence storage
|
// abstracts persistence storage
|
||||||
|
|
@ -15,12 +19,13 @@ trait MapStorage extends Storage {
|
||||||
def getMapStorageRangeFor(name: String, start: Option[AnyRef], finish: Option[AnyRef], count: Int): List[Tuple2[AnyRef, AnyRef]]
|
def getMapStorageRangeFor(name: String, start: Option[AnyRef], finish: Option[AnyRef], count: Int): List[Tuple2[AnyRef, AnyRef]]
|
||||||
}
|
}
|
||||||
|
|
||||||
// for vectors
|
// for Vectors
|
||||||
trait VectorStorage extends Storage {
|
trait VectorStorage extends Storage {
|
||||||
def insertVectorStorageEntryFor(name: String, element: AnyRef)
|
def insertVectorStorageEntryFor(name: String, element: AnyRef)
|
||||||
def insertVectorStorageEntriesFor(name: String, elements: List[AnyRef])
|
def insertVectorStorageEntriesFor(name: String, elements: List[AnyRef])
|
||||||
|
def updateVectorStorageEntryFor(name: String, index: Int, elem: AnyRef)
|
||||||
def getVectorStorageEntryFor(name: String, index: Int): AnyRef
|
def getVectorStorageEntryFor(name: String, index: Int): AnyRef
|
||||||
def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[AnyRef]
|
def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): RandomAccessSeq[AnyRef]
|
||||||
def getVectorStorageSizeFor(name: String): Int
|
def getVectorStorageSizeFor(name: String): Int
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
6
akka.iml
Executable file → Normal file
6
akka.iml
Executable file → Normal file
|
|
@ -1,14 +1,8 @@
|
||||||
<?xml version="1.0" encoding="UTF-8"?>
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
<module relativePaths="true" MavenProjectsManager.isMavenModule="true" type="JAVA_MODULE" version="4">
|
<module relativePaths="true" MavenProjectsManager.isMavenModule="true" type="JAVA_MODULE" version="4">
|
||||||
<component name="FacetManager">
|
|
||||||
<facet type="Scala" name="Scala">
|
|
||||||
<configuration />
|
|
||||||
</facet>
|
|
||||||
</component>
|
|
||||||
<component name="NewModuleRootManager" LANGUAGE_LEVEL="JDK_1_5" inherit-compiler-output="false">
|
<component name="NewModuleRootManager" LANGUAGE_LEVEL="JDK_1_5" inherit-compiler-output="false">
|
||||||
<output url="file://$MODULE_DIR$/target/classes" />
|
<output url="file://$MODULE_DIR$/target/classes" />
|
||||||
<output-test url="file://$MODULE_DIR$/target/test-classes" />
|
<output-test url="file://$MODULE_DIR$/target/test-classes" />
|
||||||
<exclude-output />
|
|
||||||
<content url="file://$MODULE_DIR$">
|
<content url="file://$MODULE_DIR$">
|
||||||
<excludeFolder url="file://$MODULE_DIR$/target" />
|
<excludeFolder url="file://$MODULE_DIR$/target" />
|
||||||
</content>
|
</content>
|
||||||
|
|
|
||||||
Binary file not shown.
Loading…
Add table
Add a link
Reference in a new issue