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

@ -14,8 +14,6 @@ import se.scalablesolutions.akka.util.{HashCode, Logging}
import scala.collection.mutable.HashMap
import org.scala_tools.javautils.Imports._
import java.util.concurrent.ConcurrentHashMap
import java.util.{Timer, TimerTask}
import java.io.IOException
@ -79,6 +77,8 @@ object AMQP {
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class AMQPSupervisor extends Actor {
import scala.collection.JavaConversions._
private val connections = new ConcurrentHashMap[FaultTolerantConnectionActor, FaultTolerantConnectionActor]
faultHandler = Some(OneForOneStrategy(5, 5000))
@ -138,7 +138,7 @@ object AMQP {
}
override def shutdown = {
connections.values.asScala.foreach(_ ! Stop)
asMap(connections).valuesIterator.foreach(_ ! Stop)
exit
}
@ -361,9 +361,14 @@ object AMQP {
extends FaultTolerantConnectionActor {
consumer: Consumer =>
import scala.collection.JavaConversions._
faultHandler = Some(OneForOneStrategy(5, 5000))
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]
setupChannel
@ -395,7 +400,7 @@ object AMQP {
throw cause
case Stop =>
listeners.elements.toList.map(_._2).foreach(unregisterListener(_))
listeners.iterator.toList.map(_._2).foreach(unregisterListener(_))
disconnect
exit
@ -411,8 +416,8 @@ object AMQP {
protected def setupChannel = {
connection = connectionFactory.newConnection(hostname, port)
channel = connection.createChannel
channel.exchangeDeclare(exchangeName.toString, exchangeType.toString, passive, durable, autoDelete, configurationArguments.asJava)
listeners.elements.toList.map(_._2).foreach(registerListener)
channel.exchangeDeclare(exchangeName.toString, exchangeType.toString, passive, durable, autoDelete, jConfigMap)
listeners.iterator.toList.map(_._2).foreach(registerListener)
if (shutdownListener.isDefined) connection.addShutdownListener(shutdownListener.get)
}
@ -426,7 +431,7 @@ object AMQP {
listener.queueName,
passive, durable,
listener.exclusive, listener.autoDelete,
configurationArguments.asJava)
jConfigMap)
}
log.debug("Binding new queue for MessageConsumerListener [%s]", listener.queueName)
@ -460,7 +465,7 @@ object AMQP {
"MessageConsumerListener [" + listener + "] does not have a tag")
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(
"Could not find message listener for tag [%s]; can't shut listener down", listenerTag)
case Some(listener) =>
@ -480,7 +485,7 @@ object AMQP {
"Can't unregister message consumer listener [%s]; no such listener",
listener.toString(exchangeName))
case Some(listener) =>
listeners - listener
listeners -= listener
listener.tag match {
case None => log.warning(
"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.util.ExchangeHelper
import scala.collection.jcl.{Map => MapWrapper}
/**
* An immutable representation of a Camel message. Actor classes that mix in
* se.scalablesolutions.akka.camel.Producer or
@ -16,12 +14,7 @@ import scala.collection.jcl.{Map => MapWrapper}
*
* @author Martin Krasser
*/
case class Message(val body: Any, val headers: Map[String, Any]) {
/**
* Creates a message with a body and an empty header map.
*/
def this(body: Any) = this(body, Map.empty)
case class Message(val body: Any, val headers: Map[String, Any] = Map.empty) {
/**
* 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
@ -58,23 +51,23 @@ case class Message(val body: Any, val headers: Map[String, Any]) {
/**
* 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.
*/
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.
*/
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
* 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
*/
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.
@ -224,8 +217,10 @@ class CamelMessageAdapter(val cm: CamelMessage) {
*/
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) =
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
* representation via Message.canonicalize.
*/
protected def produceOneway(msg: Any): Unit =
protected def produceOnewaySync(msg: Any): Unit =
template.send(endpointUri, createInOnlyExchange.fromRequestMessage(Message.canonicalize(msg)))
/**
@ -90,7 +90,7 @@ trait Producer { self: Actor =>
* representation via Message.canonicalize.
* @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 requestProcessor = new Processor() {
def process(exchange: Exchange) = exchange.fromRequestMessage(cmsg)
@ -126,9 +126,9 @@ trait Producer { self: Actor =>
*/
protected def produce: PartialFunction[Any, Unit] = {
case msg => {
if ( oneway && !async) produceOneway(msg)
if ( oneway && !async) produceOnewaySync(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)
}
}

View file

@ -135,7 +135,7 @@ class ActorProducer(val ep: ActorEndpoint) extends DefaultProducer(ep) {
private def targetById(id: String) = ActorRegistry.actorsFor(id) match {
case Nil => None
case actor :: Nil => Some(actor)
case actors => Some(actors.first)
case actors => Some(actors.head)
}
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 id actor identifier
* @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
*/

View file

@ -26,9 +26,9 @@ class CamelExchangeAdapterTest extends JUnitSuite {
}
@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")
val e2 = sampleInOut.fromFailureMessage(Failure(new Exception("test2"), Map.empty))
val e2 = sampleInOut.fromFailureMessage(Failure(new Exception("test2")))
assert(e2.getException.getMessage === "test2")
}

View file

@ -1,17 +1,18 @@
package se.scalablesolutions.akka.cluster.jgroups
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 org.scala_tools.javautils.Imports._
/**
* Clustering support via JGroups.
* @Author Viktor Klang
*/
class JGroupsClusterActor extends BasicClusterActor {
import scala.collection.JavaConversions._
import se.scalablesolutions.akka.remote.ClusterActor._
type ADDR_T = Address
@volatile private var isActive = false
@ -31,13 +32,13 @@ class JGroupsClusterActor extends BasicClusterActor {
def setState(state: Array[Byte]): 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 =
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 =
if (isActive) me send Zombie(a)
if (isActive) me ! Zombie(a)
def block: Unit =
log debug "UNSUPPORTED: JGroupsClusterActor::block" //TODO HotSwap to a buffering body
@ -60,7 +61,7 @@ class JGroupsClusterActor extends BasicClusterActor {
super.shutdown
log debug ("Shutting down %s", toString)
isActive = false
channel.foreach(_.shutdown)
channel.foreach(Util shutdown _)
channel = None
}
}

View file

@ -71,8 +71,9 @@ class ShoalClusterActor extends BasicClusterActor {
* Creates a CallBack instance that deals with the cluster signalling
*/
protected def createCallback : CallBack = {
import org.scala_tools.javautils.Imports._
import scala.collection.JavaConversions._
import ClusterActor._
val me = this
new CallBack {
def processNotification(signal : Signal) {
@ -80,10 +81,10 @@ class ShoalClusterActor extends BasicClusterActor {
signal.acquire()
if(isActive) {
signal match {
case ms : MessageSignal => me send Message[ADDR_T](ms.getMemberToken,ms.getMessage)
case jns : JoinNotificationSignal => me send View[ADDR_T](Set[ADDR_T]() ++ jns.getCurrentCoreMembers.asScala - serverName)
case fss : FailureSuspectedSignal => me send Zombie[ADDR_T](fss.getMemberToken)
case fns : FailureNotificationSignal => me send Zombie[ADDR_T](fns.getMemberToken)
case ms : MessageSignal => me ! Message[ADDR_T](ms.getMemberToken,ms.getMessage)
case jns : JoinNotificationSignal => me ! View[ADDR_T](Set[ADDR_T]() ++ jns.getCurrentCoreMembers - serverName)
case fss : FailureSuspectedSignal => me ! Zombie[ADDR_T](fss.getMemberToken)
case fns : FailureNotificationSignal => me ! Zombie[ADDR_T](fns.getMemberToken)
case _ => log.debug("Unhandled signal: [%s]",signal)
}
}

View file

@ -12,7 +12,6 @@ class AkkaBroadcaster extends org.atmosphere.jersey.JerseyBroadcaster {
val caster = new Actor {
def receive = { case f : Function0[_] => f() }
start
}
@ -22,6 +21,6 @@ class AkkaBroadcaster extends org.atmosphere.jersey.JerseyBroadcaster {
}
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.
*/
override def createCometSupportResolver() : CometSupportResolver = {
import org.scala_tools.javautils.Imports._
import scala.collection.JavaConversions._
new DefaultCometSupportResolver(config) {
type CS = CometSupport[_ <: AtmosphereResource[_,_]]
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 x :: Nil => newCometSupport(x.asInstanceOf[Class[_ <: CS]])
case _ => super.resolveMultipleNativeSupportConflict(available)

View file

@ -19,7 +19,7 @@ import se.scalablesolutions.akka.util.{Bootable, Logging}
trait BootableCometActorService extends Bootable with Logging {
self : BootableActorLoaderService =>
import config.Config._
import se.scalablesolutions.akka.config.Config._
val REST_HOSTNAME = config.getString("akka.rest.hostname", "localhost")
val REST_URL = "http://" + REST_HOSTNAME
@ -44,8 +44,8 @@ trait BootableCometActorService extends Bootable with Logging {
adapter.setServletInstance(new AkkaServlet)
adapter.setContextPath(uri.getPath)
adapter.addInitParameter("cometSupport", "org.atmosphere.container.GrizzlyCometSupport")
if (HOME.isDefined) adapter.setRootFolder(HOME.get + "/deploy/root")
log.info("REST service root path [%s] and context path [%s]", adapter.getRootFolder, adapter.getContextPath)
if (HOME.isDefined) adapter.addRootFolder(HOME.get + "/deploy/root")
log.info("REST service root path [%s] and context path [%s]", adapter.getRootFolders, adapter.getContextPath)
val ah = new com.sun.grizzly.arp.DefaultAsyncHandler
ah.addAsyncFilter(new com.sun.grizzly.comet.CometAsyncFilter)
@ -56,15 +56,14 @@ trait BootableCometActorService extends Bootable with Logging {
t.setEnableAsyncExecution(true)
t.setAsyncHandler(ah)
t.listen
t }
t
}
log.info("REST service started successfully. Listening to port [%s]", REST_PORT)
}
}
abstract override def onUnload = {
super.onUnload
if (jerseySelectorThread.isDefined) {
log.info("Shutting down REST service (Jersey)")
jerseySelectorThread.get.stopEndpoint

View file

@ -262,9 +262,10 @@ private[akka] sealed class ActiveObjectAspect {
}
private def localDispatch(joinPoint: JoinPoint): AnyRef = {
import Actor.Sender.Self
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 {
val result = actor !! Invocation(joinPoint, false, isVoid(rtti))
if (result.isDefined) result.get
@ -274,7 +275,7 @@ private[akka] sealed class ActiveObjectAspect {
private def remoteDispatch(joinPoint: JoinPoint): AnyRef = {
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 requestBuilder = RemoteRequest.newBuilder
.setId(RemoteRequestIdFactory.nextId)
@ -354,7 +355,7 @@ private[akka] sealed class ActiveObjectAspect {
object Dispatcher {
val ZERO_ITEM_CLASS_ARRAY = Array[Class[_]]()
val ZERO_ITEM_OBJECT_ARRAY = Array[Object[_]]()
val ZERO_ITEM_OBJECT_ARRAY = Array[Object]()
}
/**
@ -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 PORT = config.getInt("akka.remote.server.port", 9999)
object Sender {
implicit val Self: Option[Actor] = None
object Sender{
object Self
}
/**
@ -144,12 +144,11 @@ 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/>
* This actor can <b>not</b> respond to any messages but can be used as a simple way to
* spawn a lightweight thread to process some task.
* <p/>
* The actor is started when created.
* NOTE: If used from within an Actor then has to be qualified with 'Actor.spawn' since
* there is a method 'spawn[ActorType]' in the Actor trait already.
* Example:
* <pre>
* import Actor._
@ -159,11 +158,11 @@ object Actor extends Logging {
* }
* </pre>
*/
def spawn(body: => Unit): Actor = {
def spawn(body: => Unit): Unit = {
case object Spawn
new Actor() {
start
send(Spawn)
this ! Spawn
def receive = {
case Spawn => body; stop
}
@ -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.
* <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>
* actor ! message
* </pre>
* <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]) = {
//FIXME 2.8 def !(message: Any)(implicit sender: Option[Actor] = None) = {
def !(message: Any)(implicit sender: Option[Actor] = None) = {
if (_isKilled) throw new ActorKilledException("Actor [" + toString + "] has been killed, can't respond to messages")
if (_isRunning) postMessageToMailbox(message, sender)
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.
* <p/>
@ -577,7 +551,7 @@ trait Actor extends TransactionManagement with Logging {
* <p/>
* 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 (_isRunning) {
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\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\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) =>
future.completeWithResult(message)
}
@ -733,8 +708,8 @@ trait Actor extends TransactionManagement with Logging {
* <p/>
* To be invoked from within the actor itself.
*/
protected[this] def spawn[T <: Actor](actorClass: Class[T]): T = {
val actor = spawnButDoNotStart(actorClass)
protected[this] def spawn[T <: Actor : Manifest] : T = {
val actor = spawnButDoNotStart[T]
actor.start
actor
}
@ -744,8 +719,8 @@ trait Actor extends TransactionManagement with Logging {
* <p/>
* To be invoked from within the actor itself.
*/
protected[this] def spawnRemote[T <: Actor](actorClass: Class[T], hostname: String, port: Int): T = {
val actor = spawnButDoNotStart(actorClass)
protected[this] def spawnRemote[T <: Actor : Manifest](hostname: String, port: Int): T = {
val actor = spawnButDoNotStart[T]
actor.makeRemote(hostname, port)
actor.start
actor
@ -756,8 +731,8 @@ trait Actor extends TransactionManagement with Logging {
* <p/>
* To be invoked from within the actor itself.
*/
protected[this] def spawnLink[T <: Actor](actorClass: Class[T]): T = {
val actor = spawnButDoNotStart(actorClass)
protected[this] def spawnLink[T <: Actor : Manifest] : T = {
val actor = spawnButDoNotStart[T]
try {
actor.start
} finally {
@ -771,8 +746,8 @@ trait Actor extends TransactionManagement with Logging {
* <p/>
* To be invoked from within the actor itself.
*/
protected[this] def spawnLinkRemote[T <: Actor](actorClass: Class[T], hostname: String, port: Int): T = {
val actor = spawnButDoNotStart(actorClass)
protected[this] def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): T = {
val actor = spawnButDoNotStart[T]
try {
actor.makeRemote(hostname, port)
actor.start
@ -804,9 +779,11 @@ trait Actor extends TransactionManagement with Logging {
private[akka] def getSenderFuture = senderFuture
private def spawnButDoNotStart[T <: Actor](actorClass: Class[T]): T = {
val actor = actorClass.newInstance.asInstanceOf[T]
if (!dispatcher.isInstanceOf[ThreadBasedDispatcher]) actor.dispatcher = dispatcher
private def spawnButDoNotStart[T <: Actor : Manifest] : T = {
val actor = manifest[T].erasure.asInstanceOf[Class[T]].newInstance
if (!dispatcher.isInstanceOf[ThreadBasedDispatcher]) {
actor.dispatcher = dispatcher
}
actor
}
@ -833,7 +810,7 @@ trait Actor extends TransactionManagement with Logging {
requestBuilder.setSourceTarget(s.getClass.getName)
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)
@ -1072,17 +1049,14 @@ trait Actor extends TransactionManagement with Logging {
!message.isInstanceOf[List[_]] &&
!message.isInstanceOf[scala.collection.immutable.Map[_, _]] &&
!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)) {
Serializer.Java.deepClone(message)
} else message
} else message
override def hashCode(): Int = {
var result = HashCode.SEED
result = HashCode.hash(result, _uuid)
result
}
override def hashCode(): Int = HashCode.hash(HashCode.SEED, _uuid)
override def equals(that: Any): Boolean = {
that != null &&

View file

@ -110,7 +110,7 @@ object ActorRegistry extends Logging {
} else actorsByClassName.put(className, actor :: Nil)
// notify listeners
foreachListener(_ send ActorRegistered(actor))
foreachListener(_ ! ActorRegistered(actor))
}
/**
@ -121,7 +121,7 @@ object ActorRegistry extends Logging {
actorsById remove actor.getId
actorsByClassName remove actor.getClass.getName
// notify listeners
foreachListener(_ send ActorUnregistered(actor))
foreachListener(_ ! ActorUnregistered(actor))
}
/**

View file

@ -1,16 +1,6 @@
// Copyright © 2008-10 The original author or authors
//
// 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.
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package se.scalablesolutions.akka.actor
@ -23,47 +13,89 @@ class AgentException private[akka](message: String) extends RuntimeException(mes
/**
* The Agent class was strongly inspired by the agent principle in Clojure.
* Essentially, an agent wraps a shared mutable state and hides it behind
* a message-passing interface. Agents accept messages and process them on
* behalf of the wrapped state.
* <p/>
*
* 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).
* Agents provide independent, asynchronous change of individual locations.
* 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/>
*
* The submitted functions / commands take the internal state as a parameter
* and their output becomes the new internal state value.
* 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 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.
* 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.
* If an Agent is used within an enclosing transaction, then it will
* participate in that transaction.
* <p/>
*
* Example of usage:
* <pre>
* val agent = Agent(5)
*
* agent update (_ + 1)
* agent update (_ * 2)
* agent send (_ + 1)
* agent send (_ * 2)
*
* val result = agent()
* ... // use result
*
* agent.close
* </pre>
* <p/>
*
* NOTE: You can't call 'agent.get' or 'agent()' within an enclosing transaction since
* that will block the transaction indefinitely. But 'agent.update' or 'Agent(value)'
* is fine.
* Agent is also monadic, which means that you can compose operations using
* for-comprehensions. In monadic usage the original agents are not touched
* but new agents are created. So the old values (agents) are still available
* as-is. They are so-called 'persistent'.
* <p/>
*
* Original author:
* @author Vaclav Pech
* Example of monadic usage:
* <pre>
* val agent1 = Agent(3)
* val agent2 = Agent(5)
*
* for (value <- agent1) {
* result = value + 1
* }
*
* 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.
*
* Inital AKKA port by:
* @author Viktor Klang
*
* Modifications by:
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
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]()
start
this ! ValueHolder(initialValue)
this !! Value(initialValue)
/**
* Periodically handles incoming messages.
*/
def receive = {
case ValueHolder(x: T) => updateData(x)
case FunctionHolder(fun: (T => T)) => updateData(fun(value.getOrWait))
case ProcedureHolder(fun: (T => Unit)) => fun(copyStrategy(value.getOrWait))
case Value(v: T) =>
swap(v)
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
/**
* 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.
@ -105,18 +140,11 @@ 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.")
val ref = new AtomicReference[T]
val latch = new CountDownLatch(1)
get((x: T) => {ref.set(x); latch.countDown})
sendProc((v: T) => {ref.set(v); latch.countDown})
latch.await
ref.get
}
/**
* Asynchronously submits a request to read the internal state. The supplied function
* will be executed on the returned internal state value. A copy of the internal state
* will be used, depending on the underlying effective copyStrategy.
*/
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
@ -127,23 +155,48 @@ sealed class Agent[T] private (initialValue: T) extends Transactor {
/**
* Submits the provided function for execution against the internal agent's state.
*/
final def apply(message: (T => T)): Unit = this ! FunctionHolder(message)
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 ! ValueHolder(message)
final def apply(message: T): Unit = this ! Value(message)
/**
* Submits the provided function for execution against the internal agent's state.
* Submits the provided function of type 'T => T' for execution against the internal agent's state.
*/
final def update(message: (T => T)): Unit = this ! FunctionHolder(message)
final def send(message: (T) => T): Unit = this ! Function(message)
/**
* Submits a new value to be set as the new agent's internal state.
*/
// 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 send(message: T): Unit = this ! Value(message)
/**
* 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
* 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)
/**
* 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 map[B](f: (T) => B): Agent[B] = Agent(f(get))
/**
* 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 flatMap[B](f: (T) => Agent[B]): Agent[B] = Agent(f(get)())
/**
* Applies function with type 'T => B' to the agent's internal state.
* Does not change the value of the agent (this).
*/
final def foreach(f: (T) => Unit): Unit = f(get)
/**
* 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.
*
* @author Viktor Klang
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Agent {
/*
* The internal messages for passing around requests.
*/
private case class ProcedureHolder[T](fun: ((T) => Unit))
private case class FunctionHolder[T](fun: ((T) => T))
private case class ValueHolder[T](value: T)
private case class Value[T](value: T)
private case class Function[T](fun: ((T) => T))
private case class Procedure[T](fun: ((T) => Unit))
/**
* Creates a new Agent of type T with the initial value of value.

View file

@ -28,7 +28,7 @@ trait BootableActorLoaderService extends Bootable with Logging {
log.error("Could not find a deploy directory at [%s]", DEPLOY)
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)
new URLClassLoader(toDeploy.toArray, 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.util.Logging
import org.scala_tools.javautils.Imports._
case object UnSchedule
case class SchedulerException(msg: String, e: Throwable) extends RuntimeException(msg, e)
@ -66,7 +64,8 @@ object Scheduler extends Actor {
}
override def shutdown = {
schedulers.values.asScala.foreach(_ ! UnSchedule)
import scala.collection.JavaConversions._
schedulers.values.foreach(_ ! UnSchedule)
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
def getInstance[T](clazz: Class[T]): List[T] = actors.get(clazz.getName).asInstanceOf[List[T]]
def getComponentInterfaces: List[Class[_]] = List.flatten(
actors.values.toArray.toList.asInstanceOf[List[List[AnyRef]]]).map(_.getClass)
def getComponentInterfaces: List[Class[_]] =
actors.values.toArray.toList.asInstanceOf[List[List[AnyRef]]].flatten.map(_.getClass)
def isDefined(clazz: Class[_]): Boolean = actors.containsKey(clazz.getName)

View file

@ -6,12 +6,11 @@ package se.scalablesolutions.akka.config
import JavaConfig._
import java.util.{List => JList}
import java.util.{ArrayList}
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.
* 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>
*/
class ActiveObjectConfigurator {
import scala.collection.JavaConversions._
// TODO: make pluggable once we have f.e a SpringConfigurator
private val INSTANCE = new ActiveObjectGuiceConfigurator
@ -32,7 +32,7 @@ class ActiveObjectConfigurator {
* @param clazz the class for the active object
* @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.

View file

@ -12,7 +12,7 @@ import net.lag.configgy.{Configgy, ParseException}
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Config extends Logging {
val VERSION = "0.7"
val VERSION = "0.8"
// Set Multiverse options for max speed
System.setProperty("org.multiverse.MuliverseConstants.sanityChecks", "false")

View file

@ -13,7 +13,7 @@ object ConfiguratorRepository extends Logging {
private val configuration = new HashSet[Configurator]
def registerConfigurator(conf: Configurator) = synchronized {
configuration + conf
configuration += conf
}
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 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)
}
@ -166,7 +166,7 @@ abstract class BasicClusterActor extends ClusterActor {
case DeregisterLocalNode(s) => {
log debug ("DeregisterLocalNode: %s", s)
local = Node(local.endpoints - s)
local = Node(local.endpoints.filterNot(_ == s))
broadcast(Papers(local.endpoints))
}
}
@ -201,30 +201,30 @@ abstract class BasicClusterActor extends ClusterActor {
* Applies the given PartialFunction to all known RemoteAddresses
*/
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
*/
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
*/
def registerLocalNode(hostname: String, port: Int): Unit =
send(RegisterLocalNode(RemoteAddress(hostname, port)))
this ! RegisterLocalNode(RemoteAddress(hostname, port))
/**
* Deregisters a local endpoint
*/
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
*/
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)) {
val client = remoteClients(hash)
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 {
actorsFor(RemoteServer.Address(hostname, port)) + uuid
actorsFor(RemoteServer.Address(hostname, port)) += uuid
}
// TODO: add RemoteClient.unregister for ActiveObject, but first need a @shutdown callback
private[akka] def unregister(hostname: String, port: Int, uuid: String) = synchronized {
val set = actorsFor(RemoteServer.Address(hostname, port))
set - uuid
set -= uuid
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>
*/
@ChannelPipelineCoverage {val value = "all"}
@ChannelHandler.Sharable
class RemoteClientHandler(val name: String,
val futures: ConcurrentMap[Long, CompletableFuture],
val supervisors: ConcurrentMap[String, Actor],

View file

@ -4,7 +4,7 @@
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.remote.protobuf.RemoteProtocol.{RemoteRequest, RemoteReply}
@ -14,7 +14,7 @@ object RemoteProtocolBuilder {
private var SERIALIZER_JAVA: Serializer.Java = Serializer.Java
private var SERIALIZER_JAVA_JSON: Serializer.JavaJSON = Serializer.JavaJSON
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
@ -26,10 +26,9 @@ object RemoteProtocolBuilder {
def getMessage(request: RemoteRequest): Any = {
request.getProtocol match {
case SerializationProtocol.SBINARY =>
val renderer = Class.forName(
new String(request.getMessageManifest.toByteArray)).newInstance.asInstanceOf[SBinary[_ <: AnyRef]]
renderer.fromBytes(request.getMessage.toByteArray)
//case SerializationProtocol.SBINARY =>
// val renderer = Class.forName(new String(request.getMessageManifest.toByteArray)).newInstance.asInstanceOf[SBinary[_ <: AnyRef]]
// renderer.fromBytes(request.getMessage.toByteArray)
case SerializationProtocol.SCALA_JSON =>
val manifest = SERIALIZER_JAVA.in(request.getMessageManifest.toByteArray, None).asInstanceOf[String]
SERIALIZER_SCALA_JSON.in(request.getMessage.toByteArray, Some(Class.forName(manifest)))
@ -48,9 +47,9 @@ object RemoteProtocolBuilder {
def getMessage(reply: RemoteReply): Any = {
reply.getProtocol match {
case SerializationProtocol.SBINARY =>
val renderer = Class.forName(new String(reply.getMessageManifest.toByteArray)).newInstance.asInstanceOf[SBinary[_ <: AnyRef]]
renderer.fromBytes(reply.getMessage.toByteArray)
//case SerializationProtocol.SBINARY =>
// val renderer = Class.forName(new String(reply.getMessageManifest.toByteArray)).newInstance.asInstanceOf[SBinary[_ <: AnyRef]]
// renderer.fromBytes(reply.getMessage.toByteArray)
case SerializationProtocol.SCALA_JSON =>
val manifest = SERIALIZER_JAVA.in(reply.getMessageManifest.toByteArray, None).asInstanceOf[String]
SERIALIZER_SCALA_JSON.in(reply.getMessage.toByteArray, Some(Class.forName(manifest)))
@ -68,12 +67,12 @@ object RemoteProtocolBuilder {
}
def setMessage(message: Any, builder: RemoteRequest.Builder) = {
if (message.isInstanceOf[Serializable.SBinary[_]]) {
/*if (message.isInstanceOf[Serializable.SBinary[_]]) {
val serializable = message.asInstanceOf[Serializable.SBinary[_ <: Any]]
builder.setProtocol(SerializationProtocol.SBINARY)
builder.setMessage(ByteString.copyFrom(serializable.toBytes))
builder.setMessageManifest(ByteString.copyFrom(serializable.getClass.getName.getBytes))
} else if (message.isInstanceOf[Message]) {
} else*/ if (message.isInstanceOf[Message]) {
val serializable = message.asInstanceOf[Message]
builder.setProtocol(SerializationProtocol.PROTOBUF)
builder.setMessage(ByteString.copyFrom(serializable.toByteArray))
@ -96,12 +95,12 @@ object RemoteProtocolBuilder {
}
def setMessage(message: Any, builder: RemoteReply.Builder) = {
if (message.isInstanceOf[Serializable.SBinary[_]]) {
/*if (message.isInstanceOf[Serializable.SBinary[_]]) {
val serializable = message.asInstanceOf[Serializable.SBinary[_ <: Any]]
builder.setProtocol(SerializationProtocol.SBINARY)
builder.setMessage(ByteString.copyFrom(serializable.toBytes))
builder.setMessageManifest(ByteString.copyFrom(serializable.getClass.getName.getBytes))
} else if (message.isInstanceOf[Message]) {
} else*/ if (message.isInstanceOf[Message]) {
val serializable = message.asInstanceOf[Message]
builder.setProtocol(SerializationProtocol.PROTOBUF)
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>
*/
@ChannelPipelineCoverage {val value = "all"}
@ChannelHandler.Sharable
class RemoteServerHandler(
val name: String,
val openChannels: ChannelGroup,
@ -310,7 +310,7 @@ class RemoteServerHandler(
actor.!(message)(Some(remoteActor))
} else {
// couldn't find a way to reply, send the message without a source/sender
actor.send(message)
actor ! message
}
} else {
try {

View file

@ -4,63 +4,57 @@
package se.scalablesolutions.akka.serialization
import sbinary._
import sbinary.Operations._
import sbinary.DefaultProtocol._
// --- PRIMITIVES ---
case class BinaryString(val value: String) extends Serializable.SBinary[BinaryString] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]): BinaryString = BinaryString(fromByteArray[String](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigInt(val value: BigInt) extends Serializable.SBinary[BinaryBigInt] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigInt(fromByteArray[BigInt](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigDecimal(val value: BigDecimal) extends Serializable.SBinary[BinaryBigDecimal] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimal(fromByteArray[BigDecimal](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryLong(val value: Long) extends Serializable.SBinary[BinaryLong] {
import sbinary.DefaultProtocol._
def this() = this(0L)
def fromBytes(bytes: Array[Byte]) = BinaryLong(fromByteArray[Long](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryInt(val value: Int) extends Serializable.SBinary[BinaryInt] {
import sbinary.DefaultProtocol._
def this() = this(0)
def fromBytes(bytes: Array[Byte]) = BinaryInt(fromByteArray[Int](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryDouble(val value: Double) extends Serializable.SBinary[BinaryDouble] {
import sbinary.DefaultProtocol._
def this() = this(0.0D)
def fromBytes(bytes: Array[Byte]) = BinaryDouble(fromByteArray[Double](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryFloat(val value: Float) extends Serializable.SBinary[BinaryFloat] {
import sbinary.DefaultProtocol._
def this() = this(0.0F)
def fromBytes(bytes: Array[Byte]) = BinaryFloat(fromByteArray[Float](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBoolean(val value: Boolean) extends Serializable.SBinary[BinaryBoolean] {
import sbinary.DefaultProtocol._
def this() = this(true)
def fromBytes(bytes: Array[Byte]) = BinaryBoolean(fromByteArray[Boolean](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryByte(val value: Byte) extends Serializable.SBinary[BinaryByte] {
import sbinary.DefaultProtocol._
def this() = this(0x00)
def fromBytes(bytes: Array[Byte]) = BinaryByte(fromByteArray[Byte](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryChar(val value: Char) extends Serializable.SBinary[BinaryChar] {
import sbinary.DefaultProtocol._
def this() = this(' ')
def fromBytes(bytes: Array[Byte]) = BinaryChar(fromByteArray[Char](bytes))
def toBytes: Array[Byte] = toByteArray(value)
@ -68,117 +62,98 @@ case class BinaryChar(val value: Char) extends Serializable.SBinary[BinaryChar]
// --- ARRAYS ---
case class BinaryStringArray(val value: Array[String]) extends Serializable.SBinary[BinaryStringArray] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringArray(fromByteArray[Array[String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigIntArray(val value: Array[BigInt]) extends Serializable.SBinary[BinaryBigIntArray] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigIntArray(fromByteArray[Array[BigInt]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigDecimalArray(val value: Array[BigDecimal]) extends Serializable.SBinary[BinaryBigDecimalArray] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalArray(fromByteArray[Array[BigDecimal]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryIntArray(val value: Array[Int]) extends Serializable.SBinary[BinaryIntArray] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryIntArray(fromByteArray[Array[Int]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryLongArray(val value: Array[Long]) extends Serializable.SBinary[BinaryLongArray] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryLongArray(fromByteArray[Array[Long]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryDoubleArray(val value: Array[Double]) extends Serializable.SBinary[BinaryDoubleArray] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryDoubleArray(fromByteArray[Array[Double]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryFloatArray(val value: Array[Float]) extends Serializable.SBinary[BinaryFloatArray] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryFloatArray(fromByteArray[Array[Float]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBooleanArray(val value: Array[Boolean]) extends Serializable.SBinary[BinaryBooleanArray] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBooleanArray(fromByteArray[Array[Boolean]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryByteArray(val value: Array[Byte]) extends Serializable.SBinary[BinaryByteArray] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryByteArray(fromByteArray[Array[Byte]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryClassArray(val value: Array[Class[_]]) extends Serializable.SBinary[BinaryClassArray] {
import sbinary.DefaultProtocol._
/*case class BinaryClassArray(val value: Array[Class[_]]) extends Serializable.SBinary[BinaryClassArray] {
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryClassArray(fromByteArray[Array[Class[_]]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
}*/
// --- LISTS ---
case class BinaryStringList(val value: List[String]) extends Serializable.SBinary[BinaryStringList] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringList(fromByteArray[List[String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigIntList(val value: List[BigInt]) extends Serializable.SBinary[BinaryBigIntList] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigIntList(fromByteArray[List[BigInt]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigDecimalList(val value: List[BigDecimal]) extends Serializable.SBinary[BinaryBigDecimalList] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalList(fromByteArray[List[BigDecimal]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryLongList(val value: List[Long]) extends Serializable.SBinary[BinaryLongList] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryLongList(fromByteArray[List[Long]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryIntList(val value: List[Int]) extends Serializable.SBinary[BinaryIntList] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryIntList(fromByteArray[List[Int]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryDoubleList(val value: List[Double]) extends Serializable.SBinary[BinaryDoubleList] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryDoubleList(fromByteArray[List[Double]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryFloatList(val value: List[Float]) extends Serializable.SBinary[BinaryFloatList] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryFloatList(fromByteArray[List[Float]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBooleanList(val value: List[Boolean]) extends Serializable.SBinary[BinaryBooleanList] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBooleanList(fromByteArray[List[Boolean]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryClassList(val value: List[Class[_]]) extends Serializable.SBinary[BinaryClassList] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryClassList(fromByteArray[List[Class[_]]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
@ -186,103 +161,86 @@ case class BinaryClassList(val value: List[Class[_]]) extends Serializable.SBina
// --- TUPLES ---
case class BinaryStringStringTuple(val value: Tuple2[String, String]) extends Serializable.SBinary[BinaryStringStringTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringStringTuple(fromByteArray[Tuple2[String, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigIntBigIntTuple(val value: Tuple2[BigInt, BigInt]) extends Serializable.SBinary[BinaryBigIntBigIntTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigIntBigIntTuple(fromByteArray[Tuple2[BigInt, BigInt]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigDecimalBigDecimalTuple(val value: Tuple2[BigDecimal, BigDecimal]) extends Serializable.SBinary[BinaryBigDecimalBigDecimalTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalBigDecimalTuple(fromByteArray[Tuple2[BigDecimal, BigDecimal]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryLongLongTuple(val value: Tuple2[Long, Long]) extends Serializable.SBinary[BinaryLongLongTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryLongLongTuple(fromByteArray[Tuple2[Long, Long]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryIntIntTuple(val value: Tuple2[Int, Int]) extends Serializable.SBinary[BinaryIntIntTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryIntIntTuple(fromByteArray[Tuple2[Int, Int]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryDoubleDoubleTuple(val value: Tuple2[Double, Double]) extends Serializable.SBinary[BinaryDoubleDoubleTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryDoubleDoubleTuple(fromByteArray[Tuple2[Double, Double]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryFloatFloatTuple(val value: Tuple2[Float, Float]) extends Serializable.SBinary[BinaryFloatFloatTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryFloatFloatTuple(fromByteArray[Tuple2[Float, Float]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBooleanBooleanTuple(val value: Tuple2[Boolean, Boolean]) extends Serializable.SBinary[BinaryBooleanBooleanTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBooleanBooleanTuple(fromByteArray[Tuple2[Boolean, Boolean]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryClassClassTuple(val value: Tuple2[Class[_], Class[_]]) extends Serializable.SBinary[BinaryClassClassTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryClassClassTuple(fromByteArray[Tuple2[Class[_], Class[_]]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryByteArrayByteArrayTuple(val value: Tuple2[Array[Byte], Array[Byte]]) extends Serializable.SBinary[BinaryByteArrayByteArrayTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryByteArrayByteArrayTuple(fromByteArray[Tuple2[Array[Byte], Array[Byte]]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigIntStringTuple(val value: Tuple2[BigInt, String]) extends Serializable.SBinary[BinaryBigIntStringTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigIntStringTuple(fromByteArray[Tuple2[BigInt, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigDecimalStringTuple(val value: Tuple2[BigDecimal, String]) extends Serializable.SBinary[BinaryBigDecimalStringTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalStringTuple(fromByteArray[Tuple2[BigDecimal, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryLongStringTuple(val value: Tuple2[Long, String]) extends Serializable.SBinary[BinaryLongStringTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryLongStringTuple(fromByteArray[Tuple2[Long, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryIntStringTuple(val value: Tuple2[Int, String]) extends Serializable.SBinary[BinaryIntStringTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryIntStringTuple(fromByteArray[Tuple2[Int, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryDoubleStringTuple(val value: Tuple2[Double, String]) extends Serializable.SBinary[BinaryDoubleStringTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryDoubleStringTuple(fromByteArray[Tuple2[Double, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryFloatStringTuple(val value: Tuple2[Float, String]) extends Serializable.SBinary[BinaryFloatStringTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryFloatStringTuple(fromByteArray[Tuple2[Float, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBooleanStringTuple(val value: Tuple2[Boolean, String]) extends Serializable.SBinary[BinaryBooleanStringTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBooleanStringTuple(fromByteArray[Tuple2[Boolean, String]](bytes))
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)
}
case class BinaryStringByteArrayTuple(val value: Tuple2[String, Array[Byte]]) extends Serializable.SBinary[BinaryStringByteArrayTuple] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringByteArrayTuple(fromByteArray[Tuple2[String, Array[Byte]]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
@ -302,73 +259,61 @@ case class BinaryStringByteArrayTuple(val value: Tuple2[String, Array[Byte]]) ex
// --- MAPS ---
case class BinaryStringStringMap(val value: Map[String, String]) extends Serializable.SBinary[BinaryStringStringMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringStringMap(fromByteArray[Map[String, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigIntStringMap(val value: Map[BigInt, String]) extends Serializable.SBinary[BinaryBigIntStringMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigIntStringMap(fromByteArray[Map[BigInt, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryBigDecimalStringMap(val value: Map[BigDecimal, String]) extends Serializable.SBinary[BinaryBigDecimalStringMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryBigDecimalStringMap(fromByteArray[Map[BigDecimal, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryLongStringMap(val value: Map[Long, String]) extends Serializable.SBinary[BinaryLongStringMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryLongStringMap(fromByteArray[Map[Long, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryIntStringMap(val value: Map[Int, String]) extends Serializable.SBinary[BinaryIntStringMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryIntStringMap(fromByteArray[Map[Int, String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryClassStringMap(val value: Map[Class[_], String]) extends Serializable.SBinary[BinaryClassStringMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryClassStringMap(fromByteArray[Map[Class[_], String]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryStringBigIntMap(val value: Map[String, BigInt]) extends Serializable.SBinary[BinaryStringBigIntMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringBigIntMap(fromByteArray[Map[String, BigInt]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryStringBigDecimalMap(val value: Map[String, BigDecimal]) extends Serializable.SBinary[BinaryStringBigDecimalMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringBigDecimalMap(fromByteArray[Map[String, BigDecimal]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryStringLongMap(val value: Map[String, Long]) extends Serializable.SBinary[BinaryStringLongMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringLongMap(fromByteArray[Map[String, Long]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryStringIntMap(val value: Map[String, Int]) extends Serializable.SBinary[BinaryStringIntMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringIntMap(fromByteArray[Map[String, Int]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryStringClassMap(val value: Map[String, Class[_]]) extends Serializable.SBinary[BinaryStringClassMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringClassMap(fromByteArray[Map[String, Class[_]]](bytes))
def toBytes: Array[Byte] = toByteArray(value)
}
case class BinaryStringByteArrayMap(val value: Map[String, Array[Byte]]) extends Serializable.SBinary[BinaryStringByteArrayMap] {
import sbinary.DefaultProtocol._
def this() = this(null)
def fromBytes(bytes: Array[Byte]) = BinaryStringByteArrayMap(fromByteArray[Map[String, Array[Byte]]](bytes))
def toBytes: Array[Byte] = toByteArray(value)

View file

@ -8,8 +8,7 @@ import org.codehaus.jackson.map.ObjectMapper
import com.google.protobuf.Message
import scala.reflect.Manifest
import reflect.Manifest
import sbinary.DefaultProtocol
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>
*/
object SBinary extends SBinary
trait SBinary {
class SBinary {
import sbinary._
import sbinary.Operations._
import sbinary.DefaultProtocol._
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) = {
val thread = new IsolatedEventBasedThread(body).start
thread send Start
thread ! Start
thread
}
@ -93,9 +93,9 @@ import se.scalablesolutions.akka.dispatch.CompletableFuture
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 = {
val ref = value.get
@ -104,13 +104,13 @@ import se.scalablesolutions.akka.dispatch.CompletableFuture
val out = new Out(this)
blockedReaders.offer(out)
val result = out !! Get
out send Exit
out ! Exit
result.getOrElse(throw new DataFlowVariableException(
"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")
}
override def elements: Iterator[T] = new Iterator[T] {
def iterator: Iterator[T] = new Iterator[T] {
private val iter = queue.iterator
def hasNext: Boolean = iter.hasNext
def next: T = { val ref = iter.next; ref() }

View file

@ -46,7 +46,7 @@ trait PersistentDataStructure
*/
@serializable
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])
@ -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)
}
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 elements = root.elements
def iterator = root.elements
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 {
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
}
@ -152,7 +152,7 @@ private[stm] class CollisionNode[K, +V](val hash: Int, bucket: List[(K, V)]) ext
} 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) {
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 }
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 + ")"
}
@ -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
}
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 mask = 1 << i

View file

@ -103,36 +103,29 @@ object Transaction extends TransactionManagement with Logging {
/**
* See ScalaDoc on Transaction class.
*/
def map[T](f: => T)(implicit transactionFamilyName: String): T =
atomic {f}
def map[T](f: => T): T = atomic {f}
/**
* See ScalaDoc on Transaction class.
*/
def flatMap[T](f: => T)(implicit transactionFamilyName: String): T =
atomic {f}
def flatMap[T](f: => T): T = atomic {f}
/**
* See ScalaDoc on Transaction class.
*/
def foreach(f: => Unit)(implicit transactionFamilyName: String): Unit =
atomic {f}
def foreach(f: => Unit): Unit = atomic {f}
/**
* See ScalaDoc on Transaction class.
*/
def atomic[T](body: => T)(implicit transactionFamilyName: String): T = {
// FIXME use Transaction Builder and set the transactionFamilyName
// defaultTxBuilder.setFamilyName(transactionFamilyName)
// new TransactionTemplate[T](defaultTxBuilder.build) {
def atomic[T](body: => T): T = {
var isTopLevelTransaction = true
new TransactionTemplate[T]() {
def execute(mtx: MultiverseTransaction): T = {
val result = body
val txSet = getTransactionSetInScope
log.trace("Committing transaction [%s]\n\twith family name [%s]\n\tby joining transaction set [%s]",
mtx, transactionFamilyName, txSet)
log.trace("Committing transaction [%s]\n\tby joining transaction set [%s]", mtx, txSet)
txSet.joinCommit(mtx)
// FIXME tryJoinCommit(mtx, TransactionManagement.TRANSACTION_TIMEOUT, TimeUnit.MILLISECONDS)
@ -208,7 +201,7 @@ object Transaction extends TransactionManagement with Logging {
def commit = synchronized {
log.trace("Committing transaction %s", toString)
atomic0 {
persistentStateMap.values.foreach(_.commit)
persistentStateMap.valuesIterator.foreach(_.commit)
}
status = TransactionStatus.Completed
}

View file

@ -152,7 +152,7 @@ class TransactionalRef[T] extends Transactional {
def elements: Iterator[T] = {
ensureIsInTransaction
if (isEmpty) Iterator.empty else Iterator.fromValues(ref.get)
if (isEmpty) Iterator.empty else Iterator(ref.get)
}
def toList: List[T] = {
@ -191,11 +191,24 @@ class TransactionalMap[K, V] extends Transactional with scala.collection.mutable
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 remove(key: K) = ref.swap(ref.get.get - key)
def +=(kv: (K, V)) = {
put(kv._1,kv._2)
this
}
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)
@ -206,19 +219,21 @@ class TransactionalMap[K, V] extends Transactional with scala.collection.mutable
oldValue
}
def update(key: K, value: V) = {
override def update(key: K, value: V) = {
val map = ref.get.get
val oldValue = map.get(key)
ref.swap(map.update(key, value))
}
def elements: Iterator[(K, V)] = ref.get.get.elements
def iterator = ref.get.get.iterator
override def elements: Iterator[(K, V)] = ref.get.get.iterator
override def contains(key: K): Boolean = ref.get.get.contains(key)
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);
@ -238,7 +253,7 @@ object TransactionalVector {
*
* @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
private[this] val ref = TransactionalRef[Vector[T]]

View file

@ -44,7 +44,7 @@ import Vector._
*/
@serializable
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
/*
@ -104,7 +104,7 @@ class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail
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] = {
if (tail.length < 32) {
@ -224,7 +224,7 @@ class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail
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 i = 0
@ -236,7 +236,7 @@ class Vector[+T] private (val length: Int, shift: Int, root: Array[AnyRef], tail
back
}
override def map[A](f: (T)=>A): Vector[A] = {
def map[A](f: (T)=>A): Vector[A] = {
var back = new Vector[A]
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 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] = {
if (from < 0) {

View file

@ -16,15 +16,12 @@ class AgentTest extends junit.framework.TestCase
with Suite with MustMatchers
with ActorTestUtil with Logging {
implicit val txFamilyName = "test"
@Test def testSendFun = verify(new TestActor {
def test = {
val agent = Agent(5)
handle(agent) {
agent update (_ + 1)
agent update (_ * 2)
agent send (_ + 1)
agent send (_ * 2)
val result = agent()
result must be(12)
}
@ -35,21 +32,34 @@ with ActorTestUtil with Logging {
def test = {
val agent = Agent(5)
handle(agent) {
agent update 6
agent send 6
val result = agent()
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
val agent = Agent(5)
val tx = transactor {
case Go => agent update (_ + 1)
case Go => agent send (_ + 1)
}
tx send Go
Thread.sleep(5000)
tx ! Go
Thread.sleep(1000)
val result = agent()
result must be(6)
agent.close
@ -63,16 +73,53 @@ with ActorTestUtil with Logging {
val agent = Agent(5)
val tx = transactor {
case Go =>
agent update (_ * 2)
agent send (_ * 2)
try { agent() }
catch {
case _ => latch.countDown
}
}
tx send Go
tx ! Go
latch.await // FIXME should await with timeout and fail if timeout
agent.close
tx.stop
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 {
import Actor.Sender.Self
akka.config.Config.config
se.scalablesolutions.akka.config.Config.config
val HOSTNAME = "localhost"
val PORT1 = 9990

View file

@ -5,6 +5,9 @@
package se.scalablesolutions.akka
import se.scalablesolutions.akka.serialization.Serializable
import sbinary._
import sbinary.Operations._
import sbinary.DefaultProtocol._
sealed abstract class TestMessage
case object Ping extends TestMessage

View file

@ -30,7 +30,7 @@ class PerformanceTest extends JUnitSuite {
case object BLUE 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 Change(colour: Colour)
@ -159,7 +159,7 @@ class PerformanceTest extends JUnitSuite {
case object BLUE 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 Change(colour: Colour)

View file

@ -18,7 +18,6 @@ object Log {
var oneWayLog: String = ""
}
@serializable class RemotePingPong1Actor extends Actor {
dispatcher = Dispatchers.newThreadBasedDispatcher(this)
def receive = {
@ -74,7 +73,8 @@ object Log {
class RemoteSupervisorTest extends JUnitSuite {
import Actor.Sender.Self
akka.config.Config.config
se.scalablesolutions.akka.config.Config.config
new Thread(new Runnable() {
def run = {
RemoteNode.start

View file

@ -60,7 +60,7 @@ class ServerInitiatedRemoteActorTest extends JUnitSuite {
import ServerInitiatedRemoteActorTest._
import Actor.Sender.Self
akka.config.Config.config
se.scalablesolutions.akka.config.Config.config
private val unit = TimeUnit.MILLISECONDS

View file

@ -13,7 +13,7 @@ object ActorShutdownRunner {
val myActor = new MyActor
myActor.start
myActor.send("test")
myActor ! "test"
myActor.stop
}
}

View file

@ -27,4 +27,9 @@ public class Foo extends se.scalablesolutions.akka.serialization.Serializable.Ja
if (true) throw new RuntimeException("expected");
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] = {
case a if routes.isDefinedAt(a) =>
if (self.sender.isDefined) routes(a) forward transform(a)
else routes(a) send transform(a)
else routes(a) ! transform(a)
}
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>
*/
trait CassandraSession extends Closeable with Flushable {
import scala.collection.jcl.Conversions._
import org.scala_tools.javautils.Imports._
import scala.collection.JavaConversions._
import java.util.{Map => JMap, List => JList}
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 = {
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)
}
@ -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, batch: Map[String, List[ColumnOrSuperColumn]]): Unit = ++|(key, batch)
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) {
val columnOrSuperColumn = new ColumnOrSuperColumn
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 {
_ ++| (name, batch, CONSISTENCY_LEVEL)

View file

@ -71,10 +71,6 @@ object SoftRefPool {
def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,SoftReferenceObjectPool] {
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] {

View file

@ -98,9 +98,20 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
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] = {
register
@ -112,9 +123,10 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
newAndUpdatedEntries.update(key, value)
}
def remove(key: K) = {
override def remove(key: K) = {
register
removedEntries.add(key)
newAndUpdatedEntries.get(key)
}
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 }
}
def iterator = elements
override def elements: Iterator[Tuple2[K, V]] = {
new Iterator[Tuple2[K, V]] {
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>
*/
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 updatedElems = TransactionalState.newMap[Int, 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)
}
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]
storage.getVectorStorageRangeFor(uuid, start, finish, count).foreach(buffer.append(_))
buffer
@ -389,14 +403,20 @@ trait PersistentQueue[A] extends scala.collection.mutable.Queue[A]
override def isEmpty: Boolean =
size == 0
override def +=(elem: A): Unit = enqueue(elem)
override def ++=(elems: Iterator[A]): Unit = enqueue(elems.toList: _*)
override def ++=(elems: Iterable[A]): Unit = this ++= elems.elements
override def +=(elem: A) = {
enqueue(elem)
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] =
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")
private def register = {
@ -490,7 +510,7 @@ trait PersistentSortedSet[A]
inStorage(elem) match {
case Some(f) => f
case None =>
throw new Predef.NoSuchElementException(elem + " not present")
throw new NoSuchElementException(elem + " not present")
}
}
@ -498,6 +518,11 @@ trait PersistentSortedSet[A]
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)] = {
// need to operate on the whole range
// get all from the underlying storage
@ -512,7 +537,7 @@ trait PersistentSortedSet[A]
else if (end >= l) (l - 1)
else end
// 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 = {

View file

@ -11,6 +11,8 @@ import se.scalablesolutions.akka.config.Config.config
import sjson.json.Serializer._
import java.util.NoSuchElementException
import com.mongodb._
import java.util.{Map=>JMap, List=>JList, ArrayList=>JArrayList}
@ -125,7 +127,7 @@ private[akka] object MongoStorageBackend extends
val m =
nullSafeFindOne(name) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(dbo) =>
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
}
@ -143,7 +145,7 @@ private[akka] object MongoStorageBackend extends
val m =
nullSafeFindOne(name) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(dbo) =>
dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
}
@ -162,7 +164,7 @@ private[akka] object MongoStorageBackend extends
else count
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 =
for(s <- n)
yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]]))
@ -181,7 +183,7 @@ private[akka] object MongoStorageBackend extends
}
} catch {
case e =>
throw new Predef.NoSuchElementException(e.getMessage)
throw new NoSuchElementException(e.getMessage)
}
}
@ -221,7 +223,7 @@ private[akka] object MongoStorageBackend extends
val o =
nullSafeFindOne(name) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(dbo) =>
dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
@ -230,7 +232,7 @@ private[akka] object MongoStorageBackend extends
o.get(index).asInstanceOf[Array[Byte]])
} catch {
case e =>
throw new Predef.NoSuchElementException(e.getMessage)
throw new NoSuchElementException(e.getMessage)
}
}
@ -240,7 +242,7 @@ private[akka] object MongoStorageBackend extends
val o =
nullSafeFindOne(name) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(dbo) =>
dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
@ -254,7 +256,7 @@ private[akka] object MongoStorageBackend extends
yield serializer.in[AnyRef](e.asInstanceOf[Array[Byte]])
} catch {
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 _root_.dispatch.json._
import _root_.dispatch.json.Js._
import java.util.NoSuchElementException
@scala.reflect.BeanInfo case class Foo(no: Int, name: String)
class MongoStorageSpec extends TestCase {
@ -111,12 +112,12 @@ class MongoStorageSpec extends TestCase {
try {
MongoStorageBackend.getVectorStorageEntryFor("U-A1", 1)
fail("should throw an exception")
} catch {case e: Predef.NoSuchElementException => {}}
} catch {case e: NoSuchElementException => {}}
try {
MongoStorageBackend.getVectorStorageRangeFor("U-A1", Some(2), None, 12)
fail("should throw an exception")
} catch {case e: Predef.NoSuchElementException => {}}
} catch {case e: NoSuchElementException => {}}
}
@Test
@ -198,7 +199,7 @@ class MongoStorageSpec extends TestCase {
try {
MongoStorageBackend.getMapStorageFor("U-M2")
fail("should throw an exception")
} catch {case e: Predef.NoSuchElementException => {}}
} catch {case e: NoSuchElementException => {}}
changeSetM.clear
}

View file

@ -109,7 +109,7 @@ private [akka] object RedisStorageBackend extends
def removeMapStorageFor(name: String): Unit = withErrorHandling {
db.keys("%s:*".format(encode(name.getBytes))) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(keys) =>
keys.foreach(db.delete(_))
}
@ -122,7 +122,7 @@ private [akka] object RedisStorageBackend extends
def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = withErrorHandling {
db.get(makeRedisKey(name, key)) match {
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)
}
}
@ -138,7 +138,7 @@ private [akka] object RedisStorageBackend extends
def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = withErrorHandling {
db.keys("%s:*".format(new String(encode(name.getBytes)))) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(keys) =>
keys.map(key => (makeKeyFromRedisKey(key)._2, db.get(key).get.getBytes)).toList
}
@ -174,7 +174,7 @@ private [akka] object RedisStorageBackend extends
.toList
case ((Some(s), None, c)) if c > 0 =>
wholeSorted.from(s)
.elements
.iterator
.take(count)
.map(e => (e._1.getBytes, e._2))
.toList
@ -206,7 +206,7 @@ private [akka] object RedisStorageBackend extends
def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = withErrorHandling {
db.lindex(new String(encode(name.getBytes)), index) match {
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
}
}
@ -226,7 +226,7 @@ private [akka] object RedisStorageBackend extends
else count
db.lrange(new String(encode(name.getBytes)), s, s + cnt - 1) match {
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) =>
l map (_.get.getBytes)
}
@ -235,7 +235,7 @@ private [akka] object RedisStorageBackend extends
def getVectorStorageSizeFor(name: String): Int = {
db.llen(new String(encode(name.getBytes))) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(l) => l
}
}
@ -247,7 +247,7 @@ private [akka] object RedisStorageBackend extends
def getRefStorageFor(name: String): Option[Array[Byte]] = withErrorHandling {
db.get(new String(encode(name.getBytes))) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(s) => Some(s.getBytes)
}
}
@ -256,7 +256,7 @@ private [akka] object RedisStorageBackend extends
db.incr(new String(encode(name.getBytes))) match {
case Some(i) => Some(i)
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 {
case Some(i) => Some(i)
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 {
case Some(i) => Some(i)
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 {
case Some(i) => Some(i)
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 {
db.lpop(new String(encode(name.getBytes))) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(s) =>
Some(s.getBytes)
}
@ -304,7 +304,7 @@ private [akka] object RedisStorageBackend extends
def size(name: String): Int = withErrorHandling {
db.llen(new String(encode(name.getBytes))) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(l) => l
}
}
@ -316,14 +316,14 @@ private [akka] object RedisStorageBackend extends
case 1 =>
db.lindex(new String(encode(name.getBytes)), start) match {
case None =>
throw new Predef.NoSuchElementException("No element at " + start)
throw new NoSuchElementException("No element at " + start)
case Some(s) =>
List(s.getBytes)
}
case n =>
db.lrange(new String(encode(name.getBytes)), start, start + count - 1) match {
case None =>
throw new Predef.NoSuchElementException(
throw new NoSuchElementException(
"No element found between " + start + " and " + (start + count - 1))
case Some(es) =>
es.map(_.get.getBytes)
@ -359,7 +359,7 @@ private [akka] object RedisStorageBackend extends
def zcard(name: String): Int = withErrorHandling {
db.zcard(new String(encode(name.getBytes))) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
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 {
db.zrange(new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC, false) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(s) =>
s.map(_.get.getBytes)
}
@ -384,7 +384,7 @@ private [akka] object RedisStorageBackend extends
db.zrangeWithScore(
new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC) match {
case None =>
throw new Predef.NoSuchElementException(name + " not present")
throw new NoSuchElementException(name + " not present")
case Some(l) =>
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)
assertEquals(BigInt(1000), (bactor !! Balance("a-123")).get)
assertEquals(7, (bactor !! LogSize).get)
val c: Int = (bactor !! LogSize).get
assertTrue(7 == c)
}
@Test
@ -131,7 +132,8 @@ class RedisPersistentActorSpec extends TestCase {
assertEquals(BigInt(5000), (bactor !! Balance("a-123")).get)
// should not count the failed one
assertEquals(3, (bactor !! LogSize).get)
val c: Int = (bactor !! LogSize).get
assertTrue(3 == c)
}
@Test
@ -152,6 +154,7 @@ class RedisPersistentActorSpec extends TestCase {
assertEquals(BigInt(5000), (bactor !! (Balance("a-123"), 5000)).get)
// 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-124")
qa !! NQ("a-125")
assertEquals(3, (qa !! SZ).get)
val t: Int = (qa !! SZ).get
assertTrue(3 == t)
}
@Test
@ -68,11 +69,13 @@ class RedisPersistentQSpec extends TestCase {
qa !! NQ("a-123")
qa !! NQ("a-124")
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-124", (qa !! DQ).get)
assertEquals("a-125", (qa !! DQ).get)
assertEquals(0, (qa !! SZ).get)
val t: Int = (qa !! SZ).get
assertTrue(0 == t)
}
@Test
@ -85,11 +88,14 @@ class RedisPersistentQSpec extends TestCase {
qa !! NQ("a-123")
qa !! NQ("a-124")
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(2, (qa !! SZ).get)
val s: Int = (qa !! SZ).get
assertTrue(2 == s)
qa !! MNDQ(List("a-126", "a-127"), 2, failer)
assertEquals(2, (qa !! SZ).get)
val u: Int = (qa !! SZ).get
assertTrue(2 == u)
}
@Test
@ -104,22 +110,26 @@ class RedisPersistentQSpec extends TestCase {
qa !! NQ("a-124")
qa !! NQ("a-125")
assertEquals(3, (qa !! SZ).get)
val t: Int = (qa !! SZ).get
assertTrue(3 == t)
// dequeue 1
assertEquals("a-123", (qa !! DQ).get)
// size == 2
assertEquals(2, (qa !! SZ).get)
val s: Int = (qa !! SZ).get
assertTrue(2 == s)
// enqueue 2, dequeue 2 => size == 2
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
qa !! NQ("a-128")
qa !! NQ("a-129")
assertEquals(4, (qa !! SZ).get)
val v: Int = (qa !! SZ).get
assertTrue(4 == v)
// enqueue 1 => size 5
// dequeue 6 => fail transaction
@ -128,6 +138,7 @@ class RedisPersistentQSpec extends TestCase {
qa !! MNDQ(List("a-130"), 6, failer)
} 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)
}
catch {
case e: Predef.NoSuchElementException =>
case e: NoSuchElementException =>
e.getMessage should endWith("not present")
}
}

View file

@ -7,6 +7,7 @@ import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import se.scalablesolutions.akka.serialization.Serializable
import se.scalablesolutions.akka.serialization.Serializer._
import RedisStorageBackend._
@ -38,16 +39,19 @@ class RedisStorageBackendSpec extends
"T-1", "debasish.language".getBytes).get) should equal("java")
}
/**
it("should enter a custom object for transaction T-1") {
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)
}
**/
it("should enter key/values for another transaction T-2") {
insertMapStorageEntryFor("T-2", "debasish.age".getBytes, "49".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)
}
@ -94,9 +98,10 @@ class RedisStorageBackendSpec extends
insertVectorStorageEntryFor("T-3", "debasish".getBytes)
insertVectorStorageEntryFor("T-3", "maulindu".getBytes)
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)
getVectorStorageSizeFor("T-3") should equal(4)
getVectorStorageSizeFor("T-3") should equal(3)
}
}
@ -108,9 +113,11 @@ class RedisStorageBackendSpec extends
insertRefStorageFor("T-4", "1200".getBytes)
new String(getRefStorageFor("T-4").get) should equal("1200")
val n = Name(100, "debasish", "kolkata")
insertRefStorageFor("T-4", n.toBytes)
n.fromBytes(getRefStorageFor("T-4").get) should equal(n)
// val n = Name(100, "debasish", "kolkata")
// insertRefStorageFor("T-4", Java.out(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)
extends Serializable.SBinary[Name] {
import sbinary._
import sbinary.Operations._
import sbinary.DefaultProtocol._
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>
*/
class AkkaServlet extends ServletContainer {
import org.scala_tools.javautils.Imports._
import scala.collection.JavaConversions._
override def initiate(resourceConfig: ResourceConfig, webApplication: WebApplication) = {
val configurators = ConfiguratorRepository.getConfigurators
resourceConfig.getClasses.addAll(configurators.flatMap(_.getComponentInterfaces).asJava)
resourceConfig.getClasses.addAll(configurators.flatMap(_.getComponentInterfaces))
resourceConfig.getProperties.put(
"com.sun.jersey.spi.container.ResourceFilters",
config.getList("akka.rest.filters").mkString(","))

View file

@ -166,7 +166,7 @@ trait ChatManagement { this: Actor =>
* Creates and links a RedisChatStorage.
*/
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.Loc._
import _root_.net.liftweb.http.auth._
import _root_.net.liftweb.common._
import Helpers._
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
* to modify lift's environment
*/
class Boot {
class Boot extends Logging {
def boot {
// where to search snippet
LiftRules.addToPackages("sample.lift")
LiftRules.httpAuthProtectedResource.prepend {
case (ParsePath("liftcount" :: Nil, _, _, _)) => Full(AuthRole("admin"))
case (Req("liftcount" :: Nil, _, _)) => Full(AuthRole("admin"))
}
LiftRules.authentication = HttpBasicAuthentication("lift") {
case ("someuser", "1234", req) => {
Log.info("You are now authenticated !")
log.info("You are now authenticated !")
userRoles(AuthRole("admin"))
true
}

View file

@ -1,4 +1,4 @@
import _root_.bootstrap.liftweb.Boot
/*import _root_.bootstrap.liftweb.Boot
import _root_.scala.tools.nsc.MainGenericRunner
object LiftConsole {
@ -13,3 +13,4 @@ object LiftConsole {
exit(0)
}
}
*/

View file

@ -87,7 +87,7 @@ class PubSub extends Actor {
@Broadcast
@Path("/topic/{topic}/{message}/")
@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 receive = { case _ => }
@ -148,7 +148,7 @@ class Chat extends Actor {
@POST
@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"))
@Produces(Array("text/html"))
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) = {
val m = an.toString
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.config.ScalaConfig._
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
class Boot {

View file

@ -93,8 +93,8 @@ class AkkaSecurityFilterFactory extends ResourceFilterFactory with Logging {
case Some(r) if r.isInstanceOf[Response] =>
throw new WebApplicationException(r.asInstanceOf[Response])
case None => throw new WebApplicationException(408)
case x => {
log.error("Authenticator replied with unexpected result [%s]", x);
case unknown => {
log.warning("Authenticator replied with unexpected result [%s]", unknown);
throw new WebApplicationException(Response.Status.INTERNAL_SERVER_ERROR)
}
}
@ -257,9 +257,9 @@ trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] {
protected val invalidateNonces: PartialFunction[Any, Unit] = {
case InvalidateNonces =>
val ts = System.currentTimeMillis
nonceMap.retain((k, v) => (ts - v) < nonceValidityPeriod)
case e =>
log.info("Don't know what to do with: " + e)
nonceMap.filter(tuple => (ts - tuple._2) < nonceValidityPeriod)
case unknown =>
log.error("Don't know what to do with: ", unknown)
}
//Schedule the invalidation of nonces

View file

@ -4,7 +4,7 @@
package se.scalablesolutions.akka.security
import config.ScalaConfig._
import se.scalablesolutions.akka.config.ScalaConfig._
import org.scalatest.Suite
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>
*/
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>
<akka>
version = "0.7"
version = "0.8"
# FQN to the class doing initial active object/actor
# 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>
<groupId>sbinary</groupId>
<artifactId>sbinary</artifactId>
<version>0.3</version>
<version>2.8.0.Beta1-2.8.0.Beta1-0.3.1-SNAPSHOT</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>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.name=akka
project.version=0.7
scala.version=2.7.7
project.version=0.8
scala.version=2.8.0.Beta1
sbt.version=0.7.1
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 java.io.File
import sbt.CompileOrder._
import scala.Array
import java.util.jar.Attributes
import java.util.jar.Attributes.Name._
import java.io.File
class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
// ------------------------------------------------------------
// project versions
val JERSEY_VERSION = "1.1.5"
val ATMO_VERSION = "0.5.4"
val CASSANDRA_VERSION = "0.5.0"
abstract class AkkaDefaults(info: ProjectInfo) extends DefaultProject(info) with AutoCompilerPlugins {
// ------------------------------------------------------------
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")
home
}
val encodingUtf8 = List("-encoding", "UTF-8")
lazy val deployPath = Path.fromFile(new java.io.File(akkaHome + "/deploy"))
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.")
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
val embeddedrepo = "embedded repo" at new File(akkaHome, "embedded-repo").toURI.toString
val sunjdmk = "sunjdmk" at "http://wp5.e-taxonomy.eu/cdmlib/mavenrepo"
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_snapshots = "Codehaus Snapshots" at "http://snapshots.repository.codehaus.org"
val jboss = "jBoss" at "http://repository.jboss.org/maven2"
val guiceyfruit = "GuiceyFruit" at "http://guiceyfruit.googlecode.com/svn/repo/releases/"
val google = "google" at "http://google-maven-repository.googlecode.com/svn/repository"
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
@ -97,107 +241,46 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
// examples
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
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 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_htdisttp = "net.databinder" % "dispatch-http_2.7.7" % "0.6.4" % "compile"
val sjson = "sjson.json" % "sjson" % "0.4" % "compile"
val sbinary = "sbinary" % "sbinary" % "0.3" % "compile"
val dispatch_json = "net.databinder" % "dispatch-json_2.8.0.Beta1" % "0.6.6" % "compile"
val dispatch_htdisttp = "net.databinder" % "dispatch-http_2.8.0.Beta1" % "0.6.6" % "compile"
val sjson = "sjson.json" % "sjson" % "0.5-SNAPSHOT-2.8.Beta1" % "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_core = "org.codehaus.jackson" % "jackson-core-asl" % "1.2.1" % "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"
// testing
val scalatest = "org.scalatest" % "scalatest" % "1.0" % "test"
val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "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_core = "org.codehaus.aspectwerkz" % "aspectwerkz-jdk5" % "2.1" % "compile"
val configgy = "net.lag" % "configgy" % "1.4.7" % "compile"
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
val configgy = "net.lag" % "configgy" % "2.8.0.Beta1-1.5-SNAPSHOT" % "compile"
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 protobuf = "com.google.protobuf" % "protobuf-java" % "2.2.0" % "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 rabbit = "com.rabbitmq" % "amqp-client" % "1.7.2"
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
val rabbit = "com.rabbitmq" % "amqp-client" % "1.7.2" % "compile"
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 stax_api = "javax.xml.stream" % "stax-api" % "1.0-2" % "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_contrib = "com.sun.jersey.contribs" % "jersey-scala" % JERSEY_VERSION % "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 servlet = "javax.servlet" % "servlet-api" % "2.5" % "compile"
val atmo = "org.atmosphere" % "atmosphere-annotations" % ATMO_VERSION % "compile"
val atmo_jersey = "org.atmosphere" % "atmosphere-jersey" % 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"
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
val scalatest = "org.scalatest" % "scalatest" % "1.0" % "test"
val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "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 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 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
val scalatest = "org.scalatest" % "scalatest" % "1.0" % "test"
val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "test"
val junit = "junit" % "junit" % "4.5" % "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 commons_pool = "commons-pool" % "commons-pool" % "1.5.1" % "compile"
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
val commons_pool = "commons-pool" % "commons-pool" % "1.5.4" % "compile"
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
}
class AkkaRedisProject(info: ProjectInfo) extends DefaultProject(info) {
val redis = "com.redis" % "redisclient" % "1.2" % "compile"
class AkkaRedisProject(info: ProjectInfo) extends AkkaDefaults(info) {
val redis = "com.redis" % "redisclient" % "2.8.0.Beta1-1.2" % "compile"
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"
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 high_scale = "org.apache.cassandra" % "high-scale-lib" % 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 log4j = "log4j" % "log4j" % "1.2.15" % "test"
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) {
@ -285,15 +369,15 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
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"
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_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) {
@ -303,24 +387,25 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
new AkkaShoalProject(_), akka_core)
}
class AkkaKernelProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val dist = deployTask(info, distPath) dependsOn(`package`) describedAs("Deploying")
class AkkaKernelProject(info: ProjectInfo) extends AkkaDefaults(info) {
lazy val dist = deployTask(info, distPath, true, true, true) dependsOn(`package`, packageDocs, packageSrc) describedAs("Deploying")
}
class AkkaSpringProject(info: ProjectInfo) extends DefaultProject(info) {
val spring_beans = "org.springframework" % "spring-beans" % "3.0.1.RELEASE"
val spring_context = "org.springframework" % "spring-context" % "3.0.1.RELEASE"
class AkkaSpringProject(info: ProjectInfo) extends AkkaDefaults(info) {
val spring_beans = "org.springframework" % "spring-beans" % "3.0.1.RELEASE" % "compile"
val spring_context = "org.springframework" % "spring-context" % "3.0.1.RELEASE" % "compile"
// testing
val scalatest = "org.scalatest" % "scalatest" % "1.0" % "test"
val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "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
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 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 jersey_server = "com.sun.jersey" % "jersey-server" % 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"
}
class AkkaSampleChatProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val dist = deployTask(info, deployPath) dependsOn(`package`) describedAs("Deploying")
class AkkaSampleChatProject(info: ProjectInfo) extends AkkaDefaults(info) {
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 lift = "net.liftweb" % "lift-webkit" % "1.1-M6" % "compile"
val lift_util = "net.liftweb" % "lift-util" % "1.1-M6" % "compile"
val lift = "net.liftweb" % "lift-webkit" % LIFT_VERSION % "compile"
val lift_util = "net.liftweb" % "lift-util" % LIFT_VERSION % "compile"
val servlet = "javax.servlet" % "servlet-api" % "2.5" % "compile"
// testing
val jetty = "org.mortbay.jetty" % "jetty" % "6.1.22" % "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) {
lazy val dist = deployTask(info, deployPath) dependsOn(`package`) describedAs("Deploying")
class AkkaSampleRestJavaProject(info: ProjectInfo) extends AkkaDefaults(info) {
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"
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 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_jms = "org.apache.camel" % "camel-jms" % "2.2.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 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"
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) {
@ -384,42 +469,4 @@ class AkkaParent(info: ProjectInfo) extends DefaultProject(info) {
lazy val akka_sample_security = project("akka-sample-security", "akka-sample-security",
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
cd $AKKA_HOME
VERSION=akka_2.7.7-0.7
TARGET_DIR=dist/$1
VERSION=akka_2.8.0.Beta1-0.8
TARGET_DIR=dist/$VERSION/$1
shift 1
VMARGS=$@