added init tx state hook for active objects, rewrote mongodb test

This commit is contained in:
jboner 2009-09-23 09:02:14 +02:00
parent 6cc3d87b5f
commit cdd8a35176
17 changed files with 538 additions and 593 deletions

View file

@ -25,11 +25,12 @@ class ActiveObjectInvocationTimeoutException(msg: String) extends ActiveObjectEx
object Annotations {
import se.scalablesolutions.akka.annotation._
val oneway = classOf[oneway]
val transactionrequired = classOf[transactionrequired]
val prerestart = classOf[prerestart]
val postrestart = classOf[postrestart]
val immutable = classOf[immutable]
val oneway = classOf[oneway]
val transactionrequired = classOf[transactionrequired]
val prerestart = classOf[prerestart]
val postrestart = classOf[postrestart]
val immutable = classOf[immutable]
val inittransactionalstate = classOf[inittransactionalstate]
}
/**
@ -397,6 +398,7 @@ private[akka] class Dispatcher(val callbacks: Option[RestartCallbacks]) extends
private[actor] var target: Option[AnyRef] = None
private var preRestart: Option[Method] = None
private var postRestart: Option[Method] = None
private var initTxState: Option[Method] = None
private[actor] def initialize(targetClass: Class[_], targetInstance: AnyRef) = {
if (targetClass.isAnnotationPresent(Annotations.transactionrequired)) makeTransactionRequired
@ -417,8 +419,8 @@ private[akka] class Dispatcher(val callbacks: Option[RestartCallbacks]) extends
}
// See if we have any annotation defined restart callbacks
if (!preRestart.isDefined) preRestart = methods.find( m => m.isAnnotationPresent(Annotations.prerestart))
if (!postRestart.isDefined) postRestart = methods.find( m => m.isAnnotationPresent(Annotations.postrestart))
if (!preRestart.isDefined) preRestart = methods.find(m => m.isAnnotationPresent(Annotations.prerestart))
if (!postRestart.isDefined) postRestart = methods.find(m => m.isAnnotationPresent(Annotations.postrestart))
if (preRestart.isDefined && preRestart.get.getParameterTypes.length != 0)
throw new IllegalStateException("Method annotated with @prerestart or defined as a restart callback in [" + targetClass.getName + "] must have a zero argument definition")
@ -427,6 +429,11 @@ private[akka] class Dispatcher(val callbacks: Option[RestartCallbacks]) extends
if (preRestart.isDefined) preRestart.get.setAccessible(true)
if (postRestart.isDefined) postRestart.get.setAccessible(true)
// see if we have a method annotated with @inittransactionalstate, if so invoke it
initTxState = methods.find(m => m.isAnnotationPresent(Annotations.inittransactionalstate))
if (initTxState.isDefined && initTxState.get.getParameterTypes.length != 0) throw new IllegalStateException("Method annotated with @inittransactionalstate must have a zero argument definition")
if (initTxState.isDefined) initTxState.get.setAccessible(true)
}
override def receive: PartialFunction[Any, Unit] = {
@ -450,6 +457,13 @@ private[akka] class Dispatcher(val callbacks: Option[RestartCallbacks]) extends
} catch { case e: InvocationTargetException => throw e.getCause }
}
override protected def initTransactionalState() {
try {
if (initTxState.isDefined && target.isDefined) initTxState.get.invoke(target.get, ZERO_ITEM_OBJECT_ARRAY: _*)
} catch { case e: InvocationTargetException => throw e.getCause }
}
private def serializeArguments(joinpoint: JoinPoint) = {
val args = joinpoint.getRtti.asInstanceOf[MethodRtti].getParameterValues
var unserializable = false

View file

@ -63,9 +63,11 @@ trait Actor extends Logging with TransactionManagement {
private var hotswap: Option[PartialFunction[Any, Unit]] = None
private var config: Option[AnyRef] = None
@volatile protected[this] var isTransactional = false
@volatile protected[this] var remoteAddress: Option[InetSocketAddress] = None
@volatile protected[akka] var supervisor: Option[Actor] = None
protected[Actor] var mailbox: MessageQueue = _
protected[this] var senderFuture: Option[CompletableFutureResult] = None
protected[this] val linkedActors = new CopyOnWriteArraySet[Actor]
@ -169,15 +171,6 @@ trait Actor extends Logging with TransactionManagement {
*/
protected def init(config: AnyRef) = {}
/**
* User overridable callback/setting.
*
* Optional callback method that is called during initialization.
* Used to initialize transactional state.
* To be implemented by subclassing actor.
*/
protected def initializeTransactionalState = {}
/**
* User overridable callback/setting.
*
@ -194,6 +187,14 @@ trait Actor extends Logging with TransactionManagement {
*/
protected def postRestart(reason: AnyRef, config: Option[AnyRef]) = {}
/**
* User overridable callback/setting.
*
* Optional callback method that is called during termination.
* To be implemented by subclassing actor.
*/
protected def initTransactionalState() = {}
/**
* User overridable callback/setting.
*
@ -552,10 +553,10 @@ trait Actor extends Logging with TransactionManagement {
private val lifeCycle: PartialFunction[Any, Unit] = {
case Init(config) => init(config)
case TransactionalInit => initializeTransactionalState
case HotSwap(code) => hotswap = code
case Restart(reason) => restart(reason)
case Exit(dead, reason) => handleTrapExit(dead, reason)
case TransactionalInit => initTransactionalState
}
private[this] def handleTrapExit(dead: Actor, reason: Throwable): Unit = {

View file

@ -1,101 +0,0 @@
/**
* Copyright (C) 2009 Scalable Solutions.
*/
package se.scalablesolutions.akka.camel
/*
import config.ActiveObjectGuiceConfigurator
import annotation.oneway
import config.ScalaConfig._
import com.google.inject.{AbstractModule, Scopes}
//import com.jteigen.scalatest.JUnit4Runner
import org.apache.camel.component.bean.ProxyHelper
import org.junit.runner.RunWith
import org.scalatest._
import org.scalatest.matchers._
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.camel.CamelContext
import org.apache.camel.Endpoint
import org.apache.camel.Exchange
import org.apache.camel.Processor
import org.apache.camel.Producer
import org.apache.camel.builder.RouteBuilder
import org.apache.camel.impl.DefaultCamelContext
// REQUIRES: -Djava.naming.factory.initial=org.apache.camel.util.jndi.CamelInitialContextFactory
*/
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*
//@RunWith(classOf[JUnit4Runner])
class CamelSpec extends Spec with ShouldMatchers {
describe("A Camel routing scheme") {
it("should route message from direct:test to actor A using @Bean endpoint") {
val latch = new CountDownLatch(1);
val conf = new ActiveObjectGuiceConfigurator
conf.configure(
RestartStrategy(AllForOne, 3, 5000),
Component(
"camelfoo",
classOf[CamelFoo],
classOf[CamelFooImpl],
LifeCycle(Permanent, 1000),
1000) ::
Nil
).addRoutes(new RouteBuilder() {
def configure = {
from("direct:test").to("bean:camelfoo").process(new Processor() {
def process(e: Exchange) = {
println("Received exchange: " + e.getIn())
latch.countDown
}
})
}}
).supervise
val endpoint = conf.getRoutingEndpoint("direct:test")
val proxy = ProxyHelper.createProxy(endpoint, classOf[CamelFoo])
proxy.foo("hello there")
val exchange = endpoint.createExchange
println("----- " + exchange)
exchange.getIn().setBody("hello there")
val producer = endpoint.createProducer
println("----- " + producer)
producer.process(exchange)
// now lets sleep for a while
val received = latch.await(5, TimeUnit.SECONDS)
received should equal (true)
conf.stop
}
}
}
trait CamelFoo {
@oneway def foo(msg: String)
}
trait CamelBar {
def bar(msg: String): String
}
class CamelFooImpl extends CamelFoo {
def foo(msg: String) = println("CamelFoo.foo:" + msg)
}
class CamelBarImpl extends CamelBar {
def bar(msg: String) = msg + "return_bar "
}
*/

View file

@ -27,15 +27,9 @@ class InMemStatefulActor extends Actor {
timeout = 100000
makeTransactionRequired
//dispatcher = se.scalablesolutions.akka.reactor.Dispatchers.newThreadBasedDispatcher(this)
private var mapState: TransactionalMap[String, String] = _
private var vectorState: TransactionalVector[String] = _
private var refState: TransactionalRef[String] = _
override def initializeTransactionalState = {
mapState = TransactionalState.newMap[String, String]
vectorState = TransactionalState.newVector[String]
refState = TransactionalState.newRef[String]
}
private lazy val mapState: TransactionalMap[String, String] = TransactionalState.newMap[String, String]
private lazy val vectorState: TransactionalVector[String] = TransactionalState.newVector[String]
private lazy val refState: TransactionalRef[String] = TransactionalState.newRef[String]
def receive: PartialFunction[Any, Unit] = {
case GetMapState(key) =>

View file

@ -9,11 +9,11 @@ public class AllTest extends TestCase {
TestSuite suite = new TestSuite("All Java tests");
suite.addTestSuite(InMemoryStateTest.class);
suite.addTestSuite(InMemNestedStateTest.class);
suite.addTestSuite(RemoteInMemoryStateTest.class);
suite.addTestSuite(ActiveObjectGuiceConfiguratorTest.class);
//suite.addTestSuite(PersistentStateTest.class);
//suite.addTestSuite(PersistentNestedStateTest.class);
suite.addTestSuite(RemoteInMemoryStateTest.class);
//suite.addTestSuite(RemotePersistentStateTest.class);
suite.addTestSuite(ActiveObjectGuiceConfiguratorTest.class);
//suite.addTestSuite(RestTest.class);
return suite;
}

View file

@ -3,15 +3,22 @@ package se.scalablesolutions.akka.api;
import se.scalablesolutions.akka.annotation.transactionrequired;
import se.scalablesolutions.akka.annotation.prerestart;
import se.scalablesolutions.akka.annotation.postrestart;
import se.scalablesolutions.akka.annotation.inittransactionalstate;
import se.scalablesolutions.akka.state.*;
@transactionrequired
public class InMemStateful {
private TransactionalState factory = new TransactionalState();
private TransactionalMap<String, String> mapState = factory.newMap();
private TransactionalVector<String> vectorState = factory.newVector();
private TransactionalRef<String> refState = factory.newRef();
private TransactionalMap<String, String> mapState;
private TransactionalVector<String> vectorState;
private TransactionalRef<String> refState;
@inittransactionalstate
public void init() {
mapState = TransactionalState.newMap();
vectorState = TransactionalState.newVector();
refState = TransactionalState.newRef();
}
public String getMapState(String key) {
return (String)mapState.get(key).get();
}

View file

@ -1,15 +1,21 @@
package se.scalablesolutions.akka.api;
import se.scalablesolutions.akka.annotation.transactionrequired;
import se.scalablesolutions.akka.annotation.inittransactionalstate;
import se.scalablesolutions.akka.state.*;
@transactionrequired
public class InMemStatefulNested {
private TransactionalState factory = new TransactionalState();
private TransactionalMap<String, String> mapState = factory.newMap();
private TransactionalVector<String> vectorState = factory.newVector();
private TransactionalRef<String> refState = factory.newRef();
private TransactionalMap<String, String> mapState;
private TransactionalVector<String> vectorState;
private TransactionalRef<String> refState;
@inittransactionalstate
public void init() {
mapState = TransactionalState.newMap();
vectorState = TransactionalState.newVector();
refState = TransactionalState.newRef();
}
public String getMapState(String key) {
return (String)mapState.get(key).get();

View file

@ -35,8 +35,8 @@ public class InMemoryStateTest extends TestCase {
}
protected void tearDown() {
conf.stop();
}
conf.stop();
}
public void testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess() {
InMemStateful stateful = conf.getInstance(InMemStateful.class);

View file

@ -1,11 +1,16 @@
package se.scalablesolutions.akka.api;
import se.scalablesolutions.akka.state.*;
import se.scalablesolutions.akka.annotation.inittransactionalstate;
public class PersistentClasher {
private PersistentState factory = new PersistentState();
private PersistentMap state = factory.newMap(new CassandraStorageConfig());
private PersistentMap state;
@inittransactionalstate
public void init() {
state = PersistentState.newMap(new CassandraStorageConfig());
}
public String getState(String key) {
return (String)state.get(key).get();
}

View file

@ -1,16 +1,22 @@
package se.scalablesolutions.akka.api;
import se.scalablesolutions.akka.annotation.inittransactionalstate;
import se.scalablesolutions.akka.annotation.transactionrequired;
import se.scalablesolutions.akka.state.*;
@transactionrequired
public class PersistentStateful {
private PersistentState factory = new PersistentState();
private PersistentMap mapState = factory.newMap(new CassandraStorageConfig());
private PersistentVector vectorState = factory.newVector(new CassandraStorageConfig());;
private PersistentRef refState = factory.newRef(new CassandraStorageConfig());
private PersistentMap mapState;
private PersistentVector vectorState;
private PersistentRef refState;
@inittransactionalstate
public void init() {
mapState = PersistentState.newMap(new CassandraStorageConfig());
vectorState = PersistentState.newVector(new CassandraStorageConfig());
refState = PersistentState.newRef(new CassandraStorageConfig());
}
public String getMapState(String key) {
return (String) mapState.get(key).get();
}

View file

@ -1,15 +1,21 @@
package se.scalablesolutions.akka.api;
import se.scalablesolutions.akka.annotation.inittransactionalstate;
import se.scalablesolutions.akka.annotation.transactionrequired;
import se.scalablesolutions.akka.state.*;
@transactionrequired
public class PersistentStatefulNested {
private PersistentState factory = new PersistentState();
private PersistentMap mapState = factory.newMap(new CassandraStorageConfig());
private PersistentVector vectorState = factory.newVector(new CassandraStorageConfig());;
private PersistentRef refState = factory.newRef(new CassandraStorageConfig());
private PersistentMap mapState;
private PersistentVector vectorState;
private PersistentRef refState;
@inittransactionalstate
public void init() {
mapState = PersistentState.newMap(new CassandraStorageConfig());
vectorState = PersistentState.newVector(new CassandraStorageConfig());
refState = PersistentState.newRef(new CassandraStorageConfig());
}
public String getMapState(String key) {
return (String) mapState.get(key).get();

View file

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

View file

@ -31,15 +31,9 @@ class CassandraPersistentActor extends Actor {
timeout = 100000
makeTransactionRequired
private var mapState: PersistentMap = _
private var vectorState: PersistentVector = _
private var refState: PersistentRef = _
override def initializeTransactionalState = {
mapState = PersistentState.newMap(CassandraStorageConfig())
vectorState = PersistentState.newVector(CassandraStorageConfig())
refState = PersistentState.newRef(CassandraStorageConfig())
}
private lazy val mapState: PersistentMap = PersistentState.newMap(CassandraStorageConfig())
private lazy val vectorState: PersistentVector = PersistentState.newVector(CassandraStorageConfig())
private lazy val refState: PersistentRef = PersistentState.newRef(CassandraStorageConfig())
def receive: PartialFunction[Any, Unit] = {
case GetMapState(key) =>

View file

@ -29,12 +29,8 @@ case object LogSize
class BankAccountActor extends Actor {
makeTransactionRequired
private var accountState: PersistentMap = _
private var txnLog: PersistentVector = _
override def initializeTransactionalState = {
accountState = PersistentState.newMap(MongoStorageConfig())
txnLog = PersistentState.newVector(MongoStorageConfig())
}
private lazy val accountState: PersistentMap = PersistentState.newMap(MongoStorageConfig())
private lazy val txnLog: PersistentVector = PersistentState.newVector(MongoStorageConfig())
def receive: PartialFunction[Any, Unit] = {
// check balance

View file

@ -1,337 +1,337 @@
package se.scalablesolutions.akka.state
import junit.framework.TestCase
import org.junit.{Test, Before}
import org.junit.Assert._
import dispatch.json._
import dispatch.json.Js._
class MongoStorageSpec extends TestCase {
val changeSetV = new scala.collection.mutable.ArrayBuffer[AnyRef]
val changeSetM = new scala.collection.mutable.HashMap[AnyRef, AnyRef]
override def setUp = {
MongoStorage.coll.drop
}
@Test
def testVectorInsertForTransactionId = {
changeSetV += "debasish" // string
changeSetV += List(1, 2, 3) // Scala List
changeSetV += List(100, 200)
MongoStorage.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
assertEquals(
3,
MongoStorage.getVectorStorageSizeFor("U-A1"))
changeSetV.clear
// changeSetV should be reinitialized
changeSetV += List(12, 23, 45)
changeSetV += "maulindu"
MongoStorage.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
assertEquals(
5,
MongoStorage.getVectorStorageSizeFor("U-A1"))
// add more to the same changeSetV
changeSetV += "ramanendu"
changeSetV += Map(1 -> "dg", 2 -> "mc")
// add for a diff transaction
MongoStorage.insertVectorStorageEntriesFor("U-A2", changeSetV.toList)
assertEquals(
4,
MongoStorage.getVectorStorageSizeFor("U-A2"))
// previous transaction change set should remain same
assertEquals(
5,
MongoStorage.getVectorStorageSizeFor("U-A1"))
// test single element entry
MongoStorage.insertVectorStorageEntryFor("U-A1", Map(1->1, 2->4, 3->9))
assertEquals(
6,
MongoStorage.getVectorStorageSizeFor("U-A1"))
}
@Test
def testVectorFetchForKeys = {
// initially everything 0
assertEquals(
0,
MongoStorage.getVectorStorageSizeFor("U-A2"))
assertEquals(
0,
MongoStorage.getVectorStorageSizeFor("U-A1"))
// get some stuff
changeSetV += "debasish"
changeSetV += List(BigDecimal(12), BigDecimal(13), BigDecimal(14))
MongoStorage.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
assertEquals(
2,
MongoStorage.getVectorStorageSizeFor("U-A1"))
val JsString(str) = MongoStorage.getVectorStorageEntryFor("U-A1", 0).asInstanceOf[JsString]
assertEquals("debasish", str)
import dispatch.json.Js._
val l = MongoStorage.getVectorStorageEntryFor("U-A1", 1).asInstanceOf[JsValue]
val num_list = list ! num
val num_list(l0) = l
assertEquals(List(12, 13, 14), l0)
changeSetV.clear
changeSetV += Map(1->1, 2->4, 3->9)
changeSetV += BigInt(2310)
changeSetV += List(100, 200, 300)
MongoStorage.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
assertEquals(
5,
MongoStorage.getVectorStorageSizeFor("U-A1"))
val r =
MongoStorage.getVectorStorageRangeFor("U-A1", Some(1), None, 3)
assertEquals(3, r.size)
val lr = r(0).asInstanceOf[JsValue]
val num_list(l1) = lr
assertEquals(List(12, 13, 14), l1)
}
@Test
def testVectorFetchForNonExistentKeys = {
try {
MongoStorage.getVectorStorageEntryFor("U-A1", 1)
fail("should throw an exception")
} catch {case e: Predef.NoSuchElementException => {}}
try {
MongoStorage.getVectorStorageRangeFor("U-A1", Some(2), None, 12)
fail("should throw an exception")
} catch {case e: Predef.NoSuchElementException => {}}
}
@Test
def testMapInsertForTransactionId = {
case class Foo(no: Int, name: String)
fillMap
// add some more to changeSet
changeSetM += "5" -> Foo(12, "dg")
changeSetM += "6" -> java.util.Calendar.getInstance.getTime
// insert all into Mongo
MongoStorage.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
assertEquals(
6,
MongoStorage.getMapStorageSizeFor("U-M1"))
// individual insert api
MongoStorage.insertMapStorageEntryFor("U-M1", "7", "akka")
MongoStorage.insertMapStorageEntryFor("U-M1", "8", List(23, 25))
assertEquals(
8,
MongoStorage.getMapStorageSizeFor("U-M1"))
// add the same changeSet for another transaction
MongoStorage.insertMapStorageEntriesFor("U-M2", changeSetM.toList)
assertEquals(
6,
MongoStorage.getMapStorageSizeFor("U-M2"))
// the first transaction should remain the same
assertEquals(
8,
MongoStorage.getMapStorageSizeFor("U-M1"))
changeSetM.clear
}
@Test
def testMapContents = {
fillMap
MongoStorage.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
MongoStorage.getMapStorageEntryFor("U-M1", "2") match {
case Some(x) => {
val JsString(str) = x.asInstanceOf[JsValue]
assertEquals("peter", str)
}
case None => fail("should fetch peter")
}
MongoStorage.getMapStorageEntryFor("U-M1", "4") match {
case Some(x) => {
val num_list = list ! num
val num_list(l0) = x.asInstanceOf[JsValue]
assertEquals(3, l0.size)
}
case None => fail("should fetch list")
}
MongoStorage.getMapStorageEntryFor("U-M1", "3") match {
case Some(x) => {
val num_list = list ! num
val num_list(l0) = x.asInstanceOf[JsValue]
assertEquals(2, l0.size)
}
case None => fail("should fetch list")
}
// get the entire map
val l: List[Tuple2[AnyRef, AnyRef]] =
MongoStorage.getMapStorageFor("U-M1")
assertEquals(4, l.size)
assertTrue(l.map(_._1).contains("1"))
assertTrue(l.map(_._1).contains("2"))
assertTrue(l.map(_._1).contains("3"))
assertTrue(l.map(_._1).contains("4"))
val JsString(str) = l.filter(_._1 == "2").first._2
assertEquals(str, "peter")
// trying to fetch for a non-existent transaction will throw
try {
MongoStorage.getMapStorageFor("U-M2")
fail("should throw an exception")
} catch {case e: Predef.NoSuchElementException => {}}
changeSetM.clear
}
@Test
def testMapContentsByRange = {
fillMap
changeSetM += "5" -> Map(1 -> "dg", 2 -> "mc")
MongoStorage.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
// specify start and count
val l: List[Tuple2[AnyRef, AnyRef]] =
MongoStorage.getMapStorageRangeFor(
"U-M1", Some(Integer.valueOf(2)), None, 3)
assertEquals(3, l.size)
assertEquals("3", l(0)._1.asInstanceOf[String])
val lst = l(0)._2.asInstanceOf[JsValue]
val num_list = list ! num
val num_list(l0) = lst
assertEquals(List(100, 200), l0)
assertEquals("4", l(1)._1.asInstanceOf[String])
val ls = l(1)._2.asInstanceOf[JsValue]
val num_list(l1) = ls
assertEquals(List(10, 20, 30), l1)
// specify start, finish and count where finish - start == count
assertEquals(3,
MongoStorage.getMapStorageRangeFor(
"U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(5)), 3).size)
// specify start, finish and count where finish - start > count
assertEquals(3,
MongoStorage.getMapStorageRangeFor(
"U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(9)), 3).size)
// do not specify start or finish
assertEquals(3,
MongoStorage.getMapStorageRangeFor(
"U-M1", None, None, 3).size)
// specify finish and count
assertEquals(3,
MongoStorage.getMapStorageRangeFor(
"U-M1", None, Some(Integer.valueOf(3)), 3).size)
// specify start, finish and count where finish < start
assertEquals(3,
MongoStorage.getMapStorageRangeFor(
"U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(1)), 3).size)
changeSetM.clear
}
@Test
def testMapStorageRemove = {
fillMap
changeSetM += "5" -> Map(1 -> "dg", 2 -> "mc")
MongoStorage.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
assertEquals(5,
MongoStorage.getMapStorageSizeFor("U-M1"))
// remove key "3"
MongoStorage.removeMapStorageFor("U-M1", "3")
assertEquals(4,
MongoStorage.getMapStorageSizeFor("U-M1"))
try {
MongoStorage.getMapStorageEntryFor("U-M1", "3")
fail("should throw exception")
} catch { case e => {}}
// remove the whole stuff
MongoStorage.removeMapStorageFor("U-M1")
try {
MongoStorage.getMapStorageFor("U-M1")
fail("should throw exception")
} catch { case e: NoSuchElementException => {}}
changeSetM.clear
}
private def fillMap = {
changeSetM += "1" -> "john"
changeSetM += "2" -> "peter"
changeSetM += "3" -> List(100, 200)
changeSetM += "4" -> List(10, 20, 30)
changeSetM
}
@Test
def testRefStorage = {
MongoStorage.getRefStorageFor("U-R1") match {
case None =>
case Some(o) => fail("should be None")
}
val m = Map("1"->1, "2"->4, "3"->9)
MongoStorage.insertRefStorageFor("U-R1", m)
MongoStorage.getRefStorageFor("U-R1") match {
case None => fail("should not be empty")
case Some(r) => {
val a = r.asInstanceOf[JsValue]
val m1 = Symbol("1") ? num
val m2 = Symbol("2") ? num
val m3 = Symbol("3") ? num
val m1(n1) = a
val m2(n2) = a
val m3(n3) = a
assertEquals(n1, 1)
assertEquals(n2, 4)
assertEquals(n3, 9)
}
}
// insert another one
// the previous one should be replaced
val b = List("100", "jonas")
MongoStorage.insertRefStorageFor("U-R1", b)
MongoStorage.getRefStorageFor("U-R1") match {
case None => fail("should not be empty")
case Some(r) => {
val a = r.asInstanceOf[JsValue]
val str_lst = list ! str
val str_lst(l) = a
assertEquals(b, l)
}
}
}
}
package se.scalablesolutions.akka.state
import junit.framework.TestCase
import org.junit.{Test, Before}
import org.junit.Assert._
import dispatch.json._
import dispatch.json.Js._
class MongoStorageSpec extends TestCase {
val changeSetV = new scala.collection.mutable.ArrayBuffer[AnyRef]
val changeSetM = new scala.collection.mutable.HashMap[AnyRef, AnyRef]
override def setUp = {
MongoStorage.coll.drop
}
@Test
def testVectorInsertForTransactionId = {
changeSetV += "debasish" // string
changeSetV += List(1, 2, 3) // Scala List
changeSetV += List(100, 200)
MongoStorage.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
assertEquals(
3,
MongoStorage.getVectorStorageSizeFor("U-A1"))
changeSetV.clear
// changeSetV should be reinitialized
changeSetV += List(12, 23, 45)
changeSetV += "maulindu"
MongoStorage.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
assertEquals(
5,
MongoStorage.getVectorStorageSizeFor("U-A1"))
// add more to the same changeSetV
changeSetV += "ramanendu"
changeSetV += Map(1 -> "dg", 2 -> "mc")
// add for a diff transaction
MongoStorage.insertVectorStorageEntriesFor("U-A2", changeSetV.toList)
assertEquals(
4,
MongoStorage.getVectorStorageSizeFor("U-A2"))
// previous transaction change set should remain same
assertEquals(
5,
MongoStorage.getVectorStorageSizeFor("U-A1"))
// test single element entry
MongoStorage.insertVectorStorageEntryFor("U-A1", Map(1->1, 2->4, 3->9))
assertEquals(
6,
MongoStorage.getVectorStorageSizeFor("U-A1"))
}
@Test
def testVectorFetchForKeys = {
// initially everything 0
assertEquals(
0,
MongoStorage.getVectorStorageSizeFor("U-A2"))
assertEquals(
0,
MongoStorage.getVectorStorageSizeFor("U-A1"))
// get some stuff
changeSetV += "debasish"
changeSetV += List(BigDecimal(12), BigDecimal(13), BigDecimal(14))
MongoStorage.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
assertEquals(
2,
MongoStorage.getVectorStorageSizeFor("U-A1"))
val JsString(str) = MongoStorage.getVectorStorageEntryFor("U-A1", 0).asInstanceOf[JsString]
assertEquals("debasish", str)
import dispatch.json.Js._
val l = MongoStorage.getVectorStorageEntryFor("U-A1", 1).asInstanceOf[JsValue]
val num_list = list ! num
val num_list(l0) = l
assertEquals(List(12, 13, 14), l0)
changeSetV.clear
changeSetV += Map(1->1, 2->4, 3->9)
changeSetV += BigInt(2310)
changeSetV += List(100, 200, 300)
MongoStorage.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
assertEquals(
5,
MongoStorage.getVectorStorageSizeFor("U-A1"))
val r =
MongoStorage.getVectorStorageRangeFor("U-A1", Some(1), None, 3)
assertEquals(3, r.size)
val lr = r(0).asInstanceOf[JsValue]
val num_list(l1) = lr
assertEquals(List(12, 13, 14), l1)
}
@Test
def testVectorFetchForNonExistentKeys = {
try {
MongoStorage.getVectorStorageEntryFor("U-A1", 1)
fail("should throw an exception")
} catch {case e: Predef.NoSuchElementException => {}}
try {
MongoStorage.getVectorStorageRangeFor("U-A1", Some(2), None, 12)
fail("should throw an exception")
} catch {case e: Predef.NoSuchElementException => {}}
}
@Test
def testMapInsertForTransactionId = {
case class Foo(no: Int, name: String)
fillMap
// add some more to changeSet
changeSetM += "5" -> Foo(12, "dg")
changeSetM += "6" -> java.util.Calendar.getInstance.getTime
// insert all into Mongo
MongoStorage.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
assertEquals(
6,
MongoStorage.getMapStorageSizeFor("U-M1"))
// individual insert api
MongoStorage.insertMapStorageEntryFor("U-M1", "7", "akka")
MongoStorage.insertMapStorageEntryFor("U-M1", "8", List(23, 25))
assertEquals(
8,
MongoStorage.getMapStorageSizeFor("U-M1"))
// add the same changeSet for another transaction
MongoStorage.insertMapStorageEntriesFor("U-M2", changeSetM.toList)
assertEquals(
6,
MongoStorage.getMapStorageSizeFor("U-M2"))
// the first transaction should remain the same
assertEquals(
8,
MongoStorage.getMapStorageSizeFor("U-M1"))
changeSetM.clear
}
@Test
def testMapContents = {
fillMap
MongoStorage.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
MongoStorage.getMapStorageEntryFor("U-M1", "2") match {
case Some(x) => {
val JsString(str) = x.asInstanceOf[JsValue]
assertEquals("peter", str)
}
case None => fail("should fetch peter")
}
MongoStorage.getMapStorageEntryFor("U-M1", "4") match {
case Some(x) => {
val num_list = list ! num
val num_list(l0) = x.asInstanceOf[JsValue]
assertEquals(3, l0.size)
}
case None => fail("should fetch list")
}
MongoStorage.getMapStorageEntryFor("U-M1", "3") match {
case Some(x) => {
val num_list = list ! num
val num_list(l0) = x.asInstanceOf[JsValue]
assertEquals(2, l0.size)
}
case None => fail("should fetch list")
}
// get the entire map
val l: List[Tuple2[AnyRef, AnyRef]] =
MongoStorage.getMapStorageFor("U-M1")
assertEquals(4, l.size)
assertTrue(l.map(_._1).contains("1"))
assertTrue(l.map(_._1).contains("2"))
assertTrue(l.map(_._1).contains("3"))
assertTrue(l.map(_._1).contains("4"))
val JsString(str) = l.filter(_._1 == "2").first._2
assertEquals(str, "peter")
// trying to fetch for a non-existent transaction will throw
try {
MongoStorage.getMapStorageFor("U-M2")
fail("should throw an exception")
} catch {case e: Predef.NoSuchElementException => {}}
changeSetM.clear
}
@Test
def testMapContentsByRange = {
fillMap
changeSetM += "5" -> Map(1 -> "dg", 2 -> "mc")
MongoStorage.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
// specify start and count
val l: List[Tuple2[AnyRef, AnyRef]] =
MongoStorage.getMapStorageRangeFor(
"U-M1", Some(Integer.valueOf(2)), None, 3)
assertEquals(3, l.size)
assertEquals("3", l(0)._1.asInstanceOf[String])
val lst = l(0)._2.asInstanceOf[JsValue]
val num_list = list ! num
val num_list(l0) = lst
assertEquals(List(100, 200), l0)
assertEquals("4", l(1)._1.asInstanceOf[String])
val ls = l(1)._2.asInstanceOf[JsValue]
val num_list(l1) = ls
assertEquals(List(10, 20, 30), l1)
// specify start, finish and count where finish - start == count
assertEquals(3,
MongoStorage.getMapStorageRangeFor(
"U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(5)), 3).size)
// specify start, finish and count where finish - start > count
assertEquals(3,
MongoStorage.getMapStorageRangeFor(
"U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(9)), 3).size)
// do not specify start or finish
assertEquals(3,
MongoStorage.getMapStorageRangeFor(
"U-M1", None, None, 3).size)
// specify finish and count
assertEquals(3,
MongoStorage.getMapStorageRangeFor(
"U-M1", None, Some(Integer.valueOf(3)), 3).size)
// specify start, finish and count where finish < start
assertEquals(3,
MongoStorage.getMapStorageRangeFor(
"U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(1)), 3).size)
changeSetM.clear
}
@Test
def testMapStorageRemove = {
fillMap
changeSetM += "5" -> Map(1 -> "dg", 2 -> "mc")
MongoStorage.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
assertEquals(5,
MongoStorage.getMapStorageSizeFor("U-M1"))
// remove key "3"
MongoStorage.removeMapStorageFor("U-M1", "3")
assertEquals(4,
MongoStorage.getMapStorageSizeFor("U-M1"))
try {
MongoStorage.getMapStorageEntryFor("U-M1", "3")
fail("should throw exception")
} catch { case e => {}}
// remove the whole stuff
MongoStorage.removeMapStorageFor("U-M1")
try {
MongoStorage.getMapStorageFor("U-M1")
fail("should throw exception")
} catch { case e: NoSuchElementException => {}}
changeSetM.clear
}
private def fillMap = {
changeSetM += "1" -> "john"
changeSetM += "2" -> "peter"
changeSetM += "3" -> List(100, 200)
changeSetM += "4" -> List(10, 20, 30)
changeSetM
}
@Test
def testRefStorage = {
MongoStorage.getRefStorageFor("U-R1") match {
case None =>
case Some(o) => fail("should be None")
}
val m = Map("1"->1, "2"->4, "3"->9)
MongoStorage.insertRefStorageFor("U-R1", m)
MongoStorage.getRefStorageFor("U-R1") match {
case None => fail("should not be empty")
case Some(r) => {
val a = r.asInstanceOf[JsValue]
val m1 = Symbol("1") ? num
val m2 = Symbol("2") ? num
val m3 = Symbol("3") ? num
val m1(n1) = a
val m2(n2) = a
val m3(n3) = a
assertEquals(n1, 1)
assertEquals(n2, 4)
assertEquals(n3, 9)
}
}
// insert another one
// the previous one should be replaced
val b = List("100", "jonas")
MongoStorage.insertRefStorageFor("U-R1", b)
MongoStorage.getRefStorageFor("U-R1") match {
case None => fail("should not be empty")
case Some(r) => {
val a = r.asInstanceOf[JsValue]
val str_lst = list ! str
val str_lst(l) = a
assertEquals(b, l)
}
}
}
}

189
akka.iws
View file

@ -2,13 +2,23 @@
<project relativePaths="false" version="4">
<component name="ChangeListManager">
<list default="true" readonly="true" id="188c966f-a83c-4d3a-9128-54d5a2947a12" name="Default" comment="">
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-persistence/src/main/scala/MongoStorage.scala" afterPath="$PROJECT_DIR$/akka-persistence/src/main/scala/MongoStorage.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka.iml" afterPath="$PROJECT_DIR$/akka.iml" />
<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-persistence/src/main/scala/CassandraSession.scala" afterPath="$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraSession.scala" />
<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-persistence/src/test/scala/MongoPersistentActorSpec.scala" afterPath="$PROJECT_DIR$/akka-persistence/src/test/scala/MongoPersistentActorSpec.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-actors/src/main/scala/actor/ActiveObject.scala" afterPath="$PROJECT_DIR$/akka-actors/src/main/scala/actor/ActiveObject.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/PersistentStatefulNested.java" afterPath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/PersistentStatefulNested.java" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka.iws" afterPath="$PROJECT_DIR$/akka.iws" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-persistence/src/main/scala/Storage.scala" afterPath="$PROJECT_DIR$/akka-persistence/src/main/scala/Storage.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-persistence/src/main/scala/PersistentState.scala" afterPath="$PROJECT_DIR$/akka-persistence/src/main/scala/PersistentState.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/changes.xml" afterPath="$PROJECT_DIR$/changes.xml" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-persistence/src/test/scala/MongoStorageSpec.scala" afterPath="$PROJECT_DIR$/akka-persistence/src/test/scala/MongoStorageSpec.scala" />
<change type="DELETED" beforePath="$PROJECT_DIR$/akka-actors/src/test/scala/CamelSpec.scala" afterPath="" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/PersistentClasher.java" afterPath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/PersistentClasher.java" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemStateful.java" afterPath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemStateful.java" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/AllTest.java" afterPath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/AllTest.java" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-persistence/src/test/scala/CassandraPersistentActorSpec.scala" afterPath="$PROJECT_DIR$/akka-persistence/src/test/scala/CassandraPersistentActorSpec.scala" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/PersistentStateful.java" afterPath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/PersistentStateful.java" />
<change type="MODIFICATION" beforePath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemoryStateTest.java" afterPath="$PROJECT_DIR$/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemoryStateTest.java" />
<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" />
</list>
<ignored path=".idea/workspace.xml" />
<ignored path="akka.iws" />
@ -103,7 +113,26 @@
</component>
<component name="FileColors" enabled="false" enabledForTabs="false" />
<component name="FileEditorManager">
<leaf />
<leaf>
<file leaf-file-name="ActiveObject.scala" pinned="false" current="true" current-in-tab="true">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/actor/ActiveObject.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="471" column="49" selection-start="21206" selection-end="21206" vertical-scroll-proportion="1.4898785">
<folding />
</state>
</provider>
</entry>
</file>
<file leaf-file-name="Actor.scala" pinned="false" current="false" current-in-tab="false">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/actor/Actor.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="24" column="29" selection-start="733" selection-end="733" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
</file>
</leaf>
</component>
<component name="FindManager">
<FindUsagesManager>
@ -120,13 +149,14 @@
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/stm/Transaction.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/test/scala/InMemoryActorSpec.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-persistence/src/test/scala/AllTest.scala" />
<option value="$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraStorage.scala" />
<option value="$PROJECT_DIR$/akka-persistence/src/test/scala/CassandraPersistentActorSpec.scala" />
<option value="$PROJECT_DIR$/akka-persistence/src/test/scala/MongoPersistentActorSpec.scala" />
<option value="$PROJECT_DIR$/akka-persistence/src/main/scala/PersistentState.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionalState.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/actor/Actor.scala" />
<option value="$PROJECT_DIR$/akka-actors/src/main/scala/actor/ActiveObject.scala" />
</list>
</option>
</component>
@ -182,6 +212,10 @@
<sortByType />
</navigator>
<panes>
<pane id="Favorites" />
<pane id="ProjectPane">
<subPane />
</pane>
<pane id="Scope">
<subPane subId="Problems">
<PATH>
@ -208,9 +242,6 @@
</PATH>
</subPane>
</pane>
<pane id="ProjectPane">
<subPane />
</pane>
<pane id="PackagesPane">
<subPane>
<PATH>
@ -221,7 +252,6 @@
</PATH>
</subPane>
</pane>
<pane id="Favorites" />
</panes>
</component>
<component name="PropertiesComponent">
@ -547,12 +577,12 @@
</method>
</configuration>
<list size="6">
<item index="0" class="java.lang.String" itemvalue="JUnit.STMBugReproducer" />
<item index="1" class="java.lang.String" itemvalue="JUnit.RemoteInMemoryStateTest" />
<item index="2" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec" />
<item index="3" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec.testVectorShouldRollbackStateForStatefulServerInCaseOfFailure" />
<item index="4" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec.testRefShouldNotRollbackStateForStatefulServerInCaseOfSuccess" />
<item index="5" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec.testOneWayRefShouldRollbackStateForStatefulServerInCaseOfFailure" />
<item index="0" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec.testVectorShouldRollbackStateForStatefulServerInCaseOfFailure" />
<item index="1" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec.testRefShouldNotRollbackStateForStatefulServerInCaseOfSuccess" />
<item index="2" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec.testOneWayRefShouldRollbackStateForStatefulServerInCaseOfFailure" />
<item index="3" class="java.lang.String" itemvalue="JUnit.STMBugReproducer" />
<item index="4" class="java.lang.String" itemvalue="JUnit.RemoteInMemoryStateTest" />
<item index="5" class="java.lang.String" itemvalue="JUnit.InMemoryActorSpec" />
</list>
<configuration name="&lt;template&gt;" type="WebApp" default="true" selected="false">
<Host>localhost</Host>
@ -587,24 +617,22 @@
</list>
</option>
<option name="created" value="1252679070245" />
<option name="updated" value="1252772169806" />
<option name="updated" value="1253386287653" />
</task>
</list>
</option>
</component>
<component name="ToolWindowManager">
<frame x="4" y="22" width="1436" height="878" extended-state="6" />
<editor active="false" />
<frame x="4" y="22" width="1436" height="878" extended-state="0" />
<editor active="true" />
<layout>
<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="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="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="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="true" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="true" weight="0.19264069" sideWeight="0.6747449" 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.32908162" sideWeight="0.5" order="1" side_tool="false" />
<window_info id="Project" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="true" weight="0.19264069" sideWeight="0.6747449" 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.32908162" 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" />
@ -615,6 +643,8 @@
<window_info id="IDEtalk Messages" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="7" side_tool="false" />
<window_info id="Version Control" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="7" 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.32908162" 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.32908162" sideWeight="0.5" order="7" 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="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" />
@ -652,11 +682,6 @@
<option name="FILTER_TARGETS" value="false" />
</component>
<component name="editorHistoryManager">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/nio/RemoteServer.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="73" column="37" selection-start="2679" selection-end="2679" 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="90" column="0" selection-start="2585" selection-end="2585" vertical-scroll-proportion="0.0">
@ -666,91 +691,81 @@
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/reactor/Reactor.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="32" column="45" selection-start="666" selection-end="666" vertical-scroll-proportion="0.0">
<folding />
</state>
<state line="32" column="45" selection-start="666" selection-end="666" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-rest/src/main/scala/ActorComponentProviderFactory.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="14" column="6" selection-start="335" selection-end="335" vertical-scroll-proportion="0.0">
<folding />
</state>
<state line="14" column="6" selection-start="335" selection-end="335" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/reactor/MessageDispatcherBase.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="47" column="21" selection-start="1413" selection-end="1413" vertical-scroll-proportion="0.0">
<folding />
</state>
<state line="47" column="21" selection-start="1413" selection-end="1413" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraSession.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="27" column="7" selection-start="637" selection-end="637" vertical-scroll-proportion="-0.3102493">
<folding />
</state>
<state line="27" column="7" selection-start="637" selection-end="637" vertical-scroll-proportion="-0.3102493" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionManagement.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="16" column="0" selection-start="405" selection-end="405" vertical-scroll-proportion="0.0">
<state line="16" column="0" selection-start="405" selection-end="405" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="12" column="4" selection-start="453" selection-end="453" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/main/scala/CassandraStorage.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="67" column="0" selection-start="2484" selection-end="2484" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/CassandraPersistentActorSpec.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="42" column="0" selection-start="969" selection-end="1352" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/MongoPersistentActorSpec.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="33" column="0" selection-start="1045" selection-end="1045" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/main/scala/PersistentState.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="266" column="0" selection-start="9174" selection-end="9174" 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="30" column="59" selection-start="1165" selection-end="1165" vertical-scroll-proportion="0.0" />
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionalState.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="34" column="20" selection-start="780" selection-end="780" vertical-scroll-proportion="0.19538671" />
</provider>
</entry>
<entry file="jar:///System/Library/Frameworks/JavaVM.framework/Versions/1.6.0/Home/../Classes/classes.jar!/java/lang/reflect/Method.class">
<provider selected="true" editor-type-id="text-editor">
<state line="64" column="38" selection-start="2983" selection-end="2987" vertical-scroll-proportion="-21.961538">
<folding />
</state>
</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="457" column="29" selection-start="16736" selection-end="16736" vertical-scroll-proportion="0.0">
<state line="24" column="29" selection-start="733" selection-end="733" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/AllTest.scala">
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/actor/ActiveObject.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="12" column="4" selection-start="453" selection-end="453" 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="67" column="0" selection-start="2484" selection-end="2484" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/CassandraPersistentActorSpec.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="42" column="0" selection-start="969" selection-end="1352" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/test/scala/MongoPersistentActorSpec.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="33" column="0" selection-start="1045" selection-end="1045" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-persistence/src/main/scala/PersistentState.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="266" column="0" selection-start="9174" selection-end="9174" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/test/scala/InMemoryActorSpec.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="30" column="59" selection-start="1165" selection-end="1165" vertical-scroll-proportion="0.0">
<folding />
</state>
</provider>
</entry>
<entry file="file://$PROJECT_DIR$/akka-actors/src/main/scala/stm/TransactionalState.scala">
<provider selected="true" editor-type-id="text-editor">
<state line="34" column="20" selection-start="780" selection-end="780" vertical-scroll-proportion="0.19538671">
<state line="471" column="49" selection-start="21206" selection-end="21206" vertical-scroll-proportion="1.4898785">
<folding />
</state>
</provider>

View file

@ -21,10 +21,12 @@ see http://maven.apache.org/plugins/maven-changes-plugin/usage.html for full gui
<action dev="Debasish Ghosh" type="add">Transparent JSON serialization of Scala objects based on SJSON </action>
<action dev="Debasish Ghosh" type="add">MongoDB backed actor example</action>
<action dev="Tim Perrett" type="add">Support for using the Lift Web framework with Actors</action>
<action dev="Viktor Klang" type="add">Security module for authentication and authorization</action>
<action dev="Viktor Klang" type="add">Security module for HTTP-based authentication and authorization</action>
<action dev="Viktor Klang" type="add">Support for using Scala XML tags in RESTful Actors (scala-jersey)</action>
<action dev="Viktor Klang" type="add">Support for Comet Actors using Atmosphere</action>
<action dev="Jonas Bon&#233;r" type="add">AMQP integration; abstracted as actors in a supervisor hierarchy. Impl AMQP 0.9.1</action>
<action dev="Jonas Bon&#233;r" type="add">Rewritten STM, now integrated with Multiverse STM</action>
<action dev="Jonas Bon&#233;r" type="add">Added STM API for atomic {..} and or {..} else {..}</action>
<action dev="Jonas Bon&#233;r" type="add">Modularization of distribution into a thin core (actors, remoting and STM) and the rest in submodules</action>
<action dev="Jonas Bon&#233;r" type="add">JSON serialization for Java objects (using Jackson)</action>
<action dev="Jonas Bon&#233;r" type="add">JSON serialization for Scala objects (using SJSON)</action>