merged with upstream
This commit is contained in:
commit
2f32b850f6
78 changed files with 827 additions and 716 deletions
2
LICENSE
2
LICENSE
|
|
@ -17,4 +17,4 @@ the License.
|
||||||
---------------
|
---------------
|
||||||
|
|
||||||
Licenses for dependency projects can be found here:
|
Licenses for dependency projects can be found here:
|
||||||
[http://doc.akkasource.org/licenses]
|
[http://doc.akkasource.org/licenses]
|
||||||
|
|
|
||||||
|
|
@ -14,8 +14,6 @@ import se.scalablesolutions.akka.util.{HashCode, Logging}
|
||||||
|
|
||||||
import scala.collection.mutable.HashMap
|
import scala.collection.mutable.HashMap
|
||||||
|
|
||||||
import org.scala_tools.javautils.Imports._
|
|
||||||
|
|
||||||
import java.util.concurrent.ConcurrentHashMap
|
import java.util.concurrent.ConcurrentHashMap
|
||||||
import java.util.{Timer, TimerTask}
|
import java.util.{Timer, TimerTask}
|
||||||
import java.io.IOException
|
import java.io.IOException
|
||||||
|
|
@ -79,6 +77,8 @@ object AMQP {
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
class AMQPSupervisor extends Actor {
|
class AMQPSupervisor extends Actor {
|
||||||
|
import scala.collection.JavaConversions._
|
||||||
|
|
||||||
private val connections = new ConcurrentHashMap[FaultTolerantConnectionActor, FaultTolerantConnectionActor]
|
private val connections = new ConcurrentHashMap[FaultTolerantConnectionActor, FaultTolerantConnectionActor]
|
||||||
|
|
||||||
faultHandler = Some(OneForOneStrategy(5, 5000))
|
faultHandler = Some(OneForOneStrategy(5, 5000))
|
||||||
|
|
@ -138,7 +138,7 @@ object AMQP {
|
||||||
}
|
}
|
||||||
|
|
||||||
override def shutdown = {
|
override def shutdown = {
|
||||||
connections.values.asScala.foreach(_ ! Stop)
|
asMap(connections).valuesIterator.foreach(_ ! Stop)
|
||||||
exit
|
exit
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -361,8 +361,13 @@ object AMQP {
|
||||||
extends FaultTolerantConnectionActor {
|
extends FaultTolerantConnectionActor {
|
||||||
consumer: Consumer =>
|
consumer: Consumer =>
|
||||||
|
|
||||||
|
import scala.collection.JavaConversions._
|
||||||
|
|
||||||
faultHandler = Some(OneForOneStrategy(5, 5000))
|
faultHandler = Some(OneForOneStrategy(5, 5000))
|
||||||
trapExit = List(classOf[Throwable])
|
trapExit = List(classOf[Throwable])
|
||||||
|
|
||||||
|
//FIXME use better strategy to convert scala.immutable.Map to java.util.Map
|
||||||
|
private val jConfigMap = configurationArguments.foldLeft(new java.util.HashMap[String,Object]){ (m,kv) => { m.put(kv._1,kv._2); m } }
|
||||||
|
|
||||||
private val listeners = new HashMap[MessageConsumerListener, MessageConsumerListener]
|
private val listeners = new HashMap[MessageConsumerListener, MessageConsumerListener]
|
||||||
|
|
||||||
|
|
@ -395,7 +400,7 @@ object AMQP {
|
||||||
throw cause
|
throw cause
|
||||||
|
|
||||||
case Stop =>
|
case Stop =>
|
||||||
listeners.elements.toList.map(_._2).foreach(unregisterListener(_))
|
listeners.iterator.toList.map(_._2).foreach(unregisterListener(_))
|
||||||
disconnect
|
disconnect
|
||||||
exit
|
exit
|
||||||
|
|
||||||
|
|
@ -411,8 +416,8 @@ object AMQP {
|
||||||
protected def setupChannel = {
|
protected def setupChannel = {
|
||||||
connection = connectionFactory.newConnection(hostname, port)
|
connection = connectionFactory.newConnection(hostname, port)
|
||||||
channel = connection.createChannel
|
channel = connection.createChannel
|
||||||
channel.exchangeDeclare(exchangeName.toString, exchangeType.toString, passive, durable, autoDelete, configurationArguments.asJava)
|
channel.exchangeDeclare(exchangeName.toString, exchangeType.toString, passive, durable, autoDelete, jConfigMap)
|
||||||
listeners.elements.toList.map(_._2).foreach(registerListener)
|
listeners.iterator.toList.map(_._2).foreach(registerListener)
|
||||||
if (shutdownListener.isDefined) connection.addShutdownListener(shutdownListener.get)
|
if (shutdownListener.isDefined) connection.addShutdownListener(shutdownListener.get)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -426,7 +431,7 @@ object AMQP {
|
||||||
listener.queueName,
|
listener.queueName,
|
||||||
passive, durable,
|
passive, durable,
|
||||||
listener.exclusive, listener.autoDelete,
|
listener.exclusive, listener.autoDelete,
|
||||||
configurationArguments.asJava)
|
jConfigMap)
|
||||||
}
|
}
|
||||||
|
|
||||||
log.debug("Binding new queue for MessageConsumerListener [%s]", listener.queueName)
|
log.debug("Binding new queue for MessageConsumerListener [%s]", listener.queueName)
|
||||||
|
|
@ -460,7 +465,7 @@ object AMQP {
|
||||||
"MessageConsumerListener [" + listener + "] does not have a tag")
|
"MessageConsumerListener [" + listener + "] does not have a tag")
|
||||||
listener.tag.get == listenerTag
|
listener.tag.get == listenerTag
|
||||||
}
|
}
|
||||||
listeners.elements.toList.map(_._2).find(hasTag(_, listenerTag)) match {
|
listeners.iterator.toList.map(_._2).find(hasTag(_, listenerTag)) match {
|
||||||
case None => log.error(
|
case None => log.error(
|
||||||
"Could not find message listener for tag [%s]; can't shut listener down", listenerTag)
|
"Could not find message listener for tag [%s]; can't shut listener down", listenerTag)
|
||||||
case Some(listener) =>
|
case Some(listener) =>
|
||||||
|
|
@ -480,7 +485,7 @@ object AMQP {
|
||||||
"Can't unregister message consumer listener [%s]; no such listener",
|
"Can't unregister message consumer listener [%s]; no such listener",
|
||||||
listener.toString(exchangeName))
|
listener.toString(exchangeName))
|
||||||
case Some(listener) =>
|
case Some(listener) =>
|
||||||
listeners - listener
|
listeners -= listener
|
||||||
listener.tag match {
|
listener.tag match {
|
||||||
case None => log.warning(
|
case None => log.warning(
|
||||||
"Can't unregister message consumer listener [%s]; no listener tag",
|
"Can't unregister message consumer listener [%s]; no listener tag",
|
||||||
|
|
|
||||||
|
|
@ -7,8 +7,6 @@ package se.scalablesolutions.akka.camel
|
||||||
import org.apache.camel.{Exchange, Message => CamelMessage}
|
import org.apache.camel.{Exchange, Message => CamelMessage}
|
||||||
import org.apache.camel.util.ExchangeHelper
|
import org.apache.camel.util.ExchangeHelper
|
||||||
|
|
||||||
import scala.collection.jcl.{Map => MapWrapper}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An immutable representation of a Camel message. Actor classes that mix in
|
* An immutable representation of a Camel message. Actor classes that mix in
|
||||||
* se.scalablesolutions.akka.camel.Producer or
|
* se.scalablesolutions.akka.camel.Producer or
|
||||||
|
|
@ -16,12 +14,7 @@ import scala.collection.jcl.{Map => MapWrapper}
|
||||||
*
|
*
|
||||||
* @author Martin Krasser
|
* @author Martin Krasser
|
||||||
*/
|
*/
|
||||||
case class Message(val body: Any, val headers: Map[String, Any]) {
|
case class Message(val body: Any, val headers: Map[String, Any] = Map.empty) {
|
||||||
/**
|
|
||||||
* Creates a message with a body and an empty header map.
|
|
||||||
*/
|
|
||||||
def this(body: Any) = this(body, Map.empty)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the body of the message converted to the type given by the <code>clazz</code>
|
* Returns the body of the message converted to the type given by the <code>clazz</code>
|
||||||
* argument. Conversion is done using Camel's type converter. The type converter is obtained
|
* argument. Conversion is done using Camel's type converter. The type converter is obtained
|
||||||
|
|
@ -58,23 +51,23 @@ case class Message(val body: Any, val headers: Map[String, Any]) {
|
||||||
/**
|
/**
|
||||||
* Creates a new Message with new <code>headers</code>.
|
* Creates a new Message with new <code>headers</code>.
|
||||||
*/
|
*/
|
||||||
def setHeaders(headers: Map[String, Any]) = new Message(this.body, headers)
|
def setHeaders(headers: Map[String, Any]) = copy(this.body, headers)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new Message with the <code>headers</code> argument added to the existing headers.
|
* Creates a new Message with the <code>headers</code> argument added to the existing headers.
|
||||||
*/
|
*/
|
||||||
def addHeaders(headers: Map[String, Any]) = new Message(this.body, this.headers ++ headers)
|
def addHeaders(headers: Map[String, Any]) = copy(this.body, this.headers ++ headers)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new Message with the <code>header</code> argument added to the existing headers.
|
* Creates a new Message with the <code>header</code> argument added to the existing headers.
|
||||||
*/
|
*/
|
||||||
def addHeader(header: (String, Any)) = new Message(this.body, this.headers + header)
|
def addHeader(header: (String, Any)) = copy(this.body, this.headers + header)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new Message where the header with name <code>headerName</code> is removed from
|
* Creates a new Message where the header with name <code>headerName</code> is removed from
|
||||||
* the existing headers.
|
* the existing headers.
|
||||||
*/
|
*/
|
||||||
def removeHeader(headerName: String) = new Message(this.body, this.headers - headerName)
|
def removeHeader(headerName: String) = copy(this.body, this.headers - headerName)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -115,7 +108,7 @@ object Message {
|
||||||
*
|
*
|
||||||
* @author Martin Krasser
|
* @author Martin Krasser
|
||||||
*/
|
*/
|
||||||
case class Failure(val cause: Exception, val headers: Map[String, Any])
|
case class Failure(val cause: Exception, val headers: Map[String, Any] = Map.empty)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adapter for converting an org.apache.camel.Exchange to and from Message and Failure objects.
|
* Adapter for converting an org.apache.camel.Exchange to and from Message and Failure objects.
|
||||||
|
|
@ -224,8 +217,10 @@ class CamelMessageAdapter(val cm: CamelMessage) {
|
||||||
*/
|
*/
|
||||||
def toMessage(headers: Map[String, Any]): Message = Message(cm.getBody, cmHeaders(headers, cm))
|
def toMessage(headers: Map[String, Any]): Message = Message(cm.getBody, cmHeaders(headers, cm))
|
||||||
|
|
||||||
|
import scala.collection.JavaConversions._
|
||||||
|
|
||||||
private def cmHeaders(headers: Map[String, Any], cm: CamelMessage) =
|
private def cmHeaders(headers: Map[String, Any], cm: CamelMessage) =
|
||||||
headers ++ MapWrapper[String, AnyRef](cm.getHeaders).elements
|
headers ++ cm.getHeaders
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -66,7 +66,7 @@ trait Producer { self: Actor =>
|
||||||
* @param msg: the message to produce. The message is converted to its canonical
|
* @param msg: the message to produce. The message is converted to its canonical
|
||||||
* representation via Message.canonicalize.
|
* representation via Message.canonicalize.
|
||||||
*/
|
*/
|
||||||
protected def produceOneway(msg: Any): Unit =
|
protected def produceOnewaySync(msg: Any): Unit =
|
||||||
template.send(endpointUri, createInOnlyExchange.fromRequestMessage(Message.canonicalize(msg)))
|
template.send(endpointUri, createInOnlyExchange.fromRequestMessage(Message.canonicalize(msg)))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -90,7 +90,7 @@ trait Producer { self: Actor =>
|
||||||
* representation via Message.canonicalize.
|
* representation via Message.canonicalize.
|
||||||
* @return either a response Message or a Failure object.
|
* @return either a response Message or a Failure object.
|
||||||
*/
|
*/
|
||||||
protected def produce(msg: Any): Any = {
|
protected def produceSync(msg: Any): Any = {
|
||||||
val cmsg = Message.canonicalize(msg)
|
val cmsg = Message.canonicalize(msg)
|
||||||
val requestProcessor = new Processor() {
|
val requestProcessor = new Processor() {
|
||||||
def process(exchange: Exchange) = exchange.fromRequestMessage(cmsg)
|
def process(exchange: Exchange) = exchange.fromRequestMessage(cmsg)
|
||||||
|
|
@ -126,9 +126,9 @@ trait Producer { self: Actor =>
|
||||||
*/
|
*/
|
||||||
protected def produce: PartialFunction[Any, Unit] = {
|
protected def produce: PartialFunction[Any, Unit] = {
|
||||||
case msg => {
|
case msg => {
|
||||||
if ( oneway && !async) produceOneway(msg)
|
if ( oneway && !async) produceOnewaySync(msg)
|
||||||
else if ( oneway && async) produceOnewayAsync(msg)
|
else if ( oneway && async) produceOnewayAsync(msg)
|
||||||
else if (!oneway && !async) reply(produce(msg))
|
else if (!oneway && !async) reply(produceSync(msg))
|
||||||
else /*(!oneway && async)*/ produceAsync(msg)
|
else /*(!oneway && async)*/ produceAsync(msg)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -135,7 +135,7 @@ class ActorProducer(val ep: ActorEndpoint) extends DefaultProducer(ep) {
|
||||||
private def targetById(id: String) = ActorRegistry.actorsFor(id) match {
|
private def targetById(id: String) = ActorRegistry.actorsFor(id) match {
|
||||||
case Nil => None
|
case Nil => None
|
||||||
case actor :: Nil => Some(actor)
|
case actor :: Nil => Some(actor)
|
||||||
case actors => Some(actors.first)
|
case actors => Some(actors.head)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def targetByUuid(uuid: String) = ActorRegistry.actorFor(uuid)
|
private def targetByUuid(uuid: String) = ActorRegistry.actorFor(uuid)
|
||||||
|
|
|
||||||
|
|
@ -55,7 +55,7 @@ class ConsumerPublisher extends Actor with Logging {
|
||||||
* @param endpointUri endpoint URI of the consumer actor
|
* @param endpointUri endpoint URI of the consumer actor
|
||||||
* @param id actor identifier
|
* @param id actor identifier
|
||||||
* @param uuid <code>true</code> if <code>id</code> refers to Actor.uuid, <code>false</code> if
|
* @param uuid <code>true</code> if <code>id</code> refers to Actor.uuid, <code>false</code> if
|
||||||
* <code>id</code> refers to Acotr.getId.
|
* <code>id</code> refers to Actor.getId.
|
||||||
*
|
*
|
||||||
* @author Martin Krasser
|
* @author Martin Krasser
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -26,9 +26,9 @@ class CamelExchangeAdapterTest extends JUnitSuite {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test def shouldSetExceptionFromFailureMessage = {
|
@Test def shouldSetExceptionFromFailureMessage = {
|
||||||
val e1 = sampleInOnly.fromFailureMessage(Failure(new Exception("test1"), Map.empty))
|
val e1 = sampleInOnly.fromFailureMessage(Failure(new Exception("test1")))
|
||||||
assert(e1.getException.getMessage === "test1")
|
assert(e1.getException.getMessage === "test1")
|
||||||
val e2 = sampleInOut.fromFailureMessage(Failure(new Exception("test2"), Map.empty))
|
val e2 = sampleInOut.fromFailureMessage(Failure(new Exception("test2")))
|
||||||
assert(e2.getException.getMessage === "test2")
|
assert(e2.getException.getMessage === "test2")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,17 +1,18 @@
|
||||||
package se.scalablesolutions.akka.cluster.jgroups
|
package se.scalablesolutions.akka.cluster.jgroups
|
||||||
|
|
||||||
import org.jgroups.{JChannel, View => JG_VIEW, Address, Message => JG_MSG, ExtendedMembershipListener, Receiver}
|
import org.jgroups.{JChannel, View => JG_VIEW, Address, Message => JG_MSG, ExtendedMembershipListener, Receiver}
|
||||||
|
import org.jgroups.util.Util
|
||||||
|
|
||||||
import se.scalablesolutions.akka.remote.ClusterActor._
|
|
||||||
import se.scalablesolutions.akka.remote.BasicClusterActor
|
import se.scalablesolutions.akka.remote.BasicClusterActor
|
||||||
|
|
||||||
import org.scala_tools.javautils.Imports._
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Clustering support via JGroups.
|
* Clustering support via JGroups.
|
||||||
* @Author Viktor Klang
|
* @Author Viktor Klang
|
||||||
*/
|
*/
|
||||||
class JGroupsClusterActor extends BasicClusterActor {
|
class JGroupsClusterActor extends BasicClusterActor {
|
||||||
|
import scala.collection.JavaConversions._
|
||||||
|
import se.scalablesolutions.akka.remote.ClusterActor._
|
||||||
|
|
||||||
type ADDR_T = Address
|
type ADDR_T = Address
|
||||||
|
|
||||||
@volatile private var isActive = false
|
@volatile private var isActive = false
|
||||||
|
|
@ -31,13 +32,13 @@ class JGroupsClusterActor extends BasicClusterActor {
|
||||||
def setState(state: Array[Byte]): Unit = ()
|
def setState(state: Array[Byte]): Unit = ()
|
||||||
|
|
||||||
def receive(m: JG_MSG): Unit =
|
def receive(m: JG_MSG): Unit =
|
||||||
if (isActive && m.getSrc != channel.map(_.getAddress).getOrElse(m.getSrc)) me send Message(m.getSrc,m.getRawBuffer)
|
if (isActive && m.getSrc != channel.map(_.getAddress).getOrElse(m.getSrc)) me ! Message(m.getSrc,m.getRawBuffer)
|
||||||
|
|
||||||
def viewAccepted(view: JG_VIEW): Unit =
|
def viewAccepted(view: JG_VIEW): Unit =
|
||||||
if (isActive) me send View(Set[ADDR_T]() ++ view.getMembers.asScala - channel.get.getAddress)
|
if (isActive) me ! View(Set[ADDR_T]() ++ view.getMembers - channel.get.getAddress)
|
||||||
|
|
||||||
def suspect(a: Address): Unit =
|
def suspect(a: Address): Unit =
|
||||||
if (isActive) me send Zombie(a)
|
if (isActive) me ! Zombie(a)
|
||||||
|
|
||||||
def block: Unit =
|
def block: Unit =
|
||||||
log debug "UNSUPPORTED: JGroupsClusterActor::block" //TODO HotSwap to a buffering body
|
log debug "UNSUPPORTED: JGroupsClusterActor::block" //TODO HotSwap to a buffering body
|
||||||
|
|
@ -60,7 +61,7 @@ class JGroupsClusterActor extends BasicClusterActor {
|
||||||
super.shutdown
|
super.shutdown
|
||||||
log debug ("Shutting down %s", toString)
|
log debug ("Shutting down %s", toString)
|
||||||
isActive = false
|
isActive = false
|
||||||
channel.foreach(_.shutdown)
|
channel.foreach(Util shutdown _)
|
||||||
channel = None
|
channel = None
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -71,8 +71,9 @@ class ShoalClusterActor extends BasicClusterActor {
|
||||||
* Creates a CallBack instance that deals with the cluster signalling
|
* Creates a CallBack instance that deals with the cluster signalling
|
||||||
*/
|
*/
|
||||||
protected def createCallback : CallBack = {
|
protected def createCallback : CallBack = {
|
||||||
import org.scala_tools.javautils.Imports._
|
import scala.collection.JavaConversions._
|
||||||
import ClusterActor._
|
import ClusterActor._
|
||||||
|
|
||||||
val me = this
|
val me = this
|
||||||
new CallBack {
|
new CallBack {
|
||||||
def processNotification(signal : Signal) {
|
def processNotification(signal : Signal) {
|
||||||
|
|
@ -80,10 +81,10 @@ class ShoalClusterActor extends BasicClusterActor {
|
||||||
signal.acquire()
|
signal.acquire()
|
||||||
if(isActive) {
|
if(isActive) {
|
||||||
signal match {
|
signal match {
|
||||||
case ms : MessageSignal => me send Message[ADDR_T](ms.getMemberToken,ms.getMessage)
|
case ms : MessageSignal => me ! Message[ADDR_T](ms.getMemberToken,ms.getMessage)
|
||||||
case jns : JoinNotificationSignal => me send View[ADDR_T](Set[ADDR_T]() ++ jns.getCurrentCoreMembers.asScala - serverName)
|
case jns : JoinNotificationSignal => me ! View[ADDR_T](Set[ADDR_T]() ++ jns.getCurrentCoreMembers - serverName)
|
||||||
case fss : FailureSuspectedSignal => me send Zombie[ADDR_T](fss.getMemberToken)
|
case fss : FailureSuspectedSignal => me ! Zombie[ADDR_T](fss.getMemberToken)
|
||||||
case fns : FailureNotificationSignal => me send Zombie[ADDR_T](fns.getMemberToken)
|
case fns : FailureNotificationSignal => me ! Zombie[ADDR_T](fns.getMemberToken)
|
||||||
case _ => log.debug("Unhandled signal: [%s]",signal)
|
case _ => log.debug("Unhandled signal: [%s]",signal)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -11,9 +11,8 @@ class AkkaBroadcaster extends org.atmosphere.jersey.JerseyBroadcaster {
|
||||||
name = classOf[AkkaBroadcaster].getName
|
name = classOf[AkkaBroadcaster].getName
|
||||||
|
|
||||||
val caster = new Actor {
|
val caster = new Actor {
|
||||||
def receive = { case f : Function0[_] => f() }
|
def receive = { case f : Function0[_] => f() }
|
||||||
|
start
|
||||||
start
|
|
||||||
}
|
}
|
||||||
|
|
||||||
override def destroy {
|
override def destroy {
|
||||||
|
|
@ -22,6 +21,6 @@ class AkkaBroadcaster extends org.atmosphere.jersey.JerseyBroadcaster {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected override def broadcast(r : AtmosphereResource[_,_], e : AtmosphereResourceEvent[_,_]) = {
|
protected override def broadcast(r : AtmosphereResource[_,_], e : AtmosphereResourceEvent[_,_]) = {
|
||||||
caster.send( () => super.broadcast(r,e) )
|
caster ! (() => super.broadcast(r,e))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -59,12 +59,12 @@ class AkkaServlet extends org.atmosphere.cpr.AtmosphereServlet with Logging {
|
||||||
* we need to handle that.
|
* we need to handle that.
|
||||||
*/
|
*/
|
||||||
override def createCometSupportResolver() : CometSupportResolver = {
|
override def createCometSupportResolver() : CometSupportResolver = {
|
||||||
import org.scala_tools.javautils.Imports._
|
import scala.collection.JavaConversions._
|
||||||
|
|
||||||
new DefaultCometSupportResolver(config) {
|
new DefaultCometSupportResolver(config) {
|
||||||
type CS = CometSupport[_ <: AtmosphereResource[_,_]]
|
type CS = CometSupport[_ <: AtmosphereResource[_,_]]
|
||||||
override def resolveMultipleNativeSupportConflict(available : JList[Class[_ <: CS]]) : CS = {
|
override def resolveMultipleNativeSupportConflict(available : JList[Class[_ <: CS]]) : CS = {
|
||||||
available.asScala.filter(_ != classOf[GrizzlyCometSupport]).toList match {
|
available.filter(_ != classOf[GrizzlyCometSupport]).toList match {
|
||||||
case Nil => new GrizzlyCometSupport(config)
|
case Nil => new GrizzlyCometSupport(config)
|
||||||
case x :: Nil => newCometSupport(x.asInstanceOf[Class[_ <: CS]])
|
case x :: Nil => newCometSupport(x.asInstanceOf[Class[_ <: CS]])
|
||||||
case _ => super.resolveMultipleNativeSupportConflict(available)
|
case _ => super.resolveMultipleNativeSupportConflict(available)
|
||||||
|
|
|
||||||
|
|
@ -19,7 +19,7 @@ import se.scalablesolutions.akka.util.{Bootable, Logging}
|
||||||
trait BootableCometActorService extends Bootable with Logging {
|
trait BootableCometActorService extends Bootable with Logging {
|
||||||
self : BootableActorLoaderService =>
|
self : BootableActorLoaderService =>
|
||||||
|
|
||||||
import config.Config._
|
import se.scalablesolutions.akka.config.Config._
|
||||||
|
|
||||||
val REST_HOSTNAME = config.getString("akka.rest.hostname", "localhost")
|
val REST_HOSTNAME = config.getString("akka.rest.hostname", "localhost")
|
||||||
val REST_URL = "http://" + REST_HOSTNAME
|
val REST_URL = "http://" + REST_HOSTNAME
|
||||||
|
|
@ -44,8 +44,8 @@ trait BootableCometActorService extends Bootable with Logging {
|
||||||
adapter.setServletInstance(new AkkaServlet)
|
adapter.setServletInstance(new AkkaServlet)
|
||||||
adapter.setContextPath(uri.getPath)
|
adapter.setContextPath(uri.getPath)
|
||||||
adapter.addInitParameter("cometSupport", "org.atmosphere.container.GrizzlyCometSupport")
|
adapter.addInitParameter("cometSupport", "org.atmosphere.container.GrizzlyCometSupport")
|
||||||
if (HOME.isDefined) adapter.setRootFolder(HOME.get + "/deploy/root")
|
if (HOME.isDefined) adapter.addRootFolder(HOME.get + "/deploy/root")
|
||||||
log.info("REST service root path [%s] and context path [%s]", adapter.getRootFolder, adapter.getContextPath)
|
log.info("REST service root path [%s] and context path [%s]", adapter.getRootFolders, adapter.getContextPath)
|
||||||
|
|
||||||
val ah = new com.sun.grizzly.arp.DefaultAsyncHandler
|
val ah = new com.sun.grizzly.arp.DefaultAsyncHandler
|
||||||
ah.addAsyncFilter(new com.sun.grizzly.comet.CometAsyncFilter)
|
ah.addAsyncFilter(new com.sun.grizzly.comet.CometAsyncFilter)
|
||||||
|
|
@ -55,19 +55,18 @@ trait BootableCometActorService extends Bootable with Logging {
|
||||||
t.setAdapter(adapter)
|
t.setAdapter(adapter)
|
||||||
t.setEnableAsyncExecution(true)
|
t.setEnableAsyncExecution(true)
|
||||||
t.setAsyncHandler(ah)
|
t.setAsyncHandler(ah)
|
||||||
t.listen
|
t.listen
|
||||||
t }
|
t
|
||||||
|
}
|
||||||
log.info("REST service started successfully. Listening to port [%s]", REST_PORT)
|
log.info("REST service started successfully. Listening to port [%s]", REST_PORT)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
abstract override def onUnload = {
|
abstract override def onUnload = {
|
||||||
super.onUnload
|
super.onUnload
|
||||||
|
if (jerseySelectorThread.isDefined) {
|
||||||
if (jerseySelectorThread.isDefined) {
|
log.info("Shutting down REST service (Jersey)")
|
||||||
log.info("Shutting down REST service (Jersey)")
|
jerseySelectorThread.get.stopEndpoint
|
||||||
jerseySelectorThread.get.stopEndpoint
|
}
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -262,9 +262,10 @@ private[akka] sealed class ActiveObjectAspect {
|
||||||
}
|
}
|
||||||
|
|
||||||
private def localDispatch(joinPoint: JoinPoint): AnyRef = {
|
private def localDispatch(joinPoint: JoinPoint): AnyRef = {
|
||||||
import Actor.Sender.Self
|
|
||||||
val rtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
|
val rtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
|
||||||
if (isOneWay(rtti)) actor ! Invocation(joinPoint, true, true)
|
if (isOneWay(rtti)) {
|
||||||
|
(actor ! Invocation(joinPoint, true, true) ).asInstanceOf[AnyRef]
|
||||||
|
}
|
||||||
else {
|
else {
|
||||||
val result = actor !! Invocation(joinPoint, false, isVoid(rtti))
|
val result = actor !! Invocation(joinPoint, false, isVoid(rtti))
|
||||||
if (result.isDefined) result.get
|
if (result.isDefined) result.get
|
||||||
|
|
@ -274,7 +275,7 @@ private[akka] sealed class ActiveObjectAspect {
|
||||||
|
|
||||||
private def remoteDispatch(joinPoint: JoinPoint): AnyRef = {
|
private def remoteDispatch(joinPoint: JoinPoint): AnyRef = {
|
||||||
val rtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
|
val rtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
|
||||||
val oneWay_? = isOneWay(rtti)
|
val oneWay_? = isOneWay(rtti) || isVoid(rtti)
|
||||||
val (message: Array[AnyRef], isEscaped) = escapeArguments(rtti.getParameterValues)
|
val (message: Array[AnyRef], isEscaped) = escapeArguments(rtti.getParameterValues)
|
||||||
val requestBuilder = RemoteRequest.newBuilder
|
val requestBuilder = RemoteRequest.newBuilder
|
||||||
.setId(RemoteRequestIdFactory.nextId)
|
.setId(RemoteRequestIdFactory.nextId)
|
||||||
|
|
@ -354,7 +355,7 @@ private[akka] sealed class ActiveObjectAspect {
|
||||||
|
|
||||||
object Dispatcher {
|
object Dispatcher {
|
||||||
val ZERO_ITEM_CLASS_ARRAY = Array[Class[_]]()
|
val ZERO_ITEM_CLASS_ARRAY = Array[Class[_]]()
|
||||||
val ZERO_ITEM_OBJECT_ARRAY = Array[Object[_]]()
|
val ZERO_ITEM_OBJECT_ARRAY = Array[Object]()
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -364,7 +365,7 @@ object Dispatcher {
|
||||||
*/
|
*/
|
||||||
private[akka] class Dispatcher(transactionalRequired: Boolean, val callbacks: Option[RestartCallbacks]) extends Actor {
|
private[akka] class Dispatcher(transactionalRequired: Boolean, val callbacks: Option[RestartCallbacks]) extends Actor {
|
||||||
import Dispatcher._
|
import Dispatcher._
|
||||||
|
|
||||||
private[actor] var target: Option[AnyRef] = None
|
private[actor] var target: Option[AnyRef] = None
|
||||||
private var preRestart: Option[Method] = None
|
private var preRestart: Option[Method] = None
|
||||||
private var postRestart: Option[Method] = None
|
private var postRestart: Option[Method] = None
|
||||||
|
|
@ -470,58 +471,3 @@ private[akka] class Dispatcher(transactionalRequired: Boolean, val callbacks: Op
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
|
||||||
ublic class CamelInvocationHandler implements InvocationHandler {
|
|
||||||
private final Endpoint endpoint;
|
|
||||||
private final Producer producer;
|
|
||||||
private final MethodInfoCache methodInfoCache;
|
|
||||||
|
|
||||||
public CamelInvocationHandler(Endpoint endpoint, Producer producer, MethodInfoCache methodInfoCache) {
|
|
||||||
this.endpoint = endpoint;
|
|
||||||
this.producer = producer;
|
|
||||||
this.methodInfoCache = methodInfoCache;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
|
|
||||||
BeanInvocation invocation = new BeanInvocation(method, args);
|
|
||||||
ExchangePattern pattern = ExchangePattern.InOut;
|
|
||||||
MethodInfo methodInfo = methodInfoCache.getMethodInfo(method);
|
|
||||||
if (methodInfo ne null) {
|
|
||||||
pattern = methodInfo.getPattern();
|
|
||||||
}
|
|
||||||
Exchange exchange = new DefaultExchange(endpoint, pattern);
|
|
||||||
exchange.getIn().setBody(invocation);
|
|
||||||
|
|
||||||
producer.process(exchange);
|
|
||||||
Throwable fault = exchange.getException();
|
|
||||||
if (fault ne null) {
|
|
||||||
throw new InvocationTargetException(fault);
|
|
||||||
}
|
|
||||||
if (pattern.isOutCapable()) {
|
|
||||||
return exchange.getOut().getBody();
|
|
||||||
} else {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (joinpoint.target.isInstanceOf[MessageDriven] &&
|
|
||||||
joinpoint.method.getName == "onMessage") {
|
|
||||||
val m = joinpoint.method
|
|
||||||
|
|
||||||
val endpointName = m.getDeclaringClass.getName + "." + m.getName
|
|
||||||
val activeObjectName = m.getDeclaringClass.getName
|
|
||||||
val endpoint = conf.getRoutingEndpoint(conf.lookupUriFor(m))
|
|
||||||
val producer = endpoint.createProducer
|
|
||||||
val exchange = endpoint.createExchange
|
|
||||||
exchange.getIn().setBody(joinpoint)
|
|
||||||
producer.process(exchange)
|
|
||||||
val fault = exchange.getException();
|
|
||||||
if (fault ne null) throw new InvocationTargetException(fault)
|
|
||||||
|
|
||||||
// FIXME: need some timeout and future here...
|
|
||||||
exchange.getOut.getBody
|
|
||||||
|
|
||||||
} else
|
|
||||||
*/
|
|
||||||
|
|
|
||||||
|
|
@ -81,8 +81,8 @@ object Actor extends Logging {
|
||||||
val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost")
|
val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost")
|
||||||
val PORT = config.getInt("akka.remote.server.port", 9999)
|
val PORT = config.getInt("akka.remote.server.port", 9999)
|
||||||
|
|
||||||
object Sender {
|
object Sender{
|
||||||
implicit val Self: Option[Actor] = None
|
object Self
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -109,7 +109,7 @@ object Actor extends Logging {
|
||||||
* <pre>
|
* <pre>
|
||||||
* import Actor._
|
* import Actor._
|
||||||
*
|
*
|
||||||
* val a = transactor {
|
* val a = transactor {
|
||||||
* case msg => ... // handle message
|
* case msg => ... // handle message
|
||||||
* }
|
* }
|
||||||
* </pre>
|
* </pre>
|
||||||
|
|
@ -124,7 +124,7 @@ object Actor extends Logging {
|
||||||
* The actor is started when created.
|
* The actor is started when created.
|
||||||
* Example:
|
* Example:
|
||||||
* <pre>
|
* <pre>
|
||||||
* val a = Actor.init {
|
* val a = Actor.init {
|
||||||
* ... // init stuff
|
* ... // init stuff
|
||||||
* } receive {
|
* } receive {
|
||||||
* case msg => ... // handle message
|
* case msg => ... // handle message
|
||||||
|
|
@ -144,26 +144,25 @@ object Actor extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Use to create an anonymous event-driven actor with a body but no message loop block.
|
* Use to spawn out a block of code in an event-driven actor. Will shut actor down when
|
||||||
|
* the block has been executed.
|
||||||
* <p/>
|
* <p/>
|
||||||
* This actor can <b>not</b> respond to any messages but can be used as a simple way to
|
* NOTE: If used from within an Actor then has to be qualified with 'Actor.spawn' since
|
||||||
* spawn a lightweight thread to process some task.
|
* there is a method 'spawn[ActorType]' in the Actor trait already.
|
||||||
* <p/>
|
|
||||||
* The actor is started when created.
|
|
||||||
* Example:
|
* Example:
|
||||||
* <pre>
|
* <pre>
|
||||||
* import Actor._
|
* import Actor._
|
||||||
*
|
*
|
||||||
* spawn {
|
* spawn {
|
||||||
* ... // do stuff
|
* ... // do stuff
|
||||||
* }
|
* }
|
||||||
* </pre>
|
* </pre>
|
||||||
*/
|
*/
|
||||||
def spawn(body: => Unit): Actor = {
|
def spawn(body: => Unit): Unit = {
|
||||||
case object Spawn
|
case object Spawn
|
||||||
new Actor() {
|
new Actor() {
|
||||||
start
|
start
|
||||||
send(Spawn)
|
this ! Spawn
|
||||||
def receive = {
|
def receive = {
|
||||||
case Spawn => body; stop
|
case Spawn => body; stop
|
||||||
}
|
}
|
||||||
|
|
@ -195,7 +194,7 @@ object Actor extends Logging {
|
||||||
* <pre>
|
* <pre>
|
||||||
* import Actor._
|
* import Actor._
|
||||||
*
|
*
|
||||||
* val a = actor("localhost", 9999) {
|
* val a = actor("localhost", 9999) {
|
||||||
* case msg => ... // handle message
|
* case msg => ... // handle message
|
||||||
* }
|
* }
|
||||||
* </pre>
|
* </pre>
|
||||||
|
|
@ -369,7 +368,7 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
* <p/>
|
* <p/>
|
||||||
* Example code:
|
* Example code:
|
||||||
* <pre>
|
* <pre>
|
||||||
* def receive = {
|
* def receive = {
|
||||||
* case Ping =>
|
* case Ping =>
|
||||||
* println("got a ping")
|
* println("got a ping")
|
||||||
* reply("pong")
|
* reply("pong")
|
||||||
|
|
@ -478,44 +477,19 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
* If invoked from within an actor then the actor reference is implicitly passed on as the implicit 'sender' argument.
|
* If invoked from within an actor then the actor reference is implicitly passed on as the implicit 'sender' argument.
|
||||||
* <p/>
|
* <p/>
|
||||||
*
|
*
|
||||||
* This actor 'sender' reference is then available in the receiving actor in the 'sender' member variable.
|
* This actor 'sender' reference is then available in the receiving actor in the 'sender' member variable,
|
||||||
|
* if invoked from within an Actor. If not then no sender is available.
|
||||||
* <pre>
|
* <pre>
|
||||||
* actor ! message
|
* actor ! message
|
||||||
* </pre>
|
* </pre>
|
||||||
* <p/>
|
* <p/>
|
||||||
*
|
|
||||||
* If invoked from within a *non* Actor instance then either add this import to resolve the implicit argument:
|
|
||||||
* <pre>
|
|
||||||
* import Actor.Sender.Self
|
|
||||||
* actor ! message
|
|
||||||
* </pre>
|
|
||||||
*
|
|
||||||
* Or pass in the implicit argument explicitly:
|
|
||||||
* <pre>
|
|
||||||
* actor.!(message)(Some(this))
|
|
||||||
* </pre>
|
|
||||||
*
|
|
||||||
* Or use the 'send(..)' method;
|
|
||||||
* <pre>
|
|
||||||
* actor.send(message)
|
|
||||||
* </pre>
|
|
||||||
*/
|
*/
|
||||||
def !(message: Any)(implicit sender: Option[Actor]) = {
|
def !(message: Any)(implicit sender: Option[Actor] = None) = {
|
||||||
//FIXME 2.8 def !(message: Any)(implicit sender: Option[Actor] = None) = {
|
|
||||||
if (_isKilled) throw new ActorKilledException("Actor [" + toString + "] has been killed, can't respond to messages")
|
if (_isKilled) throw new ActorKilledException("Actor [" + toString + "] has been killed, can't respond to messages")
|
||||||
if (_isRunning) postMessageToMailbox(message, sender)
|
if (_isRunning) postMessageToMailbox(message, sender)
|
||||||
else throw new IllegalStateException("Actor has not been started, you need to invoke 'actor.start' before using it")
|
else throw new IllegalStateException("Actor has not been started, you need to invoke 'actor.start' before using it")
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Same as the '!' method but does not take an implicit sender as second parameter.
|
|
||||||
*/
|
|
||||||
def send(message: Any) = {
|
|
||||||
if (_isKilled) throw new ActorKilledException("Actor [" + toString + "] has been killed, can't respond to messages")
|
|
||||||
if (_isRunning) postMessageToMailbox(message, None)
|
|
||||||
else throw new IllegalStateException("Actor has not been started, you need to invoke 'actor.start' before using it")
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sends a message asynchronously and waits on a future for a reply message.
|
* Sends a message asynchronously and waits on a future for a reply message.
|
||||||
* <p/>
|
* <p/>
|
||||||
|
|
@ -577,7 +551,7 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
* <p/>
|
* <p/>
|
||||||
* Works with both '!' and '!!'.
|
* Works with both '!' and '!!'.
|
||||||
*/
|
*/
|
||||||
def forward(message: Any)(implicit sender: Option[Actor]) = {
|
def forward(message: Any)(implicit sender: Option[Actor] = None) = {
|
||||||
if (_isKilled) throw new ActorKilledException("Actor [" + toString + "] has been killed, can't respond to messages")
|
if (_isKilled) throw new ActorKilledException("Actor [" + toString + "] has been killed, can't respond to messages")
|
||||||
if (_isRunning) {
|
if (_isRunning) {
|
||||||
val forwarder = sender.getOrElse(throw new IllegalStateException("Can't forward message when the forwarder/mediator is not an actor"))
|
val forwarder = sender.getOrElse(throw new IllegalStateException("Can't forward message when the forwarder/mediator is not an actor"))
|
||||||
|
|
@ -605,7 +579,8 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
"\n\t\t2. Send a message from an instance that is *not* an actor" +
|
"\n\t\t2. Send a message from an instance that is *not* an actor" +
|
||||||
"\n\t\t3. Send a message to an Active Object annotated with the '@oneway' annotation? " +
|
"\n\t\t3. Send a message to an Active Object annotated with the '@oneway' annotation? " +
|
||||||
"\n\tIf so, switch to '!!' (or remove '@oneway') which passes on an implicit future" +
|
"\n\tIf so, switch to '!!' (or remove '@oneway') which passes on an implicit future" +
|
||||||
"\n\tthat will be bound by the argument passed to 'reply'. Alternatively, you can use setReplyToAddress to make sure the actor can be contacted over the network.")
|
"\n\tthat will be bound by the argument passed to 'reply'." +
|
||||||
|
"\n\tAlternatively, you can use setReplyToAddress to make sure the actor can be contacted over the network.")
|
||||||
case Some(future) =>
|
case Some(future) =>
|
||||||
future.completeWithResult(message)
|
future.completeWithResult(message)
|
||||||
}
|
}
|
||||||
|
|
@ -733,8 +708,8 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
* <p/>
|
* <p/>
|
||||||
* To be invoked from within the actor itself.
|
* To be invoked from within the actor itself.
|
||||||
*/
|
*/
|
||||||
protected[this] def spawn[T <: Actor](actorClass: Class[T]): T = {
|
protected[this] def spawn[T <: Actor : Manifest] : T = {
|
||||||
val actor = spawnButDoNotStart(actorClass)
|
val actor = spawnButDoNotStart[T]
|
||||||
actor.start
|
actor.start
|
||||||
actor
|
actor
|
||||||
}
|
}
|
||||||
|
|
@ -744,8 +719,8 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
* <p/>
|
* <p/>
|
||||||
* To be invoked from within the actor itself.
|
* To be invoked from within the actor itself.
|
||||||
*/
|
*/
|
||||||
protected[this] def spawnRemote[T <: Actor](actorClass: Class[T], hostname: String, port: Int): T = {
|
protected[this] def spawnRemote[T <: Actor : Manifest](hostname: String, port: Int): T = {
|
||||||
val actor = spawnButDoNotStart(actorClass)
|
val actor = spawnButDoNotStart[T]
|
||||||
actor.makeRemote(hostname, port)
|
actor.makeRemote(hostname, port)
|
||||||
actor.start
|
actor.start
|
||||||
actor
|
actor
|
||||||
|
|
@ -756,8 +731,8 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
* <p/>
|
* <p/>
|
||||||
* To be invoked from within the actor itself.
|
* To be invoked from within the actor itself.
|
||||||
*/
|
*/
|
||||||
protected[this] def spawnLink[T <: Actor](actorClass: Class[T]): T = {
|
protected[this] def spawnLink[T <: Actor : Manifest] : T = {
|
||||||
val actor = spawnButDoNotStart(actorClass)
|
val actor = spawnButDoNotStart[T]
|
||||||
try {
|
try {
|
||||||
actor.start
|
actor.start
|
||||||
} finally {
|
} finally {
|
||||||
|
|
@ -771,8 +746,8 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
* <p/>
|
* <p/>
|
||||||
* To be invoked from within the actor itself.
|
* To be invoked from within the actor itself.
|
||||||
*/
|
*/
|
||||||
protected[this] def spawnLinkRemote[T <: Actor](actorClass: Class[T], hostname: String, port: Int): T = {
|
protected[this] def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): T = {
|
||||||
val actor = spawnButDoNotStart(actorClass)
|
val actor = spawnButDoNotStart[T]
|
||||||
try {
|
try {
|
||||||
actor.makeRemote(hostname, port)
|
actor.makeRemote(hostname, port)
|
||||||
actor.start
|
actor.start
|
||||||
|
|
@ -804,9 +779,11 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
|
|
||||||
private[akka] def getSenderFuture = senderFuture
|
private[akka] def getSenderFuture = senderFuture
|
||||||
|
|
||||||
private def spawnButDoNotStart[T <: Actor](actorClass: Class[T]): T = {
|
private def spawnButDoNotStart[T <: Actor : Manifest] : T = {
|
||||||
val actor = actorClass.newInstance.asInstanceOf[T]
|
val actor = manifest[T].erasure.asInstanceOf[Class[T]].newInstance
|
||||||
if (!dispatcher.isInstanceOf[ThreadBasedDispatcher]) actor.dispatcher = dispatcher
|
if (!dispatcher.isInstanceOf[ThreadBasedDispatcher]) {
|
||||||
|
actor.dispatcher = dispatcher
|
||||||
|
}
|
||||||
actor
|
actor
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -833,7 +810,7 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
requestBuilder.setSourceTarget(s.getClass.getName)
|
requestBuilder.setSourceTarget(s.getClass.getName)
|
||||||
requestBuilder.setSourceUuid(s.uuid)
|
requestBuilder.setSourceUuid(s.uuid)
|
||||||
|
|
||||||
val (host, port) = s._replyToAddress.map(a => (a.getHostName,a.getPort)).getOrElse((Actor.HOSTNAME,Actor.PORT))
|
val (host, port) = s._replyToAddress.map(a => (a.getHostName,a.getPort)).getOrElse((Actor.HOSTNAME, Actor.PORT))
|
||||||
|
|
||||||
Actor.log.debug("Setting sending actor as %s @ %s:%s", s.getClass.getName, host, port)
|
Actor.log.debug("Setting sending actor as %s @ %s:%s", s.getClass.getName, host, port)
|
||||||
|
|
||||||
|
|
@ -1072,17 +1049,14 @@ trait Actor extends TransactionManagement with Logging {
|
||||||
!message.isInstanceOf[List[_]] &&
|
!message.isInstanceOf[List[_]] &&
|
||||||
!message.isInstanceOf[scala.collection.immutable.Map[_, _]] &&
|
!message.isInstanceOf[scala.collection.immutable.Map[_, _]] &&
|
||||||
!message.isInstanceOf[scala.collection.immutable.Set[_]] &&
|
!message.isInstanceOf[scala.collection.immutable.Set[_]] &&
|
||||||
!message.isInstanceOf[scala.collection.immutable.Tree[_, _]] &&
|
//Removed in Scala 2.8
|
||||||
|
//!message.isInstanceOf[scala.collection.immutable.Tree[_, _]] &&
|
||||||
!message.getClass.isAnnotationPresent(Annotations.immutable)) {
|
!message.getClass.isAnnotationPresent(Annotations.immutable)) {
|
||||||
Serializer.Java.deepClone(message)
|
Serializer.Java.deepClone(message)
|
||||||
} else message
|
} else message
|
||||||
} else message
|
} else message
|
||||||
|
|
||||||
override def hashCode(): Int = {
|
override def hashCode(): Int = HashCode.hash(HashCode.SEED, _uuid)
|
||||||
var result = HashCode.SEED
|
|
||||||
result = HashCode.hash(result, _uuid)
|
|
||||||
result
|
|
||||||
}
|
|
||||||
|
|
||||||
override def equals(that: Any): Boolean = {
|
override def equals(that: Any): Boolean = {
|
||||||
that != null &&
|
that != null &&
|
||||||
|
|
|
||||||
|
|
@ -110,7 +110,7 @@ object ActorRegistry extends Logging {
|
||||||
} else actorsByClassName.put(className, actor :: Nil)
|
} else actorsByClassName.put(className, actor :: Nil)
|
||||||
|
|
||||||
// notify listeners
|
// notify listeners
|
||||||
foreachListener(_ send ActorRegistered(actor))
|
foreachListener(_ ! ActorRegistered(actor))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -121,7 +121,7 @@ object ActorRegistry extends Logging {
|
||||||
actorsById remove actor.getId
|
actorsById remove actor.getId
|
||||||
actorsByClassName remove actor.getClass.getName
|
actorsByClassName remove actor.getClass.getName
|
||||||
// notify listeners
|
// notify listeners
|
||||||
foreachListener(_ send ActorUnregistered(actor))
|
foreachListener(_ ! ActorUnregistered(actor))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -1,17 +1,7 @@
|
||||||
// Copyright © 2008-10 The original author or authors
|
/**
|
||||||
//
|
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
|
||||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
*/
|
||||||
// you may not use this file except in compliance with the License.
|
|
||||||
// You may obtain a copy of the License at
|
|
||||||
//
|
|
||||||
// http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
//
|
|
||||||
// Unless required by applicable law or agreed to in writing, software
|
|
||||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
// See the License for the specific language governing permissions and
|
|
||||||
// limitations under the License.
|
|
||||||
|
|
||||||
package se.scalablesolutions.akka.actor
|
package se.scalablesolutions.akka.actor
|
||||||
|
|
||||||
import se.scalablesolutions.akka.stm.Ref
|
import se.scalablesolutions.akka.stm.Ref
|
||||||
|
|
@ -22,48 +12,90 @@ import java.util.concurrent.CountDownLatch
|
||||||
class AgentException private[akka](message: String) extends RuntimeException(message)
|
class AgentException private[akka](message: String) extends RuntimeException(message)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The Agent class was strongly inspired by the agent principle in Clojure.
|
* The Agent class was strongly inspired by the agent principle in Clojure.
|
||||||
* Essentially, an agent wraps a shared mutable state and hides it behind
|
* <p/>
|
||||||
* a message-passing interface. Agents accept messages and process them on
|
|
||||||
* behalf of the wrapped state.
|
|
||||||
*
|
|
||||||
* Typically agents accept functions / commands as messages and ensure the
|
|
||||||
* submitted commands are executed against the internal agent's state in a
|
|
||||||
* thread-safe manner (sequentially).
|
|
||||||
*
|
|
||||||
* The submitted functions / commands take the internal state as a parameter
|
|
||||||
* and their output becomes the new internal state value.
|
|
||||||
*
|
|
||||||
* The code that is submitted to an agent doesn't need to pay attention to
|
|
||||||
* threading or synchronization, the agent will provide such guarantees by itself.
|
|
||||||
*
|
*
|
||||||
* If an Agent is used within an enclosing transaction, then it will participate
|
* Agents provide independent, asynchronous change of individual locations.
|
||||||
* in that transaction.
|
* Agents are bound to a single storage location for their lifetime, and
|
||||||
|
* only allow mutation of that location (to a new state) to occur as a
|
||||||
|
* result of an action. Actions are functions (with, optionally, additional
|
||||||
|
* arguments) that are asynchronously applied to an Agent's state and whose
|
||||||
|
* return value becomes the Agent's new state. Because the set of functions
|
||||||
|
* is open, the set of actions supported by an Agent is also open, a sharp
|
||||||
|
* contrast to pattern matching message handling loops provided by Actors.
|
||||||
|
* <p/>
|
||||||
|
*
|
||||||
|
* Agents are reactive, not autonomous - there is no imperative message loop
|
||||||
|
* and no blocking receive. The state of an Agent should be itself immutable
|
||||||
|
* (preferably an instance of one of Akka's persistent collections), and the
|
||||||
|
* state of an Agent is always immediately available for reading by any
|
||||||
|
* thread (using the '()' function) without any messages, i.e. observation
|
||||||
|
* does not require cooperation or coordination.
|
||||||
|
* <p/>
|
||||||
|
*
|
||||||
|
* The actions of all Agents get interleaved amongst threads in a thread pool.
|
||||||
|
* At any point in time, at most one action for each Agent is being executed.
|
||||||
|
* Actions dispatched to an agent from another single agent or thread will
|
||||||
|
* occur in the order they were sent, potentially interleaved with actions
|
||||||
|
* dispatched to the same agent from other sources.
|
||||||
|
* <p/>
|
||||||
|
*
|
||||||
|
* If an Agent is used within an enclosing transaction, then it will
|
||||||
|
* participate in that transaction.
|
||||||
|
* <p/>
|
||||||
*
|
*
|
||||||
* Example of usage:
|
* Example of usage:
|
||||||
* <pre>
|
* <pre>
|
||||||
* val agent = Agent(5)
|
* val agent = Agent(5)
|
||||||
*
|
*
|
||||||
* agent update (_ + 1)
|
* agent send (_ + 1)
|
||||||
* agent update (_ * 2)
|
* agent send (_ * 2)
|
||||||
*
|
*
|
||||||
* val result = agent()
|
* val result = agent()
|
||||||
* ... // use result
|
* ... // use result
|
||||||
*
|
*
|
||||||
* agent.close
|
* agent.close
|
||||||
* </pre>
|
* </pre>
|
||||||
|
* <p/>
|
||||||
*
|
*
|
||||||
* NOTE: You can't call 'agent.get' or 'agent()' within an enclosing transaction since
|
* Agent is also monadic, which means that you can compose operations using
|
||||||
* that will block the transaction indefinitely. But 'agent.update' or 'Agent(value)'
|
* for-comprehensions. In monadic usage the original agents are not touched
|
||||||
* is fine.
|
* but new agents are created. So the old values (agents) are still available
|
||||||
|
* as-is. They are so-called 'persistent'.
|
||||||
|
* <p/>
|
||||||
*
|
*
|
||||||
* Original author:
|
* Example of monadic usage:
|
||||||
* @author Vaclav Pech
|
* <pre>
|
||||||
|
* val agent1 = Agent(3)
|
||||||
|
* val agent2 = Agent(5)
|
||||||
*
|
*
|
||||||
* Inital AKKA port by:
|
* for (value <- agent1) {
|
||||||
* @author Viktor Klang
|
* result = value + 1
|
||||||
|
* }
|
||||||
*
|
*
|
||||||
* Modifications by:
|
* val agent3 =
|
||||||
|
* for (value <- agent1) yield value + 1
|
||||||
|
*
|
||||||
|
* val agent4 = for {
|
||||||
|
* value1 <- agent1
|
||||||
|
* value2 <- agent2
|
||||||
|
* } yield value1 + value2
|
||||||
|
*
|
||||||
|
* agent1.close
|
||||||
|
* agent2.close
|
||||||
|
* agent3.close
|
||||||
|
* agent4.close
|
||||||
|
* </pre>
|
||||||
|
* <p/>
|
||||||
|
*
|
||||||
|
* IMPORTANT:
|
||||||
|
* You can *not* call 'agent.get', 'agent()' or use the monadic 'foreach',
|
||||||
|
* 'map and 'flatMap' within an enclosing transaction since that would block
|
||||||
|
* the transaction indefinitely. But all other operations are fine. The system
|
||||||
|
* will raise an error (e.g. *not* deadlock) if you try to do so, so as long as
|
||||||
|
* you test your application thoroughly you should be fine.
|
||||||
|
*
|
||||||
|
* @author Viktor Klang
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
sealed class Agent[T] private (initialValue: T) extends Transactor {
|
sealed class Agent[T] private (initialValue: T) extends Transactor {
|
||||||
|
|
@ -71,15 +103,18 @@ sealed class Agent[T] private (initialValue: T) extends Transactor {
|
||||||
private lazy val value = Ref[T]()
|
private lazy val value = Ref[T]()
|
||||||
|
|
||||||
start
|
start
|
||||||
this ! ValueHolder(initialValue)
|
this !! Value(initialValue)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Periodically handles incoming messages.
|
* Periodically handles incoming messages.
|
||||||
*/
|
*/
|
||||||
def receive = {
|
def receive = {
|
||||||
case ValueHolder(x: T) => updateData(x)
|
case Value(v: T) =>
|
||||||
case FunctionHolder(fun: (T => T)) => updateData(fun(value.getOrWait))
|
swap(v)
|
||||||
case ProcedureHolder(fun: (T => Unit)) => fun(copyStrategy(value.getOrWait))
|
case Function(fun: (T => T)) =>
|
||||||
|
swap(fun(value.getOrWait))
|
||||||
|
case Procedure(proc: (T => Unit)) =>
|
||||||
|
proc(copyStrategy(value.getOrElse(throw new AgentException("Could not read Agent's value; value is null"))))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -87,11 +122,11 @@ sealed class Agent[T] private (initialValue: T) extends Transactor {
|
||||||
*/
|
*/
|
||||||
protected def copyStrategy(t: T): T = t
|
protected def copyStrategy(t: T): T = t
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Updates the internal state with the value provided as a by-name parameter.
|
* Performs a CAS operation, atomically swapping the internal state with the value
|
||||||
*/
|
* provided as a by-name parameter.
|
||||||
private final def updateData(newData: => T): Unit = value.swap(newData)
|
*/
|
||||||
|
private final def swap(newData: => T): Unit = value.swap(newData)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Submits a request to read the internal state.
|
* Submits a request to read the internal state.
|
||||||
|
|
@ -105,46 +140,64 @@ sealed class Agent[T] private (initialValue: T) extends Transactor {
|
||||||
"Can't call Agent.get within an enclosing transaction.\n\tWould block indefinitely.\n\tPlease refactor your code.")
|
"Can't call Agent.get within an enclosing transaction.\n\tWould block indefinitely.\n\tPlease refactor your code.")
|
||||||
val ref = new AtomicReference[T]
|
val ref = new AtomicReference[T]
|
||||||
val latch = new CountDownLatch(1)
|
val latch = new CountDownLatch(1)
|
||||||
get((x: T) => {ref.set(x); latch.countDown})
|
sendProc((v: T) => {ref.set(v); latch.countDown})
|
||||||
latch.await
|
latch.await
|
||||||
ref.get
|
ref.get
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Asynchronously submits a request to read the internal state. The supplied function
|
* Submits a request to read the internal state. A copy of the internal state will be
|
||||||
* will be executed on the returned internal state value. A copy of the internal state
|
* returned, depending on the underlying effective copyStrategy. Internally leverages
|
||||||
* will be used, depending on the underlying effective copyStrategy.
|
* the asynchronous getValue() method and then waits for its result on a CountDownLatch.
|
||||||
*/
|
*/
|
||||||
final def get(message: (T => Unit)): Unit = this ! ProcedureHolder(message)
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Submits a request to read the internal state. A copy of the internal state will be
|
|
||||||
* returned, depending on the underlying effective copyStrategy. Internally leverages
|
|
||||||
* the asynchronous getValue() method and then waits for its result on a CountDownLatch.
|
|
||||||
*/
|
|
||||||
final def apply(): T = get
|
final def apply(): T = get
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Submits the provided function for execution against the internal agent's state.
|
||||||
|
*/
|
||||||
|
final def apply(message: (T => T)): Unit = this ! Function(message)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Submits a new value to be set as the new agent's internal state.
|
||||||
|
*/
|
||||||
|
final def apply(message: T): Unit = this ! Value(message)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Submits the provided function of type 'T => T' for execution against the internal agent's state.
|
||||||
|
*/
|
||||||
|
final def send(message: (T) => T): Unit = this ! Function(message)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Submits a new value to be set as the new agent's internal state.
|
||||||
|
*/
|
||||||
|
final def send(message: T): Unit = this ! Value(message)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Submits the provided function for execution against the internal agent's state.
|
* Asynchronously submits a procedure of type 'T => Unit' to read the internal state.
|
||||||
*/
|
* The supplied procedure will be executed on the returned internal state value. A copy
|
||||||
final def apply(message: (T => T)): Unit = this ! FunctionHolder(message)
|
* of the internal state will be used, depending on the underlying effective copyStrategy.
|
||||||
|
* Does not change the value of the agent (this).
|
||||||
|
*/
|
||||||
|
final def sendProc(f: (T) => Unit): Unit = this ! Procedure(f)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Submits a new value to be set as the new agent's internal state.
|
* Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result.
|
||||||
*/
|
* Does not change the value of the agent (this).
|
||||||
final def apply(message: T): Unit = this ! ValueHolder(message)
|
*/
|
||||||
|
final def map[B](f: (T) => B): Agent[B] = Agent(f(get))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Submits the provided function for execution against the internal agent's state.
|
* Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result.
|
||||||
*/
|
* Does not change the value of the agent (this).
|
||||||
final def update(message: (T => T)): Unit = this ! FunctionHolder(message)
|
*/
|
||||||
|
final def flatMap[B](f: (T) => Agent[B]): Agent[B] = Agent(f(get)())
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Submits a new value to be set as the new agent's internal state.
|
* Applies function with type 'T => B' to the agent's internal state.
|
||||||
*/
|
* Does not change the value of the agent (this).
|
||||||
// FIXME Change to 'send' when we have Scala 2.8 and we can remove the Actor.send method
|
*/
|
||||||
final def update(message: T): Unit = this ! ValueHolder(message)
|
final def foreach(f: (T) => Unit): Unit = f(get)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Closes the agents and makes it eligable for garbage collection.
|
* Closes the agents and makes it eligable for garbage collection.
|
||||||
*
|
*
|
||||||
|
|
@ -154,16 +207,19 @@ sealed class Agent[T] private (initialValue: T) extends Transactor {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Provides factory methods to create Agents.
|
* Provides factory methods to create Agents.
|
||||||
*/
|
*
|
||||||
object Agent {
|
* @author Viktor Klang
|
||||||
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
|
*/
|
||||||
|
object Agent {
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* The internal messages for passing around requests.
|
* The internal messages for passing around requests.
|
||||||
*/
|
*/
|
||||||
private case class ProcedureHolder[T](fun: ((T) => Unit))
|
private case class Value[T](value: T)
|
||||||
private case class FunctionHolder[T](fun: ((T) => T))
|
private case class Function[T](fun: ((T) => T))
|
||||||
private case class ValueHolder[T](value: T)
|
private case class Procedure[T](fun: ((T) => Unit))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new Agent of type T with the initial value of value.
|
* Creates a new Agent of type T with the initial value of value.
|
||||||
|
|
@ -177,4 +233,4 @@ object Agent {
|
||||||
def apply[T](value: T, newCopyStrategy: (T) => T) = new Agent(value) {
|
def apply[T](value: T, newCopyStrategy: (T) => T) = new Agent(value) {
|
||||||
override def copyStrategy(t: T) = newCopyStrategy(t)
|
override def copyStrategy(t: T) = newCopyStrategy(t)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -28,7 +28,7 @@ trait BootableActorLoaderService extends Bootable with Logging {
|
||||||
log.error("Could not find a deploy directory at [%s]", DEPLOY)
|
log.error("Could not find a deploy directory at [%s]", DEPLOY)
|
||||||
System.exit(-1)
|
System.exit(-1)
|
||||||
}
|
}
|
||||||
val toDeploy = for (f <- DEPLOY_DIR.listFiles().toArray.toList.asInstanceOf[List[File]]) yield f.toURL
|
val toDeploy = for (f <- DEPLOY_DIR.listFiles().toArray.toList.asInstanceOf[List[File]]) yield f.toURI.toURL
|
||||||
log.info("Deploying applications from [%s]: [%s]", DEPLOY, toDeploy.toArray.toList)
|
log.info("Deploying applications from [%s]: [%s]", DEPLOY, toDeploy.toArray.toList)
|
||||||
new URLClassLoader(toDeploy.toArray, getClass.getClassLoader)
|
new URLClassLoader(toDeploy.toArray, getClass.getClassLoader)
|
||||||
} else getClass.getClassLoader)
|
} else getClass.getClassLoader)
|
||||||
|
|
|
||||||
|
|
@ -19,8 +19,6 @@ import se.scalablesolutions.akka.config.ScalaConfig._
|
||||||
import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy}
|
import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy}
|
||||||
import se.scalablesolutions.akka.util.Logging
|
import se.scalablesolutions.akka.util.Logging
|
||||||
|
|
||||||
import org.scala_tools.javautils.Imports._
|
|
||||||
|
|
||||||
case object UnSchedule
|
case object UnSchedule
|
||||||
case class SchedulerException(msg: String, e: Throwable) extends RuntimeException(msg, e)
|
case class SchedulerException(msg: String, e: Throwable) extends RuntimeException(msg, e)
|
||||||
|
|
||||||
|
|
@ -66,7 +64,8 @@ object Scheduler extends Actor {
|
||||||
}
|
}
|
||||||
|
|
||||||
override def shutdown = {
|
override def shutdown = {
|
||||||
schedulers.values.asScala.foreach(_ ! UnSchedule)
|
import scala.collection.JavaConversions._
|
||||||
|
schedulers.values.foreach(_ ! UnSchedule)
|
||||||
service.shutdown
|
service.shutdown
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -89,8 +89,8 @@ sealed class Supervisor private[akka] (handler: FaultHandlingStrategy, trapExcep
|
||||||
// Cheating, should really go through the dispatcher rather than direct access to a CHM
|
// Cheating, should really go through the dispatcher rather than direct access to a CHM
|
||||||
def getInstance[T](clazz: Class[T]): List[T] = actors.get(clazz.getName).asInstanceOf[List[T]]
|
def getInstance[T](clazz: Class[T]): List[T] = actors.get(clazz.getName).asInstanceOf[List[T]]
|
||||||
|
|
||||||
def getComponentInterfaces: List[Class[_]] = List.flatten(
|
def getComponentInterfaces: List[Class[_]] =
|
||||||
actors.values.toArray.toList.asInstanceOf[List[List[AnyRef]]]).map(_.getClass)
|
actors.values.toArray.toList.asInstanceOf[List[List[AnyRef]]].flatten.map(_.getClass)
|
||||||
|
|
||||||
def isDefined(clazz: Class[_]): Boolean = actors.containsKey(clazz.getName)
|
def isDefined(clazz: Class[_]): Boolean = actors.containsKey(clazz.getName)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -6,12 +6,11 @@ package se.scalablesolutions.akka.config
|
||||||
|
|
||||||
import JavaConfig._
|
import JavaConfig._
|
||||||
|
|
||||||
|
import java.util.{List => JList}
|
||||||
|
import java.util.{ArrayList}
|
||||||
|
|
||||||
import com.google.inject._
|
import com.google.inject._
|
||||||
|
|
||||||
import org.scala_tools.javautils.Imports._
|
|
||||||
|
|
||||||
import java.util.{List=>JList, ArrayList}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Configurator for the Active Objects. Used to do declarative configuration of supervision.
|
* Configurator for the Active Objects. Used to do declarative configuration of supervision.
|
||||||
* It also does dependency injection with and into Active Objects using dependency injection
|
* It also does dependency injection with and into Active Objects using dependency injection
|
||||||
|
|
@ -23,6 +22,7 @@ import java.util.{List=>JList, ArrayList}
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
class ActiveObjectConfigurator {
|
class ActiveObjectConfigurator {
|
||||||
|
import scala.collection.JavaConversions._
|
||||||
// TODO: make pluggable once we have f.e a SpringConfigurator
|
// TODO: make pluggable once we have f.e a SpringConfigurator
|
||||||
private val INSTANCE = new ActiveObjectGuiceConfigurator
|
private val INSTANCE = new ActiveObjectGuiceConfigurator
|
||||||
|
|
||||||
|
|
@ -32,7 +32,7 @@ class ActiveObjectConfigurator {
|
||||||
* @param clazz the class for the active object
|
* @param clazz the class for the active object
|
||||||
* @return a list with all the active objects for the class
|
* @return a list with all the active objects for the class
|
||||||
*/
|
*/
|
||||||
def getInstances[T](clazz: Class[T]): JList[T] = INSTANCE.getInstance(clazz).asJava
|
def getInstances[T](clazz: Class[T]): JList[T] = INSTANCE.getInstance(clazz).foldLeft(new ArrayList[T]){ (l, i) => l add i ; l }
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the first item in a list of all active objects that has been put under supervision for the class specified.
|
* Returns the first item in a list of all active objects that has been put under supervision for the class specified.
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,7 @@ import net.lag.configgy.{Configgy, ParseException}
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
object Config extends Logging {
|
object Config extends Logging {
|
||||||
val VERSION = "0.7"
|
val VERSION = "0.8"
|
||||||
|
|
||||||
// Set Multiverse options for max speed
|
// Set Multiverse options for max speed
|
||||||
System.setProperty("org.multiverse.MuliverseConstants.sanityChecks", "false")
|
System.setProperty("org.multiverse.MuliverseConstants.sanityChecks", "false")
|
||||||
|
|
|
||||||
|
|
@ -13,7 +13,7 @@ object ConfiguratorRepository extends Logging {
|
||||||
private val configuration = new HashSet[Configurator]
|
private val configuration = new HashSet[Configurator]
|
||||||
|
|
||||||
def registerConfigurator(conf: Configurator) = synchronized {
|
def registerConfigurator(conf: Configurator) = synchronized {
|
||||||
configuration + conf
|
configuration += conf
|
||||||
}
|
}
|
||||||
|
|
||||||
def getConfigurators: List[Configurator] = synchronized {
|
def getConfigurators: List[Configurator] = synchronized {
|
||||||
|
|
|
||||||
|
|
@ -152,7 +152,7 @@ abstract class BasicClusterActor extends ClusterActor {
|
||||||
|
|
||||||
case Papers(x) => remotes = remotes + (src -> Node(x))
|
case Papers(x) => remotes = remotes + (src -> Node(x))
|
||||||
|
|
||||||
case RelayedMessage(c, m) => ActorRegistry.actorsFor(c).foreach(_ send m)
|
case RelayedMessage(c, m) => ActorRegistry.actorsFor(c).foreach(_ ! m)
|
||||||
|
|
||||||
case unknown => log debug ("Unknown message: %s", unknown.toString)
|
case unknown => log debug ("Unknown message: %s", unknown.toString)
|
||||||
}
|
}
|
||||||
|
|
@ -166,7 +166,7 @@ abstract class BasicClusterActor extends ClusterActor {
|
||||||
|
|
||||||
case DeregisterLocalNode(s) => {
|
case DeregisterLocalNode(s) => {
|
||||||
log debug ("DeregisterLocalNode: %s", s)
|
log debug ("DeregisterLocalNode: %s", s)
|
||||||
local = Node(local.endpoints - s)
|
local = Node(local.endpoints.filterNot(_ == s))
|
||||||
broadcast(Papers(local.endpoints))
|
broadcast(Papers(local.endpoints))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -201,30 +201,30 @@ abstract class BasicClusterActor extends ClusterActor {
|
||||||
* Applies the given PartialFunction to all known RemoteAddresses
|
* Applies the given PartialFunction to all known RemoteAddresses
|
||||||
*/
|
*/
|
||||||
def lookup[T](handleRemoteAddress: PartialFunction[RemoteAddress, T]): Option[T] =
|
def lookup[T](handleRemoteAddress: PartialFunction[RemoteAddress, T]): Option[T] =
|
||||||
remotes.values.toList.flatMap(_.endpoints).find(handleRemoteAddress isDefinedAt _).map(handleRemoteAddress)
|
remotes.valuesIterator.toList.flatMap(_.endpoints).find(handleRemoteAddress isDefinedAt _).map(handleRemoteAddress)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Applies the given function to all remote addresses known
|
* Applies the given function to all remote addresses known
|
||||||
*/
|
*/
|
||||||
def foreach(f: (RemoteAddress) => Unit): Unit = remotes.values.toList.flatMap(_.endpoints).foreach(f)
|
def foreach(f: (RemoteAddress) => Unit): Unit = remotes.valuesIterator.toList.flatMap(_.endpoints).foreach(f)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Registers a local endpoint
|
* Registers a local endpoint
|
||||||
*/
|
*/
|
||||||
def registerLocalNode(hostname: String, port: Int): Unit =
|
def registerLocalNode(hostname: String, port: Int): Unit =
|
||||||
send(RegisterLocalNode(RemoteAddress(hostname, port)))
|
this ! RegisterLocalNode(RemoteAddress(hostname, port))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deregisters a local endpoint
|
* Deregisters a local endpoint
|
||||||
*/
|
*/
|
||||||
def deregisterLocalNode(hostname: String, port: Int): Unit =
|
def deregisterLocalNode(hostname: String, port: Int): Unit =
|
||||||
send(DeregisterLocalNode(RemoteAddress(hostname, port)))
|
this ! DeregisterLocalNode(RemoteAddress(hostname, port))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Broadcasts the specified message to all Actors of type Class on all known Nodes
|
* Broadcasts the specified message to all Actors of type Class on all known Nodes
|
||||||
*/
|
*/
|
||||||
def relayMessage(to: Class[_ <: Actor], msg: AnyRef): Unit =
|
def relayMessage(to: Class[_ <: Actor], msg: AnyRef): Unit =
|
||||||
send(RelayedMessage(to.getName, msg))
|
this ! RelayedMessage(to.getName, msg)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -127,7 +127,7 @@ object RemoteClient extends Logging {
|
||||||
if (remoteClients.contains(hash)) {
|
if (remoteClients.contains(hash)) {
|
||||||
val client = remoteClients(hash)
|
val client = remoteClients(hash)
|
||||||
client.shutdown
|
client.shutdown
|
||||||
remoteClients - hash
|
remoteClients -= hash
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -140,13 +140,13 @@ object RemoteClient extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
private[akka] def register(hostname: String, port: Int, uuid: String) = synchronized {
|
private[akka] def register(hostname: String, port: Int, uuid: String) = synchronized {
|
||||||
actorsFor(RemoteServer.Address(hostname, port)) + uuid
|
actorsFor(RemoteServer.Address(hostname, port)) += uuid
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: add RemoteClient.unregister for ActiveObject, but first need a @shutdown callback
|
// TODO: add RemoteClient.unregister for ActiveObject, but first need a @shutdown callback
|
||||||
private[akka] def unregister(hostname: String, port: Int, uuid: String) = synchronized {
|
private[akka] def unregister(hostname: String, port: Int, uuid: String) = synchronized {
|
||||||
val set = actorsFor(RemoteServer.Address(hostname, port))
|
val set = actorsFor(RemoteServer.Address(hostname, port))
|
||||||
set - uuid
|
set -= uuid
|
||||||
if (set.isEmpty) shutdownClientFor(new InetSocketAddress(hostname, port))
|
if (set.isEmpty) shutdownClientFor(new InetSocketAddress(hostname, port))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -267,7 +267,7 @@ class RemoteClientPipelineFactory(name: String,
|
||||||
/**
|
/**
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
@ChannelPipelineCoverage {val value = "all"}
|
@ChannelHandler.Sharable
|
||||||
class RemoteClientHandler(val name: String,
|
class RemoteClientHandler(val name: String,
|
||||||
val futures: ConcurrentMap[Long, CompletableFuture],
|
val futures: ConcurrentMap[Long, CompletableFuture],
|
||||||
val supervisors: ConcurrentMap[String, Actor],
|
val supervisors: ConcurrentMap[String, Actor],
|
||||||
|
|
|
||||||
|
|
@ -4,7 +4,7 @@
|
||||||
|
|
||||||
package se.scalablesolutions.akka.remote
|
package se.scalablesolutions.akka.remote
|
||||||
|
|
||||||
import se.scalablesolutions.akka.serialization.Serializable.SBinary
|
//import se.scalablesolutions.akka.serialization.Serializable.SBinary
|
||||||
import se.scalablesolutions.akka.serialization.{Serializer, Serializable, SerializationProtocol}
|
import se.scalablesolutions.akka.serialization.{Serializer, Serializable, SerializationProtocol}
|
||||||
import se.scalablesolutions.akka.remote.protobuf.RemoteProtocol.{RemoteRequest, RemoteReply}
|
import se.scalablesolutions.akka.remote.protobuf.RemoteProtocol.{RemoteRequest, RemoteReply}
|
||||||
|
|
||||||
|
|
@ -14,7 +14,7 @@ object RemoteProtocolBuilder {
|
||||||
private var SERIALIZER_JAVA: Serializer.Java = Serializer.Java
|
private var SERIALIZER_JAVA: Serializer.Java = Serializer.Java
|
||||||
private var SERIALIZER_JAVA_JSON: Serializer.JavaJSON = Serializer.JavaJSON
|
private var SERIALIZER_JAVA_JSON: Serializer.JavaJSON = Serializer.JavaJSON
|
||||||
private var SERIALIZER_SCALA_JSON: Serializer.ScalaJSON = Serializer.ScalaJSON
|
private var SERIALIZER_SCALA_JSON: Serializer.ScalaJSON = Serializer.ScalaJSON
|
||||||
private var SERIALIZER_SBINARY: Serializer.SBinary = Serializer.SBinary
|
//private var SERIALIZER_SBINARY: Serializer.SBinary = Serializer.SBinary
|
||||||
private var SERIALIZER_PROTOBUF: Serializer.Protobuf = Serializer.Protobuf
|
private var SERIALIZER_PROTOBUF: Serializer.Protobuf = Serializer.Protobuf
|
||||||
|
|
||||||
|
|
||||||
|
|
@ -26,10 +26,9 @@ object RemoteProtocolBuilder {
|
||||||
|
|
||||||
def getMessage(request: RemoteRequest): Any = {
|
def getMessage(request: RemoteRequest): Any = {
|
||||||
request.getProtocol match {
|
request.getProtocol match {
|
||||||
case SerializationProtocol.SBINARY =>
|
//case SerializationProtocol.SBINARY =>
|
||||||
val renderer = Class.forName(
|
// val renderer = Class.forName(new String(request.getMessageManifest.toByteArray)).newInstance.asInstanceOf[SBinary[_ <: AnyRef]]
|
||||||
new String(request.getMessageManifest.toByteArray)).newInstance.asInstanceOf[SBinary[_ <: AnyRef]]
|
// renderer.fromBytes(request.getMessage.toByteArray)
|
||||||
renderer.fromBytes(request.getMessage.toByteArray)
|
|
||||||
case SerializationProtocol.SCALA_JSON =>
|
case SerializationProtocol.SCALA_JSON =>
|
||||||
val manifest = SERIALIZER_JAVA.in(request.getMessageManifest.toByteArray, None).asInstanceOf[String]
|
val manifest = SERIALIZER_JAVA.in(request.getMessageManifest.toByteArray, None).asInstanceOf[String]
|
||||||
SERIALIZER_SCALA_JSON.in(request.getMessage.toByteArray, Some(Class.forName(manifest)))
|
SERIALIZER_SCALA_JSON.in(request.getMessage.toByteArray, Some(Class.forName(manifest)))
|
||||||
|
|
@ -48,9 +47,9 @@ object RemoteProtocolBuilder {
|
||||||
|
|
||||||
def getMessage(reply: RemoteReply): Any = {
|
def getMessage(reply: RemoteReply): Any = {
|
||||||
reply.getProtocol match {
|
reply.getProtocol match {
|
||||||
case SerializationProtocol.SBINARY =>
|
//case SerializationProtocol.SBINARY =>
|
||||||
val renderer = Class.forName(new String(reply.getMessageManifest.toByteArray)).newInstance.asInstanceOf[SBinary[_ <: AnyRef]]
|
// val renderer = Class.forName(new String(reply.getMessageManifest.toByteArray)).newInstance.asInstanceOf[SBinary[_ <: AnyRef]]
|
||||||
renderer.fromBytes(reply.getMessage.toByteArray)
|
// renderer.fromBytes(reply.getMessage.toByteArray)
|
||||||
case SerializationProtocol.SCALA_JSON =>
|
case SerializationProtocol.SCALA_JSON =>
|
||||||
val manifest = SERIALIZER_JAVA.in(reply.getMessageManifest.toByteArray, None).asInstanceOf[String]
|
val manifest = SERIALIZER_JAVA.in(reply.getMessageManifest.toByteArray, None).asInstanceOf[String]
|
||||||
SERIALIZER_SCALA_JSON.in(reply.getMessage.toByteArray, Some(Class.forName(manifest)))
|
SERIALIZER_SCALA_JSON.in(reply.getMessage.toByteArray, Some(Class.forName(manifest)))
|
||||||
|
|
@ -68,12 +67,12 @@ object RemoteProtocolBuilder {
|
||||||
}
|
}
|
||||||
|
|
||||||
def setMessage(message: Any, builder: RemoteRequest.Builder) = {
|
def setMessage(message: Any, builder: RemoteRequest.Builder) = {
|
||||||
if (message.isInstanceOf[Serializable.SBinary[_]]) {
|
/*if (message.isInstanceOf[Serializable.SBinary[_]]) {
|
||||||
val serializable = message.asInstanceOf[Serializable.SBinary[_ <: Any]]
|
val serializable = message.asInstanceOf[Serializable.SBinary[_ <: Any]]
|
||||||
builder.setProtocol(SerializationProtocol.SBINARY)
|
builder.setProtocol(SerializationProtocol.SBINARY)
|
||||||
builder.setMessage(ByteString.copyFrom(serializable.toBytes))
|
builder.setMessage(ByteString.copyFrom(serializable.toBytes))
|
||||||
builder.setMessageManifest(ByteString.copyFrom(serializable.getClass.getName.getBytes))
|
builder.setMessageManifest(ByteString.copyFrom(serializable.getClass.getName.getBytes))
|
||||||
} else if (message.isInstanceOf[Message]) {
|
} else*/ if (message.isInstanceOf[Message]) {
|
||||||
val serializable = message.asInstanceOf[Message]
|
val serializable = message.asInstanceOf[Message]
|
||||||
builder.setProtocol(SerializationProtocol.PROTOBUF)
|
builder.setProtocol(SerializationProtocol.PROTOBUF)
|
||||||
builder.setMessage(ByteString.copyFrom(serializable.toByteArray))
|
builder.setMessage(ByteString.copyFrom(serializable.toByteArray))
|
||||||
|
|
@ -96,12 +95,12 @@ object RemoteProtocolBuilder {
|
||||||
}
|
}
|
||||||
|
|
||||||
def setMessage(message: Any, builder: RemoteReply.Builder) = {
|
def setMessage(message: Any, builder: RemoteReply.Builder) = {
|
||||||
if (message.isInstanceOf[Serializable.SBinary[_]]) {
|
/*if (message.isInstanceOf[Serializable.SBinary[_]]) {
|
||||||
val serializable = message.asInstanceOf[Serializable.SBinary[_ <: Any]]
|
val serializable = message.asInstanceOf[Serializable.SBinary[_ <: Any]]
|
||||||
builder.setProtocol(SerializationProtocol.SBINARY)
|
builder.setProtocol(SerializationProtocol.SBINARY)
|
||||||
builder.setMessage(ByteString.copyFrom(serializable.toBytes))
|
builder.setMessage(ByteString.copyFrom(serializable.toBytes))
|
||||||
builder.setMessageManifest(ByteString.copyFrom(serializable.getClass.getName.getBytes))
|
builder.setMessageManifest(ByteString.copyFrom(serializable.getClass.getName.getBytes))
|
||||||
} else if (message.isInstanceOf[Message]) {
|
} else*/ if (message.isInstanceOf[Message]) {
|
||||||
val serializable = message.asInstanceOf[Message]
|
val serializable = message.asInstanceOf[Message]
|
||||||
builder.setProtocol(SerializationProtocol.PROTOBUF)
|
builder.setProtocol(SerializationProtocol.PROTOBUF)
|
||||||
builder.setMessage(ByteString.copyFrom(serializable.toByteArray))
|
builder.setMessage(ByteString.copyFrom(serializable.toByteArray))
|
||||||
|
|
|
||||||
|
|
@ -244,7 +244,7 @@ class RemoteServerPipelineFactory(
|
||||||
/**
|
/**
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
@ChannelPipelineCoverage {val value = "all"}
|
@ChannelHandler.Sharable
|
||||||
class RemoteServerHandler(
|
class RemoteServerHandler(
|
||||||
val name: String,
|
val name: String,
|
||||||
val openChannels: ChannelGroup,
|
val openChannels: ChannelGroup,
|
||||||
|
|
@ -310,7 +310,7 @@ class RemoteServerHandler(
|
||||||
actor.!(message)(Some(remoteActor))
|
actor.!(message)(Some(remoteActor))
|
||||||
} else {
|
} else {
|
||||||
// couldn't find a way to reply, send the message without a source/sender
|
// couldn't find a way to reply, send the message without a source/sender
|
||||||
actor.send(message)
|
actor ! message
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
try {
|
try {
|
||||||
|
|
|
||||||
|
|
@ -4,63 +4,57 @@
|
||||||
|
|
||||||
package se.scalablesolutions.akka.serialization
|
package se.scalablesolutions.akka.serialization
|
||||||
|
|
||||||
|
import sbinary._
|
||||||
|
import sbinary.Operations._
|
||||||
|
import sbinary.DefaultProtocol._
|
||||||
|
|
||||||
// --- PRIMITIVES ---
|
// --- PRIMITIVES ---
|
||||||
case class BinaryString(val value: String) extends Serializable.SBinary[BinaryString] {
|
case class BinaryString(val value: String) extends Serializable.SBinary[BinaryString] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]): BinaryString = BinaryString(fromByteArray[String](bytes))
|
def fromBytes(bytes: Array[Byte]): BinaryString = BinaryString(fromByteArray[String](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigInt(val value: BigInt) extends Serializable.SBinary[BinaryBigInt] {
|
case class BinaryBigInt(val value: BigInt) extends Serializable.SBinary[BinaryBigInt] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigInt(fromByteArray[BigInt](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigInt(fromByteArray[BigInt](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigDecimal(val value: BigDecimal) extends Serializable.SBinary[BinaryBigDecimal] {
|
case class BinaryBigDecimal(val value: BigDecimal) extends Serializable.SBinary[BinaryBigDecimal] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimal(fromByteArray[BigDecimal](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimal(fromByteArray[BigDecimal](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryLong(val value: Long) extends Serializable.SBinary[BinaryLong] {
|
case class BinaryLong(val value: Long) extends Serializable.SBinary[BinaryLong] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(0L)
|
def this() = this(0L)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryLong(fromByteArray[Long](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryLong(fromByteArray[Long](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryInt(val value: Int) extends Serializable.SBinary[BinaryInt] {
|
case class BinaryInt(val value: Int) extends Serializable.SBinary[BinaryInt] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(0)
|
def this() = this(0)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryInt(fromByteArray[Int](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryInt(fromByteArray[Int](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryDouble(val value: Double) extends Serializable.SBinary[BinaryDouble] {
|
case class BinaryDouble(val value: Double) extends Serializable.SBinary[BinaryDouble] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(0.0D)
|
def this() = this(0.0D)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryDouble(fromByteArray[Double](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryDouble(fromByteArray[Double](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryFloat(val value: Float) extends Serializable.SBinary[BinaryFloat] {
|
case class BinaryFloat(val value: Float) extends Serializable.SBinary[BinaryFloat] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(0.0F)
|
def this() = this(0.0F)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryFloat(fromByteArray[Float](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryFloat(fromByteArray[Float](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBoolean(val value: Boolean) extends Serializable.SBinary[BinaryBoolean] {
|
case class BinaryBoolean(val value: Boolean) extends Serializable.SBinary[BinaryBoolean] {
|
||||||
import sbinary.DefaultProtocol._
|
def this() = this(true)
|
||||||
def this() = this(true)
|
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBoolean(fromByteArray[Boolean](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBoolean(fromByteArray[Boolean](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryByte(val value: Byte) extends Serializable.SBinary[BinaryByte] {
|
case class BinaryByte(val value: Byte) extends Serializable.SBinary[BinaryByte] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(0x00)
|
def this() = this(0x00)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryByte(fromByteArray[Byte](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryByte(fromByteArray[Byte](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryChar(val value: Char) extends Serializable.SBinary[BinaryChar] {
|
case class BinaryChar(val value: Char) extends Serializable.SBinary[BinaryChar] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(' ')
|
def this() = this(' ')
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryChar(fromByteArray[Char](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryChar(fromByteArray[Char](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
|
|
@ -68,117 +62,98 @@ case class BinaryChar(val value: Char) extends Serializable.SBinary[BinaryChar]
|
||||||
|
|
||||||
// --- ARRAYS ---
|
// --- ARRAYS ---
|
||||||
case class BinaryStringArray(val value: Array[String]) extends Serializable.SBinary[BinaryStringArray] {
|
case class BinaryStringArray(val value: Array[String]) extends Serializable.SBinary[BinaryStringArray] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringArray(fromByteArray[Array[String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringArray(fromByteArray[Array[String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigIntArray(val value: Array[BigInt]) extends Serializable.SBinary[BinaryBigIntArray] {
|
case class BinaryBigIntArray(val value: Array[BigInt]) extends Serializable.SBinary[BinaryBigIntArray] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigIntArray(fromByteArray[Array[BigInt]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigIntArray(fromByteArray[Array[BigInt]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigDecimalArray(val value: Array[BigDecimal]) extends Serializable.SBinary[BinaryBigDecimalArray] {
|
case class BinaryBigDecimalArray(val value: Array[BigDecimal]) extends Serializable.SBinary[BinaryBigDecimalArray] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalArray(fromByteArray[Array[BigDecimal]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalArray(fromByteArray[Array[BigDecimal]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryIntArray(val value: Array[Int]) extends Serializable.SBinary[BinaryIntArray] {
|
case class BinaryIntArray(val value: Array[Int]) extends Serializable.SBinary[BinaryIntArray] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryIntArray(fromByteArray[Array[Int]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryIntArray(fromByteArray[Array[Int]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryLongArray(val value: Array[Long]) extends Serializable.SBinary[BinaryLongArray] {
|
case class BinaryLongArray(val value: Array[Long]) extends Serializable.SBinary[BinaryLongArray] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryLongArray(fromByteArray[Array[Long]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryLongArray(fromByteArray[Array[Long]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryDoubleArray(val value: Array[Double]) extends Serializable.SBinary[BinaryDoubleArray] {
|
case class BinaryDoubleArray(val value: Array[Double]) extends Serializable.SBinary[BinaryDoubleArray] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryDoubleArray(fromByteArray[Array[Double]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryDoubleArray(fromByteArray[Array[Double]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryFloatArray(val value: Array[Float]) extends Serializable.SBinary[BinaryFloatArray] {
|
case class BinaryFloatArray(val value: Array[Float]) extends Serializable.SBinary[BinaryFloatArray] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryFloatArray(fromByteArray[Array[Float]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryFloatArray(fromByteArray[Array[Float]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBooleanArray(val value: Array[Boolean]) extends Serializable.SBinary[BinaryBooleanArray] {
|
case class BinaryBooleanArray(val value: Array[Boolean]) extends Serializable.SBinary[BinaryBooleanArray] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBooleanArray(fromByteArray[Array[Boolean]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBooleanArray(fromByteArray[Array[Boolean]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryByteArray(val value: Array[Byte]) extends Serializable.SBinary[BinaryByteArray] {
|
case class BinaryByteArray(val value: Array[Byte]) extends Serializable.SBinary[BinaryByteArray] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryByteArray(fromByteArray[Array[Byte]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryByteArray(fromByteArray[Array[Byte]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryClassArray(val value: Array[Class[_]]) extends Serializable.SBinary[BinaryClassArray] {
|
/*case class BinaryClassArray(val value: Array[Class[_]]) extends Serializable.SBinary[BinaryClassArray] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryClassArray(fromByteArray[Array[Class[_]]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryClassArray(fromByteArray[Array[Class[_]]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}*/
|
||||||
|
|
||||||
// --- LISTS ---
|
// --- LISTS ---
|
||||||
case class BinaryStringList(val value: List[String]) extends Serializable.SBinary[BinaryStringList] {
|
case class BinaryStringList(val value: List[String]) extends Serializable.SBinary[BinaryStringList] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringList(fromByteArray[List[String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringList(fromByteArray[List[String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigIntList(val value: List[BigInt]) extends Serializable.SBinary[BinaryBigIntList] {
|
case class BinaryBigIntList(val value: List[BigInt]) extends Serializable.SBinary[BinaryBigIntList] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigIntList(fromByteArray[List[BigInt]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigIntList(fromByteArray[List[BigInt]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigDecimalList(val value: List[BigDecimal]) extends Serializable.SBinary[BinaryBigDecimalList] {
|
case class BinaryBigDecimalList(val value: List[BigDecimal]) extends Serializable.SBinary[BinaryBigDecimalList] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalList(fromByteArray[List[BigDecimal]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalList(fromByteArray[List[BigDecimal]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryLongList(val value: List[Long]) extends Serializable.SBinary[BinaryLongList] {
|
case class BinaryLongList(val value: List[Long]) extends Serializable.SBinary[BinaryLongList] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryLongList(fromByteArray[List[Long]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryLongList(fromByteArray[List[Long]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryIntList(val value: List[Int]) extends Serializable.SBinary[BinaryIntList] {
|
case class BinaryIntList(val value: List[Int]) extends Serializable.SBinary[BinaryIntList] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryIntList(fromByteArray[List[Int]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryIntList(fromByteArray[List[Int]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryDoubleList(val value: List[Double]) extends Serializable.SBinary[BinaryDoubleList] {
|
case class BinaryDoubleList(val value: List[Double]) extends Serializable.SBinary[BinaryDoubleList] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryDoubleList(fromByteArray[List[Double]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryDoubleList(fromByteArray[List[Double]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryFloatList(val value: List[Float]) extends Serializable.SBinary[BinaryFloatList] {
|
case class BinaryFloatList(val value: List[Float]) extends Serializable.SBinary[BinaryFloatList] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryFloatList(fromByteArray[List[Float]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryFloatList(fromByteArray[List[Float]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBooleanList(val value: List[Boolean]) extends Serializable.SBinary[BinaryBooleanList] {
|
case class BinaryBooleanList(val value: List[Boolean]) extends Serializable.SBinary[BinaryBooleanList] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBooleanList(fromByteArray[List[Boolean]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBooleanList(fromByteArray[List[Boolean]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryClassList(val value: List[Class[_]]) extends Serializable.SBinary[BinaryClassList] {
|
case class BinaryClassList(val value: List[Class[_]]) extends Serializable.SBinary[BinaryClassList] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryClassList(fromByteArray[List[Class[_]]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryClassList(fromByteArray[List[Class[_]]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
|
|
@ -186,103 +161,86 @@ case class BinaryClassList(val value: List[Class[_]]) extends Serializable.SBina
|
||||||
|
|
||||||
// --- TUPLES ---
|
// --- TUPLES ---
|
||||||
case class BinaryStringStringTuple(val value: Tuple2[String, String]) extends Serializable.SBinary[BinaryStringStringTuple] {
|
case class BinaryStringStringTuple(val value: Tuple2[String, String]) extends Serializable.SBinary[BinaryStringStringTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
def this() = this(null)
|
||||||
def this() = this(null)
|
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringStringTuple(fromByteArray[Tuple2[String, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringStringTuple(fromByteArray[Tuple2[String, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigIntBigIntTuple(val value: Tuple2[BigInt, BigInt]) extends Serializable.SBinary[BinaryBigIntBigIntTuple] {
|
case class BinaryBigIntBigIntTuple(val value: Tuple2[BigInt, BigInt]) extends Serializable.SBinary[BinaryBigIntBigIntTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
def this() = this(null)
|
||||||
def this() = this(null)
|
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigIntBigIntTuple(fromByteArray[Tuple2[BigInt, BigInt]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigIntBigIntTuple(fromByteArray[Tuple2[BigInt, BigInt]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigDecimalBigDecimalTuple(val value: Tuple2[BigDecimal, BigDecimal]) extends Serializable.SBinary[BinaryBigDecimalBigDecimalTuple] {
|
case class BinaryBigDecimalBigDecimalTuple(val value: Tuple2[BigDecimal, BigDecimal]) extends Serializable.SBinary[BinaryBigDecimalBigDecimalTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalBigDecimalTuple(fromByteArray[Tuple2[BigDecimal, BigDecimal]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalBigDecimalTuple(fromByteArray[Tuple2[BigDecimal, BigDecimal]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryLongLongTuple(val value: Tuple2[Long, Long]) extends Serializable.SBinary[BinaryLongLongTuple] {
|
case class BinaryLongLongTuple(val value: Tuple2[Long, Long]) extends Serializable.SBinary[BinaryLongLongTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryLongLongTuple(fromByteArray[Tuple2[Long, Long]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryLongLongTuple(fromByteArray[Tuple2[Long, Long]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryIntIntTuple(val value: Tuple2[Int, Int]) extends Serializable.SBinary[BinaryIntIntTuple] {
|
case class BinaryIntIntTuple(val value: Tuple2[Int, Int]) extends Serializable.SBinary[BinaryIntIntTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryIntIntTuple(fromByteArray[Tuple2[Int, Int]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryIntIntTuple(fromByteArray[Tuple2[Int, Int]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryDoubleDoubleTuple(val value: Tuple2[Double, Double]) extends Serializable.SBinary[BinaryDoubleDoubleTuple] {
|
case class BinaryDoubleDoubleTuple(val value: Tuple2[Double, Double]) extends Serializable.SBinary[BinaryDoubleDoubleTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryDoubleDoubleTuple(fromByteArray[Tuple2[Double, Double]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryDoubleDoubleTuple(fromByteArray[Tuple2[Double, Double]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryFloatFloatTuple(val value: Tuple2[Float, Float]) extends Serializable.SBinary[BinaryFloatFloatTuple] {
|
case class BinaryFloatFloatTuple(val value: Tuple2[Float, Float]) extends Serializable.SBinary[BinaryFloatFloatTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryFloatFloatTuple(fromByteArray[Tuple2[Float, Float]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryFloatFloatTuple(fromByteArray[Tuple2[Float, Float]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBooleanBooleanTuple(val value: Tuple2[Boolean, Boolean]) extends Serializable.SBinary[BinaryBooleanBooleanTuple] {
|
case class BinaryBooleanBooleanTuple(val value: Tuple2[Boolean, Boolean]) extends Serializable.SBinary[BinaryBooleanBooleanTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBooleanBooleanTuple(fromByteArray[Tuple2[Boolean, Boolean]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBooleanBooleanTuple(fromByteArray[Tuple2[Boolean, Boolean]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryClassClassTuple(val value: Tuple2[Class[_], Class[_]]) extends Serializable.SBinary[BinaryClassClassTuple] {
|
case class BinaryClassClassTuple(val value: Tuple2[Class[_], Class[_]]) extends Serializable.SBinary[BinaryClassClassTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryClassClassTuple(fromByteArray[Tuple2[Class[_], Class[_]]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryClassClassTuple(fromByteArray[Tuple2[Class[_], Class[_]]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryByteArrayByteArrayTuple(val value: Tuple2[Array[Byte], Array[Byte]]) extends Serializable.SBinary[BinaryByteArrayByteArrayTuple] {
|
case class BinaryByteArrayByteArrayTuple(val value: Tuple2[Array[Byte], Array[Byte]]) extends Serializable.SBinary[BinaryByteArrayByteArrayTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryByteArrayByteArrayTuple(fromByteArray[Tuple2[Array[Byte], Array[Byte]]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryByteArrayByteArrayTuple(fromByteArray[Tuple2[Array[Byte], Array[Byte]]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigIntStringTuple(val value: Tuple2[BigInt, String]) extends Serializable.SBinary[BinaryBigIntStringTuple] {
|
case class BinaryBigIntStringTuple(val value: Tuple2[BigInt, String]) extends Serializable.SBinary[BinaryBigIntStringTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigIntStringTuple(fromByteArray[Tuple2[BigInt, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigIntStringTuple(fromByteArray[Tuple2[BigInt, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigDecimalStringTuple(val value: Tuple2[BigDecimal, String]) extends Serializable.SBinary[BinaryBigDecimalStringTuple] {
|
case class BinaryBigDecimalStringTuple(val value: Tuple2[BigDecimal, String]) extends Serializable.SBinary[BinaryBigDecimalStringTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalStringTuple(fromByteArray[Tuple2[BigDecimal, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalStringTuple(fromByteArray[Tuple2[BigDecimal, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryLongStringTuple(val value: Tuple2[Long, String]) extends Serializable.SBinary[BinaryLongStringTuple] {
|
case class BinaryLongStringTuple(val value: Tuple2[Long, String]) extends Serializable.SBinary[BinaryLongStringTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryLongStringTuple(fromByteArray[Tuple2[Long, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryLongStringTuple(fromByteArray[Tuple2[Long, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryIntStringTuple(val value: Tuple2[Int, String]) extends Serializable.SBinary[BinaryIntStringTuple] {
|
case class BinaryIntStringTuple(val value: Tuple2[Int, String]) extends Serializable.SBinary[BinaryIntStringTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryIntStringTuple(fromByteArray[Tuple2[Int, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryIntStringTuple(fromByteArray[Tuple2[Int, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryDoubleStringTuple(val value: Tuple2[Double, String]) extends Serializable.SBinary[BinaryDoubleStringTuple] {
|
case class BinaryDoubleStringTuple(val value: Tuple2[Double, String]) extends Serializable.SBinary[BinaryDoubleStringTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryDoubleStringTuple(fromByteArray[Tuple2[Double, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryDoubleStringTuple(fromByteArray[Tuple2[Double, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryFloatStringTuple(val value: Tuple2[Float, String]) extends Serializable.SBinary[BinaryFloatStringTuple] {
|
case class BinaryFloatStringTuple(val value: Tuple2[Float, String]) extends Serializable.SBinary[BinaryFloatStringTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryFloatStringTuple(fromByteArray[Tuple2[Float, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryFloatStringTuple(fromByteArray[Tuple2[Float, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBooleanStringTuple(val value: Tuple2[Boolean, String]) extends Serializable.SBinary[BinaryBooleanStringTuple] {
|
case class BinaryBooleanStringTuple(val value: Tuple2[Boolean, String]) extends Serializable.SBinary[BinaryBooleanStringTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBooleanStringTuple(fromByteArray[Tuple2[Boolean, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBooleanStringTuple(fromByteArray[Tuple2[Boolean, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
|
|
@ -294,7 +252,6 @@ case class BinaryClassStringTuple(val value: Tuple2[Class[_], String]) extends S
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryStringByteArrayTuple(val value: Tuple2[String, Array[Byte]]) extends Serializable.SBinary[BinaryStringByteArrayTuple] {
|
case class BinaryStringByteArrayTuple(val value: Tuple2[String, Array[Byte]]) extends Serializable.SBinary[BinaryStringByteArrayTuple] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringByteArrayTuple(fromByteArray[Tuple2[String, Array[Byte]]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringByteArrayTuple(fromByteArray[Tuple2[String, Array[Byte]]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
|
|
@ -302,74 +259,62 @@ case class BinaryStringByteArrayTuple(val value: Tuple2[String, Array[Byte]]) ex
|
||||||
|
|
||||||
// --- MAPS ---
|
// --- MAPS ---
|
||||||
case class BinaryStringStringMap(val value: Map[String, String]) extends Serializable.SBinary[BinaryStringStringMap] {
|
case class BinaryStringStringMap(val value: Map[String, String]) extends Serializable.SBinary[BinaryStringStringMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringStringMap(fromByteArray[Map[String, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringStringMap(fromByteArray[Map[String, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigIntStringMap(val value: Map[BigInt, String]) extends Serializable.SBinary[BinaryBigIntStringMap] {
|
case class BinaryBigIntStringMap(val value: Map[BigInt, String]) extends Serializable.SBinary[BinaryBigIntStringMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigIntStringMap(fromByteArray[Map[BigInt, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigIntStringMap(fromByteArray[Map[BigInt, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryBigDecimalStringMap(val value: Map[BigDecimal, String]) extends Serializable.SBinary[BinaryBigDecimalStringMap] {
|
case class BinaryBigDecimalStringMap(val value: Map[BigDecimal, String]) extends Serializable.SBinary[BinaryBigDecimalStringMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalStringMap(fromByteArray[Map[BigDecimal, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalStringMap(fromByteArray[Map[BigDecimal, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryLongStringMap(val value: Map[Long, String]) extends Serializable.SBinary[BinaryLongStringMap] {
|
case class BinaryLongStringMap(val value: Map[Long, String]) extends Serializable.SBinary[BinaryLongStringMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryLongStringMap(fromByteArray[Map[Long, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryLongStringMap(fromByteArray[Map[Long, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryIntStringMap(val value: Map[Int, String]) extends Serializable.SBinary[BinaryIntStringMap] {
|
case class BinaryIntStringMap(val value: Map[Int, String]) extends Serializable.SBinary[BinaryIntStringMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryIntStringMap(fromByteArray[Map[Int, String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryIntStringMap(fromByteArray[Map[Int, String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryClassStringMap(val value: Map[Class[_], String]) extends Serializable.SBinary[BinaryClassStringMap] {
|
case class BinaryClassStringMap(val value: Map[Class[_], String]) extends Serializable.SBinary[BinaryClassStringMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryClassStringMap(fromByteArray[Map[Class[_], String]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryClassStringMap(fromByteArray[Map[Class[_], String]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryStringBigIntMap(val value: Map[String, BigInt]) extends Serializable.SBinary[BinaryStringBigIntMap] {
|
case class BinaryStringBigIntMap(val value: Map[String, BigInt]) extends Serializable.SBinary[BinaryStringBigIntMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringBigIntMap(fromByteArray[Map[String, BigInt]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringBigIntMap(fromByteArray[Map[String, BigInt]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryStringBigDecimalMap(val value: Map[String, BigDecimal]) extends Serializable.SBinary[BinaryStringBigDecimalMap] {
|
case class BinaryStringBigDecimalMap(val value: Map[String, BigDecimal]) extends Serializable.SBinary[BinaryStringBigDecimalMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringBigDecimalMap(fromByteArray[Map[String, BigDecimal]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringBigDecimalMap(fromByteArray[Map[String, BigDecimal]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryStringLongMap(val value: Map[String, Long]) extends Serializable.SBinary[BinaryStringLongMap] {
|
case class BinaryStringLongMap(val value: Map[String, Long]) extends Serializable.SBinary[BinaryStringLongMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringLongMap(fromByteArray[Map[String, Long]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringLongMap(fromByteArray[Map[String, Long]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryStringIntMap(val value: Map[String, Int]) extends Serializable.SBinary[BinaryStringIntMap] {
|
case class BinaryStringIntMap(val value: Map[String, Int]) extends Serializable.SBinary[BinaryStringIntMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringIntMap(fromByteArray[Map[String, Int]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringIntMap(fromByteArray[Map[String, Int]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryStringClassMap(val value: Map[String, Class[_]]) extends Serializable.SBinary[BinaryStringClassMap] {
|
case class BinaryStringClassMap(val value: Map[String, Class[_]]) extends Serializable.SBinary[BinaryStringClassMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringClassMap(fromByteArray[Map[String, Class[_]]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringClassMap(fromByteArray[Map[String, Class[_]]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
case class BinaryStringByteArrayMap(val value: Map[String, Array[Byte]]) extends Serializable.SBinary[BinaryStringByteArrayMap] {
|
case class BinaryStringByteArrayMap(val value: Map[String, Array[Byte]]) extends Serializable.SBinary[BinaryStringByteArrayMap] {
|
||||||
import sbinary.DefaultProtocol._
|
|
||||||
def this() = this(null)
|
def this() = this(null)
|
||||||
def fromBytes(bytes: Array[Byte]) = BinaryStringByteArrayMap(fromByteArray[Map[String, Array[Byte]]](bytes))
|
def fromBytes(bytes: Array[Byte]) = BinaryStringByteArrayMap(fromByteArray[Map[String, Array[Byte]]](bytes))
|
||||||
def toBytes: Array[Byte] = toByteArray(value)
|
def toBytes: Array[Byte] = toByteArray(value)
|
||||||
}
|
}
|
||||||
|
|
@ -8,8 +8,7 @@ import org.codehaus.jackson.map.ObjectMapper
|
||||||
|
|
||||||
import com.google.protobuf.Message
|
import com.google.protobuf.Message
|
||||||
|
|
||||||
import scala.reflect.Manifest
|
import reflect.Manifest
|
||||||
|
|
||||||
import sbinary.DefaultProtocol
|
import sbinary.DefaultProtocol
|
||||||
|
|
||||||
import java.io.{StringWriter, ByteArrayOutputStream, ObjectOutputStream}
|
import java.io.{StringWriter, ByteArrayOutputStream, ObjectOutputStream}
|
||||||
|
|
|
||||||
|
|
@ -164,7 +164,9 @@ object Serializer {
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
object SBinary extends SBinary
|
object SBinary extends SBinary
|
||||||
trait SBinary {
|
class SBinary {
|
||||||
|
import sbinary._
|
||||||
|
import sbinary.Operations._
|
||||||
import sbinary.DefaultProtocol._
|
import sbinary.DefaultProtocol._
|
||||||
|
|
||||||
def deepClone[T <: AnyRef](obj: T)(implicit w : Writes[T], r : Reads[T]): T = in[T](out[T](obj), None)
|
def deepClone[T <: AnyRef](obj: T)(implicit w : Writes[T], r : Reads[T]): T = in[T](out[T](obj), None)
|
||||||
|
|
|
||||||
|
|
@ -27,7 +27,7 @@ import se.scalablesolutions.akka.dispatch.CompletableFuture
|
||||||
|
|
||||||
def thread(body: => Unit) = {
|
def thread(body: => Unit) = {
|
||||||
val thread = new IsolatedEventBasedThread(body).start
|
val thread = new IsolatedEventBasedThread(body).start
|
||||||
thread send Start
|
thread ! Start
|
||||||
thread
|
thread
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -93,9 +93,9 @@ import se.scalablesolutions.akka.dispatch.CompletableFuture
|
||||||
|
|
||||||
private[this] val in = new In(this)
|
private[this] val in = new In(this)
|
||||||
|
|
||||||
def <<(ref: DataFlowVariable[T]) = in send Set(ref())
|
def <<(ref: DataFlowVariable[T]) = in ! Set(ref())
|
||||||
|
|
||||||
def <<(value: T) = in send Set(value)
|
def <<(value: T) = in ! Set(value)
|
||||||
|
|
||||||
def apply(): T = {
|
def apply(): T = {
|
||||||
val ref = value.get
|
val ref = value.get
|
||||||
|
|
@ -104,13 +104,13 @@ import se.scalablesolutions.akka.dispatch.CompletableFuture
|
||||||
val out = new Out(this)
|
val out = new Out(this)
|
||||||
blockedReaders.offer(out)
|
blockedReaders.offer(out)
|
||||||
val result = out !! Get
|
val result = out !! Get
|
||||||
out send Exit
|
out ! Exit
|
||||||
result.getOrElse(throw new DataFlowVariableException(
|
result.getOrElse(throw new DataFlowVariableException(
|
||||||
"Timed out (after " + TIME_OUT + " milliseconds) while waiting for result"))
|
"Timed out (after " + TIME_OUT + " milliseconds) while waiting for result"))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def shutdown = in send Exit
|
def shutdown = in ! Exit
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -144,7 +144,7 @@ import se.scalablesolutions.akka.dispatch.CompletableFuture
|
||||||
"Access by index other than '0' is not supported by DataFlowStream")
|
"Access by index other than '0' is not supported by DataFlowStream")
|
||||||
}
|
}
|
||||||
|
|
||||||
override def elements: Iterator[T] = new Iterator[T] {
|
def iterator: Iterator[T] = new Iterator[T] {
|
||||||
private val iter = queue.iterator
|
private val iter = queue.iterator
|
||||||
def hasNext: Boolean = iter.hasNext
|
def hasNext: Boolean = iter.hasNext
|
||||||
def next: T = { val ref = iter.next; ref() }
|
def next: T = { val ref = iter.next; ref() }
|
||||||
|
|
|
||||||
|
|
@ -46,7 +46,7 @@ trait PersistentDataStructure
|
||||||
*/
|
*/
|
||||||
@serializable
|
@serializable
|
||||||
final class HashTrie[K, +V] private (root: Node[K, V]) extends Map[K, V] with PersistentDataStructure {
|
final class HashTrie[K, +V] private (root: Node[K, V]) extends Map[K, V] with PersistentDataStructure {
|
||||||
lazy val size = root.size
|
override lazy val size = root.size
|
||||||
|
|
||||||
def this() = this(new EmptyNode[K])
|
def this() = this(new EmptyNode[K])
|
||||||
|
|
||||||
|
|
@ -56,11 +56,11 @@ final class HashTrie[K, +V] private (root: Node[K, V]) extends Map[K, V] with Pe
|
||||||
case (k, v) => update(k, v)
|
case (k, v) => update(k, v)
|
||||||
}
|
}
|
||||||
|
|
||||||
def update[A >: V](key: K, value: A) = new HashTrie(root(0, key, key.hashCode) = value)
|
override def update[A >: V](key: K, value: A) = new HashTrie(root(0, key, key.hashCode) = value)
|
||||||
|
|
||||||
def -(key: K) = new HashTrie(root.remove(key, key.hashCode))
|
def -(key: K) = new HashTrie(root.remove(key, key.hashCode))
|
||||||
|
|
||||||
def elements = root.elements
|
def iterator = root.elements
|
||||||
|
|
||||||
def empty[A]: HashTrie[K, A] = new HashTrie(new EmptyNode[K])
|
def empty[A]: HashTrie[K, A] = new HashTrie(new EmptyNode[K])
|
||||||
|
|
||||||
|
|
@ -68,7 +68,7 @@ final class HashTrie[K, +V] private (root: Node[K, V]) extends Map[K, V] with Pe
|
||||||
}
|
}
|
||||||
|
|
||||||
object HashTrie {
|
object HashTrie {
|
||||||
def apply[K, V](pairs: (K, V)*) = pairs.foldLeft(new HashTrie[K, V]) { _ + _ }
|
def apply[K, V](pairs: (K, V)*) = pairs.foldLeft((new HashTrie[K, V]).asInstanceOf[Map[K,V]]) { _ + _ }
|
||||||
|
|
||||||
def unapplySeq[K, V](map: HashTrie[K, V]) = map.toSeq
|
def unapplySeq[K, V](map: HashTrie[K, V]) = map.toSeq
|
||||||
}
|
}
|
||||||
|
|
@ -152,7 +152,7 @@ private[stm] class CollisionNode[K, +V](val hash: Int, bucket: List[(K, V)]) ext
|
||||||
} yield v
|
} yield v
|
||||||
}
|
}
|
||||||
|
|
||||||
def update[A >: V](shift: Int, key: K, hash: Int, value: A): Node[K, A] = {
|
override def update[A >: V](shift: Int, key: K, hash: Int, value: A): Node[K, A] = {
|
||||||
if (this.hash == hash) {
|
if (this.hash == hash) {
|
||||||
var found = false
|
var found = false
|
||||||
|
|
||||||
|
|
@ -169,7 +169,7 @@ private[stm] class CollisionNode[K, +V](val hash: Int, bucket: List[(K, V)]) ext
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def remove(key: K, hash: Int) = {
|
override def remove(key: K, hash: Int) = {
|
||||||
val newBucket = bucket filter { case (k, _) => k != key }
|
val newBucket = bucket filter { case (k, _) => k != key }
|
||||||
|
|
||||||
if (newBucket.length == bucket.length) this else {
|
if (newBucket.length == bucket.length) this else {
|
||||||
|
|
@ -180,7 +180,9 @@ private[stm] class CollisionNode[K, +V](val hash: Int, bucket: List[(K, V)]) ext
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def elements = bucket.elements
|
def iterator = bucket.iterator
|
||||||
|
|
||||||
|
def elements = bucket.iterator
|
||||||
|
|
||||||
override def toString = "CollisionNode(" + bucket.toString + ")"
|
override def toString = "CollisionNode(" + bucket.toString + ")"
|
||||||
}
|
}
|
||||||
|
|
@ -202,7 +204,7 @@ private[stm] class BitmappedNode[K, +V](shift: Int)(table: Array[Node[K, V]], bi
|
||||||
if ((bits & mask) == mask) table(i)(key, hash) else None
|
if ((bits & mask) == mask) table(i)(key, hash) else None
|
||||||
}
|
}
|
||||||
|
|
||||||
def update[A >: V](levelShift: Int, key: K, hash: Int, value: A): Node[K, A] = {
|
override def update[A >: V](levelShift: Int, key: K, hash: Int, value: A): Node[K, A] = {
|
||||||
val i = (hash >>> shift) & 0x01f
|
val i = (hash >>> shift) & 0x01f
|
||||||
val mask = 1 << i
|
val mask = 1 << i
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -103,36 +103,29 @@ object Transaction extends TransactionManagement with Logging {
|
||||||
/**
|
/**
|
||||||
* See ScalaDoc on Transaction class.
|
* See ScalaDoc on Transaction class.
|
||||||
*/
|
*/
|
||||||
def map[T](f: => T)(implicit transactionFamilyName: String): T =
|
def map[T](f: => T): T = atomic {f}
|
||||||
atomic {f}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* See ScalaDoc on Transaction class.
|
* See ScalaDoc on Transaction class.
|
||||||
*/
|
*/
|
||||||
def flatMap[T](f: => T)(implicit transactionFamilyName: String): T =
|
def flatMap[T](f: => T): T = atomic {f}
|
||||||
atomic {f}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* See ScalaDoc on Transaction class.
|
* See ScalaDoc on Transaction class.
|
||||||
*/
|
*/
|
||||||
def foreach(f: => Unit)(implicit transactionFamilyName: String): Unit =
|
def foreach(f: => Unit): Unit = atomic {f}
|
||||||
atomic {f}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* See ScalaDoc on Transaction class.
|
* See ScalaDoc on Transaction class.
|
||||||
*/
|
*/
|
||||||
def atomic[T](body: => T)(implicit transactionFamilyName: String): T = {
|
def atomic[T](body: => T): T = {
|
||||||
// FIXME use Transaction Builder and set the transactionFamilyName
|
|
||||||
// defaultTxBuilder.setFamilyName(transactionFamilyName)
|
|
||||||
// new TransactionTemplate[T](defaultTxBuilder.build) {
|
|
||||||
var isTopLevelTransaction = true
|
var isTopLevelTransaction = true
|
||||||
new TransactionTemplate[T]() {
|
new TransactionTemplate[T]() {
|
||||||
def execute(mtx: MultiverseTransaction): T = {
|
def execute(mtx: MultiverseTransaction): T = {
|
||||||
val result = body
|
val result = body
|
||||||
|
|
||||||
val txSet = getTransactionSetInScope
|
val txSet = getTransactionSetInScope
|
||||||
log.trace("Committing transaction [%s]\n\twith family name [%s]\n\tby joining transaction set [%s]",
|
log.trace("Committing transaction [%s]\n\tby joining transaction set [%s]", mtx, txSet)
|
||||||
mtx, transactionFamilyName, txSet)
|
|
||||||
txSet.joinCommit(mtx)
|
txSet.joinCommit(mtx)
|
||||||
|
|
||||||
// FIXME tryJoinCommit(mtx, TransactionManagement.TRANSACTION_TIMEOUT, TimeUnit.MILLISECONDS)
|
// FIXME tryJoinCommit(mtx, TransactionManagement.TRANSACTION_TIMEOUT, TimeUnit.MILLISECONDS)
|
||||||
|
|
@ -208,7 +201,7 @@ object Transaction extends TransactionManagement with Logging {
|
||||||
def commit = synchronized {
|
def commit = synchronized {
|
||||||
log.trace("Committing transaction %s", toString)
|
log.trace("Committing transaction %s", toString)
|
||||||
atomic0 {
|
atomic0 {
|
||||||
persistentStateMap.values.foreach(_.commit)
|
persistentStateMap.valuesIterator.foreach(_.commit)
|
||||||
}
|
}
|
||||||
status = TransactionStatus.Completed
|
status = TransactionStatus.Completed
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -152,7 +152,7 @@ class TransactionalRef[T] extends Transactional {
|
||||||
|
|
||||||
def elements: Iterator[T] = {
|
def elements: Iterator[T] = {
|
||||||
ensureIsInTransaction
|
ensureIsInTransaction
|
||||||
if (isEmpty) Iterator.empty else Iterator.fromValues(ref.get)
|
if (isEmpty) Iterator.empty else Iterator(ref.get)
|
||||||
}
|
}
|
||||||
|
|
||||||
def toList: List[T] = {
|
def toList: List[T] = {
|
||||||
|
|
@ -191,11 +191,24 @@ class TransactionalMap[K, V] extends Transactional with scala.collection.mutable
|
||||||
|
|
||||||
ref.swap(new HashTrie[K, V])
|
ref.swap(new HashTrie[K, V])
|
||||||
|
|
||||||
def -=(key: K) = remove(key)
|
def -=(key: K) = {
|
||||||
|
remove(key)
|
||||||
|
this
|
||||||
|
}
|
||||||
|
|
||||||
def +=(key: K, value: V) = put(key, value)
|
def +=(key: K, value: V) = put(key, value)
|
||||||
|
|
||||||
|
def +=(kv: (K, V)) = {
|
||||||
|
put(kv._1,kv._2)
|
||||||
|
this
|
||||||
|
}
|
||||||
|
|
||||||
def remove(key: K) = ref.swap(ref.get.get - key)
|
override def remove(key: K) = {
|
||||||
|
val map = ref.get.get
|
||||||
|
val oldValue = map.get(key)
|
||||||
|
ref.swap(ref.get.get - key)
|
||||||
|
oldValue
|
||||||
|
}
|
||||||
|
|
||||||
def get(key: K): Option[V] = ref.get.get.get(key)
|
def get(key: K): Option[V] = ref.get.get.get(key)
|
||||||
|
|
||||||
|
|
@ -206,19 +219,21 @@ class TransactionalMap[K, V] extends Transactional with scala.collection.mutable
|
||||||
oldValue
|
oldValue
|
||||||
}
|
}
|
||||||
|
|
||||||
def update(key: K, value: V) = {
|
override def update(key: K, value: V) = {
|
||||||
val map = ref.get.get
|
val map = ref.get.get
|
||||||
val oldValue = map.get(key)
|
val oldValue = map.get(key)
|
||||||
ref.swap(map.update(key, value))
|
ref.swap(map.update(key, value))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def iterator = ref.get.get.iterator
|
||||||
|
|
||||||
def elements: Iterator[(K, V)] = ref.get.get.elements
|
override def elements: Iterator[(K, V)] = ref.get.get.iterator
|
||||||
|
|
||||||
override def contains(key: K): Boolean = ref.get.get.contains(key)
|
override def contains(key: K): Boolean = ref.get.get.contains(key)
|
||||||
|
|
||||||
override def clear = ref.swap(new HashTrie[K, V])
|
override def clear = ref.swap(new HashTrie[K, V])
|
||||||
|
|
||||||
def size: Int = ref.get.get.size
|
override def size: Int = ref.get.get.size
|
||||||
|
|
||||||
override def hashCode: Int = System.identityHashCode(this);
|
override def hashCode: Int = System.identityHashCode(this);
|
||||||
|
|
||||||
|
|
@ -238,7 +253,7 @@ object TransactionalVector {
|
||||||
*
|
*
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
class TransactionalVector[T] extends Transactional with RandomAccessSeq[T] {
|
class TransactionalVector[T] extends Transactional with IndexedSeq[T] {
|
||||||
val uuid = UUID.newUuid.toString
|
val uuid = UUID.newUuid.toString
|
||||||
|
|
||||||
private[this] val ref = TransactionalRef[Vector[T]]
|
private[this] val ref = TransactionalRef[Vector[T]]
|
||||||
|
|
|
||||||
|
|
@ -44,7 +44,7 @@ import Vector._
|
||||||
*/
|
*/
|
||||||
@serializable
|
@serializable
|
||||||
class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail: Array[AnyRef])
|
class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail: Array[AnyRef])
|
||||||
extends RandomAccessSeq[T] with PersistentDataStructure { outer =>
|
extends IndexedSeq[T] with PersistentDataStructure { outer =>
|
||||||
private val tailOff = length - tail.length
|
private val tailOff = length - tail.length
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
@ -104,7 +104,7 @@ class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail
|
||||||
ret
|
ret
|
||||||
}
|
}
|
||||||
|
|
||||||
override def ++[A >: T](other: Iterable[A]) = other.foldLeft(this:Vector[A]) { _ + _ }
|
def ++[A >: T](other: Iterable[A]) = other.foldLeft(this:Vector[A]) { _ + _ }
|
||||||
|
|
||||||
def +[A >: T](obj: A): Vector[A] = {
|
def +[A >: T](obj: A): Vector[A] = {
|
||||||
if (tail.length < 32) {
|
if (tail.length < 32) {
|
||||||
|
|
@ -224,7 +224,7 @@ class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail
|
||||||
back
|
back
|
||||||
}
|
}
|
||||||
|
|
||||||
override def flatMap[A](f: (T)=>Iterable[A]): Vector[A] = {
|
def flatMap[A](f: (T)=>Iterable[A]): Vector[A] = {
|
||||||
var back = new Vector[A]
|
var back = new Vector[A]
|
||||||
var i = 0
|
var i = 0
|
||||||
|
|
||||||
|
|
@ -236,7 +236,7 @@ class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail
|
||||||
back
|
back
|
||||||
}
|
}
|
||||||
|
|
||||||
override def map[A](f: (T)=>A): Vector[A] = {
|
def map[A](f: (T)=>A): Vector[A] = {
|
||||||
var back = new Vector[A]
|
var back = new Vector[A]
|
||||||
var i = 0
|
var i = 0
|
||||||
|
|
||||||
|
|
@ -254,7 +254,7 @@ class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail
|
||||||
override def apply(i: Int) = outer.apply(length - i - 1)
|
override def apply(i: Int) = outer.apply(length - i - 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def subseq(from: Int, end: Int) = subVector(from, end)
|
def subseq(from: Int, end: Int) = subVector(from, end)
|
||||||
|
|
||||||
def subVector(from: Int, end: Int): Vector[T] = {
|
def subVector(from: Int, end: Int): Vector[T] = {
|
||||||
if (from < 0) {
|
if (from < 0) {
|
||||||
|
|
|
||||||
|
|
@ -16,15 +16,12 @@ class AgentTest extends junit.framework.TestCase
|
||||||
with Suite with MustMatchers
|
with Suite with MustMatchers
|
||||||
with ActorTestUtil with Logging {
|
with ActorTestUtil with Logging {
|
||||||
|
|
||||||
implicit val txFamilyName = "test"
|
|
||||||
|
|
||||||
@Test def testSendFun = verify(new TestActor {
|
@Test def testSendFun = verify(new TestActor {
|
||||||
def test = {
|
def test = {
|
||||||
val agent = Agent(5)
|
val agent = Agent(5)
|
||||||
handle(agent) {
|
handle(agent) {
|
||||||
agent update (_ + 1)
|
agent send (_ + 1)
|
||||||
agent update (_ * 2)
|
agent send (_ * 2)
|
||||||
|
|
||||||
val result = agent()
|
val result = agent()
|
||||||
result must be(12)
|
result must be(12)
|
||||||
}
|
}
|
||||||
|
|
@ -35,21 +32,34 @@ with ActorTestUtil with Logging {
|
||||||
def test = {
|
def test = {
|
||||||
val agent = Agent(5)
|
val agent = Agent(5)
|
||||||
handle(agent) {
|
handle(agent) {
|
||||||
agent update 6
|
agent send 6
|
||||||
val result = agent()
|
val result = agent()
|
||||||
result must be(6)
|
result must be(6)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
})
|
})
|
||||||
|
|
||||||
@Test def testOneAgentUpdateWithinEnlosingTransactionSuccess = {
|
@Test def testSendProc = verify(new TestActor {
|
||||||
|
def test = {
|
||||||
|
val agent = Agent(5)
|
||||||
|
var result = 0
|
||||||
|
handle(agent) {
|
||||||
|
agent sendProc (result += _)
|
||||||
|
agent sendProc (result += _)
|
||||||
|
Thread.sleep(1000)
|
||||||
|
result must be(10)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
|
@Test def testOneAgentsendWithinEnlosingTransactionSuccess = {
|
||||||
case object Go
|
case object Go
|
||||||
val agent = Agent(5)
|
val agent = Agent(5)
|
||||||
val tx = transactor {
|
val tx = transactor {
|
||||||
case Go => agent update (_ + 1)
|
case Go => agent send (_ + 1)
|
||||||
}
|
}
|
||||||
tx send Go
|
tx ! Go
|
||||||
Thread.sleep(5000)
|
Thread.sleep(1000)
|
||||||
val result = agent()
|
val result = agent()
|
||||||
result must be(6)
|
result must be(6)
|
||||||
agent.close
|
agent.close
|
||||||
|
|
@ -63,16 +73,53 @@ with ActorTestUtil with Logging {
|
||||||
val agent = Agent(5)
|
val agent = Agent(5)
|
||||||
val tx = transactor {
|
val tx = transactor {
|
||||||
case Go =>
|
case Go =>
|
||||||
agent update (_ * 2)
|
agent send (_ * 2)
|
||||||
try { agent() }
|
try { agent() }
|
||||||
catch {
|
catch {
|
||||||
case _ => latch.countDown
|
case _ => latch.countDown
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
tx send Go
|
tx ! Go
|
||||||
latch.await // FIXME should await with timeout and fail if timeout
|
latch.await // FIXME should await with timeout and fail if timeout
|
||||||
agent.close
|
agent.close
|
||||||
tx.stop
|
tx.stop
|
||||||
assert(true)
|
assert(true)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test def testAgentForeach = verify(new TestActor {
|
||||||
|
def test = {
|
||||||
|
val agent1 = Agent(3)
|
||||||
|
var result = 0
|
||||||
|
for (first <- agent1) {
|
||||||
|
result = first + 1
|
||||||
|
}
|
||||||
|
result must be(4)
|
||||||
|
agent1.close
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
|
@Test def testAgentMap = verify(new TestActor {
|
||||||
|
def test = {
|
||||||
|
val agent1 = Agent(3)
|
||||||
|
val result = for (first <- agent1) yield first + 1
|
||||||
|
result() must be(4)
|
||||||
|
result.close
|
||||||
|
agent1.close
|
||||||
|
}
|
||||||
|
})
|
||||||
|
|
||||||
|
@Test def testAgentFlatMap = verify(new TestActor {
|
||||||
|
def test = {
|
||||||
|
val agent1 = Agent(3)
|
||||||
|
val agent2 = Agent(5)
|
||||||
|
val result = for {
|
||||||
|
first <- agent1
|
||||||
|
second <- agent2
|
||||||
|
} yield second + first
|
||||||
|
result() must be(8)
|
||||||
|
result.close
|
||||||
|
agent1.close
|
||||||
|
agent2.close
|
||||||
|
}
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -49,7 +49,7 @@ class RemoteActorSpecActorAsyncSender extends Actor {
|
||||||
class ClientInitiatedRemoteActorTest extends JUnitSuite {
|
class ClientInitiatedRemoteActorTest extends JUnitSuite {
|
||||||
import Actor.Sender.Self
|
import Actor.Sender.Self
|
||||||
|
|
||||||
akka.config.Config.config
|
se.scalablesolutions.akka.config.Config.config
|
||||||
|
|
||||||
val HOSTNAME = "localhost"
|
val HOSTNAME = "localhost"
|
||||||
val PORT1 = 9990
|
val PORT1 = 9990
|
||||||
|
|
|
||||||
|
|
@ -5,6 +5,9 @@
|
||||||
package se.scalablesolutions.akka
|
package se.scalablesolutions.akka
|
||||||
|
|
||||||
import se.scalablesolutions.akka.serialization.Serializable
|
import se.scalablesolutions.akka.serialization.Serializable
|
||||||
|
import sbinary._
|
||||||
|
import sbinary.Operations._
|
||||||
|
import sbinary.DefaultProtocol._
|
||||||
|
|
||||||
sealed abstract class TestMessage
|
sealed abstract class TestMessage
|
||||||
case object Ping extends TestMessage
|
case object Ping extends TestMessage
|
||||||
|
|
|
||||||
|
|
@ -30,7 +30,7 @@ class PerformanceTest extends JUnitSuite {
|
||||||
case object BLUE extends Colour
|
case object BLUE extends Colour
|
||||||
case object FADED extends Colour
|
case object FADED extends Colour
|
||||||
|
|
||||||
val colours = Array(BLUE, RED, YELLOW)
|
val colours = Array[Colour](BLUE, RED, YELLOW)
|
||||||
|
|
||||||
case class Meet(from: Actor, colour: Colour)
|
case class Meet(from: Actor, colour: Colour)
|
||||||
case class Change(colour: Colour)
|
case class Change(colour: Colour)
|
||||||
|
|
@ -159,7 +159,7 @@ class PerformanceTest extends JUnitSuite {
|
||||||
case object BLUE extends Colour
|
case object BLUE extends Colour
|
||||||
case object FADED extends Colour
|
case object FADED extends Colour
|
||||||
|
|
||||||
val colours = Array(BLUE, RED, YELLOW)
|
val colours = Array[Colour](BLUE, RED, YELLOW)
|
||||||
|
|
||||||
case class Meet(colour: Colour)
|
case class Meet(colour: Colour)
|
||||||
case class Change(colour: Colour)
|
case class Change(colour: Colour)
|
||||||
|
|
|
||||||
|
|
@ -18,7 +18,6 @@ object Log {
|
||||||
var oneWayLog: String = ""
|
var oneWayLog: String = ""
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@serializable class RemotePingPong1Actor extends Actor {
|
@serializable class RemotePingPong1Actor extends Actor {
|
||||||
dispatcher = Dispatchers.newThreadBasedDispatcher(this)
|
dispatcher = Dispatchers.newThreadBasedDispatcher(this)
|
||||||
def receive = {
|
def receive = {
|
||||||
|
|
@ -74,7 +73,8 @@ object Log {
|
||||||
class RemoteSupervisorTest extends JUnitSuite {
|
class RemoteSupervisorTest extends JUnitSuite {
|
||||||
import Actor.Sender.Self
|
import Actor.Sender.Self
|
||||||
|
|
||||||
akka.config.Config.config
|
se.scalablesolutions.akka.config.Config.config
|
||||||
|
|
||||||
new Thread(new Runnable() {
|
new Thread(new Runnable() {
|
||||||
def run = {
|
def run = {
|
||||||
RemoteNode.start
|
RemoteNode.start
|
||||||
|
|
|
||||||
|
|
@ -60,7 +60,7 @@ class ServerInitiatedRemoteActorTest extends JUnitSuite {
|
||||||
import ServerInitiatedRemoteActorTest._
|
import ServerInitiatedRemoteActorTest._
|
||||||
|
|
||||||
import Actor.Sender.Self
|
import Actor.Sender.Self
|
||||||
akka.config.Config.config
|
se.scalablesolutions.akka.config.Config.config
|
||||||
|
|
||||||
private val unit = TimeUnit.MILLISECONDS
|
private val unit = TimeUnit.MILLISECONDS
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -13,7 +13,7 @@ object ActorShutdownRunner {
|
||||||
|
|
||||||
val myActor = new MyActor
|
val myActor = new MyActor
|
||||||
myActor.start
|
myActor.start
|
||||||
myActor.send("test")
|
myActor ! "test"
|
||||||
myActor.stop
|
myActor.stop
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -34,4 +34,4 @@ object RemoteServerAndClusterShutdownRunner {
|
||||||
s2.shutdown
|
s2.shutdown
|
||||||
s3.shutdown
|
s3.shutdown
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -27,4 +27,9 @@ public class Foo extends se.scalablesolutions.akka.serialization.Serializable.Ja
|
||||||
if (true) throw new RuntimeException("expected");
|
if (true) throw new RuntimeException("expected");
|
||||||
return "test";
|
return "test";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int $tag() throws java.rmi.RemoteException
|
||||||
|
{
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -49,7 +49,7 @@ trait Dispatcher { self: Actor =>
|
||||||
protected def dispatch: PartialFunction[Any, Unit] = {
|
protected def dispatch: PartialFunction[Any, Unit] = {
|
||||||
case a if routes.isDefinedAt(a) =>
|
case a if routes.isDefinedAt(a) =>
|
||||||
if (self.sender.isDefined) routes(a) forward transform(a)
|
if (self.sender.isDefined) routes(a) forward transform(a)
|
||||||
else routes(a) send transform(a)
|
else routes(a) ! transform(a)
|
||||||
}
|
}
|
||||||
|
|
||||||
def receive = dispatch
|
def receive = dispatch
|
||||||
|
|
|
||||||
|
|
@ -24,8 +24,7 @@ import org.apache.thrift.protocol._
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
trait CassandraSession extends Closeable with Flushable {
|
trait CassandraSession extends Closeable with Flushable {
|
||||||
import scala.collection.jcl.Conversions._
|
import scala.collection.JavaConversions._
|
||||||
import org.scala_tools.javautils.Imports._
|
|
||||||
import java.util.{Map => JMap, List => JList}
|
import java.util.{Map => JMap, List => JList}
|
||||||
|
|
||||||
protected val client: Cassandra.Client
|
protected val client: Cassandra.Client
|
||||||
|
|
@ -92,7 +91,7 @@ trait CassandraSession extends Closeable with Flushable {
|
||||||
|
|
||||||
def ++|(key: String, batch: Map[String, List[ColumnOrSuperColumn]], consistencyLevel: Int): Unit = {
|
def ++|(key: String, batch: Map[String, List[ColumnOrSuperColumn]], consistencyLevel: Int): Unit = {
|
||||||
val jmap = new java.util.HashMap[String, JList[ColumnOrSuperColumn]]
|
val jmap = new java.util.HashMap[String, JList[ColumnOrSuperColumn]]
|
||||||
for (entry <- batch; (key, value) = entry) jmap.put(key, value.asJava)
|
for (entry <- batch; (key, value) = entry) jmap.put(key, new java.util.ArrayList(value))
|
||||||
client.batch_insert(keyspace, key, jmap, consistencyLevel)
|
client.batch_insert(keyspace, key, jmap, consistencyLevel)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -132,7 +131,6 @@ trait CassandraSession extends Closeable with Flushable {
|
||||||
|
|
||||||
def insert(key: String, colPath: ColumnPath, value: Array[Byte], timestamp: Long, consistencyLevel: Int) = ++|(key, colPath, value, timestamp, consistencyLevel)
|
def insert(key: String, colPath: ColumnPath, value: Array[Byte], timestamp: Long, consistencyLevel: Int) = ++|(key, colPath, value, timestamp, consistencyLevel)
|
||||||
|
|
||||||
|
|
||||||
def insert(key: String, batch: Map[String, List[ColumnOrSuperColumn]]): Unit = ++|(key, batch)
|
def insert(key: String, batch: Map[String, List[ColumnOrSuperColumn]]): Unit = ++|(key, batch)
|
||||||
|
|
||||||
def insert(key: String, batch: Map[String, List[ColumnOrSuperColumn]], consistencyLevel: Int): Unit = ++|(key, batch, consistencyLevel)
|
def insert(key: String, batch: Map[String, List[ColumnOrSuperColumn]], consistencyLevel: Int): Unit = ++|(key, batch, consistencyLevel)
|
||||||
|
|
|
||||||
|
|
@ -156,7 +156,7 @@ private[akka] object CassandraStorageBackend extends
|
||||||
for (entry <- entries) {
|
for (entry <- entries) {
|
||||||
val columnOrSuperColumn = new ColumnOrSuperColumn
|
val columnOrSuperColumn = new ColumnOrSuperColumn
|
||||||
columnOrSuperColumn.setColumn(new Column(entry._1, entry._2, System.currentTimeMillis))
|
columnOrSuperColumn.setColumn(new Column(entry._1, entry._2, System.currentTimeMillis))
|
||||||
batch + (MAP_COLUMN_PARENT.getColumn_family -> List(columnOrSuperColumn))
|
batch += (MAP_COLUMN_PARENT.getColumn_family -> List(columnOrSuperColumn))
|
||||||
}
|
}
|
||||||
sessions.withSession {
|
sessions.withSession {
|
||||||
_ ++| (name, batch, CONSISTENCY_LEVEL)
|
_ ++| (name, batch, CONSISTENCY_LEVEL)
|
||||||
|
|
|
||||||
|
|
@ -71,10 +71,6 @@ object SoftRefPool {
|
||||||
def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,SoftReferenceObjectPool] {
|
def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,SoftReferenceObjectPool] {
|
||||||
val impl = new SoftReferenceObjectPool(toPoolableObjectFactory(factory))
|
val impl = new SoftReferenceObjectPool(toPoolableObjectFactory(factory))
|
||||||
}
|
}
|
||||||
|
|
||||||
def apply[T](factory: PoolItemFactory[T], initSize: Int) = new PoolBridge[T,SoftReferenceObjectPool] {
|
|
||||||
val impl = new SoftReferenceObjectPool(toPoolableObjectFactory(factory),initSize)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
trait TransportFactory[T <: TTransport] extends PoolItemFactory[T] {
|
trait TransportFactory[T <: TTransport] extends PoolItemFactory[T] {
|
||||||
|
|
|
||||||
|
|
@ -98,10 +98,21 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
|
||||||
removedEntries.clear
|
removedEntries.clear
|
||||||
}
|
}
|
||||||
|
|
||||||
def -=(key: K) = remove(key)
|
def -=(key: K) = {
|
||||||
|
remove(key)
|
||||||
|
this
|
||||||
|
}
|
||||||
|
|
||||||
def +=(key: K, value: V) = put(key, value)
|
override def +=(kv : (K,V)) = {
|
||||||
|
put(kv._1,kv._2)
|
||||||
|
this
|
||||||
|
}
|
||||||
|
|
||||||
|
def +=(key: K, value: V) = {
|
||||||
|
put(key, value)
|
||||||
|
this
|
||||||
|
}
|
||||||
|
|
||||||
override def put(key: K, value: V): Option[V] = {
|
override def put(key: K, value: V): Option[V] = {
|
||||||
register
|
register
|
||||||
newAndUpdatedEntries.put(key, value)
|
newAndUpdatedEntries.put(key, value)
|
||||||
|
|
@ -112,9 +123,10 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
|
||||||
newAndUpdatedEntries.update(key, value)
|
newAndUpdatedEntries.update(key, value)
|
||||||
}
|
}
|
||||||
|
|
||||||
def remove(key: K) = {
|
override def remove(key: K) = {
|
||||||
register
|
register
|
||||||
removedEntries.add(key)
|
removedEntries.add(key)
|
||||||
|
newAndUpdatedEntries.get(key)
|
||||||
}
|
}
|
||||||
|
|
||||||
def slice(start: Option[K], count: Int): List[Tuple2[K, V]] =
|
def slice(start: Option[K], count: Int): List[Tuple2[K, V]] =
|
||||||
|
|
@ -147,6 +159,8 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
|
||||||
} catch { case e: Exception => None }
|
} catch { case e: Exception => None }
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def iterator = elements
|
||||||
|
|
||||||
override def elements: Iterator[Tuple2[K, V]] = {
|
override def elements: Iterator[Tuple2[K, V]] = {
|
||||||
new Iterator[Tuple2[K, V]] {
|
new Iterator[Tuple2[K, V]] {
|
||||||
private val originalList: List[Tuple2[K, V]] = try {
|
private val originalList: List[Tuple2[K, V]] = try {
|
||||||
|
|
@ -175,7 +189,7 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
|
||||||
*
|
*
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
trait PersistentVector[T] extends RandomAccessSeq[T] with Transactional with Committable {
|
trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committable {
|
||||||
protected val newElems = TransactionalState.newVector[T]
|
protected val newElems = TransactionalState.newVector[T]
|
||||||
protected val updatedElems = TransactionalState.newMap[Int, T]
|
protected val updatedElems = TransactionalState.newMap[Int, T]
|
||||||
protected val removedElems = TransactionalState.newVector[T]
|
protected val removedElems = TransactionalState.newVector[T]
|
||||||
|
|
@ -204,9 +218,9 @@ trait PersistentVector[T] extends RandomAccessSeq[T] with Transactional with Com
|
||||||
else storage.getVectorStorageEntryFor(uuid, index)
|
else storage.getVectorStorageEntryFor(uuid, index)
|
||||||
}
|
}
|
||||||
|
|
||||||
override def slice(start: Int, count: Int): RandomAccessSeq[T] = slice(Some(start), None, count)
|
override def slice(start: Int, count: Int): IndexedSeq[T] = slice(Some(start), None, count)
|
||||||
|
|
||||||
def slice(start: Option[Int], finish: Option[Int], count: Int): RandomAccessSeq[T] = {
|
def slice(start: Option[Int], finish: Option[Int], count: Int): IndexedSeq[T] = {
|
||||||
val buffer = new scala.collection.mutable.ArrayBuffer[T]
|
val buffer = new scala.collection.mutable.ArrayBuffer[T]
|
||||||
storage.getVectorStorageRangeFor(uuid, start, finish, count).foreach(buffer.append(_))
|
storage.getVectorStorageRangeFor(uuid, start, finish, count).foreach(buffer.append(_))
|
||||||
buffer
|
buffer
|
||||||
|
|
@ -389,14 +403,20 @@ trait PersistentQueue[A] extends scala.collection.mutable.Queue[A]
|
||||||
override def isEmpty: Boolean =
|
override def isEmpty: Boolean =
|
||||||
size == 0
|
size == 0
|
||||||
|
|
||||||
override def +=(elem: A): Unit = enqueue(elem)
|
override def +=(elem: A) = {
|
||||||
override def ++=(elems: Iterator[A]): Unit = enqueue(elems.toList: _*)
|
enqueue(elem)
|
||||||
override def ++=(elems: Iterable[A]): Unit = this ++= elems.elements
|
this
|
||||||
|
}
|
||||||
|
override def ++=(elems: Iterator[A]) = {
|
||||||
|
enqueue(elems.toList: _*)
|
||||||
|
this
|
||||||
|
}
|
||||||
|
def ++=(elems: Iterable[A]): Unit = this ++= elems.iterator
|
||||||
|
|
||||||
override def dequeueFirst(p: A => Boolean): Option[A] =
|
override def dequeueFirst(p: A => Boolean): Option[A] =
|
||||||
throw new UnsupportedOperationException("dequeueFirst not supported")
|
throw new UnsupportedOperationException("dequeueFirst not supported")
|
||||||
|
|
||||||
override def dequeueAll(p: A => Boolean): Seq[A] =
|
override def dequeueAll(p: A => Boolean): scala.collection.mutable.Seq[A] =
|
||||||
throw new UnsupportedOperationException("dequeueAll not supported")
|
throw new UnsupportedOperationException("dequeueAll not supported")
|
||||||
|
|
||||||
private def register = {
|
private def register = {
|
||||||
|
|
@ -490,13 +510,18 @@ trait PersistentSortedSet[A]
|
||||||
inStorage(elem) match {
|
inStorage(elem) match {
|
||||||
case Some(f) => f
|
case Some(f) => f
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(elem + " not present")
|
throw new NoSuchElementException(elem + " not present")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
implicit def order(x: (A, Float)) = new Ordered[(A, Float)] {
|
implicit def order(x: (A, Float)) = new Ordered[(A, Float)] {
|
||||||
def compare(that: (A, Float)) = x._2 compare that._2
|
def compare(that: (A, Float)) = x._2 compare that._2
|
||||||
}
|
}
|
||||||
|
|
||||||
|
implicit def ordering = new scala.math.Ordering[(A,Float)] {
|
||||||
|
def compare(x: (A, Float),y : (A,Float)) = x._2 compare y._2
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
def zrange(start: Int, end: Int): List[(A, Float)] = {
|
def zrange(start: Int, end: Int): List[(A, Float)] = {
|
||||||
// need to operate on the whole range
|
// need to operate on the whole range
|
||||||
|
|
@ -512,7 +537,7 @@ trait PersistentSortedSet[A]
|
||||||
else if (end >= l) (l - 1)
|
else if (end >= l) (l - 1)
|
||||||
else end
|
else end
|
||||||
// slice is open at the end, we need a closed end range
|
// slice is open at the end, we need a closed end range
|
||||||
ts.elements.slice(s, e + 1).toList
|
ts.iterator.slice(s, e + 1).toList
|
||||||
}
|
}
|
||||||
|
|
||||||
private def register = {
|
private def register = {
|
||||||
|
|
|
||||||
|
|
@ -11,6 +11,8 @@ import se.scalablesolutions.akka.config.Config.config
|
||||||
|
|
||||||
import sjson.json.Serializer._
|
import sjson.json.Serializer._
|
||||||
|
|
||||||
|
import java.util.NoSuchElementException
|
||||||
|
|
||||||
import com.mongodb._
|
import com.mongodb._
|
||||||
|
|
||||||
import java.util.{Map=>JMap, List=>JList, ArrayList=>JArrayList}
|
import java.util.{Map=>JMap, List=>JList, ArrayList=>JArrayList}
|
||||||
|
|
@ -125,7 +127,7 @@ private[akka] object MongoStorageBackend extends
|
||||||
val m =
|
val m =
|
||||||
nullSafeFindOne(name) match {
|
nullSafeFindOne(name) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(dbo) =>
|
case Some(dbo) =>
|
||||||
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
|
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
|
||||||
}
|
}
|
||||||
|
|
@ -143,7 +145,7 @@ private[akka] object MongoStorageBackend extends
|
||||||
val m =
|
val m =
|
||||||
nullSafeFindOne(name) match {
|
nullSafeFindOne(name) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(dbo) =>
|
case Some(dbo) =>
|
||||||
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
|
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
|
||||||
}
|
}
|
||||||
|
|
@ -162,7 +164,7 @@ private[akka] object MongoStorageBackend extends
|
||||||
else count
|
else count
|
||||||
|
|
||||||
val n =
|
val n =
|
||||||
List(m.keySet.toArray: _*).asInstanceOf[List[String]].sort((e1, e2) => (e1 compareTo e2) < 0).slice(s, s + cnt)
|
List(m.keySet.toArray: _*).asInstanceOf[List[String]].sortWith((e1, e2) => (e1 compareTo e2) < 0).slice(s, s + cnt)
|
||||||
val vals =
|
val vals =
|
||||||
for(s <- n)
|
for(s <- n)
|
||||||
yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]]))
|
yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]]))
|
||||||
|
|
@ -181,7 +183,7 @@ private[akka] object MongoStorageBackend extends
|
||||||
}
|
}
|
||||||
} catch {
|
} catch {
|
||||||
case e =>
|
case e =>
|
||||||
throw new Predef.NoSuchElementException(e.getMessage)
|
throw new NoSuchElementException(e.getMessage)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -221,7 +223,7 @@ private[akka] object MongoStorageBackend extends
|
||||||
val o =
|
val o =
|
||||||
nullSafeFindOne(name) match {
|
nullSafeFindOne(name) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
|
|
||||||
case Some(dbo) =>
|
case Some(dbo) =>
|
||||||
dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
|
dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
|
||||||
|
|
@ -230,7 +232,7 @@ private[akka] object MongoStorageBackend extends
|
||||||
o.get(index).asInstanceOf[Array[Byte]])
|
o.get(index).asInstanceOf[Array[Byte]])
|
||||||
} catch {
|
} catch {
|
||||||
case e =>
|
case e =>
|
||||||
throw new Predef.NoSuchElementException(e.getMessage)
|
throw new NoSuchElementException(e.getMessage)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -240,7 +242,7 @@ private[akka] object MongoStorageBackend extends
|
||||||
val o =
|
val o =
|
||||||
nullSafeFindOne(name) match {
|
nullSafeFindOne(name) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
|
|
||||||
case Some(dbo) =>
|
case Some(dbo) =>
|
||||||
dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
|
dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
|
||||||
|
|
@ -254,7 +256,7 @@ private[akka] object MongoStorageBackend extends
|
||||||
yield serializer.in[AnyRef](e.asInstanceOf[Array[Byte]])
|
yield serializer.in[AnyRef](e.asInstanceOf[Array[Byte]])
|
||||||
} catch {
|
} catch {
|
||||||
case e =>
|
case e =>
|
||||||
throw new Predef.NoSuchElementException(e.getMessage)
|
throw new NoSuchElementException(e.getMessage)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -6,6 +6,7 @@ import org.junit.{Test, Before}
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
import _root_.dispatch.json._
|
import _root_.dispatch.json._
|
||||||
import _root_.dispatch.json.Js._
|
import _root_.dispatch.json.Js._
|
||||||
|
import java.util.NoSuchElementException
|
||||||
|
|
||||||
@scala.reflect.BeanInfo case class Foo(no: Int, name: String)
|
@scala.reflect.BeanInfo case class Foo(no: Int, name: String)
|
||||||
class MongoStorageSpec extends TestCase {
|
class MongoStorageSpec extends TestCase {
|
||||||
|
|
@ -111,12 +112,12 @@ class MongoStorageSpec extends TestCase {
|
||||||
try {
|
try {
|
||||||
MongoStorageBackend.getVectorStorageEntryFor("U-A1", 1)
|
MongoStorageBackend.getVectorStorageEntryFor("U-A1", 1)
|
||||||
fail("should throw an exception")
|
fail("should throw an exception")
|
||||||
} catch {case e: Predef.NoSuchElementException => {}}
|
} catch {case e: NoSuchElementException => {}}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
MongoStorageBackend.getVectorStorageRangeFor("U-A1", Some(2), None, 12)
|
MongoStorageBackend.getVectorStorageRangeFor("U-A1", Some(2), None, 12)
|
||||||
fail("should throw an exception")
|
fail("should throw an exception")
|
||||||
} catch {case e: Predef.NoSuchElementException => {}}
|
} catch {case e: NoSuchElementException => {}}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -198,7 +199,7 @@ class MongoStorageSpec extends TestCase {
|
||||||
try {
|
try {
|
||||||
MongoStorageBackend.getMapStorageFor("U-M2")
|
MongoStorageBackend.getMapStorageFor("U-M2")
|
||||||
fail("should throw an exception")
|
fail("should throw an exception")
|
||||||
} catch {case e: Predef.NoSuchElementException => {}}
|
} catch {case e: NoSuchElementException => {}}
|
||||||
|
|
||||||
changeSetM.clear
|
changeSetM.clear
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -109,7 +109,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
def removeMapStorageFor(name: String): Unit = withErrorHandling {
|
def removeMapStorageFor(name: String): Unit = withErrorHandling {
|
||||||
db.keys("%s:*".format(encode(name.getBytes))) match {
|
db.keys("%s:*".format(encode(name.getBytes))) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(keys) =>
|
case Some(keys) =>
|
||||||
keys.foreach(db.delete(_))
|
keys.foreach(db.delete(_))
|
||||||
}
|
}
|
||||||
|
|
@ -122,7 +122,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = withErrorHandling {
|
def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = withErrorHandling {
|
||||||
db.get(makeRedisKey(name, key)) match {
|
db.get(makeRedisKey(name, key)) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(new String(key) + " not present")
|
throw new NoSuchElementException(new String(key) + " not present")
|
||||||
case Some(s) => Some(s.getBytes)
|
case Some(s) => Some(s.getBytes)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -138,7 +138,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = withErrorHandling {
|
def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = withErrorHandling {
|
||||||
db.keys("%s:*".format(new String(encode(name.getBytes)))) match {
|
db.keys("%s:*".format(new String(encode(name.getBytes)))) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(keys) =>
|
case Some(keys) =>
|
||||||
keys.map(key => (makeKeyFromRedisKey(key)._2, db.get(key).get.getBytes)).toList
|
keys.map(key => (makeKeyFromRedisKey(key)._2, db.get(key).get.getBytes)).toList
|
||||||
}
|
}
|
||||||
|
|
@ -174,7 +174,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
.toList
|
.toList
|
||||||
case ((Some(s), None, c)) if c > 0 =>
|
case ((Some(s), None, c)) if c > 0 =>
|
||||||
wholeSorted.from(s)
|
wholeSorted.from(s)
|
||||||
.elements
|
.iterator
|
||||||
.take(count)
|
.take(count)
|
||||||
.map(e => (e._1.getBytes, e._2))
|
.map(e => (e._1.getBytes, e._2))
|
||||||
.toList
|
.toList
|
||||||
|
|
@ -206,7 +206,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = withErrorHandling {
|
def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = withErrorHandling {
|
||||||
db.lindex(new String(encode(name.getBytes)), index) match {
|
db.lindex(new String(encode(name.getBytes)), index) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " does not have element at " + index)
|
throw new NoSuchElementException(name + " does not have element at " + index)
|
||||||
case Some(e) => e.getBytes
|
case Some(e) => e.getBytes
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -226,7 +226,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
else count
|
else count
|
||||||
db.lrange(new String(encode(name.getBytes)), s, s + cnt - 1) match {
|
db.lrange(new String(encode(name.getBytes)), s, s + cnt - 1) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " does not have elements in the range specified")
|
throw new NoSuchElementException(name + " does not have elements in the range specified")
|
||||||
case Some(l) =>
|
case Some(l) =>
|
||||||
l map (_.get.getBytes)
|
l map (_.get.getBytes)
|
||||||
}
|
}
|
||||||
|
|
@ -235,7 +235,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
def getVectorStorageSizeFor(name: String): Int = {
|
def getVectorStorageSizeFor(name: String): Int = {
|
||||||
db.llen(new String(encode(name.getBytes))) match {
|
db.llen(new String(encode(name.getBytes))) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(l) => l
|
case Some(l) => l
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -247,7 +247,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
def getRefStorageFor(name: String): Option[Array[Byte]] = withErrorHandling {
|
def getRefStorageFor(name: String): Option[Array[Byte]] = withErrorHandling {
|
||||||
db.get(new String(encode(name.getBytes))) match {
|
db.get(new String(encode(name.getBytes))) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(s) => Some(s.getBytes)
|
case Some(s) => Some(s.getBytes)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -256,7 +256,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
db.incr(new String(encode(name.getBytes))) match {
|
db.incr(new String(encode(name.getBytes))) match {
|
||||||
case Some(i) => Some(i)
|
case Some(i) => Some(i)
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.IllegalArgumentException(name + " exception in incr")
|
throw new IllegalArgumentException(name + " exception in incr")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -264,7 +264,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
db.incrby(new String(encode(name.getBytes)), by) match {
|
db.incrby(new String(encode(name.getBytes)), by) match {
|
||||||
case Some(i) => Some(i)
|
case Some(i) => Some(i)
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.IllegalArgumentException(name + " exception in incrby")
|
throw new IllegalArgumentException(name + " exception in incrby")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -272,7 +272,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
db.decr(new String(encode(name.getBytes))) match {
|
db.decr(new String(encode(name.getBytes))) match {
|
||||||
case Some(i) => Some(i)
|
case Some(i) => Some(i)
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.IllegalArgumentException(name + " exception in decr")
|
throw new IllegalArgumentException(name + " exception in decr")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -280,7 +280,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
db.decrby(new String(encode(name.getBytes)), by) match {
|
db.decrby(new String(encode(name.getBytes)), by) match {
|
||||||
case Some(i) => Some(i)
|
case Some(i) => Some(i)
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.IllegalArgumentException(name + " exception in decrby")
|
throw new IllegalArgumentException(name + " exception in decrby")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -294,7 +294,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
def dequeue(name: String): Option[Array[Byte]] = withErrorHandling {
|
def dequeue(name: String): Option[Array[Byte]] = withErrorHandling {
|
||||||
db.lpop(new String(encode(name.getBytes))) match {
|
db.lpop(new String(encode(name.getBytes))) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(s) =>
|
case Some(s) =>
|
||||||
Some(s.getBytes)
|
Some(s.getBytes)
|
||||||
}
|
}
|
||||||
|
|
@ -304,7 +304,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
def size(name: String): Int = withErrorHandling {
|
def size(name: String): Int = withErrorHandling {
|
||||||
db.llen(new String(encode(name.getBytes))) match {
|
db.llen(new String(encode(name.getBytes))) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(l) => l
|
case Some(l) => l
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -316,14 +316,14 @@ private [akka] object RedisStorageBackend extends
|
||||||
case 1 =>
|
case 1 =>
|
||||||
db.lindex(new String(encode(name.getBytes)), start) match {
|
db.lindex(new String(encode(name.getBytes)), start) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException("No element at " + start)
|
throw new NoSuchElementException("No element at " + start)
|
||||||
case Some(s) =>
|
case Some(s) =>
|
||||||
List(s.getBytes)
|
List(s.getBytes)
|
||||||
}
|
}
|
||||||
case n =>
|
case n =>
|
||||||
db.lrange(new String(encode(name.getBytes)), start, start + count - 1) match {
|
db.lrange(new String(encode(name.getBytes)), start, start + count - 1) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(
|
throw new NoSuchElementException(
|
||||||
"No element found between " + start + " and " + (start + count - 1))
|
"No element found between " + start + " and " + (start + count - 1))
|
||||||
case Some(es) =>
|
case Some(es) =>
|
||||||
es.map(_.get.getBytes)
|
es.map(_.get.getBytes)
|
||||||
|
|
@ -359,7 +359,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
def zcard(name: String): Int = withErrorHandling {
|
def zcard(name: String): Int = withErrorHandling {
|
||||||
db.zcard(new String(encode(name.getBytes))) match {
|
db.zcard(new String(encode(name.getBytes))) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(l) => l
|
case Some(l) => l
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -374,7 +374,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
def zrange(name: String, start: Int, end: Int): List[Array[Byte]] = withErrorHandling {
|
def zrange(name: String, start: Int, end: Int): List[Array[Byte]] = withErrorHandling {
|
||||||
db.zrange(new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC, false) match {
|
db.zrange(new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC, false) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(s) =>
|
case Some(s) =>
|
||||||
s.map(_.get.getBytes)
|
s.map(_.get.getBytes)
|
||||||
}
|
}
|
||||||
|
|
@ -384,7 +384,7 @@ private [akka] object RedisStorageBackend extends
|
||||||
db.zrangeWithScore(
|
db.zrangeWithScore(
|
||||||
new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC) match {
|
new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC) match {
|
||||||
case None =>
|
case None =>
|
||||||
throw new Predef.NoSuchElementException(name + " not present")
|
throw new NoSuchElementException(name + " not present")
|
||||||
case Some(l) =>
|
case Some(l) =>
|
||||||
l.map{ case (elem, score) => (elem.get.getBytes, score.get.toFloat) }
|
l.map{ case (elem, score) => (elem.get.getBytes, score.get.toFloat) }
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -111,7 +111,8 @@ class RedisPersistentActorSpec extends TestCase {
|
||||||
bactor !! Debit("a-123", 8000, failer)
|
bactor !! Debit("a-123", 8000, failer)
|
||||||
assertEquals(BigInt(1000), (bactor !! Balance("a-123")).get)
|
assertEquals(BigInt(1000), (bactor !! Balance("a-123")).get)
|
||||||
|
|
||||||
assertEquals(7, (bactor !! LogSize).get)
|
val c: Int = (bactor !! LogSize).get
|
||||||
|
assertTrue(7 == c)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -131,7 +132,8 @@ class RedisPersistentActorSpec extends TestCase {
|
||||||
assertEquals(BigInt(5000), (bactor !! Balance("a-123")).get)
|
assertEquals(BigInt(5000), (bactor !! Balance("a-123")).get)
|
||||||
|
|
||||||
// should not count the failed one
|
// should not count the failed one
|
||||||
assertEquals(3, (bactor !! LogSize).get)
|
val c: Int = (bactor !! LogSize).get
|
||||||
|
assertTrue(3 == c)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -152,6 +154,7 @@ class RedisPersistentActorSpec extends TestCase {
|
||||||
assertEquals(BigInt(5000), (bactor !! (Balance("a-123"), 5000)).get)
|
assertEquals(BigInt(5000), (bactor !! (Balance("a-123"), 5000)).get)
|
||||||
|
|
||||||
// should not count the failed one
|
// should not count the failed one
|
||||||
assertEquals(3, (bactor !! LogSize).get)
|
val c: Int = (bactor !! LogSize).get
|
||||||
|
assertTrue(3 == c)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -58,7 +58,8 @@ class RedisPersistentQSpec extends TestCase {
|
||||||
qa !! NQ("a-123")
|
qa !! NQ("a-123")
|
||||||
qa !! NQ("a-124")
|
qa !! NQ("a-124")
|
||||||
qa !! NQ("a-125")
|
qa !! NQ("a-125")
|
||||||
assertEquals(3, (qa !! SZ).get)
|
val t: Int = (qa !! SZ).get
|
||||||
|
assertTrue(3 == t)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -68,11 +69,13 @@ class RedisPersistentQSpec extends TestCase {
|
||||||
qa !! NQ("a-123")
|
qa !! NQ("a-123")
|
||||||
qa !! NQ("a-124")
|
qa !! NQ("a-124")
|
||||||
qa !! NQ("a-125")
|
qa !! NQ("a-125")
|
||||||
assertEquals(3, (qa !! SZ).get)
|
val s: Int = (qa !! SZ).get
|
||||||
|
assertTrue(3 == s)
|
||||||
assertEquals("a-123", (qa !! DQ).get)
|
assertEquals("a-123", (qa !! DQ).get)
|
||||||
assertEquals("a-124", (qa !! DQ).get)
|
assertEquals("a-124", (qa !! DQ).get)
|
||||||
assertEquals("a-125", (qa !! DQ).get)
|
assertEquals("a-125", (qa !! DQ).get)
|
||||||
assertEquals(0, (qa !! SZ).get)
|
val t: Int = (qa !! SZ).get
|
||||||
|
assertTrue(0 == t)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -85,11 +88,14 @@ class RedisPersistentQSpec extends TestCase {
|
||||||
qa !! NQ("a-123")
|
qa !! NQ("a-123")
|
||||||
qa !! NQ("a-124")
|
qa !! NQ("a-124")
|
||||||
qa !! NQ("a-125")
|
qa !! NQ("a-125")
|
||||||
assertEquals(3, (qa !! SZ).get)
|
val t: Int = (qa !! SZ).get
|
||||||
|
assertTrue(3 == t)
|
||||||
assertEquals("a-123", (qa !! DQ).get)
|
assertEquals("a-123", (qa !! DQ).get)
|
||||||
assertEquals(2, (qa !! SZ).get)
|
val s: Int = (qa !! SZ).get
|
||||||
|
assertTrue(2 == s)
|
||||||
qa !! MNDQ(List("a-126", "a-127"), 2, failer)
|
qa !! MNDQ(List("a-126", "a-127"), 2, failer)
|
||||||
assertEquals(2, (qa !! SZ).get)
|
val u: Int = (qa !! SZ).get
|
||||||
|
assertTrue(2 == u)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -104,22 +110,26 @@ class RedisPersistentQSpec extends TestCase {
|
||||||
qa !! NQ("a-124")
|
qa !! NQ("a-124")
|
||||||
qa !! NQ("a-125")
|
qa !! NQ("a-125")
|
||||||
|
|
||||||
assertEquals(3, (qa !! SZ).get)
|
val t: Int = (qa !! SZ).get
|
||||||
|
assertTrue(3 == t)
|
||||||
|
|
||||||
// dequeue 1
|
// dequeue 1
|
||||||
assertEquals("a-123", (qa !! DQ).get)
|
assertEquals("a-123", (qa !! DQ).get)
|
||||||
|
|
||||||
// size == 2
|
// size == 2
|
||||||
assertEquals(2, (qa !! SZ).get)
|
val s: Int = (qa !! SZ).get
|
||||||
|
assertTrue(2 == s)
|
||||||
|
|
||||||
// enqueue 2, dequeue 2 => size == 2
|
// enqueue 2, dequeue 2 => size == 2
|
||||||
qa !! MNDQ(List("a-126", "a-127"), 2, failer)
|
qa !! MNDQ(List("a-126", "a-127"), 2, failer)
|
||||||
assertEquals(2, (qa !! SZ).get)
|
val u: Int = (qa !! SZ).get
|
||||||
|
assertTrue(2 == u)
|
||||||
|
|
||||||
// enqueue 2 => size == 4
|
// enqueue 2 => size == 4
|
||||||
qa !! NQ("a-128")
|
qa !! NQ("a-128")
|
||||||
qa !! NQ("a-129")
|
qa !! NQ("a-129")
|
||||||
assertEquals(4, (qa !! SZ).get)
|
val v: Int = (qa !! SZ).get
|
||||||
|
assertTrue(4 == v)
|
||||||
|
|
||||||
// enqueue 1 => size 5
|
// enqueue 1 => size 5
|
||||||
// dequeue 6 => fail transaction
|
// dequeue 6 => fail transaction
|
||||||
|
|
@ -128,6 +138,7 @@ class RedisPersistentQSpec extends TestCase {
|
||||||
qa !! MNDQ(List("a-130"), 6, failer)
|
qa !! MNDQ(List("a-130"), 6, failer)
|
||||||
} catch { case e: Exception => {} }
|
} catch { case e: Exception => {} }
|
||||||
|
|
||||||
assertEquals(4, (qa !! SZ).get)
|
val w: Int = (qa !! SZ).get
|
||||||
|
assertTrue(4 == w)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -144,7 +144,7 @@ class RedisPersistentSortedSetSpec extends
|
||||||
qa !! REMOVE(h7)
|
qa !! REMOVE(h7)
|
||||||
}
|
}
|
||||||
catch {
|
catch {
|
||||||
case e: Predef.NoSuchElementException =>
|
case e: NoSuchElementException =>
|
||||||
e.getMessage should endWith("not present")
|
e.getMessage should endWith("not present")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -7,6 +7,7 @@ import org.scalatest.junit.JUnitRunner
|
||||||
import org.junit.runner.RunWith
|
import org.junit.runner.RunWith
|
||||||
|
|
||||||
import se.scalablesolutions.akka.serialization.Serializable
|
import se.scalablesolutions.akka.serialization.Serializable
|
||||||
|
import se.scalablesolutions.akka.serialization.Serializer._
|
||||||
|
|
||||||
import RedisStorageBackend._
|
import RedisStorageBackend._
|
||||||
|
|
||||||
|
|
@ -38,16 +39,19 @@ class RedisStorageBackendSpec extends
|
||||||
"T-1", "debasish.language".getBytes).get) should equal("java")
|
"T-1", "debasish.language".getBytes).get) should equal("java")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
it("should enter a custom object for transaction T-1") {
|
it("should enter a custom object for transaction T-1") {
|
||||||
val n = Name(100, "debasish", "kolkata")
|
val n = Name(100, "debasish", "kolkata")
|
||||||
insertMapStorageEntryFor("T-1", "debasish.identity".getBytes, n.toBytes)
|
// insertMapStorageEntryFor("T-1", "debasish.identity".getBytes, Java.out(n))
|
||||||
|
// insertMapStorageEntryFor("T-1", "debasish.identity".getBytes, n.toBytes)
|
||||||
getMapStorageSizeFor("T-1") should equal(5)
|
getMapStorageSizeFor("T-1") should equal(5)
|
||||||
}
|
}
|
||||||
|
**/
|
||||||
|
|
||||||
it("should enter key/values for another transaction T-2") {
|
it("should enter key/values for another transaction T-2") {
|
||||||
insertMapStorageEntryFor("T-2", "debasish.age".getBytes, "49".getBytes)
|
insertMapStorageEntryFor("T-2", "debasish.age".getBytes, "49".getBytes)
|
||||||
insertMapStorageEntryFor("T-2", "debasish.spouse".getBytes, "paramita".getBytes)
|
insertMapStorageEntryFor("T-2", "debasish.spouse".getBytes, "paramita".getBytes)
|
||||||
getMapStorageSizeFor("T-1") should equal(5)
|
getMapStorageSizeFor("T-1") should equal(4)
|
||||||
getMapStorageSizeFor("T-2") should equal(2)
|
getMapStorageSizeFor("T-2") should equal(2)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -94,9 +98,10 @@ class RedisStorageBackendSpec extends
|
||||||
insertVectorStorageEntryFor("T-3", "debasish".getBytes)
|
insertVectorStorageEntryFor("T-3", "debasish".getBytes)
|
||||||
insertVectorStorageEntryFor("T-3", "maulindu".getBytes)
|
insertVectorStorageEntryFor("T-3", "maulindu".getBytes)
|
||||||
val n = Name(100, "debasish", "kolkata")
|
val n = Name(100, "debasish", "kolkata")
|
||||||
insertVectorStorageEntryFor("T-3", n.toBytes)
|
// insertVectorStorageEntryFor("T-3", Java.out(n))
|
||||||
|
// insertVectorStorageEntryFor("T-3", n.toBytes)
|
||||||
insertVectorStorageEntryFor("T-3", "1200".getBytes)
|
insertVectorStorageEntryFor("T-3", "1200".getBytes)
|
||||||
getVectorStorageSizeFor("T-3") should equal(4)
|
getVectorStorageSizeFor("T-3") should equal(3)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -108,9 +113,11 @@ class RedisStorageBackendSpec extends
|
||||||
insertRefStorageFor("T-4", "1200".getBytes)
|
insertRefStorageFor("T-4", "1200".getBytes)
|
||||||
new String(getRefStorageFor("T-4").get) should equal("1200")
|
new String(getRefStorageFor("T-4").get) should equal("1200")
|
||||||
|
|
||||||
val n = Name(100, "debasish", "kolkata")
|
// val n = Name(100, "debasish", "kolkata")
|
||||||
insertRefStorageFor("T-4", n.toBytes)
|
// insertRefStorageFor("T-4", Java.out(n))
|
||||||
n.fromBytes(getRefStorageFor("T-4").get) should equal(n)
|
// insertRefStorageFor("T-4", n.toBytes)
|
||||||
|
// Java.in(getRefStorageFor("T-4").get, Some(classOf[Name])).asInstanceOf[Name] should equal(n)
|
||||||
|
// n.fromBytes(getRefStorageFor("T-4").get) should equal(n)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -216,6 +223,8 @@ class RedisStorageBackendSpec extends
|
||||||
|
|
||||||
case class Name(id: Int, name: String, address: String)
|
case class Name(id: Int, name: String, address: String)
|
||||||
extends Serializable.SBinary[Name] {
|
extends Serializable.SBinary[Name] {
|
||||||
|
import sbinary._
|
||||||
|
import sbinary.Operations._
|
||||||
import sbinary.DefaultProtocol._
|
import sbinary.DefaultProtocol._
|
||||||
|
|
||||||
def this() = this(0, null, null)
|
def this() = this(0, null, null)
|
||||||
|
|
|
||||||
|
|
@ -18,12 +18,12 @@ import com.sun.jersey.spi.container.WebApplication
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
class AkkaServlet extends ServletContainer {
|
class AkkaServlet extends ServletContainer {
|
||||||
import org.scala_tools.javautils.Imports._
|
import scala.collection.JavaConversions._
|
||||||
|
|
||||||
override def initiate(resourceConfig: ResourceConfig, webApplication: WebApplication) = {
|
override def initiate(resourceConfig: ResourceConfig, webApplication: WebApplication) = {
|
||||||
val configurators = ConfiguratorRepository.getConfigurators
|
val configurators = ConfiguratorRepository.getConfigurators
|
||||||
|
|
||||||
resourceConfig.getClasses.addAll(configurators.flatMap(_.getComponentInterfaces).asJava)
|
resourceConfig.getClasses.addAll(configurators.flatMap(_.getComponentInterfaces))
|
||||||
resourceConfig.getProperties.put(
|
resourceConfig.getProperties.put(
|
||||||
"com.sun.jersey.spi.container.ResourceFilters",
|
"com.sun.jersey.spi.container.ResourceFilters",
|
||||||
config.getList("akka.rest.filters").mkString(","))
|
config.getList("akka.rest.filters").mkString(","))
|
||||||
|
|
|
||||||
|
|
@ -166,7 +166,7 @@ trait ChatManagement { this: Actor =>
|
||||||
* Creates and links a RedisChatStorage.
|
* Creates and links a RedisChatStorage.
|
||||||
*/
|
*/
|
||||||
trait RedisChatStorageFactory { this: Actor =>
|
trait RedisChatStorageFactory { this: Actor =>
|
||||||
val storage: ChatStorage = spawnLink(classOf[RedisChatStorage]) // starts and links ChatStorage
|
val storage: ChatStorage = spawnLink[RedisChatStorage] // starts and links ChatStorage
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -5,6 +5,7 @@ import _root_.net.liftweb.http._
|
||||||
import _root_.net.liftweb.sitemap._
|
import _root_.net.liftweb.sitemap._
|
||||||
import _root_.net.liftweb.sitemap.Loc._
|
import _root_.net.liftweb.sitemap.Loc._
|
||||||
import _root_.net.liftweb.http.auth._
|
import _root_.net.liftweb.http.auth._
|
||||||
|
import _root_.net.liftweb.common._
|
||||||
import Helpers._
|
import Helpers._
|
||||||
|
|
||||||
import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor}
|
import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor}
|
||||||
|
|
@ -17,18 +18,18 @@ import sample.lift.{PersistentSimpleService, SimpleService}
|
||||||
* A class that's instantiated early and run. It allows the application
|
* A class that's instantiated early and run. It allows the application
|
||||||
* to modify lift's environment
|
* to modify lift's environment
|
||||||
*/
|
*/
|
||||||
class Boot {
|
class Boot extends Logging {
|
||||||
def boot {
|
def boot {
|
||||||
// where to search snippet
|
// where to search snippet
|
||||||
LiftRules.addToPackages("sample.lift")
|
LiftRules.addToPackages("sample.lift")
|
||||||
|
|
||||||
LiftRules.httpAuthProtectedResource.prepend {
|
LiftRules.httpAuthProtectedResource.prepend {
|
||||||
case (ParsePath("liftcount" :: Nil, _, _, _)) => Full(AuthRole("admin"))
|
case (Req("liftcount" :: Nil, _, _)) => Full(AuthRole("admin"))
|
||||||
}
|
}
|
||||||
|
|
||||||
LiftRules.authentication = HttpBasicAuthentication("lift") {
|
LiftRules.authentication = HttpBasicAuthentication("lift") {
|
||||||
case ("someuser", "1234", req) => {
|
case ("someuser", "1234", req) => {
|
||||||
Log.info("You are now authenticated !")
|
log.info("You are now authenticated !")
|
||||||
userRoles(AuthRole("admin"))
|
userRoles(AuthRole("admin"))
|
||||||
true
|
true
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,4 +1,4 @@
|
||||||
import _root_.bootstrap.liftweb.Boot
|
/*import _root_.bootstrap.liftweb.Boot
|
||||||
import _root_.scala.tools.nsc.MainGenericRunner
|
import _root_.scala.tools.nsc.MainGenericRunner
|
||||||
|
|
||||||
object LiftConsole {
|
object LiftConsole {
|
||||||
|
|
@ -13,3 +13,4 @@ object LiftConsole {
|
||||||
exit(0)
|
exit(0)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
*/
|
||||||
|
|
@ -87,7 +87,7 @@ class PubSub extends Actor {
|
||||||
@Broadcast
|
@Broadcast
|
||||||
@Path("/topic/{topic}/{message}/")
|
@Path("/topic/{topic}/{message}/")
|
||||||
@Produces(Array("text/plain;charset=ISO-8859-1"))
|
@Produces(Array("text/plain;charset=ISO-8859-1"))
|
||||||
@Cluster(Array(classOf[AkkaClusterBroadcastFilter])) { val name = "foo" }
|
//FIXME @Cluster(value = Array(classOf[AkkaClusterBroadcastFilter]),name = "foo")
|
||||||
def say(@PathParam("topic") topic: Broadcaster, @PathParam("message") message: String): Broadcastable = new Broadcastable(message, topic)
|
def say(@PathParam("topic") topic: Broadcaster, @PathParam("message") message: String): Broadcastable = new Broadcastable(message, topic)
|
||||||
|
|
||||||
def receive = { case _ => }
|
def receive = { case _ => }
|
||||||
|
|
@ -148,7 +148,7 @@ class Chat extends Actor {
|
||||||
|
|
||||||
@POST
|
@POST
|
||||||
@Broadcast(Array(classOf[XSSHtmlFilter], classOf[JsonpFilter]))
|
@Broadcast(Array(classOf[XSSHtmlFilter], classOf[JsonpFilter]))
|
||||||
@Cluster(Array(classOf[AkkaClusterBroadcastFilter])) { val name = "bar" }
|
//FIXME @Cluster(value = Array(classOf[AkkaClusterBroadcastFilter]),name = "bar")
|
||||||
@Consumes(Array("application/x-www-form-urlencoded"))
|
@Consumes(Array("application/x-www-form-urlencoded"))
|
||||||
@Produces(Array("text/html"))
|
@Produces(Array("text/html"))
|
||||||
def publishMessage(form: MultivaluedMap[String, String]) =
|
def publishMessage(form: MultivaluedMap[String, String]) =
|
||||||
|
|
@ -158,7 +158,7 @@ class Chat extends Actor {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
class JsonpFilter extends BroadcastFilter[String] with Logging {
|
class JsonpFilter extends BroadcastFilter with Logging {
|
||||||
def filter(an: AnyRef) = {
|
def filter(an: AnyRef) = {
|
||||||
val m = an.toString
|
val m = an.toString
|
||||||
var name = m
|
var name = m
|
||||||
|
|
|
||||||
|
|
@ -7,7 +7,7 @@ package se.scalablesolutions.akka.security.samples
|
||||||
import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor}
|
import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor}
|
||||||
import se.scalablesolutions.akka.config.ScalaConfig._
|
import se.scalablesolutions.akka.config.ScalaConfig._
|
||||||
import se.scalablesolutions.akka.util.Logging
|
import se.scalablesolutions.akka.util.Logging
|
||||||
import se.scalablesolutions.akka.security.{DigestAuthenticationActor, UserInfo}
|
import se.scalablesolutions.akka.security.{BasicAuthenticationActor,BasicCredentials,SpnegoAuthenticationActor,DigestAuthenticationActor, UserInfo}
|
||||||
import se.scalablesolutions.akka.stm.TransactionalState
|
import se.scalablesolutions.akka.stm.TransactionalState
|
||||||
|
|
||||||
class Boot {
|
class Boot {
|
||||||
|
|
|
||||||
|
|
@ -93,8 +93,8 @@ class AkkaSecurityFilterFactory extends ResourceFilterFactory with Logging {
|
||||||
case Some(r) if r.isInstanceOf[Response] =>
|
case Some(r) if r.isInstanceOf[Response] =>
|
||||||
throw new WebApplicationException(r.asInstanceOf[Response])
|
throw new WebApplicationException(r.asInstanceOf[Response])
|
||||||
case None => throw new WebApplicationException(408)
|
case None => throw new WebApplicationException(408)
|
||||||
case x => {
|
case unknown => {
|
||||||
log.error("Authenticator replied with unexpected result [%s]", x);
|
log.warning("Authenticator replied with unexpected result [%s]", unknown);
|
||||||
throw new WebApplicationException(Response.Status.INTERNAL_SERVER_ERROR)
|
throw new WebApplicationException(Response.Status.INTERNAL_SERVER_ERROR)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -257,9 +257,9 @@ trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] {
|
||||||
protected val invalidateNonces: PartialFunction[Any, Unit] = {
|
protected val invalidateNonces: PartialFunction[Any, Unit] = {
|
||||||
case InvalidateNonces =>
|
case InvalidateNonces =>
|
||||||
val ts = System.currentTimeMillis
|
val ts = System.currentTimeMillis
|
||||||
nonceMap.retain((k, v) => (ts - v) < nonceValidityPeriod)
|
nonceMap.filter(tuple => (ts - tuple._2) < nonceValidityPeriod)
|
||||||
case e =>
|
case unknown =>
|
||||||
log.info("Don't know what to do with: " + e)
|
log.error("Don't know what to do with: ", unknown)
|
||||||
}
|
}
|
||||||
|
|
||||||
//Schedule the invalidation of nonces
|
//Schedule the invalidation of nonces
|
||||||
|
|
|
||||||
|
|
@ -4,7 +4,7 @@
|
||||||
|
|
||||||
package se.scalablesolutions.akka.security
|
package se.scalablesolutions.akka.security
|
||||||
|
|
||||||
import config.ScalaConfig._
|
import se.scalablesolutions.akka.config.ScalaConfig._
|
||||||
|
|
||||||
import org.scalatest.Suite
|
import org.scalatest.Suite
|
||||||
import org.scalatest.junit.JUnitSuite
|
import org.scalatest.junit.JUnitSuite
|
||||||
|
|
|
||||||
|
|
@ -17,7 +17,7 @@ import java.net.UnknownHostException
|
||||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||||
*/
|
*/
|
||||||
trait Logging {
|
trait Logging {
|
||||||
@transient @volatile lazy val log = Logger.get(this.getClass.getName)
|
@transient lazy val log = Logger.get(this.getClass.getName)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -15,7 +15,7 @@
|
||||||
</log>
|
</log>
|
||||||
|
|
||||||
<akka>
|
<akka>
|
||||||
version = "0.7"
|
version = "0.8"
|
||||||
|
|
||||||
# FQN to the class doing initial active object/actor
|
# FQN to the class doing initial active object/actor
|
||||||
# supervisor bootstrap, should be defined in default constructor
|
# supervisor bootstrap, should be defined in default constructor
|
||||||
|
|
|
||||||
BIN
embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar
Normal file
BIN
embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar
Normal file
Binary file not shown.
8
embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom
Executable file
8
embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom
Executable file
|
|
@ -0,0 +1,8 @@
|
||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
<groupId>com.redis</groupId>
|
||||||
|
<artifactId>redisclient</artifactId>
|
||||||
|
<version>1.1</version>
|
||||||
|
<packaging>jar</packaging>
|
||||||
|
</project>
|
||||||
Binary file not shown.
|
|
@ -0,0 +1,8 @@
|
||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
<groupId>com.redis</groupId>
|
||||||
|
<artifactId>redisclient</artifactId>
|
||||||
|
<version>2.8.0.Beta1-1.2</version>
|
||||||
|
<packaging>jar</packaging>
|
||||||
|
</project>
|
||||||
Binary file not shown.
|
|
@ -0,0 +1,8 @@
|
||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
<groupId>net.lag</groupId>
|
||||||
|
<artifactId>configgy</artifactId>
|
||||||
|
<version>2.8.0.Beta1-1.5-SNAPSHOT</version>
|
||||||
|
<packaging>jar</packaging>
|
||||||
|
</project>
|
||||||
Binary file not shown.
|
|
@ -3,6 +3,6 @@
|
||||||
<modelVersion>4.0.0</modelVersion>
|
<modelVersion>4.0.0</modelVersion>
|
||||||
<groupId>sbinary</groupId>
|
<groupId>sbinary</groupId>
|
||||||
<artifactId>sbinary</artifactId>
|
<artifactId>sbinary</artifactId>
|
||||||
<version>0.3</version>
|
<version>2.8.0.Beta1-2.8.0.Beta1-0.3.1-SNAPSHOT</version>
|
||||||
<packaging>jar</packaging>
|
<packaging>jar</packaging>
|
||||||
</project>
|
</project>
|
||||||
Binary file not shown.
Binary file not shown.
|
|
@ -0,0 +1,8 @@
|
||||||
|
<?xml version="1.0" encoding="UTF-8"?>
|
||||||
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
<groupId>sjson.json</groupId>
|
||||||
|
<artifactId>sjson</artifactId>
|
||||||
|
<version>0.5-SNAPSHOT-2.8.Beta1</version>
|
||||||
|
<packaging>jar</packaging>
|
||||||
|
</project>
|
||||||
|
|
@ -1,7 +1,7 @@
|
||||||
project.organization=se.scalablesolutions.akka
|
project.organization=se.scalablesolutions.akka
|
||||||
project.name=akka
|
project.name=akka
|
||||||
project.version=0.7
|
project.version=0.8
|
||||||
scala.version=2.7.7
|
scala.version=2.8.0.Beta1
|
||||||
sbt.version=0.7.1
|
sbt.version=0.7.1
|
||||||
def.scala.version=2.7.7
|
def.scala.version=2.7.7
|
||||||
build.scala.versions=2.7.7
|
build.scala.versions=2.8.0.Beta1
|
||||||
|
|
|
||||||
|
|
@ -35,16 +35,13 @@
|
||||||
-------------------------------------------------------------------------------*/
|
-------------------------------------------------------------------------------*/
|
||||||
|
|
||||||
import sbt._
|
import sbt._
|
||||||
import java.io.File
|
import sbt.CompileOrder._
|
||||||
|
import scala.Array
|
||||||
import java.util.jar.Attributes
|
import java.util.jar.Attributes
|
||||||
|
import java.util.jar.Attributes.Name._
|
||||||
|
import java.io.File
|
||||||
|
|
||||||
class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
|
abstract class AkkaDefaults(info: ProjectInfo) extends DefaultProject(info) with AutoCompilerPlugins {
|
||||||
|
|
||||||
// ------------------------------------------------------------
|
|
||||||
// project versions
|
|
||||||
val JERSEY_VERSION = "1.1.5"
|
|
||||||
val ATMO_VERSION = "0.5.4"
|
|
||||||
val CASSANDRA_VERSION = "0.5.0"
|
|
||||||
|
|
||||||
// ------------------------------------------------------------
|
// ------------------------------------------------------------
|
||||||
lazy val akkaHome = {
|
lazy val akkaHome = {
|
||||||
|
|
@ -53,25 +50,172 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
|
||||||
"You need to set the $AKKA_HOME environment variable to the root of the Akka distribution")
|
"You need to set the $AKKA_HOME environment variable to the root of the Akka distribution")
|
||||||
home
|
home
|
||||||
}
|
}
|
||||||
|
val encodingUtf8 = List("-encoding", "UTF-8")
|
||||||
|
|
||||||
lazy val deployPath = Path.fromFile(new java.io.File(akkaHome + "/deploy"))
|
lazy val deployPath = Path.fromFile(new java.io.File(akkaHome + "/deploy"))
|
||||||
lazy val distPath = Path.fromFile(new java.io.File(akkaHome + "/dist"))
|
lazy val distPath = Path.fromFile(new java.io.File(akkaHome + "/dist"))
|
||||||
|
|
||||||
|
override def javaCompileOptions = JavaCompileOption("-Xlint:unchecked") :: super.javaCompileOptions.toList
|
||||||
|
|
||||||
|
def distName = "%s_%s-%s.zip".format(name, buildScalaVersion, version)
|
||||||
|
|
||||||
|
// ------------------------------------------------------------
|
||||||
|
// publishing
|
||||||
|
override def managedStyle = ManagedStyle.Maven
|
||||||
|
def publishTo = Resolver.file("maven-local", Path.userHome / ".m2" / "repository" asFile)
|
||||||
|
|
||||||
|
// Credentials(Path.userHome / ".akka_publish_credentials", log)
|
||||||
|
|
||||||
|
override def documentOptions = encodingUtf8.map(SimpleDocOption(_))
|
||||||
|
override def packageDocsJar = defaultJarPath("-doc.jar")
|
||||||
|
override def packageSrcJar= defaultJarPath("-src.jar")
|
||||||
|
override def packageToPublishActions = super.packageToPublishActions ++ Seq(packageDocs, packageSrc)
|
||||||
|
|
||||||
|
override def pomExtra =
|
||||||
|
<inceptionYear>2009</inceptionYear>
|
||||||
|
<url>http://akkasource.org</url>
|
||||||
|
<organization>
|
||||||
|
<name>Scalable Solutions AB</name>
|
||||||
|
<url>http://scalablesolutions.se</url>
|
||||||
|
</organization>
|
||||||
|
<licenses>
|
||||||
|
<license>
|
||||||
|
<name>Apache 2</name>
|
||||||
|
<url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
|
||||||
|
<distribution>repo</distribution>
|
||||||
|
</license>
|
||||||
|
</licenses>
|
||||||
|
|
||||||
|
// ------------------------------------------------------------
|
||||||
|
// create executable jar
|
||||||
|
override def mainClass = Some("se.scalablesolutions.akka.kernel.Main")
|
||||||
|
|
||||||
|
override def packageOptions =
|
||||||
|
manifestClassPath.map(cp => ManifestAttributes(
|
||||||
|
(Attributes.Name.CLASS_PATH, cp),
|
||||||
|
(IMPLEMENTATION_TITLE, "Akka"),
|
||||||
|
(IMPLEMENTATION_URL, "http://akkasource.org"),
|
||||||
|
(IMPLEMENTATION_VENDOR, "The Akka Project")
|
||||||
|
)).toList :::
|
||||||
|
getMainClass(false).map(MainClass(_)).toList
|
||||||
|
|
||||||
|
// create a manifest with all akka jars and dependency jars on classpath
|
||||||
|
override def manifestClassPath = Some(allArtifacts.getFiles
|
||||||
|
.filter(_.getName.endsWith(".jar"))
|
||||||
|
.map("lib_managed/scala_%s/compile/".format(buildScalaVersion) + _.getName)
|
||||||
|
.mkString(" ") +
|
||||||
|
" scala-library.jar" +
|
||||||
|
" dist/akka-util_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-util-java_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-core_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-cluster-shoal_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-cluster-jgroups_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-rest_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-comet_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-camel_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-security_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-amqp_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-patterns_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-persistence-common_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-persistence-redis_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-persistence-mongo_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-persistence-cassandra_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-kernel_%s-%s.jar".format(buildScalaVersion, version) +
|
||||||
|
" dist/akka-spring_%s-%s.jar".format(buildScalaVersion, version)
|
||||||
|
)
|
||||||
|
|
||||||
|
// ------------------------------------------------------------
|
||||||
|
// helper functions
|
||||||
|
def removeDupEntries(paths: PathFinder) =
|
||||||
|
Path.lazyPathFinder {
|
||||||
|
val mapped = paths.get map { p => (p.relativePath, p) }
|
||||||
|
(Map() ++ mapped).values.toList
|
||||||
|
}
|
||||||
|
|
||||||
|
def allArtifacts = {
|
||||||
|
Path.fromFile(buildScalaInstance.libraryJar) +++
|
||||||
|
(removeDupEntries(runClasspath filter ClasspathUtilities.isArchive) +++
|
||||||
|
((outputPath ##) / defaultJarName) +++
|
||||||
|
mainResources +++
|
||||||
|
mainDependencies.scalaJars +++
|
||||||
|
descendents(info.projectPath, "*.conf") +++
|
||||||
|
descendents(info.projectPath / "dist", "*.jar") +++
|
||||||
|
descendents(info.projectPath / "deploy", "*.jar") +++
|
||||||
|
descendents(path("lib") ##, "*.jar") +++
|
||||||
|
descendents(configurationPath(Configurations.Compile) ##, "*.jar"))
|
||||||
|
.filter(jar => // remove redundant libs
|
||||||
|
!jar.toString.endsWith("stax-api-1.0.1.jar"))
|
||||||
|
}
|
||||||
|
|
||||||
|
def deployTask(info: ProjectInfo, toDir: Path, genJar: Boolean, genDocs: Boolean, genSource: Boolean) = task {
|
||||||
|
val projectPath = info.projectPath.toString
|
||||||
|
val moduleName = projectPath.substring(
|
||||||
|
projectPath.lastIndexOf(System.getProperty("file.separator")) + 1, projectPath.length)
|
||||||
|
|
||||||
|
// FIXME need to find out a way to grab these paths from the sbt system
|
||||||
|
|
||||||
|
// binary
|
||||||
|
if (genJar) {
|
||||||
|
val JAR_FILE_NAME = moduleName + "_%s-%s.jar".format(buildScalaVersion, version)
|
||||||
|
val JAR_FILE_PATH = projectPath + "/target/scala_%s/".format(buildScalaVersion) + JAR_FILE_NAME
|
||||||
|
val fromJar = Path.fromFile(new java.io.File(JAR_FILE_PATH))
|
||||||
|
val toJar = Path.fromFile(new java.io.File(toDir + "/" + JAR_FILE_NAME))
|
||||||
|
log.info("Deploying bits " + toJar)
|
||||||
|
FileUtilities.copyFile(fromJar, toJar, log)
|
||||||
|
}
|
||||||
|
|
||||||
|
// docs
|
||||||
|
if (genDocs) {
|
||||||
|
val DOC_FILE_NAME = moduleName + "_%s-%s-%s.jar".format(buildScalaVersion, version, "doc")
|
||||||
|
val DOC_FILE_PATH = projectPath + "/target/scala_%s/".format(buildScalaVersion) + DOC_FILE_NAME
|
||||||
|
val fromDoc = Path.fromFile(new java.io.File(DOC_FILE_PATH))
|
||||||
|
val toDoc = Path.fromFile(new java.io.File(toDir + "/" + DOC_FILE_NAME))
|
||||||
|
log.info("Deploying docs " + toDoc)
|
||||||
|
FileUtilities.copyFile(fromDoc, toDoc, log)
|
||||||
|
}
|
||||||
|
|
||||||
|
// sources
|
||||||
|
if (genSource) {
|
||||||
|
val SRC_FILE_NAME = moduleName + "_%s-%s-%s.jar".format(buildScalaVersion, version, "src")
|
||||||
|
val SRC_FILE_PATH = projectPath + "/target/scala_%s/".format(buildScalaVersion) + SRC_FILE_NAME
|
||||||
|
val fromSrc = Path.fromFile(new java.io.File(SRC_FILE_PATH))
|
||||||
|
val toSrc = Path.fromFile(new java.io.File(toDir + "/" + SRC_FILE_NAME))
|
||||||
|
log.info("Deploying sources " + toSrc)
|
||||||
|
FileUtilities.copyFile(fromSrc, toSrc, log)
|
||||||
|
}
|
||||||
|
None
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class AkkaParent(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
|
|
||||||
|
// These lines need to be here instead of in AkkaDefaults to be able to resolve project.name in build.properties
|
||||||
|
val sourceArtifact = Artifact(artifactID, "src", "jar", Some("src"), Nil, None)
|
||||||
|
val docsArtifact = Artifact(artifactID, "docs", "jar", Some("doc"), Nil, None)
|
||||||
|
|
||||||
lazy val dist = zipTask(allArtifacts, "dist", distName) dependsOn (`package`) describedAs("Zips up the distribution.")
|
lazy val dist = zipTask(allArtifacts, "dist", distName) dependsOn (`package`) describedAs("Zips up the distribution.")
|
||||||
|
|
||||||
def distName = "%s_%s-%s.zip".format(name, defScalaVersion.value, version)
|
// ------------------------------------------------------------
|
||||||
|
// project versions
|
||||||
|
val JERSEY_VERSION = "1.1.5"
|
||||||
|
val ATMO_VERSION = "0.5.4"
|
||||||
|
val CASSANDRA_VERSION = "0.5.0"
|
||||||
|
val LIFT_VERSION = "2.0-scala280-SNAPSHOT"
|
||||||
|
val SCALATEST_VERSION = "1.0.1-for-scala-2.8.0.Beta1-with-test-interfaces-0.3-SNAPSHOT"
|
||||||
|
|
||||||
// ------------------------------------------------------------
|
// ------------------------------------------------------------
|
||||||
// repositories
|
// repositories
|
||||||
val embeddedrepo = "embedded repo" at new File(akkaHome, "embedded-repo").toURI.toString
|
val embeddedrepo = "embedded repo" at new File(akkaHome, "embedded-repo").toURI.toString
|
||||||
val sunjdmk = "sunjdmk" at "http://wp5.e-taxonomy.eu/cdmlib/mavenrepo"
|
val sunjdmk = "sunjdmk" at "http://wp5.e-taxonomy.eu/cdmlib/mavenrepo"
|
||||||
val databinder = "DataBinder" at "http://databinder.net/repo"
|
val databinder = "DataBinder" at "http://databinder.net/repo"
|
||||||
val configgy = "Configgy" at "http://www.lag.net/repo"
|
// val configgy = "Configgy" at "http://www.lag.net/repo"
|
||||||
val codehaus = "Codehaus" at "http://repository.codehaus.org"
|
val codehaus = "Codehaus" at "http://repository.codehaus.org"
|
||||||
val codehaus_snapshots = "Codehaus Snapshots" at "http://snapshots.repository.codehaus.org"
|
val codehaus_snapshots = "Codehaus Snapshots" at "http://snapshots.repository.codehaus.org"
|
||||||
val jboss = "jBoss" at "http://repository.jboss.org/maven2"
|
val jboss = "jBoss" at "http://repository.jboss.org/maven2"
|
||||||
val guiceyfruit = "GuiceyFruit" at "http://guiceyfruit.googlecode.com/svn/repo/releases/"
|
val guiceyfruit = "GuiceyFruit" at "http://guiceyfruit.googlecode.com/svn/repo/releases/"
|
||||||
val google = "google" at "http://google-maven-repository.googlecode.com/svn/repository"
|
val google = "google" at "http://google-maven-repository.googlecode.com/svn/repository"
|
||||||
val m2 = "m2" at "http://download.java.net/maven/2"
|
val m2 = "m2" at "http://download.java.net/maven/2"
|
||||||
|
val scala_tools_snapshots = "scala-tools snapshots" at "http://scala-tools.org/repo-snapshots"
|
||||||
|
|
||||||
// ------------------------------------------------------------
|
// ------------------------------------------------------------
|
||||||
// project defintions
|
// project defintions
|
||||||
|
|
@ -96,108 +240,47 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
|
||||||
|
|
||||||
// examples
|
// examples
|
||||||
lazy val akka_samples = project("akka-samples", "akka-samples", new AkkaSamplesParentProject(_))
|
lazy val akka_samples = project("akka-samples", "akka-samples", new AkkaSamplesParentProject(_))
|
||||||
|
|
||||||
// ------------------------------------------------------------
|
|
||||||
// create executable jar
|
|
||||||
override def mainClass = Some("se.scalablesolutions.akka.kernel.Main")
|
|
||||||
|
|
||||||
override def packageOptions =
|
|
||||||
manifestClassPath.map(cp => ManifestAttributes((Attributes.Name.CLASS_PATH, cp))).toList :::
|
|
||||||
getMainClass(false).map(MainClass(_)).toList
|
|
||||||
|
|
||||||
// create a manifest with all akka jars and dependency jars on classpath
|
|
||||||
override def manifestClassPath = Some(allArtifacts.getFiles
|
|
||||||
.filter(_.getName.endsWith(".jar"))
|
|
||||||
.map("lib_managed/scala_%s/compile/".format(defScalaVersion.value) + _.getName)
|
|
||||||
.mkString(" ") +
|
|
||||||
" dist/akka-util_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-util-java_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-core_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-cluster-shoal_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-cluster-jgroups_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-rest_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-comet_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-camel_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-security_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-amqp_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-patterns_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-persistence-common_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-persistence-redis_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-persistence-mongo_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-persistence-cassandra_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-kernel_%s-%s.jar".format(defScalaVersion.value, version) +
|
|
||||||
" dist/akka-spring_%s-%s.jar".format(defScalaVersion.value, version)
|
|
||||||
)
|
|
||||||
|
|
||||||
// ------------------------------------------------------------
|
|
||||||
// publishing
|
|
||||||
override def managedStyle = ManagedStyle.Maven
|
|
||||||
val publishTo = Resolver.file("maven-local", Path.userHome / ".m2" / "repository" asFile)
|
|
||||||
|
|
||||||
// Credentials(Path.userHome / ".akka_publish_credentials", log)
|
|
||||||
val sourceArtifact = Artifact(artifactID, "src", "jar", Some("sources"), Nil, None)
|
|
||||||
//val docsArtifact = Artifact(artifactID, "docs", "jar", Some("javadoc"), Nil, None)
|
|
||||||
|
|
||||||
override def packageDocsJar = defaultJarPath("-javadoc.jar")
|
|
||||||
override def packageSrcJar= defaultJarPath("-sources.jar")
|
|
||||||
override def packageToPublishActions = super.packageToPublishActions ++ Seq(packageDocs, packageSrc)
|
|
||||||
|
|
||||||
override def pomExtra =
|
|
||||||
<inceptionYear>2009</inceptionYear>
|
|
||||||
<url>http://akkasource.org</url>
|
|
||||||
<organization>
|
|
||||||
<name>Scalable Solutions AB</name>
|
|
||||||
<url>http://scalablesolutions.se</url>
|
|
||||||
</organization>
|
|
||||||
<licenses>
|
|
||||||
<license>
|
|
||||||
<name>Apache 2</name>
|
|
||||||
<url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
|
|
||||||
<distribution>repo</distribution>
|
|
||||||
</license>
|
|
||||||
</licenses>
|
|
||||||
|
|
||||||
// ------------------------------------------------------------
|
// ------------------------------------------------------------
|
||||||
// subprojects
|
// subprojects
|
||||||
class AkkaCoreProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaCoreProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val netty = "org.jboss.netty" % "netty" % "3.2.0.BETA1" % "compile"
|
val netty = "org.jboss.netty" % "netty" % "3.2.0.BETA1" % "compile"
|
||||||
val commons_io = "commons-io" % "commons-io" % "1.4" % "compile"
|
val commons_io = "commons-io" % "commons-io" % "1.4" % "compile"
|
||||||
val dispatch_json = "net.databinder" % "dispatch-json_2.7.7" % "0.6.4" % "compile"
|
val dispatch_json = "net.databinder" % "dispatch-json_2.8.0.Beta1" % "0.6.6" % "compile"
|
||||||
val dispatch_htdisttp = "net.databinder" % "dispatch-http_2.7.7" % "0.6.4" % "compile"
|
val dispatch_htdisttp = "net.databinder" % "dispatch-http_2.8.0.Beta1" % "0.6.6" % "compile"
|
||||||
val sjson = "sjson.json" % "sjson" % "0.4" % "compile"
|
val sjson = "sjson.json" % "sjson" % "0.5-SNAPSHOT-2.8.Beta1" % "compile"
|
||||||
val sbinary = "sbinary" % "sbinary" % "0.3" % "compile"
|
val sbinary = "sbinary" % "sbinary" % "2.8.0.Beta1-2.8.0.Beta1-0.3.1-SNAPSHOT" % "compile"
|
||||||
val jackson = "org.codehaus.jackson" % "jackson-mapper-asl" % "1.2.1" % "compile"
|
val jackson = "org.codehaus.jackson" % "jackson-mapper-asl" % "1.2.1" % "compile"
|
||||||
val jackson_core = "org.codehaus.jackson" % "jackson-core-asl" % "1.2.1" % "compile"
|
val jackson_core = "org.codehaus.jackson" % "jackson-core-asl" % "1.2.1" % "compile"
|
||||||
val voldemort = "voldemort.store.compress" % "h2-lzf" % "1.0" % "compile"
|
val voldemort = "voldemort.store.compress" % "h2-lzf" % "1.0" % "compile"
|
||||||
val javautils = "org.scala-tools" % "javautils" % "2.7.4-0.1" % "compile"
|
|
||||||
val jsr166x = "jsr166x" % "jsr166x" % "1.0" % "compile"
|
val jsr166x = "jsr166x" % "jsr166x" % "1.0" % "compile"
|
||||||
// testing
|
// testing
|
||||||
val scalatest = "org.scalatest" % "scalatest" % "1.0" % "test"
|
val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "test"
|
||||||
val junit = "junit" % "junit" % "4.5" % "test"
|
val junit = "junit" % "junit" % "4.5" % "test"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaUtilProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaUtilProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val werkz = "org.codehaus.aspectwerkz" % "aspectwerkz-nodeps-jdk5" % "2.1" % "compile"
|
val werkz = "org.codehaus.aspectwerkz" % "aspectwerkz-nodeps-jdk5" % "2.1" % "compile"
|
||||||
val werkz_core = "org.codehaus.aspectwerkz" % "aspectwerkz-jdk5" % "2.1" % "compile"
|
val werkz_core = "org.codehaus.aspectwerkz" % "aspectwerkz-jdk5" % "2.1" % "compile"
|
||||||
val configgy = "net.lag" % "configgy" % "1.4.7" % "compile"
|
val configgy = "net.lag" % "configgy" % "2.8.0.Beta1-1.5-SNAPSHOT" % "compile"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaJavaUtilProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaJavaUtilProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val guicey = "org.guiceyfruit" % "guice-core" % "2.0-beta-4" % "compile"
|
val guicey = "org.guiceyfruit" % "guice-core" % "2.0-beta-4" % "compile"
|
||||||
val protobuf = "com.google.protobuf" % "protobuf-java" % "2.2.0" % "compile"
|
val protobuf = "com.google.protobuf" % "protobuf-java" % "2.2.0" % "compile"
|
||||||
val multiverse = "org.multiverse" % "multiverse-alpha" % "0.4" % "compile"
|
val multiverse = "org.multiverse" % "multiverse-alpha" % "0.4" % "compile"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaAMQPProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaAMQPProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val commons_io = "commons-io" % "commons-io" % "1.4" % "compile"
|
val commons_io = "commons-io" % "commons-io" % "1.4" % "compile"
|
||||||
val rabbit = "com.rabbitmq" % "amqp-client" % "1.7.2"
|
val rabbit = "com.rabbitmq" % "amqp-client" % "1.7.2" % "compile"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaRestProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaRestProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val jackson_core_asl = "org.codehaus.jackson" % "jackson-core-asl" % "1.2.1" % "compile"
|
val jackson_core_asl = "org.codehaus.jackson" % "jackson-core-asl" % "1.2.1" % "compile"
|
||||||
val stax_api = "javax.xml.stream" % "stax-api" % "1.0-2" % "compile"
|
val stax_api = "javax.xml.stream" % "stax-api" % "1.0-2" % "compile"
|
||||||
val servlet = "javax.servlet" % "servlet-api" % "2.5" % "compile"
|
val servlet = "javax.servlet" % "servlet-api" % "2.5" % "compile"
|
||||||
|
|
@ -206,62 +289,63 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
|
||||||
val jersey_json = "com.sun.jersey" % "jersey-json" % JERSEY_VERSION % "compile"
|
val jersey_json = "com.sun.jersey" % "jersey-json" % JERSEY_VERSION % "compile"
|
||||||
val jersey_contrib = "com.sun.jersey.contribs" % "jersey-scala" % JERSEY_VERSION % "compile"
|
val jersey_contrib = "com.sun.jersey.contribs" % "jersey-scala" % JERSEY_VERSION % "compile"
|
||||||
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1" % "compile"
|
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1" % "compile"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaCometProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaCometProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val grizzly = "com.sun.grizzly" % "grizzly-comet-webserver" % "1.9.18-i" % "compile"
|
val grizzly = "com.sun.grizzly" % "grizzly-comet-webserver" % "1.9.18-i" % "compile"
|
||||||
val servlet = "javax.servlet" % "servlet-api" % "2.5" % "compile"
|
val servlet = "javax.servlet" % "servlet-api" % "2.5" % "compile"
|
||||||
val atmo = "org.atmosphere" % "atmosphere-annotations" % ATMO_VERSION % "compile"
|
val atmo = "org.atmosphere" % "atmosphere-annotations" % ATMO_VERSION % "compile"
|
||||||
val atmo_jersey = "org.atmosphere" % "atmosphere-jersey" % ATMO_VERSION % "compile"
|
val atmo_jersey = "org.atmosphere" % "atmosphere-jersey" % ATMO_VERSION % "compile"
|
||||||
val atmo_runtime = "org.atmosphere" % "atmosphere-runtime" % ATMO_VERSION % "compile"
|
val atmo_runtime = "org.atmosphere" % "atmosphere-runtime" % ATMO_VERSION % "compile"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaCamelProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaCamelProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val camel_core = "org.apache.camel" % "camel-core" % "2.2.0" % "compile"
|
val camel_core = "org.apache.camel" % "camel-core" % "2.2.0" % "compile"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaPatternsProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaPatternsProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
// testing
|
// testing
|
||||||
val scalatest = "org.scalatest" % "scalatest" % "1.0" % "test"
|
val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "test"
|
||||||
val junit = "junit" % "junit" % "4.5" % "test"
|
val junit = "junit" % "junit" % "4.5" % "test"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaSecurityProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaSecurityProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val commons_logging = "commons-logging" % "commons-logging" % "1.1.1" % "compile"
|
val commons_logging = "commons-logging" % "commons-logging" % "1.1.1" % "compile"
|
||||||
val annotation = "javax.annotation" % "jsr250-api" % "1.0"
|
val annotation = "javax.annotation" % "jsr250-api" % "1.0" % "compile"
|
||||||
val jersey_server = "com.sun.jersey" % "jersey-server" % JERSEY_VERSION % "compile"
|
val jersey_server = "com.sun.jersey" % "jersey-server" % JERSEY_VERSION % "compile"
|
||||||
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1" % "compile"
|
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1" % "compile"
|
||||||
val lift_util = "net.liftweb" % "lift-util" % "1.1-M6" % "compile"
|
val lift_common = "net.liftweb" % "lift-common" % LIFT_VERSION % "compile"
|
||||||
|
val lift_util = "net.liftweb" % "lift-util" % LIFT_VERSION % "compile"
|
||||||
// testing
|
// testing
|
||||||
val scalatest = "org.scalatest" % "scalatest" % "1.0" % "test"
|
val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "test"
|
||||||
val junit = "junit" % "junit" % "4.5" % "test"
|
val junit = "junit" % "junit" % "4.5" % "test"
|
||||||
val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test"
|
val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaPersistenceCommonProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaPersistenceCommonProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val thrift = "com.facebook" % "thrift" % "1.0" % "compile"
|
val thrift = "com.facebook" % "thrift" % "1.0" % "compile"
|
||||||
val commons_pool = "commons-pool" % "commons-pool" % "1.5.1" % "compile"
|
val commons_pool = "commons-pool" % "commons-pool" % "1.5.4" % "compile"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaRedisProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaRedisProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val redis = "com.redis" % "redisclient" % "1.2" % "compile"
|
val redis = "com.redis" % "redisclient" % "2.8.0.Beta1-1.2" % "compile"
|
||||||
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
|
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaMongoProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaMongoProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val mongo = "org.mongodb" % "mongo-java-driver" % "1.1" % "compile"
|
val mongo = "org.mongodb" % "mongo-java-driver" % "1.1" % "compile"
|
||||||
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
|
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaCassandraProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaCassandraProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val cassandra = "org.apache.cassandra" % "cassandra" % CASSANDRA_VERSION % "compile"
|
val cassandra = "org.apache.cassandra" % "cassandra" % CASSANDRA_VERSION % "compile"
|
||||||
val high_scale = "org.apache.cassandra" % "high-scale-lib" % CASSANDRA_VERSION % "test"
|
val high_scale = "org.apache.cassandra" % "high-scale-lib" % CASSANDRA_VERSION % "test"
|
||||||
val cassandra_clhm = "org.apache.cassandra" % "clhm-production" % CASSANDRA_VERSION % "test"
|
val cassandra_clhm = "org.apache.cassandra" % "clhm-production" % CASSANDRA_VERSION % "test"
|
||||||
|
|
@ -271,7 +355,7 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
|
||||||
val slf4j_log4j = "org.slf4j" % "slf4j-log4j12" % "1.5.8" % "test"
|
val slf4j_log4j = "org.slf4j" % "slf4j-log4j12" % "1.5.8" % "test"
|
||||||
val log4j = "log4j" % "log4j" % "1.2.15" % "test"
|
val log4j = "log4j" % "log4j" % "1.2.15" % "test"
|
||||||
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
|
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaPersistenceParentProject(info: ProjectInfo) extends ParentProject(info) {
|
class AkkaPersistenceParentProject(info: ProjectInfo) extends ParentProject(info) {
|
||||||
|
|
@ -285,15 +369,15 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
|
||||||
new AkkaCassandraProject(_), akka_persistence_common)
|
new AkkaCassandraProject(_), akka_persistence_common)
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaJgroupsProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaJgroupsProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val jgroups = "jgroups" % "jgroups" % "2.8.0.CR7" % "compile"
|
val jgroups = "jgroups" % "jgroups" % "2.8.0.CR7" % "compile"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaShoalProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaShoalProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val shoal = "shoal-jxta" % "shoal" % "1.1-20090818" % "compile"
|
val shoal = "shoal-jxta" % "shoal" % "1.1-20090818" % "compile"
|
||||||
val shoal_extra = "shoal-jxta" % "jxta" % "1.1-20090818" % "compile"
|
val shoal_extra = "shoal-jxta" % "jxta" % "1.1-20090818" % "compile"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaClusterParentProject(info: ProjectInfo) extends ParentProject(info) {
|
class AkkaClusterParentProject(info: ProjectInfo) extends ParentProject(info) {
|
||||||
|
|
@ -303,24 +387,25 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
|
||||||
new AkkaShoalProject(_), akka_core)
|
new AkkaShoalProject(_), akka_core)
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaKernelProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaKernelProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaSpringProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaSpringProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val spring_beans = "org.springframework" % "spring-beans" % "3.0.1.RELEASE"
|
val spring_beans = "org.springframework" % "spring-beans" % "3.0.1.RELEASE" % "compile"
|
||||||
val spring_context = "org.springframework" % "spring-context" % "3.0.1.RELEASE"
|
val spring_context = "org.springframework" % "spring-context" % "3.0.1.RELEASE" % "compile"
|
||||||
|
|
||||||
// testing
|
// testing
|
||||||
val scalatest = "org.scalatest" % "scalatest" % "1.0" % "test"
|
val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "test"
|
||||||
val junit = "junit" % "junit" % "4.5" % "test"
|
val junit = "junit" % "junit" % "4.5" % "test"
|
||||||
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
// examples
|
// examples
|
||||||
class AkkaFunTestProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaFunTestProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val jackson_core_asl = "org.codehaus.jackson" % "jackson-core-asl" % "1.2.1" % "compile"
|
val jackson_core_asl = "org.codehaus.jackson" % "jackson-core-asl" % "1.2.1" % "compile"
|
||||||
val stax_api = "javax.xml.stream" % "stax-api" % "1.0-2" % "compile"
|
val stax_api = "javax.xml.stream" % "stax-api" % "1.0-2" % "compile"
|
||||||
val protobuf = "com.google.protobuf" % "protobuf-java" % "2.2.0"
|
val protobuf = "com.google.protobuf" % "protobuf-java" % "2.2.0" % "compile"
|
||||||
val grizzly = "com.sun.grizzly" % "grizzly-comet-webserver" % "1.9.18-i" % "compile"
|
val grizzly = "com.sun.grizzly" % "grizzly-comet-webserver" % "1.9.18-i" % "compile"
|
||||||
val jersey_server = "com.sun.jersey" % "jersey-server" % JERSEY_VERSION % "compile"
|
val jersey_server = "com.sun.jersey" % "jersey-server" % JERSEY_VERSION % "compile"
|
||||||
val jersey_json = "com.sun.jersey" % "jersey-json" % JERSEY_VERSION % "compile"
|
val jersey_json = "com.sun.jersey" % "jersey-json" % JERSEY_VERSION % "compile"
|
||||||
|
|
@ -330,44 +415,44 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
|
||||||
val jmock = "org.jmock" % "jmock" % "2.4.0" % "test"
|
val jmock = "org.jmock" % "jmock" % "2.4.0" % "test"
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaSampleChatProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaSampleChatProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
lazy val dist = deployTask(info, deployPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, deployPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaSampleLiftProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaSampleLiftProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val commons_logging = "commons-logging" % "commons-logging" % "1.1.1" % "compile"
|
val commons_logging = "commons-logging" % "commons-logging" % "1.1.1" % "compile"
|
||||||
val lift = "net.liftweb" % "lift-webkit" % "1.1-M6" % "compile"
|
val lift = "net.liftweb" % "lift-webkit" % LIFT_VERSION % "compile"
|
||||||
val lift_util = "net.liftweb" % "lift-util" % "1.1-M6" % "compile"
|
val lift_util = "net.liftweb" % "lift-util" % LIFT_VERSION % "compile"
|
||||||
val servlet = "javax.servlet" % "servlet-api" % "2.5" % "compile"
|
val servlet = "javax.servlet" % "servlet-api" % "2.5" % "compile"
|
||||||
// testing
|
// testing
|
||||||
val jetty = "org.mortbay.jetty" % "jetty" % "6.1.22" % "test"
|
val jetty = "org.mortbay.jetty" % "jetty" % "6.1.22" % "test"
|
||||||
val junit = "junit" % "junit" % "4.5" % "test"
|
val junit = "junit" % "junit" % "4.5" % "test"
|
||||||
lazy val dist = deployTask(info, deployPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, deployPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaSampleRestJavaProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaSampleRestJavaProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
lazy val dist = deployTask(info, deployPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, deployPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaSampleRestScalaProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaSampleRestScalaProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1.1" % "compile"
|
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1.1" % "compile"
|
||||||
lazy val dist = deployTask(info, deployPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, deployPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaSampleCamelProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaSampleCamelProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val commons_codec = "commons-codec" % "commons-codec" % "1.3" % "compile"
|
val commons_codec = "commons-codec" % "commons-codec" % "1.3" % "compile"
|
||||||
val spring_jms = "org.springframework" % "spring-jms" % "3.0.1.RELEASE"
|
val spring_jms = "org.springframework" % "spring-jms" % "3.0.1.RELEASE" % "compile"
|
||||||
val camel_jetty = "org.apache.camel" % "camel-jetty" % "2.2.0" % "compile"
|
val camel_jetty = "org.apache.camel" % "camel-jetty" % "2.2.0" % "compile"
|
||||||
val camel_jms = "org.apache.camel" % "camel-jms" % "2.2.0" % "compile"
|
val camel_jms = "org.apache.camel" % "camel-jms" % "2.2.0" % "compile"
|
||||||
val activemq_core = "org.apache.activemq" % "activemq-core" % "5.3.0" % "compile"
|
val activemq_core = "org.apache.activemq" % "activemq-core" % "5.3.0" % "compile"
|
||||||
lazy val dist = deployTask(info, deployPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, deployPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaSampleSecurityProject(info: ProjectInfo) extends DefaultProject(info) {
|
class AkkaSampleSecurityProject(info: ProjectInfo) extends AkkaDefaults(info) {
|
||||||
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1.1" % "compile"
|
val jsr311 = "javax.ws.rs" % "jsr311-api" % "1.1.1" % "compile"
|
||||||
val jsr250 = "javax.annotation" % "jsr250-api" % "1.0"
|
val jsr250 = "javax.annotation" % "jsr250-api" % "1.0" % "compile"
|
||||||
val commons_codec = "commons-codec" % "commons-codec" % "1.3" % "compile"
|
val commons_codec = "commons-codec" % "commons-codec" % "1.3" % "compile"
|
||||||
lazy val dist = deployTask(info, deployPath) dependsOn(`package`) describedAs("Deploying")
|
lazy val dist = deployTask(info, deployPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
|
||||||
}
|
}
|
||||||
|
|
||||||
class AkkaSamplesParentProject(info: ProjectInfo) extends ParentProject(info) {
|
class AkkaSamplesParentProject(info: ProjectInfo) extends ParentProject(info) {
|
||||||
|
|
@ -382,44 +467,6 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
|
||||||
lazy val akka_sample_camel = project("akka-sample-camel", "akka-sample-camel",
|
lazy val akka_sample_camel = project("akka-sample-camel", "akka-sample-camel",
|
||||||
new AkkaSampleCamelProject(_), akka_kernel)
|
new AkkaSampleCamelProject(_), akka_kernel)
|
||||||
lazy val akka_sample_security = project("akka-sample-security", "akka-sample-security",
|
lazy val akka_sample_security = project("akka-sample-security", "akka-sample-security",
|
||||||
new AkkaSampleSecurityProject(_), akka_kernel)
|
new AkkaSampleSecurityProject(_), akka_kernel)
|
||||||
}
|
|
||||||
|
|
||||||
// ------------------------------------------------------------
|
|
||||||
// helper functions
|
|
||||||
def removeDupEntries(paths: PathFinder) =
|
|
||||||
Path.lazyPathFinder {
|
|
||||||
val mapped = paths.get map { p => (p.relativePath, p) }
|
|
||||||
(Map() ++ mapped).values.toList
|
|
||||||
}
|
|
||||||
|
|
||||||
def allArtifacts = {
|
|
||||||
(removeDupEntries(runClasspath filter ClasspathUtilities.isArchive) +++
|
|
||||||
((outputPath ##) / defaultJarName) +++
|
|
||||||
mainResources +++
|
|
||||||
mainDependencies.scalaJars +++
|
|
||||||
descendents(info.projectPath, "*.conf") +++
|
|
||||||
descendents(info.projectPath / "dist", "*.jar") +++
|
|
||||||
descendents(info.projectPath / "deploy", "*.jar") +++
|
|
||||||
descendents(path("lib") ##, "*.jar") +++
|
|
||||||
descendents(configurationPath(Configurations.Compile) ##, "*.jar"))
|
|
||||||
.filter(jar => // remove redundant libs
|
|
||||||
!jar.toString.endsWith("stax-api-1.0.1.jar") &&
|
|
||||||
!jar.toString.endsWith("scala-library-2.7.5.jar") &&
|
|
||||||
!jar.toString.endsWith("scala-library-2.7.6.jar"))
|
|
||||||
}
|
|
||||||
|
|
||||||
def deployTask(info: ProjectInfo, toDir: Path) = task {
|
|
||||||
val projectPath = info.projectPath.toString
|
|
||||||
val moduleName = projectPath.substring(
|
|
||||||
projectPath.lastIndexOf(System.getProperty("file.separator")) + 1, projectPath.length)
|
|
||||||
// FIXME need to find out a way to grab these paths from the sbt system
|
|
||||||
val JAR_FILE_NAME = moduleName + "_%s-%s.jar".format(defScalaVersion.value, version)
|
|
||||||
val JAR_FILE_PATH = projectPath + "/target/scala_%s/".format(defScalaVersion.value) + JAR_FILE_NAME
|
|
||||||
|
|
||||||
val from = Path.fromFile(new java.io.File(JAR_FILE_PATH))
|
|
||||||
val to = Path.fromFile(new java.io.File(toDir + "/" + JAR_FILE_NAME))
|
|
||||||
log.info("Deploying " + to)
|
|
||||||
FileUtilities.copyFile(from, to, log)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,7 +1,7 @@
|
||||||
#!/bin/bash
|
#!/bin/bash
|
||||||
cd $AKKA_HOME
|
cd $AKKA_HOME
|
||||||
VERSION=akka_2.7.7-0.7
|
VERSION=akka_2.8.0.Beta1-0.8
|
||||||
TARGET_DIR=dist/$1
|
TARGET_DIR=dist/$VERSION/$1
|
||||||
shift 1
|
shift 1
|
||||||
VMARGS=$@
|
VMARGS=$@
|
||||||
|
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue