fixed a bunch of persistence bugs

This commit is contained in:
jboner 2009-10-06 00:07:27 +02:00
parent fe6c025f0b
commit 046fa217b1
25 changed files with 674 additions and 572 deletions

View file

@ -146,24 +146,4 @@
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
</dependencies> </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> </project>

View file

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

View file

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

View file

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

View file

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

View file

@ -5,8 +5,12 @@
package se.scalablesolutions.akka.reactor package se.scalablesolutions.akka.reactor
import java.util.List 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 { trait MessageQueue {
def append(handle: MessageInvocation) def append(handle: MessageInvocation)
@ -32,23 +36,35 @@ trait MessageDemultiplexer {
def wakeUp def wakeUp
} }
class MessageInvocation(val sender: AnyRef, class MessageInvocation(val receiver: Actor,
val message: AnyRef, val message: AnyRef,
val future: Option[CompletableFutureResult], val future: Option[CompletableFutureResult],
val tx: Option[Transaction]) { 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 var result = HashCode.SEED
result = HashCode.hash(result, sender) result = HashCode.hash(result, receiver)
result = HashCode.hash(result, message) result = HashCode.hash(result, message)
result result
} }
override def equals(that: Any): Boolean = override def equals(that: Any): Boolean = synchronized {
that != null && that != null &&
that.isInstanceOf[MessageInvocation] && that.isInstanceOf[MessageInvocation] &&
that.asInstanceOf[MessageInvocation].sender == sender && that.asInstanceOf[MessageInvocation].receiver == receiver &&
that.asInstanceOf[MessageInvocation].message == message 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 package se.scalablesolutions.akka.stm
import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import java.util.concurrent.atomic.AtomicLong
import state.Transactional import java.util.concurrent.atomic.AtomicInteger
import util.Logging
import actor.Actor 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.api.{Transaction => MultiverseTransaction}
import org.multiverse.stms.alpha.AlphaStm import org.multiverse.stms.alpha.AlphaStm
@ -15,7 +17,9 @@ import org.multiverse.utils.GlobalStmInstance
import org.multiverse.utils.TransactionThreadLocal._ import org.multiverse.utils.TransactionThreadLocal._
import org.multiverse.templates.{OrElseTemplate, AtomicTemplate} 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> * @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 @volatile private[this] var status: TransactionStatus = TransactionStatus.New
private[akka] var transaction: MultiverseTransaction = _ private[akka] var transaction: MultiverseTransaction = _
// private[this] var initMessage: Option[AnyRef] = None private[this] var message: Option[MessageInvocation] = None
// private[this] var initReceiver: Option[Actor] = None
private[this] var participants: List[String] = Nil private[this] var participants: List[String] = Nil
private[this] var precommitted: List[String] = Nil private[this] var precommitted: List[String] = Nil
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 increment = depth.incrementAndGet
def decrement = depth.decrementAndGet def decrement = depth.decrementAndGet
def isTopLevel = depth.compareAndSet(0, 0) def isTopLevel = depth.get == 0
def begin(participant: String) = synchronized { def register(uuid: String, storage: Committable) = persistentStateMap.put(uuid, storage)
// def begin(participant: String, message, receiver) = synchronized {
def begin(participant: String, msg: MessageInvocation) = synchronized {
ensureIsActiveOrNew ensureIsActiveOrNew
// initMessage = Some(message) message = Some(msg)
// initReceiver = Some(receiver)
transaction = Multiverse.STM.startUpdateTransaction("akka") 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) if (status == TransactionStatus.New) log.debug("TX BEGIN - Actor with UUID [%s] is starting NEW transaction [%s]", participant, toString)
else log.debug("Server [%s] is participating in transaction", participant) else log.debug("Actor [%s] is participating in transaction", participant)
participants ::= participant participants ::= participant
status = TransactionStatus.Active status = TransactionStatus.Active
} }
@ -109,8 +114,9 @@ object Transaction {
} }
def commit(participant: String): Boolean = synchronized { 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) { if (status == TransactionStatus.Active) {
log.debug("TX COMMIT - Committing transaction [%s] for server with UUID [%s]", toString, participant)
val haveAllPreCommitted = val haveAllPreCommitted =
if (participants.size == precommitted.size) {{ if (participants.size == precommitted.size) {{
for (part <- participants) yield { for (part <- participants) yield {
@ -119,50 +125,62 @@ object Transaction {
}}.exists(_ == true) }}.exists(_ == true)
} else false } else false
if (haveAllPreCommitted && transaction != null) { if (haveAllPreCommitted && transaction != null) {
log.debug("TX COMMIT - Committing transaction [%s] for server with UUID [%s]", toString, participant)
transaction.commit transaction.commit
transaction.reset
status = TransactionStatus.Completed
reset reset
status = TransactionStatus.Completed
Transaction.Atomic {
persistentStateMap.values.foreach(_.commit)
}
true true
} else false } else false
} else { } else {
reset
true true
} }
} }
def rollback(participant: String) = synchronized { def rollback(participant: String) = synchronized {
ensureIsActiveOrAborted 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 status = TransactionStatus.Aborted
transaction.abort transaction.abort
transaction.reset
reset reset
} }
def rollbackForRescheduling(participant: String) = synchronized { def rollbackForRescheduling(participant: String) = synchronized {
ensureIsActiveOrAborted 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 transaction.abort
reset reset
} }
def join(participant: String) = synchronized { def join(participant: String) = synchronized {
ensureIsActive 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 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 isNew = synchronized { status == TransactionStatus.New }
def isActive = synchronized { status == TransactionStatus.Active } def isActive = synchronized { status == TransactionStatus.Active }
def isCompleted = synchronized { status == TransactionStatus.Completed } def isCompleted = synchronized { status == TransactionStatus.Completed }
def isAborted = synchronized { status == TransactionStatus.Aborted } def isAborted = synchronized { status == TransactionStatus.Aborted }
private def reset = synchronized {
participants = Nil
precommitted = Nil
}
private def ensureIsActive = if (status != TransactionStatus.Active) private def ensureIsActive = if (status != TransactionStatus.Active)
throw new IllegalStateException("Expected ACTIVE transaction - current status [" + status + "]: " + toString) 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 java.util.concurrent.atomic.AtomicBoolean
import reactor.MessageInvocation import se.scalablesolutions.akka.reactor.MessageInvocation
import util.Logging import se.scalablesolutions.akka.util.Logging
import org.codehaus.aspectwerkz.proxy.Uuid // FIXME is java.util.UUID better?
import org.codehaus.aspectwerkz.proxy.Uuid
import scala.collection.mutable.HashSet
// FIXME is java.util.UUID better?
import org.multiverse.utils.TransactionThreadLocal._ import org.multiverse.utils.TransactionThreadLocal._
@ -19,9 +24,10 @@ class TransactionAwareWrapperException(val cause: Throwable, val tx: Option[Tran
} }
object TransactionManagement { object TransactionManagement {
import Config._ import se.scalablesolutions.akka.Config._
val TIME_WAITING_FOR_COMPLETION = config.getInt("akka.stm.wait-for-completion", 100) 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 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)) val TRANSACTION_ENABLED = new AtomicBoolean(config.getBool("akka.stm.service", false))
// FIXME reenable 'akka.stm.restart-on-collision' when new STM is in place // 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) 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 isTransactionalityEnabled = TRANSACTION_ENABLED.get
def disableTransactions = TRANSACTION_ENABLED.set(false) 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 override protected def initialValue: Option[Transaction] = None
} }
} }
@ -37,40 +43,40 @@ object TransactionManagement {
trait TransactionManagement extends Logging { trait TransactionManagement extends Logging {
var uuid = Uuid.newUuid.toString var uuid = Uuid.newUuid.toString
protected[this] var latestMessage: Option[MessageInvocation] = None import TransactionManagement.currentTransaction
protected[this] var messageToReschedule: Option[MessageInvocation] = None private[akka] val activeTransactions = new HashSet[Transaction]
import TransactionManagement.threadBoundTx protected def startNewTransaction(message: MessageInvocation) = {
private[akka] var activeTx: Option[Transaction] = None
protected def startNewTransaction: Option[Transaction] = {
val newTx = new Transaction val newTx = new Transaction
newTx.begin(uuid) newTx.begin(uuid, message)
val tx = Some(newTx) activeTransactions += newTx
activeTx = tx currentTransaction.set(Some(newTx))
threadBoundTx.set(tx) setThreadLocalTransaction(newTx.transaction)
setThreadLocalTransaction(tx.get.transaction)
tx
} }
protected def joinExistingTransaction = { protected def joinExistingTransaction = {
val cflowTx = threadBoundTx.get val cflowTx = currentTransaction.get
if (!activeTx.isDefined && cflowTx.isDefined) { if (activeTransactions.isEmpty && cflowTx.isDefined) {
val currentTx = cflowTx.get val currentTx = cflowTx.get
currentTx.join(uuid) 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) { protected def tryToCommitTransactions = {
val tx = activeTx.get for (tx <- activeTransactions) {
if (tx.commit(uuid)) { if (tx.commit(uuid)) activeTransactions -= tx
removeTransactionIfTopLevel else if (tx.isTopLevel) {
true println("------------ COULD NOT COMMIT -- WAITING OR TIMEOUT? ---------")
} else false //tx.retry
} else true } 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 { protected def rollback(tx: Option[Transaction]) = tx match {
case None => {} // no tx; nothing to do case None => {} // no tx; nothing to do
@ -84,39 +90,12 @@ trait TransactionManagement extends Logging {
tx.rollbackForRescheduling(uuid) tx.rollbackForRescheduling(uuid)
} }
protected def handleCollision = { protected def isInExistingTransaction = currentTransaction.get.isDefined
var nrRetries = 0
var failed = true protected def incrementTransaction = if (currentTransaction.get.isDefined) currentTransaction.get.get.increment
do {
Thread.sleep(TransactionManagement.TIME_WAITING_FOR_COMPLETION) protected def decrementTransaction = if (currentTransaction.get.isDefined) currentTransaction.get.get.decrement
nrRetries += 1
log.debug("Pending transaction [%s] not completed, waiting %s milliseconds. Attempt %s", activeTx.get.id, TransactionManagement.TIME_WAITING_FOR_COMPLETION, nrRetries) protected def removeTransactionIfTopLevel(tx: Transaction) = if (tx.isTopLevel) { activeTransactions -= tx }
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
} }

View file

@ -45,7 +45,14 @@ object TransactionalState {
@serializable @serializable
trait Transactional { trait Transactional {
// FIXME: won't work across the cluster // 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 swap(elem: T) = ref.set(elem)
def get: Option[T] = { def get: Option[T] = {
// if (ref.isNull) None if (ref.isNull) None
// else else Some(ref.get)
Some(ref.get)
} }
def getOrWait: T = ref.getOrAwait def getOrWait: T = ref.getOrAwait
def getOrElse(default: => T): T = { def getOrElse(default: => T): T = {
// if (ref.isNull) default if (ref.isNull) default
//else else ref.get
ref.get
} }
def isDefined: Boolean = !ref.isNull 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 { override def equals(other: Any) = other match {
case vec: Vector[T] => { case vec: Vector[_] => {
var back = length == vec.length var back = length == vec.length
var i = 0 var i = 0

View file

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

View file

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

View file

@ -226,6 +226,7 @@ class InMemoryActorSpec extends TestCase {
stateful !! Failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", failer) // call failing transactionrequired method stateful !! Failure("testShouldRollbackStateForStatefulServerInCaseOfFailure", "new state", failer) // call failing transactionrequired method
fail("should have thrown an exception") fail("should have thrown an exception")
} catch {case e: RuntimeException => {}} } catch {case e: RuntimeException => {}}
println("---------- BACK")
assertEquals("init", (stateful !! GetRefState).get) // check that state is == init state 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.{Test, Before}
import org.junit.Assert._ import org.junit.Assert._
import junit.framework.TestCase import junit.framework.TestCase
import se.scalablesolutions.akka.actor.Actor
class ThreadBasedDispatcherTest extends TestCase { class ThreadBasedDispatcherTest extends TestCase {
private var threadingIssueDetected: AtomicBoolean = null 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 { class TestMessageHandle(handleLatch: CountDownLatch) extends MessageInvoker {
val guardLock: Lock = new ReentrantLock val guardLock: Lock = new ReentrantLock
@ -52,7 +56,7 @@ class ThreadBasedDispatcherTest extends TestCase {
val dispatcher = new ThreadBasedDispatcher("name", new TestMessageHandle(handleLatch)) val dispatcher = new ThreadBasedDispatcher("name", new TestMessageHandle(handleLatch))
dispatcher.start dispatcher.start
for (i <- 0 until 100) { 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)) assertTrue(handleLatch.await(5, TimeUnit.SECONDS))
assertFalse(threadingIssueDetected.get) assertFalse(threadingIssueDetected.get)
@ -73,7 +77,7 @@ class ThreadBasedDispatcherTest extends TestCase {
}) })
dispatcher.start dispatcher.start
for (i <- 0 until 100) { 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)) assertTrue(handleLatch.await(5, TimeUnit.SECONDS))
assertFalse(threadingIssueDetected.get) assertFalse(threadingIssueDetected.get)

View file

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

View file

@ -158,7 +158,7 @@
<dependency> <dependency>
<groupId>org.apache.cassandra</groupId> <groupId>org.apache.cassandra</groupId>
<artifactId>cassandra</artifactId> <artifactId>cassandra</artifactId>
<version>0.4.0-trunk</version> <version>0.4.0</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.facebook</groupId> <groupId>com.facebook</groupId>
@ -320,22 +320,5 @@
</executions> </executions>
</plugin> </plugin>
</plugins> </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> </build>
</project> </project>

View file

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

View file

@ -5,12 +5,15 @@
package se.scalablesolutions.akka.state package se.scalablesolutions.akka.state
import stm.TransactionManagement import stm.TransactionManagement
import stm.TransactionManagement.currentTransaction
import akka.collection._ import akka.collection._
import org.codehaus.aspectwerkz.proxy.Uuid import org.codehaus.aspectwerkz.proxy.Uuid
import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.mutable.{ArrayBuffer, HashMap}
class NoTransactionInScopeException extends RuntimeException
sealed abstract class PersistentStateConfig sealed abstract class PersistentStateConfig
abstract class PersistentStorageConfig extends PersistentStateConfig abstract class PersistentStorageConfig extends PersistentStateConfig
case class CassandraStorageConfig extends PersistentStorageConfig case class CassandraStorageConfig extends PersistentStorageConfig
@ -62,7 +65,7 @@ object PersistentState {
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @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 newAndUpdatedEntries = TransactionalState.newMap[AnyRef, AnyRef]
protected val removedEntries = TransactionalState.newMap[AnyRef, AnyRef] protected val removedEntries = TransactionalState.newMap[AnyRef, AnyRef]
protected val shouldClearOnCommit = TransactionalRef[Boolean]() 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 // to be concretized in subclasses
val storage: MapStorage val storage: MapStorage
private[akka] def commit = { def commit = {
storage.removeMapStorageFor(uuid, removedEntries.toList) storage.removeMapStorageFor(uuid, removedEntries.toList)
storage.insertMapStorageEntriesFor(uuid, newAndUpdatedEntries.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 newAndUpdatedEntries.clear
removedEntries.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) 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) 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) storage.getMapStorageRangeFor(uuid, start, finish, count)
} catch { case e: Exception => Nil } } 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 { override def contains(key: AnyRef): Boolean = try {
newAndUpdatedEntries.contains(key) || newAndUpdatedEntries.contains(key) || storage.getMapStorageEntryFor(uuid, key).isDefined
storage.getMapStorageEntryFor(uuid, key).isDefined
} catch { case e: Exception => false } } catch { case e: Exception => false }
override def size: Int = try { 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 } 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> * @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 newElems = TransactionalState.newVector[AnyRef]
protected val updatedElems = TransactionalState.newMap[Int, AnyRef] protected val updatedElems = TransactionalState.newMap[Int, AnyRef]
protected val removedElems = TransactionalState.newVector[AnyRef] protected val removedElems = TransactionalState.newVector[AnyRef]
@ -162,7 +181,7 @@ trait PersistentVector extends RandomAccessSeq[AnyRef] with Transactional {
val storage: VectorStorage val storage: VectorStorage
private[akka] def commit = { def commit = {
// FIXME: should use batch function once the bug is resolved // FIXME: should use batch function once the bug is resolved
for (element <- newElems) storage.insertVectorStorageEntryFor(uuid, element) for (element <- newElems) storage.insertVectorStorageEntryFor(uuid, element)
for (entry <- updatedElems) storage.updateVectorStorageEntryFor(uuid, entry._1, entry._2) for (entry <- updatedElems) storage.updateVectorStorageEntryFor(uuid, entry._1, entry._2)
@ -170,9 +189,12 @@ trait PersistentVector extends RandomAccessSeq[AnyRef] with Transactional {
updatedElems.clear 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) 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. * Removes the <i>tail</i> element of this vector.
*/ */
// FIXME: implement persistent vector pop // 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) 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 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> * @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] protected val ref = new TransactionalRef[AnyRef]
val storage: RefStorage val storage: RefStorage
private[akka] def commit = if (ref.isDefined) { def commit = if (ref.isDefined) {
storage.insertRefStorageFor(uuid, ref.get) storage.insertRefStorageFor(uuid, ref.get.get)
ref.swap(null) 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) 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 = { def getOrElse(default: => AnyRef): AnyRef = {
val current = get val current = get
if (current.isDefined) current if (current.isDefined) current.get
else default else default
} }
private def register = {
if (currentTransaction.get.isEmpty) throw new NoTransactionInScopeException
currentTransaction.get.get.register(uuid, this)
}
} }
class CassandraPersistentRef extends PersistentRef { class CassandraPersistentRef extends PersistentRef {

View file

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

View file

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

View file

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

View file

@ -7,7 +7,91 @@
<option name="BUILD_JARS_ON_MAKE" value="false" /> <option name="BUILD_JARS_ON_MAKE" value="false" />
</component> </component>
<component name="CodeStyleSettingsManager"> <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" /> <option name="USE_PER_PROJECT_SETTINGS" value="false" />
</component> </component>
<component name="CompilerConfiguration"> <component name="CompilerConfiguration">

430
akka.iws
View file

@ -2,16 +2,26 @@
<project relativePaths="false" version="4"> <project relativePaths="false" version="4">
<component name="ChangeListManager" verified="true"> <component name="ChangeListManager" verified="true">
<list default="true" readonly="true" id="188c966f-a83c-4d3a-9128-54d5a2947a12" name="Default" comment=""> <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/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$/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.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="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="DELETED" beforePath="$PROJECT_DIR$/akka-persistence/src/main/scala/DataFlowVariable.scala" afterPath="" /> <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> </list>
<ignored path=".idea/workspace.xml" />
<ignored path="akka.iws" /> <ignored path="akka.iws" />
<ignored path=".idea/workspace.xml" />
<option name="TRACKING_ENABLED" value="true" /> <option name="TRACKING_ENABLED" value="true" />
<option name="SHOW_DIALOG" value="true" /> <option name="SHOW_DIALOG" value="true" />
<option name="HIGHLIGHT_CONFLICTS" value="true" /> <option name="HIGHLIGHT_CONFLICTS" value="true" />
@ -27,21 +37,6 @@
<disable_hints /> <disable_hints />
</component> </component>
<component name="DebuggerManager"> <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_any>
<breakpoint> <breakpoint>
<option name="NOTIFY_CAUGHT" value="true" /> <option name="NOTIFY_CAUGHT" value="true" />
@ -75,24 +70,7 @@
</breakpoint> </breakpoint>
</breakpoint_any> </breakpoint_any>
<breakpoint_rules /> <breakpoint_rules />
<ui_properties> <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>
</component> </component>
<component name="FavoritesManager"> <component name="FavoritesManager">
<favorites_list name="akka" /> <favorites_list name="akka" />
@ -100,91 +78,82 @@
<component name="FileColors" enabled="false" enabledForTabs="false" /> <component name="FileColors" enabled="false" enabledForTabs="false" />
<component name="FileEditorManager"> <component name="FileEditorManager">
<leaf> <leaf>
<file leaf-file-name="CassandraSession.scala" pinned="false" current="false" current-in-tab="false"> <file leaf-file-name="Actor.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraSession.scala"> <entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/actor/Actor.scala">
<provider selected="true" editor-type-id="text-editor"> <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 /> <folding />
</state> </state>
</provider> </provider>
</entry> </entry>
</file> </file>
<file leaf-file-name="storage-conf.xml" pinned="false" current="true" current-in-tab="true"> <file leaf-file-name="RemoteServer.scala" pinned="false" current="true" current-in-tab="true">
<entry file="file://$PROJECT_DIR$/config/storage-conf.xml"> <entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/nio/RemoteServer.scala">
<provider selected="true" editor-type-id="text-editor"> <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 /> <folding />
</state> </state>
</provider> </provider>
</entry> </entry>
</file> </file>
<file leaf-file-name="AllTest.scala" pinned="false" current="false" current-in-tab="false"> <file leaf-file-name="TransactionManagement.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala"> <entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionManagement.scala">
<provider selected="true" editor-type-id="text-editor"> <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 /> <folding />
</state> </state>
</provider> </provider>
</entry> </entry>
</file> </file>
<file leaf-file-name="Predef.scala" pinned="false" current="false" current-in-tab="false"> <file leaf-file-name="Scheduler.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"> <entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/actor/Scheduler.scala">
<provider selected="true" editor-type-id="text-editor"> <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 /> <folding />
</state> </state>
</provider> </provider>
</entry> </entry>
</file> </file>
<file leaf-file-name="Map.scala" pinned="false" current="false" current-in-tab="false"> <file leaf-file-name="Transaction.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"> <entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/stm/Transaction.scala">
<provider selected="true" editor-type-id="text-editor"> <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 /> <folding />
</state> </state>
</provider> </provider>
</entry> </entry>
</file> </file>
<file leaf-file-name="CassandraStorage.scala" pinned="false" current="false" current-in-tab="false"> <file leaf-file-name="Future.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraStorage.scala"> <entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/reactor/Future.scala">
<provider selected="true" editor-type-id="text-editor"> <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 /> <folding />
</state> </state>
</provider> </provider>
</entry> </entry>
</file> </file>
<file leaf-file-name="ColumnOrSuperColumn.class" pinned="false" current="false" current-in-tab="false"> <file leaf-file-name="InMemoryActorSpec.scala" 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"> <entry file="file://$PROJECT_DIR$/akka-actors/src/test/scala/InMemoryActorSpec.scala">
<provider selected="true" editor-type-id="text-editor"> <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 /> <folding />
</state> </state>
</provider> </provider>
</entry> </entry>
</file> </file>
<file leaf-file-name="Column.class" pinned="false" current="false" current-in-tab="false"> <file leaf-file-name="InMemoryStateTest.java" 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"> <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"> <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 /> <folding />
</state> </state>
</provider> </provider>
</entry> </entry>
</file> </file>
<file leaf-file-name="SimpleService.scala" pinned="false" current="false" current-in-tab="false"> <file leaf-file-name="InMemNestedStateTest.java" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-samples-scala/src/main/scala/SimpleService.scala"> <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"> <provider selected="true" editor-type-id="text-editor">
<state line="39" column="31" selection-start="1169" selection-end="1169" vertical-scroll-proportion="0.0"> <state line="40" column="14" selection-start="1523" selection-end="1523" 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">
<folding /> <folding />
</state> </state>
</provider> </provider>
@ -205,22 +174,22 @@
<component name="IdeDocumentHistory"> <component name="IdeDocumentHistory">
<option name="changedFiles"> <option name="changedFiles">
<list> <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$/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> </list>
</option> </option>
</component> </component>
@ -264,7 +233,7 @@
<option name="STATE" value="0" /> <option name="STATE" value="0" />
</component> </component>
<component name="ProjectView"> <component name="ProjectView">
<navigator currentView="PackagesPane" proportions="" version="1" splitterProportion="0.5"> <navigator currentView="ProjectPane" proportions="" version="1" splitterProportion="0.5">
<flattenPackages /> <flattenPackages />
<showMembers /> <showMembers />
<showModules /> <showModules />
@ -276,39 +245,6 @@
<sortByType /> <sortByType />
</navigator> </navigator>
<panes> <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"> <pane id="Scope">
<subPane subId="Problems"> <subPane subId="Problems">
<PATH> <PATH>
@ -335,45 +271,84 @@
</PATH> </PATH>
</subPane> </subPane>
</pane> </pane>
<pane id="ProjectPane">
<subPane />
</pane>
<pane id="PackagesPane">
<subPane />
</pane>
<pane id="Favorites" /> <pane id="Favorites" />
</panes> </panes>
</component> </component>
<component name="PropertiesComponent"> <component name="PropertiesComponent">
<property name="GoToClass.includeJavaFiles" value="false" />
<property name="project.structure.proportion" value="0.15" /> <property name="project.structure.proportion" value="0.15" />
<property name="GoToClass.includeJavaFiles" value="false" />
<property name="MemberChooser.copyJavadoc" value="false" /> <property name="MemberChooser.copyJavadoc" value="false" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder2" value="2" /> <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="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="FileHistory.git4idea.history.GitHistoryProvider_treeWidth3" value="924" />
<property name="GoToFile.includeJavaFiles" value="false" /> <property name="GoToFile.includeJavaFiles" value="false" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatWidth0" value="113" /> <property name="FileHistory.git4idea.history.GitHistoryProvider_flatWidth0" value="113" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeWidth2" value="130" /> <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="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="MemberChooser.showClasses" value="true" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder0" value="0" /> <property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder0" value="0" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder3" value="3" /> <property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder3" value="3" />
<property name="project.structure.side.proportion" value="0.2" /> <property name="project.structure.side.proportion" value="0.2" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeWidth0" value="124" /> <property name="FileHistory.git4idea.history.GitHistoryProvider_treeWidth0" value="124" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeOrder3" value="3" /> <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="GoToClass.toSaveIncludeLibraries" value="false" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeOrder0" value="0" /> <property name="FileHistory.git4idea.history.GitHistoryProvider_treeOrder0" value="0" />
<property name="RunManagerConfig.showSettingsBeforeRunnig" value="false" /> <property name="RunManagerConfig.showSettingsBeforeRunnig" value="false" />
<property name="project.structure.last.edited" value="Facets" /> <property name="project.structure.last.edited" value="Facets" />
<property name="MemberChooser.sorted" value="false" />
<property name="options.searchVisible" value="true" /> <property name="options.searchVisible" value="true" />
<property name="MemberChooser.sorted" value="false" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeOrder1" value="1" /> <property name="FileHistory.git4idea.history.GitHistoryProvider_treeOrder1" value="1" />
<property name="recentsLimit" value="5" /> <property name="recentsLimit" value="5" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_treeWidth1" value="162" /> <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="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_flatWidth1" value="166" />
<property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder1" value="1" /> <property name="FileHistory.git4idea.history.GitHistoryProvider_flatOrder1" value="1" />
</component> </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"> <configuration default="true" type="Remote" factoryName="Remote">
<option name="USE_SOCKET_TRANSPORT" value="true" /> <option name="USE_SOCKET_TRANSPORT" value="true" />
<option name="SERVER_MODE" value="false" /> <option name="SERVER_MODE" value="false" />
@ -426,14 +401,6 @@
<option name="Maven.BeforeRunTask" enabled="false" /> <option name="Maven.BeforeRunTask" enabled="false" />
</method> </method>
</configuration> </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"> <configuration default="true" type="TestNG" factoryName="TestNG" enabled="false" merge="false" runner="emma">
<module name="" /> <module name="" />
<option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" /> <option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
@ -465,6 +432,14 @@
<option name="Maven.BeforeRunTask" enabled="false" /> <option name="Maven.BeforeRunTask" enabled="false" />
</method> </method>
</configuration> </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"> <configuration default="true" type="JUnit" factoryName="JUnit" enabled="false" merge="false" runner="emma">
<module name="" /> <module name="" />
<option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" /> <option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
@ -561,7 +536,13 @@
<value defaultName="moduleWithDependencies" /> <value defaultName="moduleWithDependencies" />
</option> </option>
<envs /> <envs />
<RunnerSettings RunnerId="Debug">
<option name="DEBUG_PORT" value="53499" />
<option name="TRANSPORT" value="0" />
<option name="LOCAL" value="true" />
</RunnerSettings>
<RunnerSettings RunnerId="Run" /> <RunnerSettings RunnerId="Run" />
<ConfigurationWrapper RunnerId="Debug" />
<ConfigurationWrapper RunnerId="Run" /> <ConfigurationWrapper RunnerId="Run" />
<method> <method>
<option name="Make" enabled="false" /> <option name="Make" enabled="false" />
@ -613,17 +594,24 @@
<value defaultName="moduleWithDependencies" /> <value defaultName="moduleWithDependencies" />
</option> </option>
<envs /> <envs />
<RunnerSettings RunnerId="Debug">
<option name="DEBUG_PORT" value="52630" />
<option name="TRANSPORT" value="0" />
<option name="LOCAL" value="true" />
</RunnerSettings>
<RunnerSettings RunnerId="Run" /> <RunnerSettings RunnerId="Run" />
<ConfigurationWrapper RunnerId="Debug" />
<ConfigurationWrapper RunnerId="Run" /> <ConfigurationWrapper RunnerId="Run" />
<method> <method>
<option name="Make" enabled="false" /> <option name="Make" enabled="false" />
</method> </method>
</configuration> </configuration>
<list size="4"> <list size="5">
<item index="0" class="java.lang.String" itemvalue="JUnit.RemoteInMemoryStateTest" /> <item index="0" class="java.lang.String" itemvalue="JUnit.InMemNestedStateTest.testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess" />
<item index="1" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec" /> <item index="1" class="java.lang.String" itemvalue="JUnit.RemoteInMemoryStateTest" />
<item index="2" class="java.lang.String" itemvalue="JUnit.InMemoryStateTest" /> <item index="2" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec" />
<item index="3" class="java.lang.String" itemvalue="JUnit.InMemNestedStateTest" /> <item index="3" class="java.lang.String" itemvalue="JUnit.InMemoryStateTest" />
<item index="4" class="java.lang.String" itemvalue="JUnit.InMemNestedStateTest" />
</list> </list>
<configuration name="&lt;template&gt;" type="WebApp" default="true" selected="false"> <configuration name="&lt;template&gt;" type="WebApp" default="true" selected="false">
<Host>localhost</Host> <Host>localhost</Host>
@ -664,29 +652,29 @@
</option> </option>
</component> </component>
<component name="ToolWindowManager"> <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" /> <editor active="true" />
<layout> <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="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="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="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="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="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="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="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="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="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="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="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="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" /> <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> </layout>
@ -723,100 +711,86 @@
<option name="FILTER_TARGETS" value="false" /> <option name="FILTER_TARGETS" value="false" />
</component> </component>
<component name="editorHistoryManager"> <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"> <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> </provider>
</entry> </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"> <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> </provider>
</entry> </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"> <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> </provider>
</entry> </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"> <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> </provider>
</entry> </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"> <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> </provider>
</entry> </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"> <provider selected="true" editor-type-id="text-editor">
<state line="39" column="31" selection-start="1169" selection-end="1169" 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$/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>
</provider> </provider>
</entry> </entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala"> <entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala">
<provider selected="true" editor-type-id="text-editor"> <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 /> <folding />
</state> </state>
</provider> </provider>
</entry> </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"> <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 /> <folding />
</state> </state>
</provider> </provider>

View file

@ -19,7 +19,7 @@
# FQN to the class doing initial active object/actor # FQN to the class doing initial active object/actor
# supervisor bootstrap, should be defined in default constructor # 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> <actor>
timeout = 5000 # default timeout for future based invocations 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 hostname = "127.0.0.1" # IP address or hostname of the MongoDB DB instance
port = 27017 port = 27017
dbname = "mydb" dbname = "mydb"
storage-format = "scala-json" # Options: java, scala-json, java-json, protobuf
</mongodb> </mongodb>
</storage> </storage>
</akka> </akka>