diff --git a/kernel/src/main/scala/state/State.scala b/kernel/src/main/scala/state/State.scala
index 8967ed3dd8..e49ffb25d6 100644
--- a/kernel/src/main/scala/state/State.scala
+++ b/kernel/src/main/scala/state/State.scala
@@ -11,6 +11,8 @@ import org.codehaus.aspectwerkz.proxy.Uuid
import scala.collection.mutable.{ArrayBuffer, HashMap}
+import org.multiverse.standard.manuallyinstrumented.Ref
+
sealed abstract class TransactionalStateConfig
abstract class PersistentStorageConfig extends TransactionalStateConfig
case class CassandraStorageConfig extends PersistentStorageConfig
@@ -70,10 +72,6 @@ trait Transactional {
// FIXME: won't work across the cluster
val uuid = Uuid.newUuid.toString
- private[kernel] def begin
- private[kernel] def commit
- private[kernel] def rollback
-
protected def verifyTransaction = {
val cflowTx = TransactionManagement.threadBoundTx.get
if (!cflowTx.isDefined) {
@@ -103,62 +101,32 @@ trait TransactionalMap[K, V] extends Transactional with scala.collection.mutable
* @author Jonas Bonér
*/
class InMemoryTransactionalMap[K, V] extends TransactionalMap[K, V] {
- protected[kernel] var state = new HashTrie[K, V]
- protected[kernel] var snapshot = state
-
- // ---- For Transactional ----
- override def begin = snapshot = state
- override def commit = snapshot = state
- override def rollback = state = snapshot
+ protected[this] var ref = new TransactionalRef(new HashTrie[K, V])
// ---- Overriding scala.collection.mutable.Map behavior ----
- override def contains(key: K): Boolean = {
- verifyTransaction
- state.contains(key)
- }
+ override def contains(key: K): Boolean = ref.get.contains(key)
- override def clear = {
- verifyTransaction
- state = new HashTrie[K, V]
- }
+ override def clear = ref.set(new HashTrie[K, V])
- override def size: Int = {
- verifyTransaction
- state.size
- }
+ override def size: Int = ref.get.size
// ---- For scala.collection.mutable.Map ----
- override def remove(key: K) = {
- verifyTransaction
- state = state - key
- }
+ override def remove(key: K) = ref.set(ref.get - key)
- override def elements: Iterator[(K, V)] = {
-// verifyTransaction
- state.elements
- }
-
- override def get(key: K): Option[V] = {
- verifyTransaction
- state.get(key)
- }
+ override def elements: Iterator[(K, V)] = ref.get.elements
+
+ override def get(key: K): Option[V] = ref.get.get(key)
override def put(key: K, value: V): Option[V] = {
- verifyTransaction
- val oldValue = state.get(key)
- state = state.update(key, value)
+ val map = ref.get
+ val oldValue = map.get(key)
+ ref.set(map.update(key, value))
oldValue
}
- override def -=(key: K) = {
- verifyTransaction
- remove(key)
- }
+ override def -=(key: K) = remove(key)
- override def update(key: K, value: V) = {
- verifyTransaction
- put(key, value)
- }
+ override def update(key: K, value: V) = put(key, value)
}
/**
@@ -176,10 +144,9 @@ abstract class PersistentTransactionalMap[K, V] extends TransactionalMap[K, V] {
def getRange(start: Int, count: Int)
- // ---- For Transactional ----
- override def begin = {}
-
- override def rollback = changeSet.clear
+ def begin
+ def commit
+ def rollback = changeSet.clear
// ---- For scala.collection.mutable.Map ----
override def put(key: K, value: V): Option[V] = {
@@ -305,16 +272,15 @@ abstract class TransactionalVector[T] extends Transactional with RandomAccessSeq
* @author Jonas Bonér
*/
class InMemoryTransactionalVector[T] extends TransactionalVector[T] {
- private[kernel] var state: Vector[T] = EmptyVector
- private[kernel] var snapshot = state
+ private[kernel] var state = new Ref[Vector[T]](EmptyVector)
def add(elem: T) = {
- verifyTransaction
- state = state + elem
+ state.get.set(elem)
}
def get(index: Int): T = {
- verifyTransaction
+
+
state(index)
}
@@ -427,55 +393,52 @@ class CassandraPersistentTransactionalVector extends PersistentTransactionalVect
*
* @author Jonas Bonér
*/
-class TransactionalRef[T] extends Transactional {
- private[kernel] var ref: Option[T] = None
- private[kernel] var snapshot: Option[T] = None
+class TransactionalRef[T](elem: T) extends Transactional {
+ def this() = this(null)
- override def begin = if (ref.isDefined) snapshot = Some(ref.get)
+ private[kernel] val ref = new Ref[T](elem)
- override def commit = if (ref.isDefined) snapshot = Some(ref.get)
-
- override def rollback = if (snapshot.isDefined) ref = Some(snapshot.get)
-
- def swap(elem: T) = {
- verifyTransaction
- ref = Some(elem)
- }
+ def swap(elem: T) = ref.set(elem)
def get: Option[T] = {
- verifyTransaction
- ref
+ if (ref.isNull) None
+ else Some(ref.get)
}
def getOrElse(default: => T): T = {
- verifyTransaction
- ref.getOrElse(default)
+ if (ref.isNull) default
+ else ref.get
}
- def isDefined: Boolean = {
- verifyTransaction
- ref.isDefined
- }
+ def isDefined: Boolean = !ref.isNull)
}
-class CassandraPersistentTransactionalRef extends TransactionalRef[AnyRef] {
- override def commit = if (ref.isDefined) {
+/**
+ * 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 Jonas Bonér
+ */
+class CassandraPersistentTransactionalRef extends TransactionalRef[String] {
+ def commit = if (isDefined) {
CassandraStorage.insertRefStorageFor(uuid, ref.get)
- ref = None
+ ref.clear
}
- override def rollback = ref = None
+ def rollback = ref.clear
override def get: Option[AnyRef] = {
verifyTransaction
- CassandraStorage.getRefStorageFor(uuid)
+ if (isDefined) super.get
+ else CassandraStorage.getRefStorageFor(uuid)
}
override def isDefined: Boolean = get.isDefined
override def getOrElse(default: => AnyRef): AnyRef = {
val ref = get
- if (ref.isDefined) ref
+ if (ref.isDefined) ref.get
else default
}
-}
\ No newline at end of file
+}
diff --git a/kernel/src/main/scala/stm/Transaction.scala b/kernel/src/main/scala/stm/Transaction.scala
index b72eb7c2b2..32aefa8a13 100644
--- a/kernel/src/main/scala/stm/Transaction.scala
+++ b/kernel/src/main/scala/stm/Transaction.scala
@@ -4,9 +4,176 @@
package se.scalablesolutions.akka.kernel.stm
-import java.util.concurrent.atomic.{AtomicInteger, AtomicLong}
import kernel.state.Transactional
import kernel.util.Logging
+
+import org.multiverse.api.Transaction
+import org.multiverse.standard.DefaultStm
+import org.multiverse.utils.GlobalStmInstance
+import org.multiverse.utils.TransactionThreadLocal._
+
+
+/**
+ * @author Jonas Bonér
+ */
+object STM {
+ val stm = new DefaultStm
+ GlobalStmInstance.set(stm)
+ setThreadLocalTransaction(null)
+}
+
+
+
+/**
+ * @author Jonas Bonér
+ */
+@serializable class Transaction extends Logging {
+ log.debug("Creating a new transaction with id [%s]", id)
+
+
+ @volatile private[this] var status: TransactionStatus = TransactionStatus.New
+
+ private[this] val transactionalItems = new ChangeSet
+
+ private[this] var participants: List[String] = Nil
+ private[this] var precommitted: List[String] = Nil
+
+ private[this] val depth = new AtomicInteger(0)
+
+ def increment = synchronized { depth.incrementAndGet }
+ def decrement = synchronized { depth.decrementAndGet }
+ def isTopLevel = synchronized { depth.get == 0 }
+
+ def register(transactional: Transactional) = synchronized {
+ ensureIsActiveOrNew
+ transactionalItems + transactional
+ }
+
+ def begin(participant: String) = synchronized {
+ ensureIsActiveOrNew
+ if (status == TransactionStatus.New) log.debug("TX BEGIN - Server with UUID [%s] is starting NEW transaction [%s]", participant, toString)
+ else log.debug("Server [%s] is participating in transaction", participant)
+ participants ::= participant
+ status = TransactionStatus.Active
+ }
+
+ def precommit(participant: String) = synchronized {
+ if (status == TransactionStatus.Active) {
+ log.debug("TX PRECOMMIT - Pre-committing transaction [%s] for server with UUID [%s]", toString, participant)
+ precommitted ::= participant
+ }
+ }
+
+ def commit(participant: String): Boolean = synchronized {
+ if (status == TransactionStatus.Active) {
+ log.debug("TX COMMIT - Committing transaction [%s] for server with UUID [%s]", toString, participant)
+ val haveAllPreCommitted =
+ if (participants.size == precommitted.size) {{
+ for (part <- participants) yield {
+ if (precommitted.exists(_ == part)) true
+ else false
+ }}.exists(_ == true)
+ } else false
+ if (haveAllPreCommitted) {
+ transactionalItems.items.foreach(_.commit)
+ status = TransactionStatus.Completed
+ reset
+ true
+ } else false
+ } else {
+ reset
+ true
+ }
+ }
+
+ def rollback(participant: String) = synchronized {
+ ensureIsActiveOrAborted
+ log.debug("TX ROLLBACK - Server with UUID [%s] has initiated transaction rollback for [%s]", participant, toString)
+ transactionalItems.items.foreach(_.rollback)
+ status = TransactionStatus.Aborted
+ reset
+ }
+
+ def rollbackForRescheduling(participant: String) = synchronized {
+ ensureIsActiveOrAborted
+ log.debug("TX ROLLBACK for recheduling - Server with UUID [%s] has initiated transaction rollback for [%s]", participant, toString)
+ transactionalItems.items.foreach(_.rollback)
+ reset
+ }
+
+ def join(participant: String) = synchronized {
+ ensureIsActive
+ log.debug("TX JOIN - Server with UUID [%s] is joining transaction [%s]" , participant, toString)
+ participants ::= participant
+ }
+
+ def isNew = status == TransactionStatus.New
+ def isActive = status == TransactionStatus.Active
+ def isCompleted = status == TransactionStatus.Completed
+ def isAborted = status == TransactionStatus.Aborted
+
+ private def reset = {
+ transactionalItems.clear
+ participants = Nil
+ precommitted = Nil
+ }
+
+ private def ensureIsActive = if (status != TransactionStatus.Active)
+ throw new IllegalStateException("Expected ACTIVE transaction - current status [" + status + "]: " + toString)
+
+ private def ensureIsActiveOrAborted = if (!(status == TransactionStatus.Active || status == TransactionStatus.Aborted))
+ throw new IllegalStateException("Expected ACTIVE or ABORTED transaction - current status [" + status + "]: " + toString)
+
+ private def ensureIsActiveOrNew = if (!(status == TransactionStatus.Active || status == TransactionStatus.New))
+ throw new IllegalStateException("Expected ACTIVE or NEW transaction - current status [" + status + "]: " + toString)
+
+ // For reinitialize transaction after sending it over the wire
+ private[kernel] def reinit = synchronized {
+ import net.lag.logging.{Logger, Level}
+ if (log == null) {
+ log = Logger.get(this.getClass.getName)
+ log.setLevel(Level.ALL)
+ }
+ }
+
+ override def equals(that: Any): Boolean = synchronized {
+ that != null &&
+ that.isInstanceOf[Transaction] &&
+ that.asInstanceOf[Transaction].id == this.id
+ }
+
+ override def hashCode(): Int = id.toInt
+
+ override def toString(): String = synchronized {
+ "Transaction[" + id + ", " + status + "]"
+ }
+}
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
@serializable sealed abstract class TransactionStatus
object TransactionStatus {
@@ -20,18 +187,17 @@ object TransactionStatus {
* Represents a snapshot of the current invocation.
*
* @author Jonas Bonér
- */
object TransactionIdFactory {
// FIXME: will not work in distributed env
private val currentId = new AtomicLong(0L)
def newId = currentId.getAndIncrement
}
+ */
/**
* Represents a snapshot of the current invocation.
*
* @author Jonas Bonér
- */
@serializable class Transaction extends Logging {
val id = TransactionIdFactory.newId
@@ -154,3 +320,4 @@ object TransactionIdFactory {
"Transaction[" + id + ", " + status + "]"
}
}
+ */