Refactored STM API into Transaction.Global and Transaction.Local, fixes issues with "atomic" outside actors

This commit is contained in:
Jonas Bonér 2010-04-05 11:53:43 +02:00
parent 3ae55214bd
commit f04fbba47b
7 changed files with 171 additions and 44 deletions

View file

@ -8,7 +8,7 @@ import se.scalablesolutions.akka.dispatch._
import se.scalablesolutions.akka.config.Config._ import se.scalablesolutions.akka.config.Config._
import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy} import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy}
import se.scalablesolutions.akka.config.ScalaConfig._ import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.stm.Transaction._ import se.scalablesolutions.akka.stm.Transaction.Global._
import se.scalablesolutions.akka.stm.TransactionManagement._ import se.scalablesolutions.akka.stm.TransactionManagement._
import se.scalablesolutions.akka.stm.TransactionManagement import se.scalablesolutions.akka.stm.TransactionManagement
import se.scalablesolutions.akka.remote.protobuf.RemoteProtocol.RemoteRequest import se.scalablesolutions.akka.remote.protobuf.RemoteProtocol.RemoteRequest

View file

@ -12,7 +12,7 @@ import scala.collection.mutable.HashMap
import se.scalablesolutions.akka.util.Logging import se.scalablesolutions.akka.util.Logging
import org.multiverse.api.{Transaction => MultiverseTransaction} import org.multiverse.api.{Transaction => MultiverseTransaction, TransactionLifecycleListener, TransactionLifecycleEvent}
import org.multiverse.api.GlobalStmInstance.getGlobalStmInstance import org.multiverse.api.GlobalStmInstance.getGlobalStmInstance
import org.multiverse.api.ThreadLocalTransaction._ import org.multiverse.api.ThreadLocalTransaction._
import org.multiverse.templates.{TransactionTemplate, OrElseTemplate} import org.multiverse.templates.{TransactionTemplate, OrElseTemplate}
@ -97,9 +97,76 @@ class TransactionRetryException(message: String) extends RuntimeException(messag
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
object Transaction extends TransactionManagement with Logging { object Transaction {
val idFactory = new AtomicLong(-1L) val idFactory = new AtomicLong(-1L)
/**
* Creates a STM atomic transaction and by-passes all transactions hooks
* such as persistence etc.
*
* Only for internal usage.
*/
private[akka] def atomic0[T](body: => T): T = new TransactionTemplate[T]() {
def execute(mtx: MultiverseTransaction): T = body
}.execute()
object Local extends TransactionManagement with Logging {
/**
* See ScalaDoc on Transaction class.
*/
def map[T](f: => T): T = atomic {f}
/**
* See ScalaDoc on Transaction class.
*/
def flatMap[T](f: => T): T = atomic {f}
/**
* See ScalaDoc on Transaction class.
*/
def foreach(f: => Unit): Unit = atomic {f}
/**
* See ScalaDoc on class.
*/
def atomic[T](body: => T): T = {
new TransactionTemplate[T]() {
def execute(mtx: MultiverseTransaction): T = body
override def onStart(mtx: MultiverseTransaction) = {
val tx = new Transaction
tx.transaction = Some(mtx)
setTransaction(Some(tx))
mtx.registerLifecycleListener(new TransactionLifecycleListener() {
def notify(tx: MultiverseTransaction, event: TransactionLifecycleEvent) = event.name match {
case "postCommit" => tx.commit
case "postAbort" => tx.abort
case _ => {}
}
})
}
}.execute()
}
/**
* See ScalaDoc on class.
*/
def atomically[A](firstBody: => A) = elseBody(firstBody)
/**
* Should only be used together with <code>atomically</code> to form atomically-orElse constructs.
* See ScalaDoc on class.
*/
def elseBody[A](firstBody: => A) = new {
def orElse(secondBody: => A) = new OrElseTemplate[A] {
def run(t: MultiverseTransaction) = firstBody
def orelserun(t: MultiverseTransaction) = secondBody
}.execute()
}
}
object Global extends TransactionManagement with Logging {
/** /**
* See ScalaDoc on Transaction class. * See ScalaDoc on Transaction class.
*/ */
@ -119,7 +186,7 @@ object Transaction extends TransactionManagement with Logging {
* See ScalaDoc on Transaction class. * See ScalaDoc on Transaction class.
*/ */
def atomic[T](body: => T): T = { def atomic[T](body: => T): T = {
var isTopLevelTransaction = true var isTopLevelTransaction = false
new TransactionTemplate[T]() { new TransactionTemplate[T]() {
def execute(mtx: MultiverseTransaction): T = { def execute(mtx: MultiverseTransaction): T = {
val result = body val result = body
@ -154,40 +221,13 @@ object Transaction extends TransactionManagement with Logging {
} }
}.execute() }.execute()
} }
/**
* See ScalaDoc on class.
*/
def atomically[A](firstBody: => A) = elseBody(firstBody)
/**
* Should only be used together with <code>atomically</code> to form atomically-orElse constructs.
* See ScalaDoc on class.
*/
def elseBody[A](firstBody: => A) = new {
def orElse(secondBody: => A) = new OrElseTemplate[A] {
def run(t: MultiverseTransaction) = firstBody
def orelserun(t: MultiverseTransaction) = secondBody
}.execute()
} }
/**
* Creates a STM atomic transaction and by-passes all transactions hooks
* such as persistence etc.
*
* Only for internal usage.
*/
private[akka] def atomic0[T](body: => T): T = new TransactionTemplate[T]() {
def execute(mtx: MultiverseTransaction): T = body
}.execute()
} }
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
@serializable class Transaction extends Logging { @serializable class Transaction extends Logging {
import Transaction._
val id = Transaction.idFactory.incrementAndGet val id = Transaction.idFactory.incrementAndGet
@volatile private[this] var status: TransactionStatus = TransactionStatus.New @volatile private[this] var status: TransactionStatus = TransactionStatus.New
private[akka] var transaction: Option[MultiverseTransaction] = None private[akka] var transaction: Option[MultiverseTransaction] = None
@ -200,7 +240,7 @@ object Transaction extends TransactionManagement with Logging {
def commit = synchronized { def commit = synchronized {
log.trace("Committing transaction %s", toString) log.trace("Committing transaction %s", toString)
atomic0 { Transaction.atomic0 {
persistentStateMap.valuesIterator.foreach(_.commit) persistentStateMap.valuesIterator.foreach(_.commit)
} }
status = TransactionStatus.Completed status = TransactionStatus.Completed

View file

@ -4,6 +4,8 @@
package se.scalablesolutions.akka.stm package se.scalablesolutions.akka.stm
import se.scalablesolutions.akka.util.Logging
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import org.multiverse.api.ThreadLocalTransaction._ import org.multiverse.api.ThreadLocalTransaction._
@ -49,9 +51,10 @@ object TransactionManagement extends TransactionManagement {
} }
} }
trait TransactionManagement { trait TransactionManagement extends Logging {
private[akka] def createNewTransactionSet: CountDownCommitBarrier = { private[akka] def createNewTransactionSet: CountDownCommitBarrier = {
log.trace("Creating new transaction set")
val txSet = new CountDownCommitBarrier(1, TransactionManagement.FAIR_TRANSACTIONS) val txSet = new CountDownCommitBarrier(1, TransactionManagement.FAIR_TRANSACTIONS)
TransactionManagement.transactionSet.set(Some(txSet)) TransactionManagement.transactionSet.set(Some(txSet))
txSet txSet
@ -63,9 +66,13 @@ trait TransactionManagement {
private[akka] def setTransaction(tx: Option[Transaction]) = private[akka] def setTransaction(tx: Option[Transaction]) =
if (tx.isDefined) TransactionManagement.transaction.set(tx) if (tx.isDefined) TransactionManagement.transaction.set(tx)
private[akka] def clearTransactionSet = TransactionManagement.transactionSet.set(None) private[akka] def clearTransactionSet = {
log.trace("Clearing transaction set")
TransactionManagement.transactionSet.set(None)
}
private[akka] def clearTransaction = { private[akka] def clearTransaction = {
log.trace("Clearing transaction")
TransactionManagement.transaction.set(None) TransactionManagement.transaction.set(None)
setThreadLocalTransaction(null) setThreadLocalTransaction(null)
} }

View file

@ -4,7 +4,6 @@
package se.scalablesolutions.akka.stm package se.scalablesolutions.akka.stm
import se.scalablesolutions.akka.stm.Transaction.atomic
import se.scalablesolutions.akka.util.UUID import se.scalablesolutions.akka.util.UUID
import org.multiverse.stms.alpha.AlphaRef import org.multiverse.stms.alpha.AlphaRef

View file

@ -2,7 +2,7 @@ package se.scalablesolutions.akka.actor
import _root_.java.util.concurrent.TimeUnit import _root_.java.util.concurrent.TimeUnit
import se.scalablesolutions.akka.actor.Actor.transactor import se.scalablesolutions.akka.actor.Actor.transactor
import se.scalablesolutions.akka.stm.Transaction.atomic import se.scalablesolutions.akka.stm.Transaction.Global.atomic
import se.scalablesolutions.akka.util.Logging import se.scalablesolutions.akka.util.Logging
import org.scalatest.Suite import org.scalatest.Suite

View file

@ -0,0 +1,81 @@
package se.scalablesolutions.akka.actor
import se.scalablesolutions.akka.stm.Transaction.Local._
import se.scalablesolutions.akka.stm._
import org.scalatest.Spec
import org.scalatest.Assertions
import org.scalatest.matchers.ShouldMatchers
import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
@RunWith(classOf[JUnitRunner])
class StmSpec extends
Spec with
ShouldMatchers with
BeforeAndAfterAll {
describe("STM outside actors") {
it("should be able to do multiple consecutive atomic {..} statements") {
lazy val ref = TransactionalState.newRef[Int]
def increment = atomic {
ref.swap(ref.get.getOrElse(0) + 1)
}
def total: Int = atomic {
ref.get.getOrElse(0)
}
increment
increment
increment
total should equal(3)
}
it("should be able to do nested atomic {..} statements") {
lazy val ref = TransactionalState.newRef[Int]
def increment = atomic {
ref.swap(ref.get.getOrElse(0) + 1)
}
def total: Int = atomic {
ref.get.getOrElse(0)
}
atomic {
increment
increment
}
atomic {
increment
total should equal(3)
}
}
it("should roll back failing nested atomic {..} statements") {
lazy val ref = TransactionalState.newRef[Int]
def increment = atomic {
ref.swap(ref.get.getOrElse(0) + 1)
}
def total: Int = atomic {
ref.get.getOrElse(0)
}
try {
atomic {
increment
increment
throw new Exception
}
} catch {
case e => {}
}
total should equal(0)
}
}
}

View file

@ -10,7 +10,7 @@ import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor, RemoteActor}
import se.scalablesolutions.akka.remote.{RemoteNode, RemoteClient} import se.scalablesolutions.akka.remote.{RemoteNode, RemoteClient}
import se.scalablesolutions.akka.persistence.common.PersistentVector import se.scalablesolutions.akka.persistence.common.PersistentVector
import se.scalablesolutions.akka.persistence.redis.RedisStorage import se.scalablesolutions.akka.persistence.redis.RedisStorage
import se.scalablesolutions.akka.stm.Transaction._ import se.scalablesolutions.akka.stm.Transaction.Global._
import se.scalablesolutions.akka.config.ScalaConfig._ import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.config.OneForOneStrategy import se.scalablesolutions.akka.config.OneForOneStrategy
import se.scalablesolutions.akka.util.Logging import se.scalablesolutions.akka.util.Logging