fixed a bunch of persistence bugs

This commit is contained in:
jboner 2009-10-06 00:07:27 +02:00
parent 5b8b46d21c
commit 059502b463
25 changed files with 674 additions and 572 deletions

View file

@ -146,24 +146,4 @@
<scope>test</scope>
</dependency>
</dependencies>
<build>
<resources>
<resource>
<filtering>false</filtering>
<directory>../config</directory>
<includes>
<include>akka.conf</include>
<include>akka-reference.conf</include>
</includes>
</resource>
<resource>
<filtering>false</filtering>
<directory>src/main/resources</directory>
<includes>
<include>META-INF/*</include>
</includes>
</resource>
</resources>
</build>
</project>

View file

@ -4,21 +4,22 @@
package se.scalablesolutions.akka.actor
import com.google.protobuf.ByteString
import java.net.InetSocketAddress
import java.util.HashSet
import reactor._
import config.ScalaConfig._
import stm.TransactionManagement
import nio.protobuf.RemoteProtocol.RemoteRequest
import nio.{RemoteProtocolBuilder, RemoteClient, RemoteServer, RemoteRequestIdFactory}
import serialization.{Serializer, Serializable, SerializationProtocol}
import util.Helpers.ReadWriteLock
import util.Logging
import se.scalablesolutions.akka.Config._
import se.scalablesolutions.akka.reactor._
import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.stm.TransactionManagement._
import se.scalablesolutions.akka.stm.TransactionManagement
import se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest
import se.scalablesolutions.akka.nio.{RemoteProtocolBuilder, RemoteClient, RemoteRequestIdFactory}
import se.scalablesolutions.akka.serialization.Serializer
import se.scalablesolutions.akka.util.Helpers.ReadWriteLock
import se.scalablesolutions.akka.util.Logging
import org.multiverse.utils.TransactionThreadLocal._
import org.multiverse.api.exceptions.StmException
sealed abstract class LifecycleMessage
case class Init(config: AnyRef) extends LifecycleMessage
@ -46,7 +47,6 @@ class ActorMessageInvoker(val actor: Actor) extends MessageInvoker {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Actor {
import Config._
val TIMEOUT = config.getInt("akka.actor.timeout", 5000)
val SERIALIZE_MESSAGES = config.getBool("akka.actor.serialize-messages", false)
}
@ -68,7 +68,7 @@ trait Actor extends Logging with TransactionManagement {
@volatile protected[this] var remoteAddress: Option[InetSocketAddress] = None
@volatile protected[akka] var supervisor: Option[Actor] = None
protected[Actor] var mailbox: MessageQueue = _
protected[akka] var mailbox: MessageQueue = _
protected[this] var senderFuture: Option[CompletableFutureResult] = None
protected[this] val linkedActors = new HashSet[Actor]
protected[actor] var lifeCycleConfig: Option[LifeCycle] = None
@ -443,9 +443,8 @@ trait Actor extends Logging with TransactionManagement {
RemoteProtocolBuilder.setMessage(message, requestBuilder)
RemoteClient.clientFor(remoteAddress.get).send(requestBuilder.build)
} else {
val handle = new MessageInvocation(this, message, None, TransactionManagement.threadBoundTx.get)
mailbox.append(handle)
latestMessage = Some(handle)
val handle = new MessageInvocation(this, message, None, currentTransaction.get)
handle.send
}
}
@ -466,9 +465,8 @@ trait Actor extends Logging with TransactionManagement {
else throw new IllegalStateException("Expected a future from remote call to actor " + toString)
} else {
val future = new DefaultCompletableFutureResult(timeout)
val handle = new MessageInvocation(this, message, Some(future), TransactionManagement.threadBoundTx.get)
mailbox.append(handle)
latestMessage = Some(handle)
val handle = new MessageInvocation(this, message, Some(future), currentTransaction.get)
handle.send
future
}
}
@ -483,7 +481,7 @@ trait Actor extends Logging with TransactionManagement {
private def dispatch[T](messageHandle: MessageInvocation) = {
if (messageHandle.tx.isDefined) {
TransactionManagement.threadBoundTx.set(messageHandle.tx)
currentTransaction.set(messageHandle.tx)
setThreadLocalTransaction(messageHandle.tx.get.transaction)
}
val message = messageHandle.message //serializeMessage(messageHandle.message)
@ -499,14 +497,14 @@ trait Actor extends Logging with TransactionManagement {
if (future.isDefined) future.get.completeWithException(this, e)
else e.printStackTrace
} finally {
TransactionManagement.threadBoundTx.set(None)
currentTransaction.set(None)
setThreadLocalTransaction(null)
}
}
private def transactionalDispatch[T](messageHandle: MessageInvocation) = {
if (messageHandle.tx.isDefined) {
TransactionManagement.threadBoundTx.set(messageHandle.tx)
currentTransaction.set(messageHandle.tx)
setThreadLocalTransaction(messageHandle.tx.get.transaction)
}
@ -514,35 +512,59 @@ trait Actor extends Logging with TransactionManagement {
val future = messageHandle.future
try {
if (!tryToCommitTransaction && isTransactionTopLevel) handleCollision
tryToCommitTransactions
if (TransactionManagement.threadBoundTx.get.isDefined && !TransactionManagement.threadBoundTx.get.get.isActive) {
TransactionManagement.threadBoundTx.set(None) // need to clear threadBoundTx before call to supervisor
if (currentTransaction.get.isDefined && !currentTransaction.get.get.isActive) {
currentTransaction.set(None) // need to clear currentTransaction before call to supervisor
setThreadLocalTransaction(null)
}
if (isInExistingTransaction) joinExistingTransaction
else if (isTransactional) startNewTransaction
else if (isTransactional) startNewTransaction(messageHandle)
incrementTransaction
senderFuture = future
if (base.isDefinedAt(message)) base(message) // invoke user actor's receive partial function
else throw new IllegalArgumentException("No handler matching message [" + message + "] in " + toString)
decrementTransaction
} catch {
case e =>
rollback(activeTx)
TransactionManagement.threadBoundTx.set(None) // need to clear threadBoundTx before call to supervisor
setThreadLocalTransaction(null)
// FIXME to fix supervisor restart of remote actor for oneway calls, inject a supervisor proxy that can send notification back to client
if (supervisor.isDefined) supervisor.get ! Exit(this, e)
case e: StmException =>
e.printStackTrace
decrementTransaction
val tx = currentTransaction.get
rollback(tx)
if (!(tx.isDefined && tx.get.isTopLevel)) {
val done = tx.get.retry
if (done) {
if (future.isDefined) future.get.completeWithException(this, e)
else e.printStackTrace
} finally {
}
}
currentTransaction.set(None) // need to clear currentTransaction before call to supervisor
setThreadLocalTransaction(null)
case e =>
e.printStackTrace
decrementTransaction
if (isTransactionAborted) removeTransactionIfTopLevel
else tryToPrecommitTransaction
rescheduleClashedMessages
TransactionManagement.threadBoundTx.set(None)
val tx = currentTransaction.get
rollback(tx)
if (future.isDefined) future.get.completeWithException(this, e)
else e.printStackTrace
currentTransaction.set(None) // need to clear currentTransaction before call to supervisor
setThreadLocalTransaction(null)
// FIXME to fix supervisor restart of remote actor for oneway calls, inject a supervisor proxy that can send notification back to client
if (supervisor.isDefined) supervisor.get ! Exit(this, e)
} finally {
if (currentTransaction.get.isDefined && currentTransaction.get.get.isAborted) removeTransactionIfTopLevel(currentTransaction.get.get)
else tryToPrecommitTransactions
currentTransaction.set(None)
setThreadLocalTransaction(null)
}
}
@ -551,14 +573,6 @@ trait Actor extends Logging with TransactionManagement {
if (future.exception.isDefined) throw future.exception.get._2
else future.result.asInstanceOf[Option[T]]
private def rescheduleClashedMessages = if (messageToReschedule.isDefined) {
val handle = messageToReschedule.get
val newTx = startNewTransaction
val clone = new MessageInvocation(handle.sender, handle.message, handle.future, newTx)
log.debug("Rescheduling message %s", clone)
mailbox.append(clone) // FIXME append or prepend rescheduled messages?
}
private def base: PartialFunction[Any, Unit] = lifeCycle orElse (hotswap getOrElse receive)
private val lifeCycle: PartialFunction[Any, Unit] = {
@ -588,7 +602,7 @@ trait Actor extends Logging with TransactionManagement {
private[Actor] def restart(reason: AnyRef) = synchronized {
lifeCycleConfig match {
case None => throw new IllegalStateException("Server [" + id + "] does not have a life-cycle defined.")
case None => throw new IllegalStateException("Actor [" + id + "] does not have a life-cycle defined.")
// FIXME implement support for shutdown time
case Some(LifeCycle(scope, shutdownTime, _)) => {
@ -605,10 +619,10 @@ trait Actor extends Logging with TransactionManagement {
// log.debug("Restarting actor [%s] configured as TEMPORARY (since exited naturally).", id)
// scheduleRestart
// } else
log.info("Server [%s] configured as TEMPORARY will not be restarted (received unnatural exit message).", id)
log.info("Actor [%s] configured as TEMPORARY will not be restarted (received unnatural exit message).", id)
case Transient =>
log.info("Server [%s] configured as TRANSIENT will not be restarted.", id)
log.info("Actor [%s] configured as TRANSIENT will not be restarted.", id)
}
}
}
@ -644,7 +658,7 @@ trait Actor extends Logging with TransactionManagement {
!message.isInstanceOf[Tuple6[_,_,_,_,_,_]] &&
!message.isInstanceOf[Tuple7[_,_,_,_,_,_,_]] &&
!message.isInstanceOf[Tuple8[_,_,_,_,_,_,_,_]] &&
!message.isInstanceOf[Array[_]] &&
!message.getClass.isArray &&
!message.isInstanceOf[List[_]] &&
!message.isInstanceOf[scala.collection.immutable.Map[_,_]] &&
!message.isInstanceOf[scala.collection.immutable.Set[_]] &&

View file

@ -8,11 +8,10 @@ import java.lang.reflect.InvocationTargetException
import java.net.InetSocketAddress
import java.util.concurrent.{ConcurrentHashMap, Executors}
import actor._
import util._
import protobuf.RemoteProtocol
import protobuf.RemoteProtocol.{RemoteReply, RemoteRequest}
import serialization.{Serializer, Serializable, SerializationProtocol}
import se.scalablesolutions.akka.actor._
import se.scalablesolutions.akka.util._
import se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.{RemoteReply, RemoteRequest}
import se.scalablesolutions.akka.Config.config
import org.jboss.netty.bootstrap.ServerBootstrap
import org.jboss.netty.channel._
@ -31,7 +30,6 @@ class RemoteServer extends Logging {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object RemoteServer extends Logging {
import Config.config
val HOSTNAME = config.getString("akka.remote.hostname", "localhost")
val PORT = config.getInt("akka.remote.port", 9999)
val CONNECTION_TIMEOUT_MILLIS = config.getInt("akka.remote.connection-timeout", 1000)
@ -70,7 +68,7 @@ class RemoteServerPipelineFactory(name: String, loader: Option[ClassLoader]) ext
def getPipeline: ChannelPipeline = {
val p = Channels.pipeline()
p.addLast("frameDecoder", new LengthFieldBasedFrameDecoder(1048576, 0, 4, 0, 4))
p.addLast("protobufDecoder", new ProtobufDecoder(RemoteProtocol.RemoteRequest.getDefaultInstance))
p.addLast("protobufDecoder", new ProtobufDecoder(RemoteRequest.getDefaultInstance))
p.addLast("frameEncoder", new LengthFieldPrepender(4))
p.addLast("protobufEncoder", new ProtobufEncoder)
p.addLast("handler", new RemoteServerHandler(name, loader))
@ -234,6 +232,7 @@ class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassL
val activeObjectOrNull = activeObjects.get(name)
if (activeObjectOrNull == null) {
try {
log.info("Creating a new remote active object [%s]", name)
val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name)
else Class.forName(name)
val newInstance = activeObjectFactory.newInstance(clazz, timeout).asInstanceOf[AnyRef]
@ -252,6 +251,7 @@ class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassL
val actorOrNull = actors.get(name)
if (actorOrNull == null) {
try {
log.info("Creating a new remote actor [%s]", name)
val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name)
else Class.forName(name)
val newInstance = clazz.newInstance.asInstanceOf[Actor]

View file

@ -26,7 +26,7 @@ class EventBasedSingleThreadDispatcher(name: String) extends MessageDispatcherBa
val iter = selectedInvocations.iterator
while (iter.hasNext) {
val invocation = iter.next
val invoker = messageHandlers.get(invocation.sender)
val invoker = messageHandlers.get(invocation.receiver)
if (invoker != null) invoker.invoke(invocation)
iter.remove
}

View file

@ -97,7 +97,7 @@ class EventBasedThreadPoolDispatcher(name: String, private val concurrentMode: B
threadPoolBuilder.execute(new Runnable() {
def run = {
invoker.invoke(invocation)
free(invocation.sender)
free(invocation.receiver)
messageDemultiplexer.wakeUp
}
})
@ -119,16 +119,16 @@ class EventBasedThreadPoolDispatcher(name: String, private val concurrentMode: B
while (iterator.hasNext) {
val invocation = iterator.next
if (concurrentMode) {
val invoker = messageHandlers.get(invocation.sender)
val invoker = messageHandlers.get(invocation.receiver)
if (invocation == null) throw new IllegalStateException("Message invocation is null [" + invocation + "]")
if (invoker == null) throw new IllegalStateException("Message invoker for invocation [" + invocation + "] is null")
result.put(invocation, invoker)
} else if (!busyInvokers.contains(invocation.sender)) {
val invoker = messageHandlers.get(invocation.sender)
} else if (!busyInvokers.contains(invocation.receiver)) {
val invoker = messageHandlers.get(invocation.receiver)
if (invocation == null) throw new IllegalStateException("Message invocation is null [" + invocation + "]")
if (invoker == null) throw new IllegalStateException("Message invoker for invocation [" + invocation + "] is null")
result.put(invocation, invoker)
busyInvokers.add(invocation.sender)
busyInvokers.add(invocation.receiver)
iterator.remove
}
}

View file

@ -5,8 +5,12 @@
package se.scalablesolutions.akka.reactor
import java.util.List
import stm.Transaction
import util.HashCode
import se.scalablesolutions.akka.util.HashCode
import se.scalablesolutions.akka.stm.Transaction
import se.scalablesolutions.akka.actor.Actor
import java.util.concurrent.atomic.AtomicInteger
trait MessageQueue {
def append(handle: MessageInvocation)
@ -32,23 +36,35 @@ trait MessageDemultiplexer {
def wakeUp
}
class MessageInvocation(val sender: AnyRef,
class MessageInvocation(val receiver: Actor,
val message: AnyRef,
val future: Option[CompletableFutureResult],
val tx: Option[Transaction]) {
if (receiver == null) throw new IllegalArgumentException("receiver is null")
if (message == null) throw new IllegalArgumentException("message is null")
override def hashCode(): Int = {
private [akka] val nrOfDeliveryAttempts = new AtomicInteger(0)
def send = synchronized {
receiver.mailbox.append(this)
nrOfDeliveryAttempts.incrementAndGet
}
override def hashCode(): Int = synchronized {
var result = HashCode.SEED
result = HashCode.hash(result, sender)
result = HashCode.hash(result, receiver)
result = HashCode.hash(result, message)
result
}
override def equals(that: Any): Boolean =
override def equals(that: Any): Boolean = synchronized {
that != null &&
that.isInstanceOf[MessageInvocation] &&
that.asInstanceOf[MessageInvocation].sender == sender &&
that.asInstanceOf[MessageInvocation].receiver == receiver &&
that.asInstanceOf[MessageInvocation].message == message
override def toString(): String = "MessageInvocation[message = " + message + ", sender = " + sender + ", future = " + future + ", tx = " + tx + "]"
}
override def toString(): String = synchronized {
"MessageInvocation[message = " + message + ", receiver = " + receiver + ", future = " + future + ", tx = " + tx + "]"
}
}

View file

@ -4,10 +4,12 @@
package se.scalablesolutions.akka.stm
import java.util.concurrent.atomic.{AtomicInteger, AtomicLong}
import state.Transactional
import util.Logging
import actor.Actor
import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.atomic.AtomicInteger
import se.scalablesolutions.akka.reactor.MessageInvocation
import se.scalablesolutions.akka.util.Logging
import se.scalablesolutions.akka.state.Committable
import org.multiverse.api.{Transaction => MultiverseTransaction}
import org.multiverse.stms.alpha.AlphaStm
@ -15,7 +17,9 @@ import org.multiverse.utils.GlobalStmInstance
import org.multiverse.utils.TransactionThreadLocal._
import org.multiverse.templates.{OrElseTemplate, AtomicTemplate}
import java.util.concurrent.atomic.AtomicInteger
import scala.collection.mutable.HashMap
class TransactionRetryException(message: String) extends RuntimeException(message)
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
@ -75,28 +79,29 @@ object Transaction {
@volatile private[this] var status: TransactionStatus = TransactionStatus.New
private[akka] var transaction: MultiverseTransaction = _
// private[this] var initMessage: Option[AnyRef] = None
// private[this] var initReceiver: Option[Actor] = None
private[this] var message: Option[MessageInvocation] = None
private[this] var participants: List[String] = Nil
private[this] var precommitted: List[String] = Nil
private[this] val depth = new AtomicInteger(0)
private[this] val persistentStateMap = new HashMap[String, Committable]
private[akka] val depth = new AtomicInteger(0)
def increment = depth.incrementAndGet
def decrement = depth.decrementAndGet
def isTopLevel = depth.compareAndSet(0, 0)
def isTopLevel = depth.get == 0
def begin(participant: String) = synchronized {
// def begin(participant: String, message, receiver) = synchronized {
def register(uuid: String, storage: Committable) = persistentStateMap.put(uuid, storage)
def begin(participant: String, msg: MessageInvocation) = synchronized {
ensureIsActiveOrNew
// initMessage = Some(message)
// initReceiver = Some(receiver)
message = Some(msg)
transaction = Multiverse.STM.startUpdateTransaction("akka")
log.debug("Creating a new transaction with id [%s]", id)
log.debug("TX BEGIN - Creating a new transaction with id [%s]", id)
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)
if (status == TransactionStatus.New) log.debug("TX BEGIN - Actor with UUID [%s] is starting NEW transaction [%s]", participant, toString)
else log.debug("Actor [%s] is participating in transaction", participant)
participants ::= participant
status = TransactionStatus.Active
}
@ -109,8 +114,9 @@ object Transaction {
}
def commit(participant: String): Boolean = synchronized {
log.debug("TX COMMIT - Trying to commit transaction [%s] for server with UUID [%s]", toString, participant)
setThreadLocalTransaction(transaction)
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 {
@ -119,50 +125,62 @@ object Transaction {
}}.exists(_ == true)
} else false
if (haveAllPreCommitted && transaction != null) {
log.debug("TX COMMIT - Committing transaction [%s] for server with UUID [%s]", toString, participant)
transaction.commit
transaction.reset
status = TransactionStatus.Completed
reset
status = TransactionStatus.Completed
Transaction.Atomic {
persistentStateMap.values.foreach(_.commit)
}
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)
log.debug("TX ROLLBACK - Actor with UUID [%s] has initiated transaction rollback for [%s]", participant, toString)
status = TransactionStatus.Aborted
transaction.abort
transaction.reset
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)
log.debug("TX ROLLBACK for recheduling - Actor with UUID [%s] has initiated transaction rollback for [%s]", participant, toString)
transaction.abort
reset
}
def join(participant: String) = synchronized {
ensureIsActive
log.debug("TX JOIN - Server with UUID [%s] is joining transaction [%s]" , participant, toString)
log.debug("TX JOIN - Actor with UUID [%s] is joining transaction [%s]" , participant, toString)
participants ::= participant
}
def retry: Boolean = synchronized {
println("----- 2 " + message.isDefined)
println("----- 3 " + message.get.nrOfDeliveryAttempts)
if (message.isDefined && message.get.nrOfDeliveryAttempts.get < TransactionManagement.MAX_NR_OF_RETRIES) {
log.debug("TX RETRY - Restarting transaction [%s] resending message [%s]", transaction, message.get)
message.get.send
true
} else false
}
private def reset = synchronized {
transaction.reset
participants = Nil
precommitted = Nil
}
def isNew = synchronized { status == TransactionStatus.New }
def isActive = synchronized { status == TransactionStatus.Active }
def isCompleted = synchronized { status == TransactionStatus.Completed }
def isAborted = synchronized { status == TransactionStatus.Aborted }
private def reset = synchronized {
participants = Nil
precommitted = Nil
}
private def ensureIsActive = if (status != TransactionStatus.Active)
throw new IllegalStateException("Expected ACTIVE transaction - current status [" + status + "]: " + toString)

View file

@ -6,9 +6,14 @@ package se.scalablesolutions.akka.stm
import java.util.concurrent.atomic.AtomicBoolean
import reactor.MessageInvocation
import util.Logging
import org.codehaus.aspectwerkz.proxy.Uuid // FIXME is java.util.UUID better?
import se.scalablesolutions.akka.reactor.MessageInvocation
import se.scalablesolutions.akka.util.Logging
import org.codehaus.aspectwerkz.proxy.Uuid
import scala.collection.mutable.HashSet
// FIXME is java.util.UUID better?
import org.multiverse.utils.TransactionThreadLocal._
@ -19,9 +24,10 @@ class TransactionAwareWrapperException(val cause: Throwable, val tx: Option[Tran
}
object TransactionManagement {
import Config._
import se.scalablesolutions.akka.Config._
val TIME_WAITING_FOR_COMPLETION = config.getInt("akka.stm.wait-for-completion", 100)
val NR_OF_TIMES_WAITING_FOR_COMPLETION = config.getInt("akka.stm.wait-nr-of-times", 3)
val MAX_NR_OF_RETRIES = config.getInt("akka.stm.max-nr-of-retries", 10)
val TRANSACTION_ENABLED = new AtomicBoolean(config.getBool("akka.stm.service", false))
// FIXME reenable 'akka.stm.restart-on-collision' when new STM is in place
val RESTART_TRANSACTION_ON_COLLISION = false //akka.Kernel.config.getBool("akka.stm.restart-on-collision", true)
@ -29,7 +35,7 @@ object TransactionManagement {
def isTransactionalityEnabled = TRANSACTION_ENABLED.get
def disableTransactions = TRANSACTION_ENABLED.set(false)
private[akka] val threadBoundTx: ThreadLocal[Option[Transaction]] = new ThreadLocal[Option[Transaction]]() {
private[akka] val currentTransaction: ThreadLocal[Option[Transaction]] = new ThreadLocal[Option[Transaction]]() {
override protected def initialValue: Option[Transaction] = None
}
}
@ -37,40 +43,40 @@ object TransactionManagement {
trait TransactionManagement extends Logging {
var uuid = Uuid.newUuid.toString
protected[this] var latestMessage: Option[MessageInvocation] = None
protected[this] var messageToReschedule: Option[MessageInvocation] = None
import TransactionManagement.currentTransaction
private[akka] val activeTransactions = new HashSet[Transaction]
import TransactionManagement.threadBoundTx
private[akka] var activeTx: Option[Transaction] = None
protected def startNewTransaction: Option[Transaction] = {
protected def startNewTransaction(message: MessageInvocation) = {
val newTx = new Transaction
newTx.begin(uuid)
val tx = Some(newTx)
activeTx = tx
threadBoundTx.set(tx)
setThreadLocalTransaction(tx.get.transaction)
tx
newTx.begin(uuid, message)
activeTransactions += newTx
currentTransaction.set(Some(newTx))
setThreadLocalTransaction(newTx.transaction)
}
protected def joinExistingTransaction = {
val cflowTx = threadBoundTx.get
if (!activeTx.isDefined && cflowTx.isDefined) {
val cflowTx = currentTransaction.get
if (activeTransactions.isEmpty && cflowTx.isDefined) {
val currentTx = cflowTx.get
currentTx.join(uuid)
activeTx = Some(currentTx)
activeTransactions += currentTx
}
}
protected def tryToPrecommitTransaction = if (activeTx.isDefined) activeTx.get.precommit(uuid)
protected def tryToPrecommitTransactions = activeTransactions.foreach(_.precommit(uuid))
protected def tryToCommitTransaction: Boolean = if (activeTx.isDefined) {
val tx = activeTx.get
if (tx.commit(uuid)) {
removeTransactionIfTopLevel
true
} else false
} else true
protected def tryToCommitTransactions = {
for (tx <- activeTransactions) {
if (tx.commit(uuid)) activeTransactions -= tx
else if (tx.isTopLevel) {
println("------------ COULD NOT COMMIT -- WAITING OR TIMEOUT? ---------")
//tx.retry
} else {
// continue, try to commit on next received message
// FIXME check if TX hase timed out => throw exception
}
}
}
protected def rollback(tx: Option[Transaction]) = tx match {
case None => {} // no tx; nothing to do
@ -84,39 +90,12 @@ trait TransactionManagement extends Logging {
tx.rollbackForRescheduling(uuid)
}
protected def handleCollision = {
var nrRetries = 0
var failed = true
do {
Thread.sleep(TransactionManagement.TIME_WAITING_FOR_COMPLETION)
nrRetries += 1
log.debug("Pending transaction [%s] not completed, waiting %s milliseconds. Attempt %s", activeTx.get.id, TransactionManagement.TIME_WAITING_FOR_COMPLETION, nrRetries)
failed = !tryToCommitTransaction
} while(nrRetries < TransactionManagement.NR_OF_TIMES_WAITING_FOR_COMPLETION && failed)
if (failed) {
log.debug("Pending transaction [%s] still not completed, aborting and rescheduling message [%s]", activeTx.get.id, latestMessage)
rollback(activeTx)
if (TransactionManagement.RESTART_TRANSACTION_ON_COLLISION) messageToReschedule = Some(latestMessage.get)
else throw new TransactionRollbackException("Conflicting transactions, rolling back transaction for message [" + latestMessage + "]")
}
}
protected def isInExistingTransaction = TransactionManagement.threadBoundTx.get.isDefined
protected def isTransactionTopLevel = activeTx.isDefined && activeTx.get.isTopLevel
protected def isTransactionAborted = activeTx.isDefined && activeTx.get.isAborted
protected def incrementTransaction = if (activeTx.isDefined) activeTx.get.increment
protected def decrementTransaction = if (activeTx.isDefined) activeTx.get.decrement
protected def removeTransactionIfTopLevel = if (isTransactionTopLevel) activeTx = None
protected def reenteringExistingTransaction= if (activeTx.isDefined) {
val cflowTx = threadBoundTx.get
if (cflowTx.isDefined && cflowTx.get.id == activeTx.get.id) false
else true
} else true
protected def isInExistingTransaction = currentTransaction.get.isDefined
protected def incrementTransaction = if (currentTransaction.get.isDefined) currentTransaction.get.get.increment
protected def decrementTransaction = if (currentTransaction.get.isDefined) currentTransaction.get.get.decrement
protected def removeTransactionIfTopLevel(tx: Transaction) = if (tx.isTopLevel) { activeTransactions -= tx }
}

View file

@ -45,7 +45,14 @@ object TransactionalState {
@serializable
trait Transactional {
// FIXME: won't work across the cluster
val uuid = Uuid.newUuid.toString
var uuid = Uuid.newUuid.toString
}
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait Committable {
def commit: Unit
}
/**
@ -71,17 +78,15 @@ class TransactionalRef[T] extends Transactional {
def swap(elem: T) = ref.set(elem)
def get: Option[T] = {
// if (ref.isNull) None
// else
Some(ref.get)
if (ref.isNull) None
else Some(ref.get)
}
def getOrWait: T = ref.getOrAwait
def getOrElse(default: => T): T = {
// if (ref.isNull) default
//else
ref.get
if (ref.isNull) default
else ref.get
}
def isDefined: Boolean = !ref.isNull

View file

@ -298,7 +298,7 @@ class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail
}
override def equals(other: Any) = other match {
case vec: Vector[T] => {
case vec: Vector[_] => {
var back = length == vec.length
var i = 0

View file

@ -8,6 +8,7 @@ import java.util.concurrent.locks.ReentrantLock
import org.junit.{Test, Before}
import org.junit.Assert._
import junit.framework.TestCase
import se.scalablesolutions.akka.actor.Actor
class EventBasedSingleThreadDispatcherTest extends TestCase {
private var threadingIssueDetected: AtomicBoolean = null
@ -51,15 +52,18 @@ class EventBasedSingleThreadDispatcherTest extends TestCase {
internalTestMessagesDispatchedToHandlersAreExecutedInFIFOOrder
}
val key1 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
val key2 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
val key3 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
private def internalTestMessagesDispatchedToTheSameHandlerAreExecutedSequentially: Unit = {
val guardLock = new ReentrantLock
val handleLatch = new CountDownLatch(100)
val key = "key"
val dispatcher = new EventBasedSingleThreadDispatcher("name")
dispatcher.registerHandler(key, new TestMessageHandle(handleLatch))
dispatcher.registerHandler(key1, new TestMessageHandle(handleLatch))
dispatcher.start
for (i <- 0 until 100) {
dispatcher.messageQueue.append(new MessageInvocation(key, new Object, None, None))
dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None))
}
assertTrue(handleLatch.await(5, TimeUnit.SECONDS))
assertFalse(threadingIssueDetected.get)
@ -67,8 +71,6 @@ class EventBasedSingleThreadDispatcherTest extends TestCase {
private def internalTestMessagesDispatchedToDifferentHandlersAreExecutedSequentially: Unit = {
val handleLatch = new CountDownLatch(2)
val key1 = "key1"
val key2 = "key2"
val dispatcher = new EventBasedSingleThreadDispatcher("name")
dispatcher.registerHandler(key1, new TestMessageHandle(handleLatch))
dispatcher.registerHandler(key2, new TestMessageHandle(handleLatch))
@ -81,8 +83,6 @@ class EventBasedSingleThreadDispatcherTest extends TestCase {
private def internalTestMessagesDispatchedToHandlersAreExecutedInFIFOOrder: Unit = {
val handleLatch = new CountDownLatch(200)
val key1 = "key1"
val key2 = "key2"
val dispatcher = new EventBasedSingleThreadDispatcher("name")
dispatcher.registerHandler(key1, new MessageInvoker {
var currentValue = -1;

View file

@ -9,9 +9,13 @@ import org.junit.Before
import org.junit.Test
import org.junit.Assert._
import junit.framework.TestCase
import se.scalablesolutions.akka.actor.Actor
class EventBasedThreadPoolDispatcherTest extends TestCase {
private var threadingIssueDetected: AtomicBoolean = null
val key1 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
val key2 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
val key3 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
@Before
override def setUp = {
@ -36,7 +40,6 @@ class EventBasedThreadPoolDispatcherTest extends TestCase {
private def internalTestMessagesDispatchedToTheSameHandlerAreExecutedSequentially: Unit = {
val guardLock = new ReentrantLock
val handleLatch = new CountDownLatch(10)
val key = "key"
val dispatcher = Dispatchers.newEventBasedThreadPoolDispatcher("name")
dispatcher.withNewThreadPoolWithBoundedBlockingQueue(100)
.setCorePoolSize(2)
@ -44,7 +47,7 @@ class EventBasedThreadPoolDispatcherTest extends TestCase {
.setKeepAliveTimeInMillis(60000)
.setRejectionPolicy(new CallerRunsPolicy)
.buildThreadPool
dispatcher.registerHandler(key, new MessageInvoker {
dispatcher.registerHandler(key1, new MessageInvoker {
def invoke(message: MessageInvocation) {
try {
if (threadingIssueDetected.get) return
@ -64,7 +67,7 @@ class EventBasedThreadPoolDispatcherTest extends TestCase {
})
dispatcher.start
for (i <- 0 until 10) {
dispatcher.messageQueue.append(new MessageInvocation(key, new Object, None, None))
dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None))
}
assertTrue(handleLatch.await(5, TimeUnit.SECONDS))
assertFalse(threadingIssueDetected.get)
@ -74,8 +77,6 @@ class EventBasedThreadPoolDispatcherTest extends TestCase {
val guardLock1 = new ReentrantLock
val guardLock2 = new ReentrantLock
val handlersBarrier = new CyclicBarrier(3)
val key1 = "key1"
val key2 = "key2"
val dispatcher = Dispatchers.newEventBasedThreadPoolDispatcher("name")
dispatcher.withNewThreadPoolWithBoundedBlockingQueue(100)
.setCorePoolSize(2)
@ -119,8 +120,6 @@ class EventBasedThreadPoolDispatcherTest extends TestCase {
private def internalTestMessagesDispatchedToHandlersAreExecutedInFIFOOrder: Unit = {
val handleLatch = new CountDownLatch(200)
val key1 = "key1"
val key2 = "key2"
val dispatcher = Dispatchers.newEventBasedThreadPoolDispatcher("name")
dispatcher.withNewThreadPoolWithBoundedBlockingQueue(100)
.setCorePoolSize(2)

View file

@ -226,6 +226,7 @@ class InMemoryActorSpec extends TestCase {
stateful !! Failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", failer) // call failing transactionrequired method
fail("should have thrown an exception")
} catch {case e: RuntimeException => {}}
println("---------- BACK")
assertEquals("init", (stateful !! GetRefState).get) // check that state is == init state
}
}

View file

@ -8,9 +8,13 @@ import java.util.concurrent.locks.ReentrantLock
import org.junit.{Test, Before}
import org.junit.Assert._
import junit.framework.TestCase
import se.scalablesolutions.akka.actor.Actor
class ThreadBasedDispatcherTest extends TestCase {
private var threadingIssueDetected: AtomicBoolean = null
val key1 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
val key2 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
val key3 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
class TestMessageHandle(handleLatch: CountDownLatch) extends MessageInvoker {
val guardLock: Lock = new ReentrantLock
@ -52,7 +56,7 @@ class ThreadBasedDispatcherTest extends TestCase {
val dispatcher = new ThreadBasedDispatcher("name", new TestMessageHandle(handleLatch))
dispatcher.start
for (i <- 0 until 100) {
dispatcher.messageQueue.append(new MessageInvocation("id", new Object, None, None))
dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None))
}
assertTrue(handleLatch.await(5, TimeUnit.SECONDS))
assertFalse(threadingIssueDetected.get)
@ -73,7 +77,7 @@ class ThreadBasedDispatcherTest extends TestCase {
})
dispatcher.start
for (i <- 0 until 100) {
dispatcher.messageQueue.append(new MessageInvocation("id", new Integer(i), None, None))
dispatcher.messageQueue.append(new MessageInvocation(key1, new Integer(i), None, None))
}
assertTrue(handleLatch.await(5, TimeUnit.SECONDS))
assertFalse(threadingIssueDetected.get)

View file

@ -38,32 +38,37 @@ public class InMemNestedStateTest extends TestCase {
conf.stop();
}
public void testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess() {
public void testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess() throws Exception {
InMemStateful stateful = conf.getInstance(InMemStateful.class);
stateful.setMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "init"); // set init state
InMemStatefulNested nested = conf.getInstance(InMemStatefulNested.class);
nested.setMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "init"); // set init state
stateful.success("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "new state", nested); // transactionrequired
System.out.println("-- BACK --");
assertEquals("new state", stateful.getMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess"));
assertEquals("new state", nested.getMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess"));
}
public void testMapShouldRollbackStateForStatefulServerInCaseOfFailure() {
public void testMapShouldRollbackStateForStatefulServerInCaseOfFailure() throws InterruptedException {
InMemStateful stateful = conf.getInstance(InMemStateful.class);
stateful.setMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure", "init"); // set init state
Thread.sleep(100);
InMemStatefulNested nested = conf.getInstance(InMemStatefulNested.class);
nested.setMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure", "init"); // set init state
Thread.sleep(100);
InMemFailer failer = conf.getInstance(InMemFailer.class);
try {
stateful.failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", nested, failer); // call failing transactionrequired method
Thread.sleep(100);
fail("should have thrown an exception");
} catch (RuntimeException e) {
} // expected
assertEquals("init", stateful.getMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure")); // check that state is == init state
Thread.sleep(100);
assertEquals("init", nested.getMapState("testShouldRollbackStateForStatefulServerInCaseOfFailure")); // check that state is == init state
}
public void testVectorShouldNotRollbackStateForStatefulServerInCaseOfSuccess() {
public void testVectorShouldNotRollbackStateForStatefulServerInCaseOfSuccess() throws Exception {
InMemStateful stateful = conf.getInstance(InMemStateful.class);
stateful.setVectorState("init"); // set init state
InMemStatefulNested nested = conf.getInstance(InMemStatefulNested.class);
@ -73,22 +78,26 @@ public class InMemNestedStateTest extends TestCase {
assertEquals("new state", nested.getVectorState());
}
public void testVectorShouldRollbackStateForStatefulServerInCaseOfFailure() {
public void testVectorShouldRollbackStateForStatefulServerInCaseOfFailure() throws InterruptedException {
InMemStateful stateful = conf.getInstance(InMemStateful.class);
stateful.setVectorState("init"); // set init state
Thread.sleep(100);
InMemStatefulNested nested = conf.getInstance(InMemStatefulNested.class);
nested.setVectorState("init"); // set init state
Thread.sleep(100);
InMemFailer failer = conf.getInstance(InMemFailer.class);
try {
stateful.failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", nested, failer); // call failing transactionrequired method
Thread.sleep(100);
fail("should have thrown an exception");
} catch (RuntimeException e) {
} // expected
assertEquals("init", stateful.getVectorState()); // check that state is == init state
Thread.sleep(100);
assertEquals("init", nested.getVectorState()); // check that state is == init state
}
public void testRefShouldNotRollbackStateForStatefulServerInCaseOfSuccess() {
public void testRefShouldNotRollbackStateForStatefulServerInCaseOfSuccess() throws Exception {
InMemStateful stateful = conf.getInstance(InMemStateful.class);
InMemStatefulNested nested = conf.getInstance(InMemStatefulNested.class);
stateful.setRefState("init"); // set init state
@ -98,18 +107,22 @@ public class InMemNestedStateTest extends TestCase {
assertEquals("new state", nested.getRefState());
}
public void testRefShouldRollbackStateForStatefulServerInCaseOfFailure() {
public void testRefShouldRollbackStateForStatefulServerInCaseOfFailure() throws InterruptedException {
InMemStateful stateful = conf.getInstance(InMemStateful.class);
InMemStatefulNested nested = conf.getInstance(InMemStatefulNested.class);
stateful.setRefState("init"); // set init state
Thread.sleep(100);
nested.setRefState("init"); // set init state
Thread.sleep(100);
InMemFailer failer = conf.getInstance(InMemFailer.class);
try {
stateful.failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", nested, failer); // call failing transactionrequired method
Thread.sleep(100);
fail("should have thrown an exception");
} catch (RuntimeException e) {
} // expected
assertEquals("init", stateful.getRefState()); // check that state is == init state
Thread.sleep(100);
assertEquals("init", nested.getRefState()); // check that state is == init state
}
}

View file

@ -158,7 +158,7 @@
<dependency>
<groupId>org.apache.cassandra</groupId>
<artifactId>cassandra</artifactId>
<version>0.4.0-trunk</version>
<version>0.4.0</version>
</dependency>
<dependency>
<groupId>com.facebook</groupId>
@ -320,22 +320,5 @@
</executions>
</plugin>
</plugins>
<resources>
<resource>
<filtering>false</filtering>
<directory>../config</directory>
<includes>
<include>akka.conf</include>
<include>akka-reference.conf</include>
</includes>
</resource>
<resource>
<filtering>false</filtering>
<directory>src/main/resources</directory>
<includes>
<include>META-INF/*</include>
</includes>
</resource>
</resources>
</build>
</project>

View file

@ -50,19 +50,13 @@ class AkkaServlet extends ServletContainer with AtmosphereServletProcessor with
false
} catch { case e: IllegalStateException => true }
if (isUsingStream)
{
if (data != null)
response.getOutputStream.write(data.getBytes)
if (isUsingStream) {
if (data != null) response.getOutputStream.write(data.getBytes)
response.getOutputStream.flush
}
else
{
if (data != null)
response.getWriter.write(data)
} else {
if (data != null) response.getWriter.write(data)
response.getWriter.flush
}
event
}

View file

@ -5,12 +5,15 @@
package se.scalablesolutions.akka.state
import stm.TransactionManagement
import stm.TransactionManagement.currentTransaction
import akka.collection._
import org.codehaus.aspectwerkz.proxy.Uuid
import scala.collection.mutable.{ArrayBuffer, HashMap}
class NoTransactionInScopeException extends RuntimeException
sealed abstract class PersistentStateConfig
abstract class PersistentStorageConfig extends PersistentStateConfig
case class CassandraStorageConfig extends PersistentStorageConfig
@ -62,7 +65,7 @@ object PersistentState {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait PersistentMap extends scala.collection.mutable.Map[AnyRef, AnyRef] with Transactional {
trait PersistentMap extends scala.collection.mutable.Map[AnyRef, AnyRef] with Transactional with Committable {
protected val newAndUpdatedEntries = TransactionalState.newMap[AnyRef, AnyRef]
protected val removedEntries = TransactionalState.newMap[AnyRef, AnyRef]
protected val shouldClearOnCommit = TransactionalRef[Boolean]()
@ -70,10 +73,10 @@ trait PersistentMap extends scala.collection.mutable.Map[AnyRef, AnyRef] with Tr
// to be concretized in subclasses
val storage: MapStorage
private[akka] def commit = {
def commit = {
storage.removeMapStorageFor(uuid, removedEntries.toList)
storage.insertMapStorageEntriesFor(uuid, newAndUpdatedEntries.toList)
if (shouldClearOnCommit.isDefined & shouldClearOnCommit.get.get) storage.removeMapStorageFor(uuid)
if (shouldClearOnCommit.isDefined && shouldClearOnCommit.get.get) storage.removeMapStorageFor(uuid)
newAndUpdatedEntries.clear
removedEntries.clear
}
@ -82,11 +85,20 @@ trait PersistentMap extends scala.collection.mutable.Map[AnyRef, AnyRef] with Tr
def +=(key: AnyRef, value: AnyRef) = put(key, value)
override def put(key: AnyRef, value: AnyRef): Option[AnyRef] = newAndUpdatedEntries.put(key, value)
override def put(key: AnyRef, value: AnyRef): Option[AnyRef] = {
register
newAndUpdatedEntries.put(key, value)
}
override def update(key: AnyRef, value: AnyRef) = newAndUpdatedEntries.update(key, value)
override def update(key: AnyRef, value: AnyRef) = {
register
newAndUpdatedEntries.update(key, value)
}
def remove(key: AnyRef) = removedEntries.remove(key)
def remove(key: AnyRef) = {
register
removedEntries.remove(key)
}
def slice(start: Option[AnyRef], count: Int): List[Tuple2[AnyRef, AnyRef]] = slice(start, None, count)
@ -94,11 +106,13 @@ trait PersistentMap extends scala.collection.mutable.Map[AnyRef, AnyRef] with Tr
storage.getMapStorageRangeFor(uuid, start, finish, count)
} catch { case e: Exception => Nil }
override def clear = shouldClearOnCommit.swap(true)
override def clear = {
register
shouldClearOnCommit.swap(true)
}
override def contains(key: AnyRef): Boolean = try {
newAndUpdatedEntries.contains(key) ||
storage.getMapStorageEntryFor(uuid, key).isDefined
newAndUpdatedEntries.contains(key) || storage.getMapStorageEntryFor(uuid, key).isDefined
} catch { case e: Exception => false }
override def size: Int = try {
@ -129,6 +143,11 @@ trait PersistentMap extends scala.collection.mutable.Map[AnyRef, AnyRef] with Tr
override def hasNext: Boolean = synchronized { !elements.isEmpty }
}
}
private def register = {
if (currentTransaction.get.isEmpty) throw new NoTransactionInScopeException
currentTransaction.get.get.register(uuid, this)
}
}
/**
@ -154,7 +173,7 @@ class MongoPersistentMap extends PersistentMap {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait PersistentVector extends RandomAccessSeq[AnyRef] with Transactional {
trait PersistentVector extends RandomAccessSeq[AnyRef] with Transactional with Committable {
protected val newElems = TransactionalState.newVector[AnyRef]
protected val updatedElems = TransactionalState.newMap[Int, AnyRef]
protected val removedElems = TransactionalState.newVector[AnyRef]
@ -162,7 +181,7 @@ trait PersistentVector extends RandomAccessSeq[AnyRef] with Transactional {
val storage: VectorStorage
private[akka] def commit = {
def commit = {
// FIXME: should use batch function once the bug is resolved
for (element <- newElems) storage.insertVectorStorageEntryFor(uuid, element)
for (entry <- updatedElems) storage.updateVectorStorageEntryFor(uuid, entry._1, entry._2)
@ -170,9 +189,12 @@ trait PersistentVector extends RandomAccessSeq[AnyRef] with Transactional {
updatedElems.clear
}
def +(elem: AnyRef) = newElems + elem
def +(elem: AnyRef) = add(elem)
def add(elem: AnyRef) = newElems + elem
def add(elem: AnyRef) = {
register
newElems + elem
}
def apply(index: Int): AnyRef = get(index)
@ -193,9 +215,15 @@ trait PersistentVector extends RandomAccessSeq[AnyRef] with Transactional {
* Removes the <i>tail</i> element of this vector.
*/
// FIXME: implement persistent vector pop
def pop: AnyRef = throw new UnsupportedOperationException("need to implement persistent vector pop")
def pop: AnyRef = {
register
throw new UnsupportedOperationException("need to implement persistent vector pop")
}
def update(index: Int, newElem: AnyRef) = storage.updateVectorStorageEntryFor(uuid, index, newElem)
def update(index: Int, newElem: AnyRef) = {
register
storage.updateVectorStorageEntryFor(uuid, index, newElem)
}
override def first: AnyRef = get(0)
@ -209,6 +237,11 @@ trait PersistentVector extends RandomAccessSeq[AnyRef] with Transactional {
}
def length: Int = storage.getVectorStorageSizeFor(uuid) + newElems.length
private def register = {
if (currentTransaction.get.isEmpty) throw new NoTransactionInScopeException
currentTransaction.get.get.register(uuid, this)
}
}
/**
@ -234,17 +267,20 @@ class MongoPersistentVector extends PersistentVector {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait PersistentRef extends Transactional {
trait PersistentRef extends Transactional with Committable {
protected val ref = new TransactionalRef[AnyRef]
val storage: RefStorage
private[akka] def commit = if (ref.isDefined) {
storage.insertRefStorageFor(uuid, ref.get)
def commit = if (ref.isDefined) {
storage.insertRefStorageFor(uuid, ref.get.get)
ref.swap(null)
}
def swap(elem: AnyRef) = ref.swap(elem)
def swap(elem: AnyRef) = {
register
ref.swap(elem)
}
def get: Option[AnyRef] = if (ref.isDefined) ref.get else storage.getRefStorageFor(uuid)
@ -252,9 +288,14 @@ trait PersistentRef extends Transactional {
def getOrElse(default: => AnyRef): AnyRef = {
val current = get
if (current.isDefined) current
if (current.isDefined) current.get
else default
}
private def register = {
if (currentTransaction.get.isEmpty) throw new NoTransactionInScopeException
currentTransaction.get.get.register(uuid, this)
}
}
class CassandraPersistentRef extends PersistentRef {

View file

@ -8,7 +8,7 @@ import junit.framework.TestSuite
object AllTest extends TestCase {
def suite(): Test = {
val suite = new TestSuite("All Scala tests")
suite.addTestSuite(classOf[CassandraPersistentActorSpec])
//suite.addTestSuite(classOf[CassandraPersistentActorSpec])
//suite.addTestSuite(classOf[MongoPersistentActorSpec])
//suite.addTestSuite(classOf[MongoStorageSpec])
suite

View file

@ -11,7 +11,7 @@ import javax.ws.rs.Produces;
import se.scalablesolutions.akka.annotation.transactionrequired;
import se.scalablesolutions.akka.annotation.prerestart;
import se.scalablesolutions.akka.annotation.postrestart;
import se.scalablesolutions.akka.state.TransactionalState;
import se.scalablesolutions.akka.state.PersistentMap;
import se.scalablesolutions.akka.state.PersistentState;
import se.scalablesolutions.akka.state.TransactionalMap;
import se.scalablesolutions.akka.state.CassandraStorageConfig;
@ -29,8 +29,7 @@ public class PersistentSimpleService {
private String KEY = "COUNTER";
private boolean hasStartedTicking = false;
private PersistentState factory = new PersistentState();
private TransactionalMap<Object, Object> storage = factory.newMap(new CassandraStorageConfig());
private PersistentMap storage = PersistentState.newMap(new CassandraStorageConfig());
@GET
@Produces({"application/html"})

View file

@ -28,8 +28,7 @@ public class SimpleService {
private String KEY = "COUNTER";
private boolean hasStartedTicking = false;
private TransactionalState factory = new TransactionalState();
private TransactionalMap storage = factory.newMap();
private TransactionalMap storage = TransactionalState.newMap();
@GET
@Produces({"application/json"})

View file

@ -7,7 +7,91 @@
<option name="BUILD_JARS_ON_MAKE" value="false" />
</component>
<component name="CodeStyleSettingsManager">
<option name="PER_PROJECT_SETTINGS" />
<option name="PER_PROJECT_SETTINGS">
<value>
<ADDITIONAL_INDENT_OPTIONS fileType="groovy">
<option name="INDENT_SIZE" value="2" />
<option name="CONTINUATION_INDENT_SIZE" value="8" />
<option name="TAB_SIZE" value="4" />
<option name="USE_TAB_CHARACTER" value="false" />
<option name="SMART_TABS" value="false" />
<option name="LABEL_INDENT_SIZE" value="0" />
<option name="LABEL_INDENT_ABSOLUTE" value="false" />
</ADDITIONAL_INDENT_OPTIONS>
<ADDITIONAL_INDENT_OPTIONS fileType="gsp">
<option name="INDENT_SIZE" value="2" />
<option name="CONTINUATION_INDENT_SIZE" value="8" />
<option name="TAB_SIZE" value="4" />
<option name="USE_TAB_CHARACTER" value="false" />
<option name="SMART_TABS" value="false" />
<option name="LABEL_INDENT_SIZE" value="0" />
<option name="LABEL_INDENT_ABSOLUTE" value="false" />
</ADDITIONAL_INDENT_OPTIONS>
<ADDITIONAL_INDENT_OPTIONS fileType="java">
<option name="INDENT_SIZE" value="4" />
<option name="CONTINUATION_INDENT_SIZE" value="8" />
<option name="TAB_SIZE" value="4" />
<option name="USE_TAB_CHARACTER" value="false" />
<option name="SMART_TABS" value="false" />
<option name="LABEL_INDENT_SIZE" value="0" />
<option name="LABEL_INDENT_ABSOLUTE" value="false" />
</ADDITIONAL_INDENT_OPTIONS>
<ADDITIONAL_INDENT_OPTIONS fileType="js">
<option name="INDENT_SIZE" value="4" />
<option name="CONTINUATION_INDENT_SIZE" value="8" />
<option name="TAB_SIZE" value="4" />
<option name="USE_TAB_CHARACTER" value="false" />
<option name="SMART_TABS" value="false" />
<option name="LABEL_INDENT_SIZE" value="0" />
<option name="LABEL_INDENT_ABSOLUTE" value="false" />
</ADDITIONAL_INDENT_OPTIONS>
<ADDITIONAL_INDENT_OPTIONS fileType="jsp">
<option name="INDENT_SIZE" value="4" />
<option name="CONTINUATION_INDENT_SIZE" value="8" />
<option name="TAB_SIZE" value="4" />
<option name="USE_TAB_CHARACTER" value="false" />
<option name="SMART_TABS" value="false" />
<option name="LABEL_INDENT_SIZE" value="0" />
<option name="LABEL_INDENT_ABSOLUTE" value="false" />
</ADDITIONAL_INDENT_OPTIONS>
<ADDITIONAL_INDENT_OPTIONS fileType="scala">
<option name="INDENT_SIZE" value="2" />
<option name="CONTINUATION_INDENT_SIZE" value="8" />
<option name="TAB_SIZE" value="2" />
<option name="USE_TAB_CHARACTER" value="false" />
<option name="SMART_TABS" value="false" />
<option name="LABEL_INDENT_SIZE" value="0" />
<option name="LABEL_INDENT_ABSOLUTE" value="false" />
</ADDITIONAL_INDENT_OPTIONS>
<ADDITIONAL_INDENT_OPTIONS fileType="sql">
<option name="INDENT_SIZE" value="2" />
<option name="CONTINUATION_INDENT_SIZE" value="8" />
<option name="TAB_SIZE" value="4" />
<option name="USE_TAB_CHARACTER" value="false" />
<option name="SMART_TABS" value="false" />
<option name="LABEL_INDENT_SIZE" value="0" />
<option name="LABEL_INDENT_ABSOLUTE" value="false" />
</ADDITIONAL_INDENT_OPTIONS>
<ADDITIONAL_INDENT_OPTIONS fileType="xml">
<option name="INDENT_SIZE" value="4" />
<option name="CONTINUATION_INDENT_SIZE" value="8" />
<option name="TAB_SIZE" value="4" />
<option name="USE_TAB_CHARACTER" value="false" />
<option name="SMART_TABS" value="false" />
<option name="LABEL_INDENT_SIZE" value="0" />
<option name="LABEL_INDENT_ABSOLUTE" value="false" />
</ADDITIONAL_INDENT_OPTIONS>
<ADDITIONAL_INDENT_OPTIONS fileType="yml">
<option name="INDENT_SIZE" value="2" />
<option name="CONTINUATION_INDENT_SIZE" value="8" />
<option name="TAB_SIZE" value="4" />
<option name="USE_TAB_CHARACTER" value="false" />
<option name="SMART_TABS" value="false" />
<option name="LABEL_INDENT_SIZE" value="0" />
<option name="LABEL_INDENT_ABSOLUTE" value="false" />
</ADDITIONAL_INDENT_OPTIONS>
</value>
</option>
<option name="USE_PER_PROJECT_SETTINGS" value="false" />
</component>
<component name="CompilerConfiguration">

430
akka.iws
View file

@ -2,16 +2,26 @@
<project relativePaths="false" version="4">
<component name="ChangeListManager" verified="true">
<list default="true" readonly="true" id="188c966f-a83c-4d3a-9128-54d5a2947a12" name="Default" comment="">
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/main/scala/actor/Actor.scala" afterPath="$PROJECT_DIR$/akka-actors/src/main/scala/actor/Actor.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/main/scala/reactor/EventBasedSingleThreadDispatcher.scala" afterPath="$PROJECT_DIR$/akka-actors/src/main/scala/reactor/EventBasedSingleThreadDispatcher.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/main/scala/reactor/Reactor.scala" afterPath="$PROJECT_DIR$/akka-actors/src/main/scala/reactor/Reactor.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala" afterPath="$PROJECT_DIR$/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/test/scala/InMemoryActorSpec.scala" afterPath="$PROJECT_DIR$/akka-actors/src/test/scala/InMemoryActorSpec.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala" afterPath="$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraStorage.scala" afterPath="$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraStorage.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/main/scala/nio/RemoteServer.scala" afterPath="$PROJECT_DIR$/akka-actors/src/main/scala/nio/RemoteServer.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka.ipr" afterPath="$PROJECT_DIR$/akka.ipr" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/config/storage-conf.xml" afterPath="$PROJECT_DIR$/config/storage-conf.xml" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka.iws" afterPath="$PROJECT_DIR$/akka.iws" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraSession.scala" afterPath="$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraSession.scala" />
<change type="DELETED" beforePath="$PROJECT_DIR$/akka-persistence/src/main/scala/DataFlowVariable.scala" afterPath="" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java" afterPath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala" afterPath="$PROJECT_DIR$/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/main/scala/stm/Vector.scala" afterPath="$PROJECT_DIR$/akka-actors/src/main/scala/stm/Vector.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/main/scala/reactor/EventBasedThreadPoolDispatcher.scala" afterPath="$PROJECT_DIR$/akka-actors/src/main/scala/reactor/EventBasedThreadPoolDispatcher.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala" afterPath="$PROJECT_DIR$/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionManagement.scala" afterPath="$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionManagement.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemStatefulNested.java" afterPath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemStatefulNested.java" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-actors/src/main/scala/stm/Transaction.scala" afterPath="$PROJECT_DIR$/akka-actors/src/main/scala/stm/Transaction.scala" />
</list>
<ignored path=".idea/workspace.xml" />
<ignored path="akka.iws" />
<ignored path=".idea/workspace.xml" />
<option name="TRACKING_ENABLED" value="true" />
<option name="SHOW_DIALOG" value="true" />
<option name="HIGHLIGHT_CONFLICTS" value="true" />
@ -27,21 +37,6 @@
<disable_hints />
</component>
<component name="DebuggerManager">
<line_breakpoints>
<breakpoint url="file://$PROJECT_DIR$/akka-actors/src/main/scala/actor/Actor.scala" line="568" class="Class at Actor.scala:568" package="">
<option name="ENABLED" value="true" />
<option name="LOG_ENABLED" value="false" />
<option name="LOG_EXPRESSION_ENABLED" value="false" />
<option name="SUSPEND_POLICY" value="SuspendAll" />
<option name="COUNT_FILTER_ENABLED" value="false" />
<option name="COUNT_FILTER" value="0" />
<option name="CONDITION_ENABLED" value="false" />
<option name="CLASS_FILTERS_ENABLED" value="false" />
<option name="INSTANCE_FILTERS_ENABLED" value="false" />
<option name="CONDITION" value="" />
<option name="LOG_MESSAGE" value="" />
</breakpoint>
</line_breakpoints>
<breakpoint_any>
<breakpoint>
<option name="NOTIFY_CAUGHT" value="true" />
@ -75,24 +70,7 @@
</breakpoint>
</breakpoint_any>
<breakpoint_rules />
<ui_properties>
<property name="exception_breakpoints_flattenPackages" value="true" />
<property name="line_breakpoints_groupByClasses" value="true" />
<property name="exception_breakpoints_groupByMethods" value="false" />
<property name="method_breakpoints_viewId" value="TABLE" />
<property name="line_breakpoints_viewId" value="TABLE" />
<property name="exception_breakpoints_groupByClasses" value="true" />
<property name="method_breakpoints_groupByMethods" value="false" />
<property name="field_breakpoints_flattenPackages" value="true" />
<property name="line_breakpoints_flattenPackages" value="true" />
<property name="exception_breakpoints_viewId" value="TABLE" />
<property name="field_breakpoints_groupByClasses" value="true" />
<property name="method_breakpoints_flattenPackages" value="true" />
<property name="field_breakpoints_viewId" value="TABLE" />
<property name="line_breakpoints_groupByMethods" value="false" />
<property name="method_breakpoints_groupByClasses" value="true" />
<property name="field_breakpoints_groupByMethods" value="false" />
</ui_properties>
<ui_properties />
</component>
<component name="FavoritesManager">
<favorites_list name="akka" />
@ -100,91 +78,82 @@
<component name="FileColors" enabled="false" enabledForTabs="false" />
<component name="FileEditorManager">
<leaf>
<file leaf-file-name="CassandraSession.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraSession.scala">
<file leaf-file-name="Actor.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/actor/Actor.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="14" column="0" selection-start="268" selection-end="268" vertical-scroll-proportion="0.0">
<state line="111" column="36" selection-start="4409" selection-end="4409" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
</file>
<file leaf-file-name="storage-conf.xml" pinned="false" current="true" current-in-tab="true">
<entry file="file://$PROJECT_DIR$/config/storage-conf.xml">
<file leaf-file-name="RemoteServer.scala" pinned="false" current="true" current-in-tab="true">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/nio/RemoteServer.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="320" column="0" selection-start="14147" selection-end="14147" vertical-scroll-proportion="0.9945726">
<state line="2" column="19" selection-start="49" selection-end="49" vertical-scroll-proportion="0.015429122">
<folding />
</state>
</provider>
</entry>
</file>
<file leaf-file-name="AllTest.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala">
<file leaf-file-name="TransactionManagement.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionManagement.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="11" column="4" selection-start="395" selection-end="395" vertical-scroll-proportion="0.0">
<state line="87" column="30" selection-start="3169" selection-end="3169" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
</file>
<file leaf-file-name="Predef.scala" pinned="false" current="false" current-in-tab="false">
<entry file="jar://$MAVEN_REPOSITORY$/org/scala-lang/scala-library/2.7.5/scala-library-2.7.5-sources.jar!/scala/Predef.scala">
<file leaf-file-name="Scheduler.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/actor/Scheduler.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="67" column="42" selection-start="2802" selection-end="2802" vertical-scroll-proportion="0.0">
<state line="42" column="25" selection-start="1494" selection-end="1494" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
</file>
<file leaf-file-name="Map.scala" pinned="false" current="false" current-in-tab="false">
<entry file="jar://$MAVEN_REPOSITORY$/org/scala-lang/scala-library/2.7.5/scala-library-2.7.5-sources.jar!/scala/collection/immutable/Map.scala">
<file leaf-file-name="Transaction.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/stm/Transaction.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="10" column="71" selection-start="580" selection-end="580" vertical-scroll-proportion="0.0">
<state line="126" column="0" selection-start="4172" selection-end="4172" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
</file>
<file leaf-file-name="CassandraStorage.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraStorage.scala">
<file leaf-file-name="Future.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/reactor/Future.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="156" column="13" selection-start="5640" selection-end="5640" vertical-scroll-proportion="0.0">
<state line="59" column="50" selection-start="1822" selection-end="1822" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
</file>
<file leaf-file-name="ColumnOrSuperColumn.class" pinned="false" current="false" current-in-tab="false">
<entry file="jar://$MAVEN_REPOSITORY$/org/apache/cassandra/cassandra/0.4.0/cassandra-0.4.0.jar!/org/apache/cassandra/service/ColumnOrSuperColumn.class">
<file leaf-file-name="InMemoryActorSpec.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-actors/src/test/scala/InMemoryActorSpec.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="5" column="13" selection-start="166" selection-end="166" vertical-scroll-proportion="0.0">
<state line="111" column="6" selection-start="3960" selection-end="3960" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
</file>
<file leaf-file-name="Column.class" pinned="false" current="false" current-in-tab="false">
<entry file="jar://$MAVEN_REPOSITORY$/org/apache/cassandra/cassandra/0.4.0/cassandra-0.4.0.jar!/org/apache/cassandra/service/Column.class">
<file leaf-file-name="InMemoryStateTest.java" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemoryStateTest.java">
<provider selected="true" editor-type-id="text-editor">
<state line="5" column="13" selection-start="166" selection-end="166" vertical-scroll-proportion="0.0">
<state line="20" column="23" selection-start="533" selection-end="533" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
</file>
<file leaf-file-name="SimpleService.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-samples-scala/src/main/scala/SimpleService.scala">
<file leaf-file-name="InMemNestedStateTest.java" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java">
<provider selected="true" editor-type-id="text-editor">
<state line="39" column="31" selection-start="1169" selection-end="1169" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
</file>
<file leaf-file-name="akka-reference.conf" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/config/akka-reference.conf">
<provider selected="true" editor-type-id="text-editor">
<state line="12" column="2" selection-start="386" selection-end="386" vertical-scroll-proportion="0.0">
<state line="40" column="14" selection-start="1523" selection-end="1523" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
@ -205,22 +174,22 @@
<component name="IdeDocumentHistory">
<option name="changedFiles">
<list>
<option value="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java" />
<option value="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemoryStateTest.java" />
<option value="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemStateful.java" />
<option value="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemStatefulNested.java" />
<option value="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/RemoteInMemoryStateTest.java" />
<option value="$PROJECT_DIR$/akka-samples-scala/src/main/scala/SimpleService.scala" />
<option value="$PROJECT_DIR$/akka-kernel/src/main/scala/Kernel.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/stm/Transaction.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/actor/Actor.scala" />
<option value="$PROJECT_DIR$/akka-util-java/src/main/java/se/scalablesolutions/akka/config/Ref.java" />
<option value="$PROJECT_DIR$/akka-util-java/src/main/java/se/scalablesolutions/akka/stm/Ref.java" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionalState.scala" />
<option value="$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraStorage.scala" />
<option value="$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraSession.scala" />
<option value="$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala" />
<option value="$PROJECT_DIR$/config/storage-conf.xml" />
<option value="$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala" />
<option value="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemStatefulNested.java" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/reactor/EventBasedSingleThreadDispatcher.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/reactor/EventBasedThreadPoolDispatcher.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/stm/Vector.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/reactor/Reactor.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/test/scala/InMemoryActorSpec.scala" />
<option value="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/stm/Transaction.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionManagement.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/actor/Actor.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/nio/RemoteServer.scala" />
</list>
</option>
</component>
@ -264,7 +233,7 @@
<option name="STATE" value="0" />
</component>
<component name="ProjectView">
<navigator currentView="PackagesPane" proportions="" version="1" splitterProportion="0.5">
<navigator currentView="ProjectPane" proportions="" version="1" splitterProportion="0.5">
<flattenPackages />
<showMembers />
<showModules />
@ -276,39 +245,6 @@
<sortByType />
</navigator>
<panes>
<pane id="ProjectPane">
<subPane />
</pane>
<pane id="PackagesPane">
<subPane>
<PATH>
<PATH_ELEMENT>
<option name="myItemId" value="akka" />
<option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.PackageViewProjectNode" />
</PATH_ELEMENT>
</PATH>
<PATH>
<PATH_ELEMENT>
<option name="myItemId" value="akka" />
<option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.PackageViewProjectNode" />
</PATH_ELEMENT>
<PATH_ELEMENT>
<option name="myItemId" value="akka-persistence" />
<option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.PackageViewModuleNode" />
</PATH_ELEMENT>
</PATH>
<PATH>
<PATH_ELEMENT>
<option name="myItemId" value="akka" />
<option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.PackageViewProjectNode" />
</PATH_ELEMENT>
<PATH_ELEMENT>
<option name="myItemId" value="akka-fun-test-java" />
<option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.PackageViewModuleNode" />
</PATH_ELEMENT>
</PATH>
</subPane>
</pane>
<pane id="Scope">
<subPane subId="Problems">
<PATH>
@ -335,45 +271,84 @@
</PATH>
</subPane>
</pane>
<pane id="ProjectPane">
<subPane />
</pane>
<pane id="PackagesPane">
<subPane />
</pane>
<pane id="Favorites" />
</panes>
</component>
<component name="PropertiesComponent">
<property name="GoToClass.includeJavaFiles" value="false" />
<property name="project.structure.proportion" value="0.15" />
<property name="GoToClass.includeJavaFiles" value="false" />
<property name="MemberChooser.copyJavadoc" value="false" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder2" value="2" />
<property name="options.splitter.main.proportions" value="0.3" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeOrder2" value="2" />
<property name="options.splitter.main.proportions" value="0.3" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeWidth3" value="924" />
<property name="GoToFile.includeJavaFiles" value="false" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatWidth0" value="113" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeWidth2" value="130" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatWidth2" value="133" />
<property name="GoToClass.includeLibraries" value="false" />
<property name="options.splitter.details.proportions" value="0.2" />
<property name="GoToClass.includeLibraries" value="false" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatWidth2" value="133" />
<property name="MemberChooser.showClasses" value="true" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder0" value="0" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder3" value="3" />
<property name="project.structure.side.proportion" value="0.2" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeWidth0" value="124" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeOrder3" value="3" />
<property name="options.lastSelected" value="preferences.intentionPowerPack" />
<property name="options.lastSelected" value="preferences.sourceCode.General" />
<property name="GoToClass.toSaveIncludeLibraries" value="false" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeOrder0" value="0" />
<property name="RunManagerConfig.showSettingsBeforeRunnig" value="false" />
<property name="project.structure.last.edited" value="Facets" />
<property name="MemberChooser.sorted" value="false" />
<property name="options.searchVisible" value="true" />
<property name="MemberChooser.sorted" value="false" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeOrder1" value="1" />
<property name="recentsLimit" value="5" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeWidth1" value="162" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatWidth3" value="928" />
<property name="dynamic.classpath" value="false" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatWidth3" value="928" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatWidth1" value="166" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder1" value="1" />
</component>
<component name="RunManager" selected="JUnit.InMemNestedStateTest">
<component name="RunManager" selected="JUnit.InMemNestedStateTest.testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess">
<configuration default="false" name="InMemNestedStateTest.testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess" type="JUnit" factoryName="JUnit" temporary="true" enabled="false" merge="false" sample_coverage="true" runner="emma">
<pattern>
<option name="PATTERN" value="se.scalablesolutions.akka.api.*" />
<option name="ENABLED" value="true" />
</pattern>
<module name="akka-fun-test-java" />
<option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
<option name="ALTERNATIVE_JRE_PATH" value="" />
<option name="PACKAGE_NAME" value="se.scalablesolutions.akka.api" />
<option name="MAIN_CLASS_NAME" value="se.scalablesolutions.akka.api.InMemNestedStateTest" />
<option name="METHOD_NAME" value="testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess" />
<option name="TEST_OBJECT" value="method" />
<option name="VM_PARAMETERS" value="" />
<option name="PARAMETERS" value="" />
<option name="WORKING_DIRECTORY" value="file://$PROJECT_DIR$" />
<option name="ENV_VARIABLES" />
<option name="PASS_PARENT_ENVS" value="true" />
<option name="TEST_SEARCH_SCOPE">
<value defaultName="moduleWithDependencies" />
</option>
<envs />
<RunnerSettings RunnerId="Debug">
<option name="DEBUG_PORT" value="54866" />
<option name="TRANSPORT" value="0" />
<option name="LOCAL" value="true" />
</RunnerSettings>
<RunnerSettings RunnerId="Run" />
<ConfigurationWrapper RunnerId="Debug" />
<ConfigurationWrapper RunnerId="Run" />
<method>
<option name="Make" enabled="false" />
</method>
</configuration>
<configuration default="true" type="Remote" factoryName="Remote">
<option name="USE_SOCKET_TRANSPORT" value="true" />
<option name="SERVER_MODE" value="false" />
@ -426,14 +401,6 @@
<option name="Maven.BeforeRunTask" enabled="false" />
</method>
</configuration>
<configuration default="true" type="FlexUnitRunConfigurationType" factoryName="FlexUnit" air_descriptor_path="" air_program_params="" air_publisher_id="" air_root_dir_path="" class_name="" html_or_swf_file_path="" main_class_name="" method_name="" module_name="" output_log_level="" package_name="" port="0" run_mode="HtmlOrSwfFile" scope="Class" socket_policy_port="0" url_to_launch="http://">
<method>
<option name="AntTarget" enabled="false" />
<option name="BuildArtifacts" enabled="false" />
<option name="Make" enabled="true" />
<option name="Maven.BeforeRunTask" enabled="false" />
</method>
</configuration>
<configuration default="true" type="TestNG" factoryName="TestNG" enabled="false" merge="false" runner="emma">
<module name="" />
<option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
@ -465,6 +432,14 @@
<option name="Maven.BeforeRunTask" enabled="false" />
</method>
</configuration>
<configuration default="true" type="FlexUnitRunConfigurationType" factoryName="FlexUnit" air_descriptor_path="" air_program_params="" air_publisher_id="" air_root_dir_path="" class_name="" html_or_swf_file_path="" main_class_name="" method_name="" module_name="" output_log_level="" package_name="" port="0" run_mode="HtmlOrSwfFile" scope="Class" socket_policy_port="0" url_to_launch="http://">
<method>
<option name="AntTarget" enabled="false" />
<option name="BuildArtifacts" enabled="false" />
<option name="Make" enabled="true" />
<option name="Maven.BeforeRunTask" enabled="false" />
</method>
</configuration>
<configuration default="true" type="JUnit" factoryName="JUnit" enabled="false" merge="false" runner="emma">
<module name="" />
<option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
@ -561,7 +536,13 @@
<value defaultName="moduleWithDependencies" />
</option>
<envs />
<RunnerSettings RunnerId="Debug">
<option name="DEBUG_PORT" value="53499" />
<option name="TRANSPORT" value="0" />
<option name="LOCAL" value="true" />
</RunnerSettings>
<RunnerSettings RunnerId="Run" />
<ConfigurationWrapper RunnerId="Debug" />
<ConfigurationWrapper RunnerId="Run" />
<method>
<option name="Make" enabled="false" />
@ -613,17 +594,24 @@
<value defaultName="moduleWithDependencies" />
</option>
<envs />
<RunnerSettings RunnerId="Debug">
<option name="DEBUG_PORT" value="52630" />
<option name="TRANSPORT" value="0" />
<option name="LOCAL" value="true" />
</RunnerSettings>
<RunnerSettings RunnerId="Run" />
<ConfigurationWrapper RunnerId="Debug" />
<ConfigurationWrapper RunnerId="Run" />
<method>
<option name="Make" enabled="false" />
</method>
</configuration>
<list size="4">
<item index="0" class="java.lang.String" itemvalue="JUnit.RemoteInMemoryStateTest" />
<item index="1" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec" />
<item index="2" class="java.lang.String" itemvalue="JUnit.InMemoryStateTest" />
<item index="3" class="java.lang.String" itemvalue="JUnit.InMemNestedStateTest" />
<list size="5">
<item index="0" class="java.lang.String" itemvalue="JUnit.InMemNestedStateTest.testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess" />
<item index="1" class="java.lang.String" itemvalue="JUnit.RemoteInMemoryStateTest" />
<item index="2" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec" />
<item index="3" class="java.lang.String" itemvalue="JUnit.InMemoryStateTest" />
<item index="4" class="java.lang.String" itemvalue="JUnit.InMemNestedStateTest" />
</list>
<configuration name="&lt;template&gt;" type="WebApp" default="true" selected="false">
<Host>localhost</Host>
@ -664,29 +652,29 @@
</option>
</component>
<component name="ToolWindowManager">
<frame x="4" y="22" width="1436" height="878" extended-state="6" />
<frame x="4" y="22" width="1916" height="1178" extended-state="0" />
<editor active="true" />
<layout>
<window_info id="Maven Projects" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="3" side_tool="false" />
<window_info id="IDEtalk" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="3" side_tool="false" />
<window_info id="Changes" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="8" side_tool="false" />
<window_info id="Web" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="2" side_tool="true" />
<window_info id="IDEtalk" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="5" side_tool="false" />
<window_info id="Run" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.63745385" sideWeight="0.5" order="2" side_tool="false" />
<window_info id="TODO" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="6" side_tool="false" />
<window_info id="Project" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="true" weight="0.29076478" sideWeight="0.6651291" order="0" side_tool="false" />
<window_info id="Structure" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.25" sideWeight="0.5" order="1" side_tool="true" />
<window_info id="Messages" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.32841328" sideWeight="0.5" order="7" side_tool="false" />
<window_info id="Dependency Viewer" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="7" side_tool="false" />
<window_info id="Palette" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="3" side_tool="false" />
<window_info id="Ant Build" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.25" sideWeight="0.5" order="1" side_tool="false" />
<window_info id="Changes" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="7" side_tool="false" />
<window_info id="Run" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.41605166" sideWeight="0.5" order="2" side_tool="false" />
<window_info id="Debug" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.48214287" sideWeight="0.5" order="3" side_tool="false" />
<window_info id="Commander" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.4" sideWeight="0.5" order="0" side_tool="false" />
<window_info id="IDEtalk Messages" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="7" side_tool="false" />
<window_info id="Version Control" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.32908162" sideWeight="0.5" order="7" side_tool="false" />
<window_info id="Web Preview" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="7" side_tool="false" />
<window_info id="Project" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="true" weight="0.18595926" sideWeight="0.51291513" order="0" side_tool="false" />
<window_info id="Debug" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.4801444" sideWeight="0.5" order="3" side_tool="false" />
<window_info id="Structure" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.25" sideWeight="0.5" order="1" side_tool="true" />
<window_info id="IDEtalk Messages" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="9" side_tool="false" />
<window_info id="Version Control" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.32908162" sideWeight="0.5" order="11" side_tool="false" />
<window_info id="Web Preview" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="10" side_tool="false" />
<window_info id="Message" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="0" side_tool="false" />
<window_info id="Find" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.32780612" sideWeight="0.5" order="1" side_tool="false" />
<window_info id="Find" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.29335794" sideWeight="0.5" order="1" side_tool="false" />
<window_info id="Messages" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.32749078" sideWeight="0.5" order="12" side_tool="false" />
<window_info id="Inspection" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.4" sideWeight="0.5" order="5" side_tool="false" />
<window_info id="Dependency Viewer" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="7" side_tool="false" />
<window_info id="Palette" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="4" side_tool="false" />
<window_info id="Ant Build" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.25" sideWeight="0.5" order="1" side_tool="false" />
<window_info id="Cvs" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.25" sideWeight="0.5" order="4" side_tool="false" />
<window_info id="Hierarchy" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.25" sideWeight="0.5" order="2" side_tool="false" />
</layout>
@ -723,100 +711,86 @@
<option name="FILTER_TARGETS" value="false" />
</component>
<component name="editorHistoryManager">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionalState.scala">
<entry file="file://$PROJECT_DIR$/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="82" column="6" selection-start="1883" selection-end="1883" vertical-scroll-proportion="0.0" />
<state line="118" column="0" selection-start="4515" selection-end="4515" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java">
<entry file="file://$PROJECT_DIR$/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="65" column="14" selection-start="3397" selection-end="3397" vertical-scroll-proportion="0.0" />
<state line="159" column="0" selection-start="6245" selection-end="6245" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-util-java/src/main/java/se/scalablesolutions/akka/stm/Ref.java">
<entry file="file://$PROJECT_DIR$/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="0" column="1" selection-start="1" selection-end="1" vertical-scroll-proportion="0.0" />
<state line="12" column="50" selection-start="445" selection-end="445" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-kernel/src/main/scala/Kernel.scala">
<entry file="file://$PROJECT_DIR$/akka-samples-java/src/main/java/sample/java/SimpleService.java">
<provider selected="true" editor-type-id="text-editor">
<state line="32" column="1" selection-start="989" selection-end="989" vertical-scroll-proportion="0.0" />
<state line="12" column="34" selection-start="307" selection-end="307" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-samples-lift/src/main/scala/akka/SimpleService.scala">
<entry file="jar://$MAVEN_REPOSITORY$/org/scala-lang/scala-library/2.7.5/scala-library-2.7.5-sources.jar!/scala/Option.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="18" column="6" selection-start="597" selection-end="597" vertical-scroll-proportion="0.0" />
<state line="38" column="0" selection-start="1292" selection-end="1292" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-samples-scala/src/main/scala/SimpleService.scala">
<entry file="jar://$MAVEN_REPOSITORY$/org/scala-lang/scala-library/2.7.5/scala-library-2.7.5.jar!/scala/Some.class">
<provider selected="true" editor-type-id="text-editor">
<state line="39" column="31" selection-start="1169" selection-end="1169" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/config/akka-reference.conf">
<provider selected="true" editor-type-id="text-editor">
<state line="12" column="2" selection-start="386" selection-end="386" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="jar://$MAVEN_REPOSITORY$/org/apache/cassandra/cassandra/0.4.0/cassandra-0.4.0.jar!/org/apache/cassandra/service/Cassandra.class">
<provider selected="true" editor-type-id="text-editor">
<state line="2124" column="33" selection-start="116949" selection-end="116949" vertical-scroll-proportion="0.33333334">
<folding />
</state>
</provider>
</entry>
<entry file="jar://$MAVEN_REPOSITORY$/org/apache/cassandra/cassandra/0.4.0/cassandra-0.4.0.jar!/org/apache/cassandra/service/ColumnOrSuperColumn.class">
<provider selected="true" editor-type-id="text-editor">
<state line="5" column="13" selection-start="166" selection-end="166" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="jar://$MAVEN_REPOSITORY$/org/apache/cassandra/cassandra/0.4.0/cassandra-0.4.0.jar!/org/apache/cassandra/service/Column.class">
<provider selected="true" editor-type-id="text-editor">
<state line="5" column="13" selection-start="166" selection-end="166" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="jar://$MAVEN_REPOSITORY$/org/scala-lang/scala-library/2.7.5/scala-library-2.7.5-sources.jar!/scala/collection/immutable/Map.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="10" column="71" selection-start="580" selection-end="580" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="jar://$MAVEN_REPOSITORY$/org/scala-lang/scala-library/2.7.5/scala-library-2.7.5-sources.jar!/scala/Predef.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="67" column="42" selection-start="2802" selection-end="2802" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraStorage.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="156" column="13" selection-start="5640" selection-end="5640" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraSession.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="14" column="0" selection-start="268" selection-end="268" vertical-scroll-proportion="0.0">
<folding />
</state>
<state line="0" column="0" selection-start="0" selection-end="0" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="11" column="4" selection-start="395" selection-end="395" vertical-scroll-proportion="0.0">
<state line="0" column="0" selection-start="0" selection-end="0" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/actor/Actor.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="111" column="36" selection-start="4409" selection-end="4409" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/config/storage-conf.xml">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionManagement.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="320" column="0" selection-start="14147" selection-end="14147" vertical-scroll-proportion="0.9945726">
<state line="87" column="30" selection-start="3169" selection-end="3169" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/actor/Scheduler.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="42" column="25" selection-start="1494" selection-end="1494" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/stm/Transaction.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="126" column="0" selection-start="4172" selection-end="4172" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/reactor/Future.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="59" column="50" selection-start="1822" selection-end="1822" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/test/scala/InMemoryActorSpec.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="111" column="6" selection-start="3960" selection-end="3960" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemoryStateTest.java">
<provider selected="true" editor-type-id="text-editor">
<state line="20" column="23" selection-start="533" selection-end="533" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java">
<provider selected="true" editor-type-id="text-editor">
<state line="40" column="14" selection-start="1523" selection-end="1523" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/nio/RemoteServer.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="2" column="19" selection-start="49" selection-end="49" vertical-scroll-proportion="0.015429122">
<folding />
</state>
</provider>

View file

@ -19,7 +19,7 @@
# FQN to the class doing initial active object/actor
# supervisor bootstrap, should be defined in default constructor
boot = ["sample.java.Boot", "sample.scala.Boot", "sample.secure.Boot", "se.foldleft.akka.demo.Boot"]
boot = ["sample.java.Boot", "sample.scala.Boot", "sample.secure.Boot"]
<actor>
timeout = 5000 # default timeout for future based invocations
@ -66,7 +66,6 @@
hostname = "127.0.0.1" # IP address or hostname of the MongoDB DB instance
port = 27017
dbname = "mydb"
storage-format = "scala-json" # Options: java, scala-json, java-json, protobuf
</mongodb>
</storage>
</akka>