added init tx state hook for active objects, rewrote mongodb test
This commit is contained in:
parent
6cc3d87b5f
commit
cdd8a35176
17 changed files with 538 additions and 593 deletions
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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é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 "
|
||||
}
|
||||
*/
|
||||
|
|
@ -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) =>
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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) =>
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
189
akka.iws
|
|
@ -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="<template>" 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>
|
||||
|
|
|
|||
|
|
@ -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ér" type="add">AMQP integration; abstracted as actors in a supervisor hierarchy. Impl AMQP 0.9.1</action>
|
||||
<action dev="Jonas Bonér" type="add">Rewritten STM, now integrated with Multiverse STM</action>
|
||||
<action dev="Jonas Bonér" type="add">Added STM API for atomic {..} and or {..} else {..}</action>
|
||||
<action dev="Jonas Bonér" type="add">Modularization of distribution into a thin core (actors, remoting and STM) and the rest in submodules</action>
|
||||
<action dev="Jonas Bonér" type="add">JSON serialization for Java objects (using Jackson)</action>
|
||||
<action dev="Jonas Bonér" type="add">JSON serialization for Scala objects (using SJSON)</action>
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue