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 + "]" } } + */