merged with upstream

This commit is contained in:
Jonas Bonér 2010-04-01 08:10:52 +02:00
commit 2f32b850f6
78 changed files with 827 additions and 716 deletions

View file

@ -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]

View file

@ -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&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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",

View file

@ -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
} }
/** /**

View file

@ -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)
} }
} }

View file

@ -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)

View file

@ -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
*/ */

View file

@ -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")
} }

View file

@ -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
} }
} }

View file

@ -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)
} }
} }

View file

@ -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))
} }
} }

View file

@ -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)

View file

@ -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 }
}
} }
} }

View file

@ -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
*/

View file

@ -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 &&

View file

@ -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))
} }
/** /**

View file

@ -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&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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&#233;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)
} }
} }

View file

@ -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)

View file

@ -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
} }

View file

@ -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)

View file

@ -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&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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.

View file

@ -12,7 +12,7 @@ import net.lag.configgy.{Configgy, ParseException}
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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")

View file

@ -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 {

View file

@ -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)
} }
/** /**

View file

@ -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&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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],

View file

@ -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))

View file

@ -244,7 +244,7 @@ class RemoteServerPipelineFactory(
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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 {

View file

@ -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)
} }

View file

@ -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}

View file

@ -164,7 +164,9 @@ object Serializer {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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)

View file

@ -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() }

View file

@ -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

View file

@ -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
} }

View file

@ -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&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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]]

View file

@ -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) {

View file

@ -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
}
})
} }

View file

@ -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

View file

@ -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

View file

@ -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)

View file

@ -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

View file

@ -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

View file

@ -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
} }
} }

View file

@ -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;
}
} }

View file

@ -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

View file

@ -24,8 +24,7 @@ import org.apache.thrift.protocol._
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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)

View file

@ -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)

View file

@ -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] {

View file

@ -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&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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 = {

View file

@ -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)
} }
} }

View file

@ -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
} }

View file

@ -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) }
} }

View file

@ -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)
} }
} }

View file

@ -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)
} }
} }

View file

@ -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")
} }
} }

View file

@ -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)

View file

@ -18,12 +18,12 @@ import com.sun.jersey.spi.container.WebApplication
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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(","))

View file

@ -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
} }
/** /**

View file

@ -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
} }

View file

@ -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)
} }
} }
*/

View file

@ -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

View file

@ -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 {

View file

@ -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

View file

@ -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

View file

@ -17,7 +17,7 @@ import java.net.UnknownHostException
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;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)
} }
/** /**

View file

@ -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

View 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>

View 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>2.8.0.Beta1-1.2</version>
<packaging>jar</packaging>
</project>

View 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>net.lag</groupId>
<artifactId>configgy</artifactId>
<version>2.8.0.Beta1-1.5-SNAPSHOT</version>
<packaging>jar</packaging>
</project>

View file

@ -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>

View 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>sjson.json</groupId>
<artifactId>sjson</artifactId>
<version>0.5-SNAPSHOT-2.8.Beta1</version>
<packaging>jar</packaging>
</project>

View file

@ -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

View file

@ -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)
} }
} }

View file

@ -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=$@