Merge branch 'master' of git@github.com:jboner/akka

Conflicts:
	akka-actor/src/main/scala/dispatch/Dispatchers.scala
	akka-actor/src/main/scala/dispatch/MessageHandling.scala
	akka-actor/src/main/scala/util/ReflectiveAccess.scala
This commit is contained in:
Jonas Bonér 2010-09-24 22:49:28 +02:00
commit fe42fdf782
69 changed files with 2026 additions and 633 deletions

View file

@ -1,59 +0,0 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package se.scalablesolutions.akka.util;
/**
* NOTE:
* <p/>
* This code is based on code from the [Plasmid Replication Engine] project.
* <br/>
* Licensed under [Mozilla Public License 1.0 (MPL)].
* <p/>
* Original JavaDoc:
* <p/>
* Our distributed objects are generally named most efficiently (and cleanly)
* by their UUID's. This class provides some static helpers for using UUID's.
* If it was efficient to do in Java, I would make the uuid an normal class
* and use instances of it. However, in current JVM's, we would end up using an
* Object to represent a long, which is pretty expensive. Maybe someday. ###
* <p/>
* UUID format: currently using currentTimeMillis() for the low bits. This uses
* about 40 bits for the next 1000 years, leaving 24 bits for debugging
* and consistency data. I'm using 8 of those for a magic asci 'U' byte.
* <p/>
* Future: use one instance of Uuid per type of object for better performance
* and more detailed info (instance could be matched to its uuid's via a map or
* array). This all static version bites.###
*/
public final class UUID {
public static final long UUID_NONE = 0;
public static final long UUID_WILD = -1;
public static final long UUID_MAGICMASK = 0xff << 56;
public static final long UUID_MAGIC = 'U' << 56;
protected static long lastTime;
/**
* Generate and return a new Universally Unique ID.
* Happens to be monotonically increasing.
*/
public synchronized static long newUuid() {
long time = System.currentTimeMillis();
if (time <= lastTime) {
time = lastTime + 1;
}
lastTime = time;
return UUID_MAGIC | time;
}
/**
* Returns true if uuid could have been generated by Uuid.
*/
public static boolean isValid(final long uuid) {
return (uuid & UUID_MAGICMASK) == UUID_MAGIC
&& (uuid & ~UUID_MAGICMASK) != 0;
}
}

View file

@ -78,11 +78,10 @@ trait ActorRef extends
java.lang.Comparable[ActorRef] { scalaRef: ScalaActorRef =>
// Only mutable for RemoteServer in order to maintain identity across nodes
@volatile protected[akka] var _uuid = UUID.newUuid.toString
@volatile protected[akka] var _uuid = newUuid
@volatile protected[this] var _status: ActorRefStatus.StatusType = ActorRefStatus.UNSTARTED
@volatile protected[akka] var _homeAddress = new InetSocketAddress(RemoteServerModule.HOSTNAME, RemoteServerModule.PORT)
@volatile protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None
@volatile protected[akka] var registeredInRemoteNodeDuringSerialization = false
protected[akka] val guard = new ReentrantGuard
/**
@ -95,7 +94,7 @@ trait ActorRef extends
* that you can use a custom name to be able to retrieve the "correct" persisted state
* upon restart, remote restart etc.
*/
@BeanProperty @volatile var id: String = _uuid
@BeanProperty @volatile var id: String = _uuid.toString
/**
* User overridable callback/setting.
@ -213,7 +212,7 @@ trait ActorRef extends
/**
* Comparison only takes uuid into account.
*/
def compareTo(other: ActorRef) = this.uuid.compareTo(other.uuid)
def compareTo(other: ActorRef) = this.uuid compareTo other.uuid
/**
* Returns the uuid for the actor.
@ -266,7 +265,7 @@ trait ActorRef extends
/**
* Only for internal use. UUID is effectively final.
*/
protected[akka] def uuid_=(uid: String) = _uuid = uid
protected[akka] def uuid_=(uid: Uuid) = _uuid = uid
/**
* Akka Java API
@ -622,9 +621,9 @@ trait ActorRef extends
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit
protected[akka] def registerSupervisorAsRemoteActor: Option[String]
protected[akka] def registerSupervisorAsRemoteActor: Option[Uuid]
protected[akka] def linkedActors: JMap[String, ActorRef]
protected[akka] def linkedActors: JMap[Uuid, ActorRef]
protected[akka] def linkedActorsAsList: List[ActorRef]
@ -665,7 +664,7 @@ class LocalActorRef private[akka](
extends ActorRef with ScalaActorRef {
@volatile private[akka] var _remoteAddress: Option[InetSocketAddress] = None // only mutable to maintain identity across nodes
@volatile private[akka] var _linkedActors: Option[ConcurrentHashMap[String, ActorRef]] = None
@volatile private[akka] var _linkedActors: Option[ConcurrentHashMap[Uuid, ActorRef]] = None
@volatile private[akka] var _supervisor: Option[ActorRef] = None
@volatile private var isInInitialization = false
@volatile private var runActorInitialization = false
@ -687,7 +686,7 @@ class LocalActorRef private[akka](
private[akka] def this(factory: () => Actor) = this(Right(Some(factory)))
// used only for deserialization
private[akka] def this(__uuid: String,
private[akka] def this(__uuid: Uuid,
__id: String,
__hostname: String,
__port: Int,
@ -1114,7 +1113,7 @@ class LocalActorRef private[akka](
}
}
protected[akka] def registerSupervisorAsRemoteActor: Option[String] = guard.withGuard {
protected[akka] def registerSupervisorAsRemoteActor: Option[Uuid] = guard.withGuard {
ensureRemotingEnabled
if (_supervisor.isDefined) {
remoteAddress.foreach(address => RemoteClientModule.registerSupervisorForActor(address, this))
@ -1122,9 +1121,9 @@ class LocalActorRef private[akka](
} else None
}
protected[akka] def linkedActors: JMap[String, ActorRef] = guard.withGuard {
protected[akka] def linkedActors: JMap[Uuid, ActorRef] = guard.withGuard {
if (_linkedActors.isEmpty) {
val actors = new ConcurrentHashMap[String, ActorRef]
val actors = new ConcurrentHashMap[Uuid, ActorRef]
_linkedActors = Some(actors)
actors
} else _linkedActors.get
@ -1408,7 +1407,7 @@ private[akka] case class RemoteActorRef private[akka] (
*/
def actorClassName: String = className
protected[akka] def registerSupervisorAsRemoteActor: Option[String] = None
protected[akka] def registerSupervisorAsRemoteActor: Option[Uuid] = None
val remoteAddress: Option[InetSocketAddress] = Some(new InetSocketAddress(hostname, port))
@ -1437,7 +1436,7 @@ private[akka] case class RemoteActorRef private[akka] (
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def linkedActors: JMap[String, ActorRef] = unsupported
protected[akka] def linkedActors: JMap[Uuid, ActorRef] = unsupported
protected[akka] def linkedActorsAsList: List[ActorRef] = unsupported
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported
protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported
@ -1460,7 +1459,7 @@ trait ActorRefShared {
/**
* Returns the uuid for the actor.
*/
def uuid: String
def uuid: Uuid
/**
* Shuts down and removes all linked actors.

View file

@ -35,7 +35,7 @@ case class ActorUnregistered(actor: ActorRef) extends ActorRegistryEvent
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object ActorRegistry extends ListenerManagement {
private val actorsByUUID = new ConcurrentHashMap[String, ActorRef]
private val actorsByUUID = new ConcurrentHashMap[Uuid, ActorRef]
private val actorsById = new Index[String,ActorRef]
/**
@ -112,7 +112,7 @@ object ActorRegistry extends ListenerManagement {
/**
* Finds the actor that has a specific UUID.
*/
def actorFor(uuid: String): Option[ActorRef] = Option(actorsByUUID get uuid)
def actorFor(uuid: Uuid): Option[ActorRef] = Option(actorsByUUID get uuid)
/**
* Registers an actor in the ActorRegistry.

View file

@ -12,4 +12,11 @@ package object actor {
implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef =
ref.asInstanceOf[ActorRef]
type Uuid = com.eaio.uuid.UUID
def newUuid(): Uuid = new Uuid()
def uuidFrom(time: Long, clockSeqAndNode: Long): Uuid = new Uuid(time,clockSeqAndNode)
def uuidFrom(uuid: String): Uuid = {
new Uuid(uuid)
}
}

View file

@ -6,7 +6,8 @@ package se.scalablesolutions.akka.dispatch
import se.scalablesolutions.akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.config.Config._
import se.scalablesolutions.akka.util.{Duration, Logging, UUID}
import se.scalablesolutions.akka.util.{Duration, Logging}
import se.scalablesolutions.akka.actor.newUuid
import net.lag.configgy.ConfigMap
@ -169,7 +170,7 @@ object Dispatchers extends Logging {
* Throws: IllegalArgumentException if the value of "type" is not valid
*/
def from(cfg: ConfigMap): Option[MessageDispatcher] = {
lazy val name = cfg.getString("name", UUID.newUuid.toString)
lazy val name = cfg.getString("name", newUuid.toString)
def threadPoolConfig(b: ThreadPoolBuilder) {
b.configureIfPossible( builder => {

View file

@ -4,7 +4,7 @@
package se.scalablesolutions.akka.dispatch
import se.scalablesolutions.akka.actor.{Actor, ActorRef, ActorInitializationException}
import se.scalablesolutions.akka.actor.{Actor, ActorRef, Uuid, ActorInitializationException}
import se.scalablesolutions.akka.util.{SimpleLock, Duration, HashCode, Logging}
import se.scalablesolutions.akka.util.ReflectiveAccess.EnterpriseModule
import se.scalablesolutions.akka.AkkaException
@ -13,7 +13,6 @@ import org.multiverse.commitbarriers.CountDownCommitBarrier
import java.util.{Queue, List}
import java.util.concurrent._
import concurrent.forkjoin.LinkedTransferQueue
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
@ -62,7 +61,7 @@ final class MessageInvocation(val receiver: ActorRef,
*/
trait MessageDispatcher extends MailboxFactory with Logging {
protected val uuids = new ConcurrentSkipListSet[String]
protected val uuids = new ConcurrentSkipListSet[Uuid]
def dispatch(invocation: MessageInvocation): Unit

View file

@ -4,7 +4,7 @@
package se.scalablesolutions.akka.stm
import se.scalablesolutions.akka.util.UUID
import se.scalablesolutions.akka.actor.{newUuid, Uuid}
import org.multiverse.transactional.refs.BasicRef
@ -34,7 +34,7 @@ class Ref[T](initialValue: T) extends BasicRef[T](initialValue) with Transaction
def this() = this(null.asInstanceOf[T])
val uuid = UUID.newUuid.toString
val uuid = newUuid.toString
def swap(elem: T) = set(elem)

View file

@ -6,7 +6,7 @@ package se.scalablesolutions.akka.stm
import scala.collection.immutable.HashMap
import se.scalablesolutions.akka.util.UUID
import se.scalablesolutions.akka.actor.{newUuid}
import org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction
@ -24,7 +24,7 @@ object TransactionalMap {
class TransactionalMap[K, V](initialValue: HashMap[K, V]) extends Transactional with scala.collection.mutable.Map[K, V] {
def this() = this(HashMap[K, V]())
val uuid = UUID.newUuid.toString
val uuid = newUuid.toString
private[this] val ref = Ref(initialValue)

View file

@ -6,7 +6,7 @@ package se.scalablesolutions.akka.stm
import scala.collection.immutable.Vector
import se.scalablesolutions.akka.util.UUID
import se.scalablesolutions.akka.actor.newUuid
import org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction
@ -24,7 +24,7 @@ object TransactionalVector {
class TransactionalVector[T](initialValue: Vector[T]) extends Transactional with IndexedSeq[T] {
def this() = this(Vector[T]())
val uuid = UUID.newUuid.toString
val uuid = newUuid.toString
private[this] val ref = Ref(initialValue)

View file

@ -4,7 +4,8 @@
package se.scalablesolutions.akka
import se.scalablesolutions.akka.util.{UUID, Logging}
import se.scalablesolutions.akka.util.Logging
import se.scalablesolutions.akka.actor.newUuid
import java.io.{StringWriter, PrintWriter}
import java.net.{InetAddress, UnknownHostException}
@ -23,7 +24,7 @@ import java.net.{InetAddress, UnknownHostException}
import AkkaException._
val exceptionName = getClass.getName
val uuid = "%s_%s".format(hostname, UUID.newUuid.toString)
val uuid = "%s_%s".format(hostname, newUuid)
override val toString = "%s\n\t[%s]\n\t%s\n\t%s".format(exceptionName, uuid, message, stackTrace)

View file

@ -4,7 +4,7 @@
package se.scalablesolutions.akka.util
import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException, ActorType}
import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException, ActorType, Uuid}
import se.scalablesolutions.akka.dispatch.{Future, CompletableFuture, MessageInvocation}
import se.scalablesolutions.akka.config.{Config, ModuleNotAvailableException}
import se.scalablesolutions.akka.stm.Transaction
@ -53,8 +53,8 @@ object ReflectiveAccess extends Logging {
}
type RemoteClientObject = {
def register(hostname: String, port: Int, uuid: String): Unit
def unregister(hostname: String, port: Int, uuid: String): Unit
def register(hostname: String, port: Int, uuid: Uuid): Unit
def unregister(hostname: String, port: Int, uuid: Uuid): Unit
def clientFor(address: InetSocketAddress): RemoteClient
def clientFor(hostname: String, port: Int, loader: Option[ClassLoader]): RemoteClient
}
@ -67,12 +67,12 @@ object ReflectiveAccess extends Logging {
val remoteClientObjectInstance: Option[RemoteClientObject] =
getObjectFor("se.scalablesolutions.akka.remote.RemoteClient$")
def register(address: InetSocketAddress, uuid: String) = {
def register(address: InetSocketAddress, uuid: Uuid) = {
ensureRemotingEnabled
remoteClientObjectInstance.get.register(address.getHostName, address.getPort, uuid)
}
def unregister(address: InetSocketAddress, uuid: String) = {
def unregister(address: InetSocketAddress, uuid: Uuid) = {
ensureRemotingEnabled
remoteClientObjectInstance.get.unregister(address.getHostName, address.getPort, uuid)
}
@ -114,7 +114,7 @@ object ReflectiveAccess extends Logging {
val PORT = Config.config.getInt("akka.remote.server.port", 9999)
type RemoteServerObject = {
def registerActor(address: InetSocketAddress, uuid: String, actor: ActorRef): Unit
def registerActor(address: InetSocketAddress, uuid: Uuid, actor: ActorRef): Unit
def registerTypedActor(address: InetSocketAddress, name: String, typedActor: AnyRef): Unit
}
@ -128,7 +128,7 @@ object ReflectiveAccess extends Logging {
val remoteNodeObjectInstance: Option[RemoteNodeObject] =
getObjectFor("se.scalablesolutions.akka.remote.RemoteNode$")
def registerActor(address: InetSocketAddress, uuid: String, actorRef: ActorRef) = {
def registerActor(address: InetSocketAddress, uuid: Uuid, actorRef: ActorRef) = {
ensureRemotingEnabled
remoteServerObjectInstance.get.registerActor(address, uuid, actorRef)
}
@ -223,12 +223,12 @@ object ReflectiveAccess extends Logging {
"Feature is only available in Akka Enterprise")
def createFileBasedMailbox(
name: String, actorType: ActorType, typedActorInfo: Option[Tuple2[String, String]]): FileBasedMailbox = {
uuid: Uuid, actorType: ActorType, typedActorInfo: Option[Tuple2[String, String]]): FileBasedMailbox = {
ensureEnterpriseEnabled
createInstance(
"se.scalablesolutions.akka.cluster.FileBasedMailbox",
Array(classOf[String], classOf[ActorType], classOf[Option[Tuple2[String, String]]]),
Array(name, actorType, typedActorInfo).asInstanceOf[Array[AnyRef]],
Array(classOf[Uuid], classOf[ActorType], classOf[Option[Tuple2[String, String]]]),
Array(uuid, actorType, typedActorInfo).asInstanceOf[Array[AnyRef]],
loader)
.getOrElse(throw new IllegalActorStateException("Could not create file-based mailbox"))
.asInstanceOf[FileBasedMailbox]
@ -269,10 +269,10 @@ object ReflectiveAccess extends Logging {
Some(ctor.newInstance(args: _*).asInstanceOf[T])
} catch {
case e: java.lang.reflect.InvocationTargetException =>
log.error(e.getCause, "Could not instantiate class [%s]", fqn)
log.error(e.getCause, "Could not instantiate class [%s] due to [%s]", fqn, e.toString)
None
case e: Exception =>
log.error(e.getCause, "Could not instantiate class [%s]", fqn)
log.error(e.getCause, "Could not instantiate class [%s] due to [%s]", fqn, e.toString)
None
}

View file

@ -10,7 +10,7 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
object ExecutorBasedEventDrivenDispatcherActorSpec {
class TestActor extends Actor {
self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid)
self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid.toString)
def receive = {
case "Hello" =>
self.reply("World")
@ -23,7 +23,7 @@ object ExecutorBasedEventDrivenDispatcherActorSpec {
val oneWay = new CountDownLatch(1)
}
class OneWayTestActor extends Actor {
self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid)
self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid.toString)
def receive = {
case "OneWay" => OneWayTestActor.oneWay.countDown
}

View file

@ -13,7 +13,7 @@ import se.scalablesolutions.akka.camel.component.TypedActorComponent
import se.scalablesolutions.akka.util.Logging
/**
* Defines the lifecycle of a CamelContext. Allowed state transitions are
* Manages the lifecycle of a CamelContext. Allowed transitions are
* init -> start -> stop -> init -> ... etc.
*
* @author Martin Krasser
@ -35,7 +35,7 @@ trait CamelContextLifecycle extends Logging {
/**
* Registry in which typed actors are TEMPORARILY registered during
* creation of Camel routes to typed actors.
* creation of Camel routes to these actors.
*/
private[camel] var typedActorRegistry: Map[String, AnyRef] = _
@ -63,7 +63,7 @@ trait CamelContextLifecycle extends Logging {
def started = _started
/**
* Starts the CamelContext and ProducerTemplate.
* Starts the CamelContext and an associated ProducerTemplate.
*/
def start = {
context.start
@ -73,7 +73,7 @@ trait CamelContextLifecycle extends Logging {
}
/**
* Stops the CamelContext and ProducerTemplate.
* Stops the CamelContext and the associated ProducerTemplate.
*/
def stop = {
template.stop
@ -90,11 +90,10 @@ trait CamelContextLifecycle extends Logging {
/**
* Initializes this lifecycle object with the given CamelContext. For the passed
* CamelContext stream-caching is enabled. If applications want to disable stream-
* CamelContext, stream-caching is enabled. If applications want to disable stream-
* caching they can do so after this method returned and prior to calling start.
* This method also registers a new
* {@link se.scalablesolutions.akka.camel.component.TypedActorComponent} at
* <code>context</code> under a name defined by TypedActorComponent.InternalSchema.
* This method also registers a new TypedActorComponent at the passes CamelContext
* under a name defined by TypedActorComponent.InternalSchema.
*/
def init(context: CamelContext) {
this.typedActorComponent = new TypedActorComponent
@ -109,8 +108,7 @@ trait CamelContextLifecycle extends Logging {
}
/**
* Makes a global CamelContext and ProducerTemplate accessible to applications. The lifecycle
* of these objects is managed by se.scalablesolutions.akka.camel.CamelService.
* Manages a global CamelContext and an associated ProducerTemplate.
*/
object CamelContextManager extends CamelContextLifecycle {
override def context: CamelContext = super.context

View file

@ -12,9 +12,10 @@ import se.scalablesolutions.akka.actor.{AspectInitRegistry, ActorRegistry}
import se.scalablesolutions.akka.util.{Bootable, Logging}
/**
* Used by applications (and the Kernel) to publish consumer actors and typed actors via
* Camel endpoints and to manage the life cycle of a a global CamelContext which can be
* accessed via <code>se.scalablesolutions.akka.camel.CamelContextManager.context</code>.
* Publishes (untyped) consumer actors and typed consumer actors via Camel endpoints. Actors
* are published (asynchronously) when they are started and unpublished (asynchronously) when
* they are stopped. The CamelService is notified about actor start- and stop-events by
* registering listeners at ActorRegistry and AspectInitRegistry.
*
* @author Martin Krasser
*/
@ -29,11 +30,11 @@ trait CamelService extends Bootable with Logging {
AspectInitRegistry.addListener(publishRequestor)
/**
* Starts the CamelService. Any started actor that is a consumer actor will be (asynchronously)
* Starts this CamelService. Any started actor that is a consumer actor will be (asynchronously)
* published as Camel endpoint. Consumer actors that are started after this method returned will
* be published as well. Actor publishing is done asynchronously. A started (loaded) CamelService
* also publishes <code>@consume</code> annotated methods of typed actors that have been created
* with <code>TypedActor.newInstance(..)</code> (and <code>TypedActor.newInstance(..)</code>
* with <code>TypedActor.newInstance(..)</code> (and <code>TypedActor.newRemoteInstance(..)</code>
* on a remote node).
*/
abstract override def onLoad = {
@ -54,7 +55,8 @@ trait CamelService extends Bootable with Logging {
}
/**
* Stops the CamelService.
* Stops this CamelService. All published consumer actors and typed consumer actor methods will be
* unpublished asynchronously.
*/
abstract override def onUnload = {
// Unregister this instance as current CamelService
@ -98,24 +100,24 @@ trait CamelService extends Bootable with Logging {
def stop = onUnload
/**
* Sets an expectation of the number of upcoming endpoint activations and returns
* a {@link CountDownLatch} that can be used to wait for the activations to occur.
* Endpoint activations that occurred in the past are not considered.
* Sets an expectation on the number of upcoming endpoint activations and returns
* a CountDownLatch that can be used to wait for the activations to occur. Endpoint
* activations that occurred in the past are not considered.
*/
def expectEndpointActivationCount(count: Int): CountDownLatch =
(consumerPublisher !! SetExpectedRegistrationCount(count)).as[CountDownLatch].get
/**
* Sets an expectation of the number of upcoming endpoint de-activations and returns
* a {@link CountDownLatch} that can be used to wait for the de-activations to occur.
* Endpoint de-activations that occurred in the past are not considered.
* Sets an expectation on the number of upcoming endpoint de-activations and returns
* a CountDownLatch that can be used to wait for the de-activations to occur. Endpoint
* de-activations that occurred in the past are not considered.
*/
def expectEndpointDeactivationCount(count: Int): CountDownLatch =
(consumerPublisher !! SetExpectedUnregistrationCount(count)).as[CountDownLatch].get
}
/**
* ...
* Manages a global CamelService (the 'current' CamelService).
*
* @author Martin Krasser
*/
@ -128,11 +130,17 @@ object CamelServiceManager {
/**
* Starts a new CamelService and makes it the current CamelService.
*
* @see CamelService#start
* @see CamelService#onLoad
*/
def startCamelService = CamelServiceFactory.createCamelService.start
/**
* Stops the current CamelService.
*
* @see CamelService#stop
* @see CamelService#onUnload
*/
def stopCamelService = service.stop
@ -159,12 +167,12 @@ object CamelServiceManager {
*/
object CamelServiceFactory {
/**
* Creates a new CamelService instance
* Creates a new CamelService instance.
*/
def createCamelService: CamelService = new CamelService { }
/**
* Creates a new CamelService instance
* Creates a new CamelService instance and initializes it with the given CamelContext.
*/
def createCamelService(camelContext: CamelContext): CamelService = {
CamelContextManager.init(camelContext)

View file

@ -20,30 +20,24 @@ trait Consumer { self: Actor =>
def endpointUri: String
/**
* Determines whether two-way communications with this consumer actor should
* be done in blocking or non-blocking mode (default is non-blocking). One-way
* communications never block.
* Determines whether two-way communications between an endpoint and this consumer actor
* should be done in blocking or non-blocking mode (default is non-blocking). This method
* doesn't have any effect on one-way communications (they'll never block).
*/
def blocking = false
}
/**
* Java-friendly {@link Consumer} inherited by
* Java-friendly Consumer.
*
* <ul>
* <li>{@link UntypedConsumerActor}</li>
* <li>{@link RemoteUntypedConsumerActor}</li>
* <li>{@link UntypedConsumerTransactor}</li>
* </ul>
*
* implementations.
* @see UntypedConsumerActor
* @see RemoteUntypedConsumerActor
* @see UntypedConsumerTransactor
*
* @author Martin Krasser
*/
trait UntypedConsumer extends Consumer { self: UntypedActor =>
final override def endpointUri = getEndpointUri
final override def blocking = isBlocking
/**
@ -52,9 +46,9 @@ trait UntypedConsumer extends Consumer { self: UntypedActor =>
def getEndpointUri(): String
/**
* Determines whether two-way communications with this consumer actor should
* be done in blocking or non-blocking mode (default is non-blocking). One-way
* communications never block.
* Determines whether two-way communications between an endpoint and this consumer actor
* should be done in blocking or non-blocking mode (default is non-blocking). This method
* doesn't have any effect on one-way communications (they'll never block).
*/
def isBlocking() = super.blocking
}
@ -89,7 +83,7 @@ private[camel] object Consumer {
* reference with a target actor that implements the <code>Consumer</code> trait. The
* target <code>Consumer</code> object is passed as argument to <code>f</code>. This
* method returns <code>None</code> if <code>actorRef</code> is not a valid reference
* to a consumer actor, <code>Some</code> result otherwise.
* to a consumer actor, <code>Some</code> consumer actor otherwise.
*/
def forConsumer[T](actorRef: ActorRef)(f: Consumer => T): Option[T] = {
if (!actorRef.actor.isInstanceOf[Consumer]) None

View file

@ -28,10 +28,10 @@ private[camel] object ConsumerPublisher extends Logging {
}
/**
* Stops route to the already un-registered consumer actor.
* Stops the route to the already un-registered consumer actor.
*/
def handleConsumerUnregistered(event: ConsumerUnregistered) {
CamelContextManager.context.stopRoute(event.uuid)
CamelContextManager.context.stopRoute(event.uuid.toString)
log.info("unpublished actor %s from endpoint %s" format (event.actorRef, event.uri))
}
@ -48,7 +48,7 @@ private[camel] object ConsumerPublisher extends Logging {
}
/**
* Stops route to the already un-registered consumer actor method.
* Stops the route to the already un-registered consumer actor method.
*/
def handleConsumerMethodUnregistered(event: ConsumerMethodUnregistered) {
val targetMethod = event.method.getName
@ -62,10 +62,10 @@ private[camel] object ConsumerPublisher extends Logging {
/**
* Actor that publishes consumer actors and typed actor methods at Camel endpoints.
* The Camel context used for publishing is CamelContextManager.context. This actor
* accepts messages of type
* The Camel context used for publishing is obtained via CamelContextManager.context.
* This actor accepts messages of type
* se.scalablesolutions.akka.camel.ConsumerRegistered,
* se.scalablesolutions.akka.camel.ConsumerUnregistered.
* se.scalablesolutions.akka.camel.ConsumerUnregistered,
* se.scalablesolutions.akka.camel.ConsumerMethodRegistered and
* se.scalablesolutions.akka.camel.ConsumerMethodUnregistered.
*
@ -110,7 +110,7 @@ private[camel] case class SetExpectedRegistrationCount(num: Int)
private[camel] case class SetExpectedUnregistrationCount(num: Int)
/**
* Defines an abstract route to a target which is either an actor or an typed actor method..
* Abstract route to a target which is either an actor or an typed actor method.
*
* @param endpointUri endpoint URI of the consumer actor or typed actor method.
* @param id actor identifier or typed actor identifier (registry key).
@ -135,20 +135,20 @@ private[camel] abstract class ConsumerRoute(endpointUri: String, id: String) ext
}
/**
* Defines the route to a consumer actor.
* Defines the route to a (untyped) consumer actor.
*
* @param endpointUri endpoint URI of the consumer actor
* @param endpointUri endpoint URI of the (untyped) consumer actor
* @param uuid actor uuid
* @param blocking true for blocking in-out exchanges, false otherwise
*
* @author Martin Krasser
*/
private[camel] class ConsumerActorRoute(endpointUri: String, uuid: String, blocking: Boolean) extends ConsumerRoute(endpointUri, uuid) {
private[camel] class ConsumerActorRoute(endpointUri: String, uuid: Uuid, blocking: Boolean) extends ConsumerRoute(endpointUri, uuid.toString) {
protected override def targetUri = "actor:uuid:%s?blocking=%s" format (uuid, blocking)
}
/**
* Defines the route to an typed actor method..
* Defines the route to a typed actor method.
*
* @param endpointUri endpoint URI of the consumer actor method
* @param id typed actor identifier
@ -162,10 +162,10 @@ private[camel] class ConsumerMethodRoute(val endpointUri: String, id: String, me
/**
* A registration listener that triggers publication of consumer actors and typed actor
* methods as well as un-publication of consumer actors. This actor needs to be initialized
* with a <code>PublishRequestorInit</code> command message for obtaining a reference to
* a <code>publisher</code> actor. Before initialization it buffers all outbound messages
* and delivers them to the <code>publisher</code> when receiving a
* methods as well as un-publication of consumer actors and typed actor methods. This actor
* needs to be initialized with a <code>PublishRequestorInit</code> command message for
* obtaining a reference to a <code>publisher</code> actor. Before initialization it buffers
* all outbound messages and delivers them to the <code>publisher</code> when receiving a
* <code>PublishRequestorInit</code> message. After initialization, outbound messages are
* delivered directly without buffering.
*
@ -229,7 +229,7 @@ private[camel] sealed trait ConsumerEvent
*
* @author Martin Krasser
*/
private[camel] case class ConsumerRegistered(actorRef: ActorRef, uri: String, uuid: String, blocking: Boolean) extends ConsumerEvent
private[camel] case class ConsumerRegistered(actorRef: ActorRef, uri: String, uuid: Uuid, blocking: Boolean) extends ConsumerEvent
/**
* Event indicating that a consumer actor has been unregistered from the actor registry.
@ -240,7 +240,7 @@ private[camel] case class ConsumerRegistered(actorRef: ActorRef, uri: String, uu
*
* @author Martin Krasser
*/
private[camel] case class ConsumerUnregistered(actorRef: ActorRef, uri: String, uuid: String) extends ConsumerEvent
private[camel] case class ConsumerUnregistered(actorRef: ActorRef, uri: String, uuid: Uuid) extends ConsumerEvent
/**
* Event indicating that an typed actor proxy has been created for a typed actor. For each <code>@consume</code>
@ -273,7 +273,7 @@ private[camel] case class ConsumerMethodUnregistered(typedActor: AnyRef, init: A
*/
private[camel] object ConsumerRegistered {
/**
* Optionally creates an ConsumerRegistered event message for a consumer actor or None if
* Creates an ConsumerRegistered event message for a consumer actor or None if
* <code>actorRef</code> is not a consumer actor.
*/
def forConsumer(actorRef: ActorRef): Option[ConsumerRegistered] = {
@ -288,7 +288,7 @@ private[camel] object ConsumerRegistered {
*/
private[camel] object ConsumerUnregistered {
/**
* Optionally creates an ConsumerUnregistered event message for a consumer actor or None if
* Creates an ConsumerUnregistered event message for a consumer actor or None if
* <code>actorRef</code> is not a consumer actor.
*/
def forConsumer(actorRef: ActorRef): Option[ConsumerUnregistered] = {
@ -327,8 +327,8 @@ private[camel] object ConsumerMethod {
*/
private[camel] object ConsumerMethodRegistered {
/**
* Creates a list of ConsumerMethodRegistered event messages for an typed actor or an empty
* list if the typed actor is a proxy for an remote typed actor or the typed actor doesn't
* Creates a list of ConsumerMethodRegistered event messages for a typed actor or an empty
* list if the typed actor is a proxy for a remote typed actor or the typed actor doesn't
* have any <code>@consume</code> annotated methods.
*/
def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodRegistered] = {
@ -343,8 +343,8 @@ private[camel] object ConsumerMethodRegistered {
*/
private[camel] object ConsumerMethodUnregistered {
/**
* Creates a list of ConsumerMethodUnregistered event messages for an typed actor or an empty
* list if the typed actor is a proxy for an remote typed actor or the typed actor doesn't
* Creates a list of ConsumerMethodUnregistered event messages for a typed actor or an empty
* list if the typed actor is a proxy for a remote typed actor or the typed actor doesn't
* have any <code>@consume</code> annotated methods.
*/
def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodUnregistered] = {

View file

@ -10,7 +10,7 @@ import org.apache.camel.util.ExchangeHelper
/**
* An immutable representation of a Camel message. Actor classes that mix in
* se.scalablesolutions.akka.camel.Producer or
* se.scalablesolutions.akka.camel.Consumer use this message type for communication.
* se.scalablesolutions.akka.camel.Consumer usually use this message type for communication.
*
* @author Martin Krasser
*/
@ -264,8 +264,8 @@ class CamelMessageAdapter(val cm: CamelMessage) {
/**
* Defines conversion methods to CamelExchangeAdapter and CamelMessageAdapter.
* Imported by applications
* that implicitly want to use conversion methods of CamelExchangeAdapter and CamelMessageAdapter.
* Imported by applications that implicitly want to use conversion methods of
* CamelExchangeAdapter and CamelMessageAdapter.
*/
object CamelMessageConversion {

View file

@ -24,7 +24,7 @@ trait ProducerSupport { this: Actor =>
private val headersToCopyDefault = Set(Message.MessageExchangeId)
/**
* <code>Endpoint</code> object resolved from current CamelContext with
* <code>Endpoint</code> object resolved from the current CamelContext with
* <code>endpointUri</code>.
*/
private lazy val endpoint = CamelContextManager.context.getEndpoint(endpointUri)
@ -36,8 +36,8 @@ trait ProducerSupport { this: Actor =>
/**
* If set to false (default), this producer expects a response message from the Camel endpoint.
* If set to true, this producer communicates with the Camel endpoint with an in-only message
* exchange pattern (fire and forget).
* If set to true, this producer initiates an in-only message exchange with the Camel endpoint
* (fire and forget).
*/
def oneway: Boolean = false
@ -62,13 +62,17 @@ trait ProducerSupport { this: Actor =>
}
/**
* Produces <code>msg</code> as exchange of given <code>pattern</code> to the endpoint specified by
* <code>endpointUri</code>. After producing to the endpoint the processing result is passed as argument
* to <code>receiveAfterProduce</code>. If the result was returned synchronously by the endpoint then
* <code>receiveAfterProduce</code> is called synchronously as well. If the result was returned asynchronously,
* the <code>receiveAfterProduce</code> is called asynchronously as well. This is done by wrapping the result,
* adding it to this producers mailbox, unwrapping it once it is received and calling
* <code>receiveAfterProduce</code>. The original sender and senderFuture are thereby preserved.
* Initiates a message exchange of given <code>pattern</code> with the endpoint specified by
* <code>endpointUri</code>. The in-message of the initiated exchange is the canonical form
* of <code>msg</code>. After sending the in-message, the processing result (response) is passed
* as argument to <code>receiveAfterProduce</code>. If the response is received synchronously from
* the endpoint then <code>receiveAfterProduce</code> is called synchronously as well. If the
* response is received asynchronously, the <code>receiveAfterProduce</code> is called
* asynchronously. This is done by wrapping the response, adding it to this producers
* mailbox, unwrapping it and calling <code>receiveAfterProduce</code>. The original
* sender and senderFuture are thereby preserved.
*
* @see Message#canonicalize(Any)
*
* @param msg message to produce
* @param pattern exchange pattern
@ -106,8 +110,8 @@ trait ProducerSupport { this: Actor =>
/**
* Produces <code>msg</code> to the endpoint specified by <code>endpointUri</code>. Before the message is
* actually produced it is pre-processed by calling <code>receiveBeforeProduce</code>. If <code>oneway</code>
* is true an in-only message exchange is initiated, otherwise an in-out message exchange.
* actually sent it is pre-processed by calling <code>receiveBeforeProduce</code>. If <code>oneway</code>
* is <code>true</code>, an in-only message exchange is initiated, otherwise an in-out message exchange.
*
* @see Producer#produce(Any, ExchangePattern)
*/
@ -132,17 +136,18 @@ trait ProducerSupport { this: Actor =>
}
/**
* Called after the a result was received from the endpoint specified by <code>endpointUri</code>. The
* result is passed as argument. By default, this method replies the result back to the original sender
* if <code>oneway</code> is false. If <code>oneway</code> is true then nothing is done. This method may
* be overridden by subtraits or subclasses.
* Called after a response was received from the endpoint specified by <code>endpointUri</code>. The
* response is passed as argument. By default, this method sends the response back to the original sender
* if <code>oneway</code> is <code>false</code>. If <code>oneway</code> is <code>true</code>, nothing is
* done. This method may be overridden by subtraits or subclasses (e.g. to forward responses to another
* actor).
*/
protected def receiveAfterProduce: Receive = {
case msg => if (!oneway) self.reply(msg)
}
/**
* Creates a new Exchange with given <code>pattern</code> from the endpoint specified by
* Creates a new Exchange of given <code>pattern</code> from the endpoint specified by
* <code>endpointUri</code>.
*/
private def createExchange(pattern: ExchangePattern): Exchange = endpoint.createExchange(pattern)
@ -158,25 +163,26 @@ trait ProducerSupport { this: Actor =>
}
/**
* Mixed in by Actor implementations that produce messages to Camel endpoints.
* Mixed in by Actor implementations to produce messages to Camel endpoints.
*/
trait Producer extends ProducerSupport { this: Actor =>
/**
* Default implementation of Actor.receive
* Default implementation of Actor.receive. Any messages received by this actors
* will be produced to the endpoint specified by <code>endpointUri</code>.
*/
protected def receive = produce
}
/**
* Java-friendly {@link ProducerSupport} inherited by {@link UntypedProducerActor} implementations.
* Java-friendly ProducerSupport.
*
* @see UntypedProducerActor
*
* @author Martin Krasser
*/
trait UntypedProducer extends ProducerSupport { this: UntypedActor =>
final override def endpointUri = getEndpointUri
final override def oneway = isOneway
final override def receiveBeforeProduce = {
@ -213,10 +219,10 @@ trait UntypedProducer extends ProducerSupport { this: UntypedActor =>
def onReceiveBeforeProduce(message: Any): Any = super.receiveBeforeProduce(message)
/**
* Called after the a result was received from the endpoint specified by <code>getEndpointUri</code>. The
* result is passed as argument. By default, this method replies the result back to the original sender
* if <code>isOneway</code> returns false. If <code>isOneway</code> returns true then nothing is done. This
* method may be overridden by subclasses.
* Called after a response was received from the endpoint specified by <code>endpointUri</code>. The
* response is passed as argument. By default, this method sends the response back to the original sender
* if <code>oneway</code> is <code>false</code>. If <code>oneway</code> is <code>true</code>, nothing is
* done. This method may be overridden by subclasses (e.g. to forward responses to another actor).
*/
@throws(classOf[Exception])
def onReceiveAfterProduce(message: Any): Unit = super.receiveAfterProduce(message)

View file

@ -14,14 +14,13 @@ import jsr166x.Deque
import org.apache.camel._
import org.apache.camel.impl.{DefaultProducer, DefaultEndpoint, DefaultComponent}
import se.scalablesolutions.akka.camel.{Failure, CamelMessageConversion, Message}
import CamelMessageConversion.toExchangeAdapter
import se.scalablesolutions.akka.actor._
import se.scalablesolutions.akka.camel.{Failure, Message}
import se.scalablesolutions.akka.camel.CamelMessageConversion.toExchangeAdapter
import se.scalablesolutions.akka.dispatch.{CompletableFuture, MessageInvocation, MessageDispatcher}
import se.scalablesolutions.akka.stm.TransactionConfig
import se.scalablesolutions.akka.AkkaException
import scala.reflect.BeanProperty
import se.scalablesolutions.akka.actor._
/**
* Camel component for sending messages to and receiving replies from (untyped) actors.
@ -33,28 +32,26 @@ import se.scalablesolutions.akka.actor._
*/
class ActorComponent extends DefaultComponent {
def createEndpoint(uri: String, remaining: String, parameters: JavaMap[String, Object]): ActorEndpoint = {
val idAndUuid = idAndUuidPair(remaining)
new ActorEndpoint(uri, this, idAndUuid._1, idAndUuid._2)
val (id,uuid) = idAndUuidPair(remaining)
new ActorEndpoint(uri, this, id, uuid)
}
private def idAndUuidPair(remaining: String): Tuple2[Option[String], Option[String]] = {
remaining split ":" toList match {
case id :: Nil => (Some(id), None)
case "id" :: id :: Nil => (Some(id), None)
case "uuid" :: uuid :: Nil => (None, Some(uuid))
case _ => throw new IllegalArgumentException(
"invalid path format: %s - should be <actorid> or id:<actorid> or uuid:<actoruuid>" format remaining)
}
private def idAndUuidPair(remaining: String): Tuple2[Option[String],Option[Uuid]] = remaining match {
case null | "" => throw new IllegalArgumentException("invalid path format: [%s] - should be <actorid> or id:<actorid> or uuid:<actoruuid>" format remaining)
case id if id startsWith "id:" => (Some(id substring 3),None)
case uuid if uuid startsWith "uuid:" => (None,Some(uuidFrom(uuid substring 5)))
case id => (Some(id),None)
}
}
/**
* Camel endpoint for referencing an (untyped) actor. The actor reference is given by the endpoint URI.
* An actor can be referenced by its <code>ActorRef.id</code> or its <code>ActorRef.uuid</code>.
* Supported endpoint URI formats are
* <code>actor:&lt;actorid&gt;</code>,
* <code>actor:id:&lt;actorid&gt;</code> and
* <code>actor:uuid:&lt;actoruuid&gt;</code>.
* Camel endpoint for sending messages to and receiving replies from (untyped) actors. Actors
* are referenced using <code>actor</code> endpoint URIs of the following format:
* <code>actor:<actor-id></code>,
* <code>actor:id:<actor-id></code> and
* <code>actor:uuid:<actor-uuid></code>,
* where <code>actor-id</code> refers to <code>ActorRef.id</code> and <code>actor-uuid</code>
* refers to the String-representation od <code>ActorRef.uuid</code>.
*
* @see se.scalablesolutions.akka.camel.component.ActorComponent
* @see se.scalablesolutions.akka.camel.component.ActorProducer
@ -64,11 +61,12 @@ class ActorComponent extends DefaultComponent {
class ActorEndpoint(uri: String,
comp: ActorComponent,
val id: Option[String],
val uuid: Option[String]) extends DefaultEndpoint(uri, comp) {
val uuid: Option[Uuid]) extends DefaultEndpoint(uri, comp) {
/**
* Blocking of caller thread during two-way message exchanges with consumer actors. This is set
* via the <code>blocking=true|false</code> endpoint URI parameter. If omitted blocking is false.
* Whether to block caller thread during two-way message exchanges with (untyped) actors. This is
* set via the <code>blocking=true|false</code> endpoint URI parameter. Default value is
* <code>false</code>.
*/
@BeanProperty var blocking: Boolean = false
@ -90,9 +88,18 @@ class ActorEndpoint(uri: String,
}
/**
* Sends the in-message of an exchange to an (untyped) actor. If the exchange pattern is out-capable and
* <code>blocking</code> is enabled then the producer waits for a reply (using the !! operator),
* otherwise the ! operator is used for sending the message.
* Sends the in-message of an exchange to an (untyped) actor.
* <ul>
* <li>If the exchange pattern is out-capable and <code>blocking</code> is set to
* <code>true</code> then the producer waits for a reply, using the !! operator.</li>
* <li>If the exchange pattern is out-capable and <code>blocking</code> is set to
* <code>false</code> then the producer sends the message using the ! operator, together
* with a callback handler. The callback handler is an <code>ActorRef</code> that can be
* used by the receiving actor to asynchronously reply to the route that is sending the
* message.</li>
* <li>If the exchange pattern is in-only then the producer sends the message using the
* ! operator.</li>
* </ul>
*
* @see se.scalablesolutions.akka.camel.component.ActorComponent
* @see se.scalablesolutions.akka.camel.component.ActorEndpoint
@ -151,7 +158,7 @@ class ActorProducer(val ep: ActorEndpoint) extends DefaultProducer(ep) with Asyn
case actors => Some(actors(0))
}
private def targetByUuid(uuid: String) = ActorRegistry.actorFor(uuid)
private def targetByUuid(uuid: Uuid) = ActorRegistry.actorFor(uuid)
}
/**
@ -187,11 +194,11 @@ private[akka] object AsyncCallbackAdapter {
}
/**
* Adapts an <code>AsyncCallback</code> to <code>ActorRef.!</code>. Used by other actors to reply
* asynchronously to Camel with <code>ActorRef.reply</code>.
* Adapts an <code>ActorRef</code> to a Camel <code>AsyncCallback</code>. Used by receiving actors to reply
* asynchronously to Camel routes with <code>ActorRef.reply</code>.
* <p>
* <em>Please note</em> that this adapter can only be used locally at the moment which should not
* be a problem is most situations as Camel endpoints are only activated for local actor references,
* be a problem is most situations since Camel endpoints are only activated for local actor references,
* never for remote references.
*
* @author Martin Krasser
@ -208,8 +215,9 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
}
/**
* Writes the reply <code>message</code> to <code>exchange</code> and uses <code>callback</code> to
* generate completion notifications.
* Populates the initial <code>exchange</code> with the reply <code>message</code> and uses the
* <code>callback</code> handler to notify Camel about the asynchronous completion of the message
* exchange.
*
* @param message reply message
* @param sender ignored
@ -249,7 +257,7 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
protected[akka] def linkedActors: JavaMap[String, ActorRef] = unsupported
protected[akka] def linkedActors: JavaMap[Uuid, ActorRef] = unsupported
protected[akka] def linkedActorsAsList: List[ActorRef] = unsupported
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported
protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported

View file

@ -21,7 +21,7 @@ object TypedActorComponent {
/**
* Camel component for exchanging messages with typed actors. This component
* tries to obtain the typed actor from the <code>typedActorRegistry</code>
* tries to obtain the typed actor from its <code>typedActorRegistry</code>
* first. If it's not there it tries to obtain it from the CamelContext's registry.
*
* @see org.apache.camel.component.bean.BeanComponent
@ -32,9 +32,9 @@ class TypedActorComponent extends BeanComponent {
val typedActorRegistry = new ConcurrentHashMap[String, AnyRef]
/**
* Creates a {@link org.apache.camel.component.bean.BeanEndpoint} with a custom
* bean holder that uses <code>typedActorRegistry</code> for getting access to
* typed actors (beans).
* Creates an <code>org.apache.camel.component.bean.BeanEndpoint</code> with a custom
* bean holder that uses <code>typedActorRegistry</code> for getting access to typed
* actors (beans).
*
* @see se.scalablesolutions.akka.camel.component.TypedActorHolder
*/
@ -51,7 +51,7 @@ class TypedActorComponent extends BeanComponent {
}
/**
* {@link org.apache.camel.component.bean.BeanHolder} implementation that uses a custom
* <code>org.apache.camel.component.bean.BeanHolder</code> implementation that uses a custom
* registry for getting access to typed actors.
*
* @author Martin Krasser
@ -60,13 +60,16 @@ class TypedActorHolder(typedActorRegistry: Map[String, AnyRef], context: CamelCo
extends RegistryBean(context, name) {
/**
* Returns an {@link se.scalablesolutions.akka.camel.component.TypedActorInfo} instance.
* Returns an <code>se.scalablesolutions.akka.camel.component.TypedActorInfo</code> instance.
*/
override def getBeanInfo: BeanInfo =
new TypedActorInfo(getContext, getBean.getClass, getParameterMappingStrategy)
/**
* Obtains an typed actor from <code>typedActorRegistry</code>.
* Obtains a typed actor from <code>typedActorRegistry</code>. If the typed actor cannot
* be found then this method tries to obtain the actor from the CamelContext's registry.
*
* @return a typed actor or <code>null</code>.
*/
override def getBean: AnyRef = {
val bean = typedActorRegistry.get(getName)
@ -75,7 +78,7 @@ class TypedActorHolder(typedActorRegistry: Map[String, AnyRef], context: CamelCo
}
/**
* Provides typed actor meta information.
* Typed actor meta information.
*
* @author Martin Krasser
*/

View file

@ -8,7 +8,7 @@ import se.scalablesolutions.akka.actor.ActorRegistry
/**
* @author Martin Krasser
*/
class CamelServiceManagerSpec extends WordSpec with BeforeAndAfterAll with MustMatchers {
class CamelServiceManagerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
override def afterAll = ActorRegistry.shutdownAll

View file

@ -13,9 +13,9 @@ import se.scalablesolutions.akka.actor._
/**
* @author Martin Krasser
*/
class ConsumerSpec extends WordSpec with BeforeAndAfterAll with MustMatchers {
class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
import CamelContextManager.template
import ConsumerSpec._
import ConsumerTest._
var service: CamelService = _
@ -174,7 +174,7 @@ class ConsumerSpec extends WordSpec with BeforeAndAfterAll with MustMatchers {
}
}
object ConsumerSpec {
object ConsumerTest {
class TestConsumer(uri: String) extends Actor with Consumer {
def endpointUri = uri
protected def receive = {

View file

@ -5,9 +5,13 @@ import org.apache.camel.impl.DefaultCamelContext
import org.junit._
import org.scalatest.junit.JUnitSuite
import se.scalablesolutions.akka.actor.uuidFrom
class ActorComponentTest extends JUnitSuite {
val component: ActorComponent = ActorComponentTest.actorComponent
def testUUID = uuidFrom("93da8c80-c3fd-11df-abed-60334b120057")
@Test def shouldCreateEndpointWithIdDefined = {
val ep1: ActorEndpoint = component.createEndpoint("actor:abc").asInstanceOf[ActorEndpoint]
val ep2: ActorEndpoint = component.createEndpoint("actor:id:abc").asInstanceOf[ActorEndpoint]
@ -20,15 +24,15 @@ class ActorComponentTest extends JUnitSuite {
}
@Test def shouldCreateEndpointWithUuidDefined = {
val ep: ActorEndpoint = component.createEndpoint("actor:uuid:abc").asInstanceOf[ActorEndpoint]
assert(ep.uuid === Some("abc"))
val ep: ActorEndpoint = component.createEndpoint("actor:uuid:" + testUUID).asInstanceOf[ActorEndpoint]
assert(ep.uuid === Some(testUUID))
assert(ep.id === None)
assert(!ep.blocking)
}
@Test def shouldCreateEndpointWithBlockingSet = {
val ep: ActorEndpoint = component.createEndpoint("actor:uuid:abc?blocking=true").asInstanceOf[ActorEndpoint]
assert(ep.uuid === Some("abc"))
val ep: ActorEndpoint = component.createEndpoint("actor:uuid:"+testUUID+"?blocking=true").asInstanceOf[ActorEndpoint]
assert(ep.uuid === Some(testUUID))
assert(ep.id === None)
assert(ep.blocking)
}

View file

@ -1,13 +1,12 @@
package se.scalablesolutions.akka.camel.component
import org.apache.camel._
import org.apache.camel.builder.RouteBuilder
import org.apache.camel.impl.{DefaultCamelContext, SimpleRegistry}
import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec}
import org.apache.camel.builder.RouteBuilder
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.actor.{ActorRegistry, TypedActor}
import se.scalablesolutions.akka.camel._
import org.apache.camel.impl.{DefaultCamelContext, SimpleRegistry}
import org.apache.camel.{ResolveEndpointFailedException, ExchangePattern, Exchange, Processor}
/**
* @author Martin Krasser

View file

@ -4,16 +4,16 @@
package se.scalablesolutions.akka.persistence.cassandra
import se.scalablesolutions.akka.util.UUID
import se.scalablesolutions.akka.stm._
import se.scalablesolutions.akka.persistence.common._
import se.scalablesolutions.akka.actor.{newUuid}
object CassandraStorage extends Storage {
type ElementType = Array[Byte]
def newMap: PersistentMap[ElementType, ElementType] = newMap(UUID.newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(UUID.newUuid.toString)
def newRef: PersistentRef[ElementType] = newRef(UUID.newUuid.toString)
def newMap: PersistentMap[ElementType, ElementType] = newMap(newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(newUuid.toString)
def newRef: PersistentRef[ElementType] = newRef(newUuid.toString)
def getMap(id: String): PersistentMap[ElementType, ElementType] = newMap(id)
def getVector(id: String): PersistentVector[ElementType] = newVector(id)

View file

@ -0,0 +1,25 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you 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.
log4j.rootLogger=ERROR,R
# rolling log file ("system.log
log4j.appender.R=org.apache.log4j.DailyRollingFileAppender
log4j.appender.R.DatePattern='.'yyyy-MM-dd-HH
log4j.appender.R.layout=org.apache.log4j.PatternLayout
log4j.appender.R.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n
log4j.appender.R.File=target/logs/system.log

View file

@ -4,16 +4,16 @@
package se.scalablesolutions.akka.persistence.hbase
import se.scalablesolutions.akka.util.UUID
import se.scalablesolutions.akka.actor.{Uuid,newUuid}
import se.scalablesolutions.akka.stm._
import se.scalablesolutions.akka.persistence.common._
object HbaseStorage extends Storage {
type ElementType = Array[Byte]
def newMap: PersistentMap[ElementType, ElementType] = newMap(UUID.newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(UUID.newUuid.toString)
def newRef: PersistentRef[ElementType] = newRef(UUID.newUuid.toString)
def newMap: PersistentMap[ElementType, ElementType] = newMap(newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(newUuid.toString)
def newRef: PersistentRef[ElementType] = newRef(newUuid.toString)
def getMap(id: String): PersistentMap[ElementType, ElementType] = newMap(id)
def getVector(id: String): PersistentVector[ElementType] = newVector(id)

View file

@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.util.Bytes
*/
private[akka] object HbaseStorageBackend extends MapStorageBackend[Array[Byte], Array[Byte]] with VectorStorageBackend[Array[Byte]] with RefStorageBackend[Array[Byte]] with Logging {
val EMPTY_BYTE_ARRAY = new Array[Byte](0)
val HBASE_ZOOKEEPER_QUORUM = config.getString("akka.storage.hbase.zookeeper-quorum", "localhost")
val CONFIGURATION = new HBaseConfiguration
val REF_TABLE_NAME = "__REF_TABLE"

View file

@ -76,7 +76,7 @@ class PersistentFailerActor extends Transactor {
}
}
class HbasePersistentActorSpec extends JUnitSuite with BeforeAndAfterAll {
class HbasePersistentActorSpecTestIntegration extends JUnitSuite with BeforeAndAfterAll {
val testUtil = new HBaseTestingUtility

View file

@ -5,7 +5,7 @@ import org.scalatest.matchers.ShouldMatchers
import org.scalatest.BeforeAndAfterAll
import org.scalatest.BeforeAndAfterEach
class HbaseStorageSpec extends
class HbaseStorageSpecTestIntegration extends
Spec with
ShouldMatchers with
BeforeAndAfterAll with

View file

@ -171,7 +171,7 @@ object Storage {
import Storage._
@RunWith(classOf[JUnitRunner])
class HbaseTicket343Spec extends Spec with ShouldMatchers with BeforeAndAfterAll with BeforeAndAfterEach {
class HbaseTicket343SpecTestIntegration extends Spec with ShouldMatchers with BeforeAndAfterAll with BeforeAndAfterEach {
import org.apache.hadoop.hbase.HBaseTestingUtility

View file

@ -10,7 +10,7 @@ import org.junit.Test
import org.apache.hadoop.hbase.HBaseTestingUtility
@RunWith(classOf[JUnitRunner])
class PersistenceSpec extends Spec with BeforeAndAfterAll with ShouldMatchers {
class SimpleHbaseSpecTestIntegration extends Spec with BeforeAndAfterAll with ShouldMatchers {
import org.apache.hadoop.hbase.HBaseTestingUtility

View file

@ -6,14 +6,14 @@ package se.scalablesolutions.akka.persistence.mongo
import se.scalablesolutions.akka.stm._
import se.scalablesolutions.akka.persistence.common._
import se.scalablesolutions.akka.util.UUID
import se.scalablesolutions.akka.actor.{newUuid}
object MongoStorage extends Storage {
type ElementType = Array[Byte]
def newMap: PersistentMap[ElementType, ElementType] = newMap(UUID.newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(UUID.newUuid.toString)
def newRef: PersistentRef[ElementType] = newRef(UUID.newUuid.toString)
def newMap: PersistentMap[ElementType, ElementType] = newMap(newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(newUuid.toString)
def newRef: PersistentRef[ElementType] = newRef(newUuid.toString)
def getMap(id: String): PersistentMap[ElementType, ElementType] = newMap(id)
def getVector(id: String): PersistentVector[ElementType] = newVector(id)

View file

@ -4,18 +4,18 @@
package se.scalablesolutions.akka.persistence.redis
import se.scalablesolutions.akka.util.UUID
import se.scalablesolutions.akka.actor.{newUuid}
import se.scalablesolutions.akka.stm._
import se.scalablesolutions.akka.persistence.common._
object RedisStorage extends Storage {
type ElementType = Array[Byte]
def newMap: PersistentMap[ElementType, ElementType] = newMap(UUID.newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(UUID.newUuid.toString)
def newRef: PersistentRef[ElementType] = newRef(UUID.newUuid.toString)
override def newQueue: PersistentQueue[ElementType] = newQueue(UUID.newUuid.toString)
override def newSortedSet: PersistentSortedSet[ElementType] = newSortedSet(UUID.newUuid.toString)
def newMap: PersistentMap[ElementType, ElementType] = newMap(newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(newUuid.toString)
def newRef: PersistentRef[ElementType] = newRef(newUuid.toString)
override def newQueue: PersistentQueue[ElementType] = newQueue(newUuid.toString)
override def newSortedSet: PersistentSortedSet[ElementType] = newSortedSet(newUuid.toString)
def getMap(id: String): PersistentMap[ElementType, ElementType] = newMap(id)
def getVector(id: String): PersistentVector[ElementType] = newVector(id)

View file

@ -0,0 +1,43 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package se.scalablesolutions.akka.persistence.voldemort
import se.scalablesolutions.akka.actor.{newUuid}
import se.scalablesolutions.akka.stm._
import se.scalablesolutions.akka.persistence.common._
object VoldemortStorage extends Storage {
type ElementType = Array[Byte]
def newMap: PersistentMap[ElementType, ElementType] = newMap(newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(newUuid.toString)
def newRef: PersistentRef[ElementType] = newRef(newUuid.toString)
def getMap(id: String): PersistentMap[ElementType, ElementType] = newMap(id)
def getVector(id: String): PersistentVector[ElementType] = newVector(id)
def getRef(id: String): PersistentRef[ElementType] = newRef(id)
def newMap(id: String): PersistentMap[ElementType, ElementType] = new VoldemortPersistentMap(id)
def newVector(id: String): PersistentVector[ElementType] = new VoldemortPersistentVector(id)
def newRef(id: String): PersistentRef[ElementType] = new VoldemortPersistentRef(id)
}
class VoldemortPersistentMap(id: String) extends PersistentMapBinary {
val uuid = id
val storage = VoldemortStorageBackend
}
class VoldemortPersistentVector(id: String) extends PersistentVector[Array[Byte]] {
val uuid = id
val storage = VoldemortStorageBackend
}
class VoldemortPersistentRef(id: String) extends PersistentRef[Array[Byte]] {
val uuid = id
val storage = VoldemortStorageBackend
}

View file

@ -0,0 +1,379 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package se.scalablesolutions.akka.persistence.voldemort
import se.scalablesolutions.akka.stm._
import se.scalablesolutions.akka.persistence.common._
import se.scalablesolutions.akka.util.Logging
import se.scalablesolutions.akka.util.Helpers._
import se.scalablesolutions.akka.config.Config.config
import voldemort.client._
import java.lang.String
import voldemort.utils.ByteUtils
import voldemort.versioning.Versioned
import collection.JavaConversions
import java.nio.ByteBuffer
import collection.Map
import collection.immutable.{IndexedSeq, SortedSet, TreeSet, HashMap}
import collection.mutable.{Set, HashSet, ArrayBuffer}
import java.util.{Properties, Map => JMap}
private[akka] object VoldemortStorageBackend extends
MapStorageBackend[Array[Byte], Array[Byte]] with
VectorStorageBackend[Array[Byte]] with
RefStorageBackend[Array[Byte]] with
QueueStorageBackend[Array[Byte]] with
Logging {
val bootstrapUrlsProp = "bootstrap_urls"
val clientConfig = config.getConfigMap("akka.storage.voldemort.client") match {
case Some(configMap) => getClientConfig(configMap.asMap)
case None => getClientConfig(new HashMap[String, String] + (bootstrapUrlsProp -> "tcp://localhost:6666"))
}
val refStore = config.getString("akka.storage.voldemort.store.ref", "Refs")
val mapKeyStore = config.getString("akka.storage.voldemort.store.map-key", "MapKeys")
val mapValueStore = config.getString("akka.storage.voldemort.store.map-value", "MapValues")
val vectorStore = config.getString("akka.storage.voldemort.store.vector", "Vectors")
val queueStore = config.getString("akka.storage.voldemort.store.queue", "Queues")
var storeClientFactory: StoreClientFactory = null
var refClient: StoreClient[String, Array[Byte]] = null
var mapKeyClient: StoreClient[String, Array[Byte]] = null
var mapValueClient: StoreClient[Array[Byte], Array[Byte]] = null
var vectorClient: StoreClient[Array[Byte], Array[Byte]] = null
var queueClient: StoreClient[Array[Byte], Array[Byte]] = null
initStoreClients
val underscoreBytesUTF8 = "_".getBytes("UTF-8")
val vectorSizeIndex = -1
val queueHeadIndex = -1
val queueTailIndex = -2
case class QueueMetadata(head: Int, tail: Int) {
def size = tail - head
//worry about wrapping etc
}
implicit val byteOrder = new Ordering[Array[Byte]] {
override def compare(x: Array[Byte], y: Array[Byte]) = ByteUtils.compare(x, y)
}
def getRefStorageFor(name: String): Option[Array[Byte]] = {
val result: Array[Byte] = refClient.getValue(name)
result match {
case null => None
case _ => Some(result)
}
}
def insertRefStorageFor(name: String, element: Array[Byte]) = {
refClient.put(name, element)
}
def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = {
val allkeys: SortedSet[Array[Byte]] = getMapKeys(name)
val range = allkeys.rangeImpl(start, finish).take(count)
getKeyValues(name, range)
}
def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = {
val keys = getMapKeys(name)
getKeyValues(name, keys)
}
private def getKeyValues(name: String, keys: SortedSet[Array[Byte]]): List[(Array[Byte], Array[Byte])] = {
val all: JMap[Array[Byte], Versioned[Array[Byte]]] =
mapValueClient.getAll(JavaConversions.asIterable(keys.map {
mapKey => getKey(name, mapKey)
}))
val buf = new ArrayBuffer[(Array[Byte], Array[Byte])](all.size)
JavaConversions.asMap(all).foreach {
(entry) => {
entry match {
case (key: Array[Byte], versioned: Versioned[Array[Byte]]) => {
buf += key -> versioned.getValue
}
}
}
}
buf.toList
}
def getMapStorageSizeFor(name: String): Int = {
val keys = getMapKeys(name)
keys.size
}
def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = {
val result: Array[Byte] = mapValueClient.getValue(getKey(name, key))
result match {
case null => None
case _ => Some(result)
}
}
def removeMapStorageFor(name: String, key: Array[Byte]) = {
var keys = getMapKeys(name)
keys -= key
putMapKeys(name, keys)
mapValueClient.delete(getKey(name, key))
}
def removeMapStorageFor(name: String) = {
val keys = getMapKeys(name)
keys.foreach {
key =>
mapValueClient.delete(getKey(name, key))
}
mapKeyClient.delete(name)
}
def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]) = {
mapValueClient.put(getKey(name, key), value)
var keys = getMapKeys(name)
keys += key
putMapKeys(name, keys)
}
def insertMapStorageEntriesFor(name: String, entries: List[(Array[Byte], Array[Byte])]) = {
val newKeys = entries.map {
case (key, value) => {
mapValueClient.put(getKey(name, key), value)
key
}
}
var keys = getMapKeys(name)
keys ++= newKeys
putMapKeys(name, keys)
}
def putMapKeys(name: String, keys: SortedSet[Array[Byte]]) = {
mapKeyClient.put(name, SortedSetSerializer.toBytes(keys))
}
def getMapKeys(name: String): SortedSet[Array[Byte]] = {
SortedSetSerializer.fromBytes(mapKeyClient.getValue(name, Array.empty[Byte]))
}
def getVectorStorageSizeFor(name: String): Int = {
IntSerializer.fromBytes(vectorClient.getValue(getIndexedKey(name, vectorSizeIndex), IntSerializer.toBytes(0)))
}
def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = {
val size = getVectorStorageSizeFor(name)
val st = start.getOrElse(0)
val cnt =
if (finish.isDefined) {
val f = finish.get
if (f >= st) (f - st) else count
} else {
count
}
val seq: IndexedSeq[Array[Byte]] = (st until st + cnt).map {
index => getIndexedKey(name, index)
}
val all: JMap[Array[Byte], Versioned[Array[Byte]]] = vectorClient.getAll(JavaConversions.asIterable(seq))
var storage = new ArrayBuffer[Array[Byte]](seq.size)
storage = storage.padTo(seq.size, Array.empty[Byte])
var idx = 0;
seq.foreach {
key => {
if (all.containsKey(key)) {
storage.update(idx, all.get(key).getValue)
}
idx += 1
}
}
storage.toList
}
def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = {
vectorClient.getValue(getIndexedKey(name, index), Array.empty[Byte])
}
def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) = {
val size = getVectorStorageSizeFor(name)
vectorClient.put(getIndexedKey(name, index), elem)
if (size < index + 1) {
vectorClient.put(getIndexedKey(name, vectorSizeIndex), IntSerializer.toBytes(index + 1))
}
}
def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) = {
var size = getVectorStorageSizeFor(name)
elements.foreach {
element =>
vectorClient.put(getIndexedKey(name, size), element)
size += 1
}
vectorClient.put(getIndexedKey(name, vectorSizeIndex), IntSerializer.toBytes(size))
}
def insertVectorStorageEntryFor(name: String, element: Array[Byte]) = {
insertVectorStorageEntriesFor(name, List(element))
}
def remove(name: String): Boolean = {
false
}
def peek(name: String, start: Int, count: Int): List[Array[Byte]] = {
List(Array.empty[Byte])
}
def size(name: String): Int = {
getQueueMetadata(name).size
}
def dequeue(name: String): Option[Array[Byte]] = {
None
}
def enqueue(name: String, item: Array[Byte]): Option[Int] = {
val mdata = getQueueMetadata(name)
val key = getIndexedKey(name, mdata.tail)
queueClient.put(key, item)
queueClient.put(getIndexedKey(name, queueTailIndex), IntSerializer.toBytes(mdata.tail + 1))
Some (mdata.size + 1)
}
def getQueueMetadata(name: String): QueueMetadata = {
val keys = List(getIndexedKey(name, queueHeadIndex), getIndexedKey(name, queueTailIndex))
val qdata = JavaConversions.asMap(queueClient.getAll(JavaConversions.asIterable(keys)))
val values = keys.map {
qdata.get(_) match {
case Some(versioned) => IntSerializer.fromBytes(versioned.getValue)
case None => 0
}
}
QueueMetadata(values.head, values.tail.head)
}
/**
* Concat the ownerlenght+owner+key+ of owner so owned data will be colocated
* Store the length of owner as first byte to work around the rare case
* where ownerbytes1 + keybytes1 == ownerbytes2 + keybytes2 but ownerbytes1 != ownerbytes2
*/
def getKey(owner: String, key: Array[Byte]): Array[Byte] = {
val ownerBytes: Array[Byte] = owner.getBytes("UTF-8")
val ownerLenghtBytes: Array[Byte] = IntSerializer.toBytes(owner.length)
val theKey = new Array[Byte](ownerLenghtBytes.length + ownerBytes.length + key.length)
System.arraycopy(ownerLenghtBytes, 0, theKey, 0, ownerLenghtBytes.length)
System.arraycopy(ownerBytes, 0, theKey, ownerLenghtBytes.length, ownerBytes.length)
System.arraycopy(key, 0, theKey, ownerLenghtBytes.length + ownerBytes.length, key.length)
theKey
}
def getIndexedKey(owner: String, index: Int): Array[Byte] = {
val indexbytes = IntSerializer.toBytes(index)
val theIndexKey = new Array[Byte](underscoreBytesUTF8.length + indexbytes.length)
System.arraycopy(underscoreBytesUTF8, 0, theIndexKey, 0, underscoreBytesUTF8.length)
System.arraycopy(indexbytes, 0, theIndexKey, underscoreBytesUTF8.length, indexbytes.length)
getKey(owner, theIndexKey)
}
def getIndexFromVectorValueKey(owner: String, key: Array[Byte]): Int = {
val indexBytes = new Array[Byte](IntSerializer.bytesPerInt)
System.arraycopy(key, key.length - IntSerializer.bytesPerInt, indexBytes, 0, IntSerializer.bytesPerInt)
IntSerializer.fromBytes(indexBytes)
}
def getClientConfig(configMap: Map[String, String]): Properties = {
val properites = new Properties
configMap.foreach {
keyval => keyval match {
case (key, value) => properites.setProperty(key.asInstanceOf[java.lang.String], value.asInstanceOf[java.lang.String])
}
}
properites
}
def initStoreClients() = {
if (storeClientFactory != null) {
storeClientFactory.close
}
storeClientFactory = {
if (clientConfig.getProperty(bootstrapUrlsProp, "none").startsWith("tcp")) {
new SocketStoreClientFactory(new ClientConfig(clientConfig))
} else if (clientConfig.getProperty(bootstrapUrlsProp, "none").startsWith("http")) {
new HttpStoreClientFactory(new ClientConfig(clientConfig))
} else {
throw new IllegalArgumentException("Unknown boostrapUrl syntax" + clientConfig.getProperty(bootstrapUrlsProp, "No Bootstrap URLs defined"))
}
}
refClient = storeClientFactory.getStoreClient(refStore)
mapKeyClient = storeClientFactory.getStoreClient(mapKeyStore)
mapValueClient = storeClientFactory.getStoreClient(mapValueStore)
vectorClient = storeClientFactory.getStoreClient(vectorStore)
queueClient = storeClientFactory.getStoreClient(queueStore)
}
object IntSerializer {
val bytesPerInt = java.lang.Integer.SIZE / java.lang.Byte.SIZE
def toBytes(i: Int) = ByteBuffer.wrap(new Array[Byte](bytesPerInt)).putInt(i).array()
def fromBytes(bytes: Array[Byte]) = ByteBuffer.wrap(bytes).getInt()
def toString(obj: Int) = obj.toString
def fromString(str: String) = str.toInt
}
object SortedSetSerializer {
def toBytes(set: SortedSet[Array[Byte]]): Array[Byte] = {
val length = set.foldLeft(0) {
(total, bytes) => {
total + bytes.length + IntSerializer.bytesPerInt
}
}
val allBytes = new Array[Byte](length)
val written = set.foldLeft(0) {
(total, bytes) => {
val sizeBytes = IntSerializer.toBytes(bytes.length)
System.arraycopy(sizeBytes, 0, allBytes, total, sizeBytes.length)
System.arraycopy(bytes, 0, allBytes, total + sizeBytes.length, bytes.length)
total + sizeBytes.length + bytes.length
}
}
require(length == written, "Bytes Written Did not equal Calculated Length, written %d, length %d".format(written, length))
allBytes
}
def fromBytes(bytes: Array[Byte]): SortedSet[Array[Byte]] = {
var set = new TreeSet[Array[Byte]]
if (bytes.length > IntSerializer.bytesPerInt) {
var pos = 0
while (pos < bytes.length) {
val lengthBytes = new Array[Byte](IntSerializer.bytesPerInt)
System.arraycopy(bytes, pos, lengthBytes, 0, IntSerializer.bytesPerInt)
pos += IntSerializer.bytesPerInt
val length = IntSerializer.fromBytes(lengthBytes)
val item = new Array[Byte](length)
System.arraycopy(bytes, pos, item, 0, length)
set = set + item
pos += length
}
}
set
}
}
}

View file

@ -0,0 +1,14 @@
<cluster>
<!-- The name is just to help users identify this cluster from the gui -->
<name>akka-test</name>
<server>
<!-- The node id is a unique, sequential id beginning with 0 that identifies each server in the cluster-->
<id>0</id>
<host>localhost</host>
<http-port>8081</http-port>
<socket-port>6666</socket-port>
<admin-port>6667</admin-port>
<!-- A list of data partitions assigned to this server -->
<partitions>0,1,2,3</partitions>
</server>
</cluster>

View file

@ -0,0 +1,4 @@
node.id=0
enable.rebalancing=false
enable.bdb.engine=false
slop.enable=false

View file

@ -0,0 +1,85 @@
<stores>
<store>
<name>Refs</name>
<replication-factor>1</replication-factor>
<preferred-reads>1</preferred-reads>
<required-reads>1</required-reads>
<preferred-writes>1</preferred-writes>
<required-writes>1</required-writes>
<persistence>memory</persistence>
<routing>client</routing>
<key-serializer>
<type>string</type>
<schema-info>utf8</schema-info>
</key-serializer>
<value-serializer>
<type>identity</type>
</value-serializer>
</store>
<store>
<name>MapValues</name>
<replication-factor>1</replication-factor>
<preferred-reads>1</preferred-reads>
<required-reads>1</required-reads>
<preferred-writes>1</preferred-writes>
<required-writes>1</required-writes>
<persistence>memory</persistence>
<routing>client</routing>
<key-serializer>
<type>identity</type>
</key-serializer>
<value-serializer>
<type>identity</type>
</value-serializer>
</store>
<store>
<name>MapKeys</name>
<replication-factor>1</replication-factor>
<preferred-reads>1</preferred-reads>
<required-reads>1</required-reads>
<preferred-writes>1</preferred-writes>
<required-writes>1</required-writes>
<persistence>memory</persistence>
<routing>client</routing>
<key-serializer>
<type>string</type>
<schema-info>utf8</schema-info>
</key-serializer>
<value-serializer>
<type>identity</type>
</value-serializer>
</store>
<store>
<name>Vectors</name>
<replication-factor>1</replication-factor>
<preferred-reads>1</preferred-reads>
<required-reads>1</required-reads>
<preferred-writes>1</preferred-writes>
<required-writes>1</required-writes>
<persistence>memory</persistence>
<routing>client</routing>
<key-serializer>
<type>identity</type>
</key-serializer>
<value-serializer>
<type>identity</type>
</value-serializer>
</store>
<store>
<name>Queues</name>
<replication-factor>1</replication-factor>
<preferred-reads>1</preferred-reads>
<required-reads>1</required-reads>
<preferred-writes>1</preferred-writes>
<required-writes>1</required-writes>
<persistence>memory</persistence>
<routing>client</routing>
<key-serializer>
<type>identity</type>
</key-serializer>
<value-serializer>
<type>identity</type>
</value-serializer>
</store>
</stores>

View file

@ -0,0 +1,41 @@
package se.scalablesolutions.akka.persistence.voldemort
import org.scalatest.matchers.ShouldMatchers
import voldemort.server.{VoldemortServer, VoldemortConfig}
import org.scalatest.{Suite, BeforeAndAfterAll, FunSuite}
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import voldemort.utils.Utils
import java.io.File
import se.scalablesolutions.akka.util.{Logging}
import collection.JavaConversions
import voldemort.store.memory.InMemoryStorageConfiguration
@RunWith(classOf[JUnitRunner])
trait EmbeddedVoldemort extends BeforeAndAfterAll with Logging {
this: Suite =>
var server: VoldemortServer = null
override protected def beforeAll(): Unit = {
try {
val dir = "./akka-persistence/akka-persistence-voldemort/target/scala_2.8.0/test-resources"
val home = new File(dir)
log.info("Creating Voldemort Config")
val config = VoldemortConfig.loadFromVoldemortHome(home.getCanonicalPath)
config.setStorageConfigurations(JavaConversions.asList(List(classOf[InMemoryStorageConfiguration].getName)))
log.info("Starting Voldemort")
server = new VoldemortServer(config)
server.start
VoldemortStorageBackend.initStoreClients
log.info("Started")
} catch {
case e => log.error(e, "Error Starting Voldemort")
throw e
}
}
override protected def afterAll(): Unit = {
server.stop
}
}

View file

@ -0,0 +1,179 @@
package se.scalablesolutions.akka.persistence.voldemort
import org.scalatest.Spec
import org.scalatest.matchers.ShouldMatchers
import org.scalatest.BeforeAndAfterEach
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import se.scalablesolutions.akka.actor.{Transactor, Actor, ActorRef}
import Actor._
import BankAccountActor._
case class Balance(accountNo: String)
case class Debit(accountNo: String, amount: Int, failer: ActorRef)
case class MultiDebit(accountNo: String, amounts: List[Int], failer: ActorRef)
case class Credit(accountNo: String, amount: Int)
case class Log(start: Int, finish: Int)
case object LogSize
object BankAccountActor {
val state = "accountState"
val tx = "txnLog"
}
class BankAccountActor extends Transactor {
private val accountState = VoldemortStorage.newMap(state)
private val txnLog = VoldemortStorage.newVector(tx)
import sjson.json.DefaultProtocol._
import sjson.json.JsonSerialization._
def receive: Receive = {
// check balance
case Balance(accountNo) =>
txnLog.add(("Balance:" + accountNo).getBytes)
self.reply(
accountState.get(accountNo.getBytes)
.map(frombinary[Int](_))
.getOrElse(0))
// debit amount: can fail
case Debit(accountNo, amount, failer) =>
txnLog.add(("Debit:" + accountNo + " " + amount).getBytes)
val m = accountState.get(accountNo.getBytes)
.map(frombinary[Int](_))
.getOrElse(0)
accountState.put(accountNo.getBytes, tobinary(m - amount))
if (amount > m) failer !! "Failure"
self.reply(m - amount)
// many debits: can fail
// demonstrates true rollback even if multiple puts have been done
case MultiDebit(accountNo, amounts, failer) =>
val sum = amounts.foldRight(0)(_ + _)
txnLog.add(("MultiDebit:" + accountNo + " " + sum).getBytes)
val m = accountState.get(accountNo.getBytes)
.map(frombinary[Int](_))
.getOrElse(0)
var cbal = m
amounts.foreach {
amount =>
accountState.put(accountNo.getBytes, tobinary(m - amount))
cbal = cbal - amount
if (cbal < 0) failer !! "Failure"
}
self.reply(m - sum)
// credit amount
case Credit(accountNo, amount) =>
txnLog.add(("Credit:" + accountNo + " " + amount).getBytes)
val m = accountState.get(accountNo.getBytes)
.map(frombinary[Int](_))
.getOrElse(0)
accountState.put(accountNo.getBytes, tobinary(m + amount))
self.reply(m + amount)
case LogSize =>
self.reply(txnLog.length)
case Log(start, finish) =>
self.reply(txnLog.slice(start, finish).map(new String(_)))
}
}
@serializable class PersistentFailerActor extends Transactor {
def receive = {
case "Failure" =>
throw new RuntimeException("Expected exception; to test fault-tolerance")
}
}
@RunWith(classOf[JUnitRunner])
class VoldemortPersistentActorSuite extends
Spec with
ShouldMatchers with
BeforeAndAfterEach with EmbeddedVoldemort {
import VoldemortStorageBackend._
override def beforeEach {
removeMapStorageFor(state)
var size = getVectorStorageSizeFor(tx)
(-1 to size).foreach {
index => {
vectorClient.delete(getIndexedKey(tx, index))
}
}
}
override def afterEach {
beforeEach
}
describe("successful debit") {
it("should debit successfully") {
log.info("Succesful Debit starting")
val bactor = actorOf[BankAccountActor]
bactor.start
val failer = actorOf[PersistentFailerActor]
failer.start
bactor !! Credit("a-123", 5000)
log.info("credited")
bactor !! Debit("a-123", 3000, failer)
log.info("debited")
(bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(2000)
log.info("balane matched")
bactor !! Credit("a-123", 7000)
log.info("Credited")
(bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(9000)
log.info("Balance matched")
bactor !! Debit("a-123", 8000, failer)
log.info("Debited")
(bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(1000)
log.info("Balance matched")
(bactor !! LogSize).get.asInstanceOf[Int] should equal(7)
(bactor !! Log(0, 7)).get.asInstanceOf[Iterable[String]].size should equal(7)
}
}
describe("unsuccessful debit") {
it("debit should fail") {
val bactor = actorOf[BankAccountActor]
bactor.start
val failer = actorOf[PersistentFailerActor]
failer.start
bactor !! Credit("a-123", 5000)
(bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000)
evaluating {
bactor !! Debit("a-123", 7000, failer)
} should produce[Exception]
(bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000)
(bactor !! LogSize).get.asInstanceOf[Int] should equal(3)
}
}
describe("unsuccessful multidebit") {
it("multidebit should fail") {
val bactor = actorOf[BankAccountActor]
bactor.start
val failer = actorOf[PersistentFailerActor]
failer.start
bactor !! Credit("a-123", 5000)
(bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000)
evaluating {
bactor !! MultiDebit("a-123", List(1000, 2000, 4000), failer)
} should produce[Exception]
(bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000)
(bactor !! LogSize).get.asInstanceOf[Int] should equal(3)
}
}
}

View file

@ -0,0 +1,87 @@
package se.scalablesolutions.akka.persistence.voldemort
import org.scalatest.FunSuite
import org.scalatest.matchers.ShouldMatchers
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import se.scalablesolutions.akka.persistence.voldemort.VoldemortStorageBackend._
import se.scalablesolutions.akka.actor.{newUuid,Uuid}
import collection.immutable.TreeSet
import VoldemortStorageBackendSuite._
import se.scalablesolutions.akka.stm._
import se.scalablesolutions.akka.stm.global._
import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.persistence.common._
import se.scalablesolutions.akka.util.Logging
import se.scalablesolutions.akka.config.Config.config
@RunWith(classOf[JUnitRunner])
class VoldemortPersistentDatastructureSuite extends FunSuite with ShouldMatchers with EmbeddedVoldemort with Logging {
test("persistentRefs work as expected") {
val name = newUuid.toString
val one = "one".getBytes
atomic {
val ref = VoldemortStorage.getRef(name)
ref.isDefined should be(false)
ref.swap(one)
ref.get match {
case Some(bytes) => bytes should be(one)
case None => true should be(false)
}
}
val two = "two".getBytes
atomic {
val ref = VoldemortStorage.getRef(name)
ref.isDefined should be(true)
ref.swap(two)
ref.get match {
case Some(bytes) => bytes should be(two)
case None => true should be(false)
}
}
}
test("Persistent Vectors function as expected") {
val name = newUuid.toString
val one = "one".getBytes
val two = "two".getBytes
atomic {
val vec = VoldemortStorage.getVector(name)
vec.add(one)
}
atomic {
val vec = VoldemortStorage.getVector(name)
vec.size should be(1)
vec.add(two)
}
atomic {
val vec = VoldemortStorage.getVector(name)
vec.get(0) should be(one)
vec.get(1) should be(two)
vec.size should be(2)
vec.update(0, two)
}
atomic {
val vec = VoldemortStorage.getVector(name)
vec.get(0) should be(two)
vec.get(1) should be(two)
vec.size should be(2)
vec.update(0, Array.empty[Byte])
vec.update(1, Array.empty[Byte])
}
atomic {
val vec = VoldemortStorage.getVector(name)
vec.get(0) should be(Array.empty[Byte])
vec.get(1) should be(Array.empty[Byte])
vec.size should be(2)
}
}
}

View file

@ -0,0 +1,139 @@
package se.scalablesolutions.akka.persistence.voldemort
import org.scalatest.FunSuite
import org.scalatest.matchers.ShouldMatchers
import org.junit.runner.RunWith
import org.scalatest.junit.JUnitRunner
import se.scalablesolutions.akka.persistence.voldemort.VoldemortStorageBackend._
import se.scalablesolutions.akka.util.{Logging}
import collection.immutable.TreeSet
import VoldemortStorageBackendSuite._
@RunWith(classOf[JUnitRunner])
class VoldemortStorageBackendSuite extends FunSuite with ShouldMatchers with EmbeddedVoldemort with Logging {
test("that ref storage and retrieval works") {
val key = "testRef"
val value = "testRefValue"
val valueBytes = bytes(value)
refClient.delete(key)
refClient.getValue(key, empty) should be(empty)
refClient.put(key, valueBytes)
refClient.getValue(key) should be(valueBytes)
}
test("PersistentRef apis function as expected") {
val key = "apiTestRef"
val value = "apiTestRefValue"
val valueBytes = bytes(value)
refClient.delete(key)
getRefStorageFor(key) should be(None)
insertRefStorageFor(key, valueBytes)
getRefStorageFor(key).get should equal(valueBytes)
}
test("that map key storage and retrieval works") {
val key = "testmapKey"
val mapKeys = new TreeSet[Array[Byte]] + bytes("key1")
mapKeyClient.delete(key)
mapKeyClient.getValue(key, SortedSetSerializer.toBytes(emptySet)) should equal(SortedSetSerializer.toBytes(emptySet))
putMapKeys(key, mapKeys)
getMapKeys(key) should equal(mapKeys)
}
test("that map value storage and retrieval works") {
val key = bytes("keyForTestingMapValueClient")
val value = bytes("value for testing map value client")
mapValueClient.put(key, value)
mapValueClient.getValue(key, empty) should equal(value)
}
test("PersistentMap apis function as expected") {
val name = "theMap"
val key = bytes("mapkey")
val value = bytes("mapValue")
removeMapStorageFor(name, key)
removeMapStorageFor(name)
getMapStorageEntryFor(name, key) should be(None)
getMapStorageSizeFor(name) should be(0)
getMapStorageFor(name).length should be(0)
getMapStorageRangeFor(name, None, None, 100).length should be(0)
insertMapStorageEntryFor(name, key, value)
getMapStorageEntryFor(name, key).get should equal(value)
getMapStorageSizeFor(name) should be(1)
getMapStorageFor(name).length should be(1)
getMapStorageRangeFor(name, None, None, 100).length should be(1)
removeMapStorageFor(name, key)
removeMapStorageFor(name)
getMapStorageEntryFor(name, key) should be(None)
getMapStorageSizeFor(name) should be(0)
getMapStorageFor(name).length should be(0)
getMapStorageRangeFor(name, None, None, 100).length should be(0)
insertMapStorageEntriesFor(name, List(key -> value))
getMapStorageEntryFor(name, key).get should equal(value)
getMapStorageSizeFor(name) should be(1)
getMapStorageFor(name).length should be(1)
getMapStorageRangeFor(name, None, None, 100).length should be(1)
}
test("that vector value storage and retrieval works") {
val key = "vectorValueKey"
val index = 3
val value = bytes("some bytes")
val vecKey = getIndexedKey(key, index)
getIndexFromVectorValueKey(key, vecKey) should be(index)
vectorClient.delete(vecKey)
vectorClient.getValue(vecKey, empty) should equal(empty)
vectorClient.put(vecKey, value)
vectorClient.getValue(vecKey) should equal(value)
}
test("PersistentVector apis function as expected") {
val key = "vectorApiKey"
val value = bytes("Some bytes we want to store in a vector")
val updatedValue = bytes("Some updated bytes we want to store in a vector")
vectorClient.delete(getIndexedKey(key, vectorSizeIndex))
vectorClient.delete(getIndexedKey(key, 0))
vectorClient.delete(getIndexedKey(key, 1))
getVectorStorageEntryFor(key, 0) should be(empty)
getVectorStorageEntryFor(key, 1) should be(empty)
getVectorStorageRangeFor(key, None, None, 1).head should be(empty)
insertVectorStorageEntryFor(key, value)
//again
insertVectorStorageEntryFor(key, value)
getVectorStorageEntryFor(key, 0) should be(value)
getVectorStorageEntryFor(key, 1) should be(value)
getVectorStorageRangeFor(key, None, None, 1).head should be(value)
getVectorStorageRangeFor(key, Some(1), None, 1).head should be(value)
getVectorStorageSizeFor(key) should be(2)
updateVectorStorageEntryFor(key, 1, updatedValue)
getVectorStorageEntryFor(key, 0) should be(value)
getVectorStorageEntryFor(key, 1) should be(updatedValue)
getVectorStorageRangeFor(key, None, None, 1).head should be(value)
getVectorStorageRangeFor(key, Some(1), None, 1).head should be(updatedValue)
getVectorStorageSizeFor(key) should be(2)
}
}
object VoldemortStorageBackendSuite {
val empty = Array.empty[Byte]
val emptySet = new TreeSet[Array[Byte]]
def bytes(value: String): Array[Byte] = {
value.getBytes("UTF-8")
}
}

View file

@ -244,12 +244,12 @@ public final class RemoteProtocol {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_fieldAccessorTable;
}
// required string uuid = 1;
public static final int UUID_FIELD_NUMBER = 1;
private boolean hasUuid;
private java.lang.String uuid_ = "";
public boolean hasUuid() { return hasUuid; }
public java.lang.String getUuid() { return uuid_; }
// required string classOrServiceName = 1;
public static final int CLASSORSERVICENAME_FIELD_NUMBER = 1;
private boolean hasClassOrServiceName;
private java.lang.String classOrServiceName_ = "";
public boolean hasClassOrServiceName() { return hasClassOrServiceName; }
public java.lang.String getClassOrServiceName() { return classOrServiceName_; }
// required string actorClassname = 2;
public static final int ACTORCLASSNAME_FIELD_NUMBER = 2;
@ -276,7 +276,7 @@ public final class RemoteProtocol {
homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
}
public final boolean isInitialized() {
if (!hasUuid) return false;
if (!hasClassOrServiceName) return false;
if (!hasActorClassname) return false;
if (!hasHomeAddress) return false;
if (!getHomeAddress().isInitialized()) return false;
@ -286,8 +286,8 @@ public final class RemoteProtocol {
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
if (hasUuid()) {
output.writeString(1, getUuid());
if (hasClassOrServiceName()) {
output.writeString(1, getClassOrServiceName());
}
if (hasActorClassname()) {
output.writeString(2, getActorClassname());
@ -307,9 +307,9 @@ public final class RemoteProtocol {
if (size != -1) return size;
size = 0;
if (hasUuid()) {
if (hasClassOrServiceName()) {
size += com.google.protobuf.CodedOutputStream
.computeStringSize(1, getUuid());
.computeStringSize(1, getClassOrServiceName());
}
if (hasActorClassname()) {
size += com.google.protobuf.CodedOutputStream
@ -481,8 +481,8 @@ public final class RemoteProtocol {
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) return this;
if (other.hasUuid()) {
setUuid(other.getUuid());
if (other.hasClassOrServiceName()) {
setClassOrServiceName(other.getClassOrServiceName());
}
if (other.hasActorClassname()) {
setActorClassname(other.getActorClassname());
@ -519,7 +519,7 @@ public final class RemoteProtocol {
break;
}
case 10: {
setUuid(input.readString());
setClassOrServiceName(input.readString());
break;
}
case 18: {
@ -544,24 +544,24 @@ public final class RemoteProtocol {
}
// required string uuid = 1;
public boolean hasUuid() {
return result.hasUuid();
// required string classOrServiceName = 1;
public boolean hasClassOrServiceName() {
return result.hasClassOrServiceName();
}
public java.lang.String getUuid() {
return result.getUuid();
public java.lang.String getClassOrServiceName() {
return result.getClassOrServiceName();
}
public Builder setUuid(java.lang.String value) {
public Builder setClassOrServiceName(java.lang.String value) {
if (value == null) {
throw new NullPointerException();
}
result.hasUuid = true;
result.uuid_ = value;
result.hasClassOrServiceName = true;
result.classOrServiceName_ = value;
return this;
}
public Builder clearUuid() {
result.hasUuid = false;
result.uuid_ = getDefaultInstance().getUuid();
public Builder clearClassOrServiceName() {
result.hasClassOrServiceName = false;
result.classOrServiceName_ = getDefaultInstance().getClassOrServiceName();
return this;
}
@ -1034,12 +1034,12 @@ public final class RemoteProtocol {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_fieldAccessorTable;
}
// required string uuid = 1;
// required .UuidProtocol uuid = 1;
public static final int UUID_FIELD_NUMBER = 1;
private boolean hasUuid;
private java.lang.String uuid_ = "";
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_;
public boolean hasUuid() { return hasUuid; }
public java.lang.String getUuid() { return uuid_; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; }
// required string id = 2;
public static final int ID_FIELD_NUMBER = 2;
@ -1131,6 +1131,7 @@ public final class RemoteProtocol {
}
private void initFields() {
uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance();
supervisor_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
@ -1140,6 +1141,7 @@ public final class RemoteProtocol {
if (!hasId) return false;
if (!hasActorClassname) return false;
if (!hasOriginalAddress) return false;
if (!getUuid().isInitialized()) return false;
if (!getOriginalAddress().isInitialized()) return false;
if (hasLifeCycle()) {
if (!getLifeCycle().isInitialized()) return false;
@ -1157,7 +1159,7 @@ public final class RemoteProtocol {
throws java.io.IOException {
getSerializedSize();
if (hasUuid()) {
output.writeString(1, getUuid());
output.writeMessage(1, getUuid());
}
if (hasId()) {
output.writeString(2, getId());
@ -1206,7 +1208,7 @@ public final class RemoteProtocol {
size = 0;
if (hasUuid()) {
size += com.google.protobuf.CodedOutputStream
.computeStringSize(1, getUuid());
.computeMessageSize(1, getUuid());
}
if (hasId()) {
size += com.google.protobuf.CodedOutputStream
@ -1419,7 +1421,7 @@ public final class RemoteProtocol {
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance()) return this;
if (other.hasUuid()) {
setUuid(other.getUuid());
mergeUuid(other.getUuid());
}
if (other.hasId()) {
setId(other.getId());
@ -1486,7 +1488,12 @@ public final class RemoteProtocol {
break;
}
case 10: {
setUuid(input.readString());
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder();
if (hasUuid()) {
subBuilder.mergeFrom(getUuid());
}
input.readMessage(subBuilder, extensionRegistry);
setUuid(subBuilder.buildPartial());
break;
}
case 18: {
@ -1559,24 +1566,40 @@ public final class RemoteProtocol {
}
// required string uuid = 1;
// required .UuidProtocol uuid = 1;
public boolean hasUuid() {
return result.hasUuid();
}
public java.lang.String getUuid() {
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() {
return result.getUuid();
}
public Builder setUuid(java.lang.String value) {
public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (value == null) {
throw new NullPointerException();
}
result.hasUuid = true;
throw new NullPointerException();
}
result.hasUuid = true;
result.uuid_ = value;
return this;
}
public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) {
result.hasUuid = true;
result.uuid_ = builderForValue.build();
return this;
}
public Builder mergeUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (result.hasUuid() &&
result.uuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) {
result.uuid_ =
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial();
} else {
result.uuid_ = value;
}
result.hasUuid = true;
return this;
}
public Builder clearUuid() {
result.hasUuid = false;
result.uuid_ = getDefaultInstance().getUuid();
result.uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
return this;
}
@ -2674,12 +2697,12 @@ public final class RemoteProtocol {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_fieldAccessorTable;
}
// required string uuid = 1;
// required .UuidProtocol uuid = 1;
public static final int UUID_FIELD_NUMBER = 1;
private boolean hasUuid;
private java.lang.String uuid_ = "";
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_;
public boolean hasUuid() { return hasUuid; }
public java.lang.String getUuid() { return uuid_; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; }
// required string target = 2;
public static final int TARGET_FIELD_NUMBER = 2;
@ -2717,6 +2740,7 @@ public final class RemoteProtocol {
public java.lang.String getId() { return id_; }
private void initFields() {
uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR;
typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance();
}
@ -2725,6 +2749,7 @@ public final class RemoteProtocol {
if (!hasTarget) return false;
if (!hasTimeout) return false;
if (!hasActorType) return false;
if (!getUuid().isInitialized()) return false;
if (hasTypedActorInfo()) {
if (!getTypedActorInfo().isInitialized()) return false;
}
@ -2735,7 +2760,7 @@ public final class RemoteProtocol {
throws java.io.IOException {
getSerializedSize();
if (hasUuid()) {
output.writeString(1, getUuid());
output.writeMessage(1, getUuid());
}
if (hasTarget()) {
output.writeString(2, getTarget());
@ -2763,7 +2788,7 @@ public final class RemoteProtocol {
size = 0;
if (hasUuid()) {
size += com.google.protobuf.CodedOutputStream
.computeStringSize(1, getUuid());
.computeMessageSize(1, getUuid());
}
if (hasTarget()) {
size += com.google.protobuf.CodedOutputStream
@ -2944,7 +2969,7 @@ public final class RemoteProtocol {
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) return this;
if (other.hasUuid()) {
setUuid(other.getUuid());
mergeUuid(other.getUuid());
}
if (other.hasTarget()) {
setTarget(other.getTarget());
@ -2987,7 +3012,12 @@ public final class RemoteProtocol {
break;
}
case 10: {
setUuid(input.readString());
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder();
if (hasUuid()) {
subBuilder.mergeFrom(getUuid());
}
input.readMessage(subBuilder, extensionRegistry);
setUuid(subBuilder.buildPartial());
break;
}
case 18: {
@ -3026,24 +3056,40 @@ public final class RemoteProtocol {
}
// required string uuid = 1;
// required .UuidProtocol uuid = 1;
public boolean hasUuid() {
return result.hasUuid();
}
public java.lang.String getUuid() {
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() {
return result.getUuid();
}
public Builder setUuid(java.lang.String value) {
public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (value == null) {
throw new NullPointerException();
}
result.hasUuid = true;
throw new NullPointerException();
}
result.hasUuid = true;
result.uuid_ = value;
return this;
}
public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) {
result.hasUuid = true;
result.uuid_ = builderForValue.build();
return this;
}
public Builder mergeUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (result.hasUuid() &&
result.uuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) {
result.uuid_ =
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial();
} else {
result.uuid_ = value;
}
result.hasUuid = true;
return this;
}
public Builder clearUuid() {
result.hasUuid = false;
result.uuid_ = getDefaultInstance().getUuid();
result.uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
return this;
}
@ -3535,12 +3581,12 @@ public final class RemoteProtocol {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_fieldAccessorTable;
}
// required uint64 id = 1;
public static final int ID_FIELD_NUMBER = 1;
private boolean hasId;
private long id_ = 0L;
public boolean hasId() { return hasId; }
public long getId() { return id_; }
// required .UuidProtocol uuid = 1;
public static final int UUID_FIELD_NUMBER = 1;
private boolean hasUuid;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_;
public boolean hasUuid() { return hasUuid; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; }
// required .MessageProtocol message = 2;
public static final int MESSAGE_FIELD_NUMBER = 2;
@ -3563,12 +3609,12 @@ public final class RemoteProtocol {
public boolean hasIsOneWay() { return hasIsOneWay; }
public boolean getIsOneWay() { return isOneWay_; }
// optional string supervisorUuid = 5;
// optional .UuidProtocol supervisorUuid = 5;
public static final int SUPERVISORUUID_FIELD_NUMBER = 5;
private boolean hasSupervisorUuid;
private java.lang.String supervisorUuid_ = "";
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol supervisorUuid_;
public boolean hasSupervisorUuid() { return hasSupervisorUuid; }
public java.lang.String getSupervisorUuid() { return supervisorUuid_; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { return supervisorUuid_; }
// optional .RemoteActorRefProtocol sender = 6;
public static final int SENDER_FIELD_NUMBER = 6;
@ -3590,17 +3636,23 @@ public final class RemoteProtocol {
}
private void initFields() {
uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance();
supervisorUuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
}
public final boolean isInitialized() {
if (!hasId) return false;
if (!hasUuid) return false;
if (!hasMessage) return false;
if (!hasActorInfo) return false;
if (!hasIsOneWay) return false;
if (!getUuid().isInitialized()) return false;
if (!getMessage().isInitialized()) return false;
if (!getActorInfo().isInitialized()) return false;
if (hasSupervisorUuid()) {
if (!getSupervisorUuid().isInitialized()) return false;
}
if (hasSender()) {
if (!getSender().isInitialized()) return false;
}
@ -3613,8 +3665,8 @@ public final class RemoteProtocol {
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
if (hasId()) {
output.writeUInt64(1, getId());
if (hasUuid()) {
output.writeMessage(1, getUuid());
}
if (hasMessage()) {
output.writeMessage(2, getMessage());
@ -3626,7 +3678,7 @@ public final class RemoteProtocol {
output.writeBool(4, getIsOneWay());
}
if (hasSupervisorUuid()) {
output.writeString(5, getSupervisorUuid());
output.writeMessage(5, getSupervisorUuid());
}
if (hasSender()) {
output.writeMessage(6, getSender());
@ -3643,9 +3695,9 @@ public final class RemoteProtocol {
if (size != -1) return size;
size = 0;
if (hasId()) {
if (hasUuid()) {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(1, getId());
.computeMessageSize(1, getUuid());
}
if (hasMessage()) {
size += com.google.protobuf.CodedOutputStream
@ -3661,7 +3713,7 @@ public final class RemoteProtocol {
}
if (hasSupervisorUuid()) {
size += com.google.protobuf.CodedOutputStream
.computeStringSize(5, getSupervisorUuid());
.computeMessageSize(5, getSupervisorUuid());
}
if (hasSender()) {
size += com.google.protobuf.CodedOutputStream
@ -3833,8 +3885,8 @@ public final class RemoteProtocol {
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance()) return this;
if (other.hasId()) {
setId(other.getId());
if (other.hasUuid()) {
mergeUuid(other.getUuid());
}
if (other.hasMessage()) {
mergeMessage(other.getMessage());
@ -3846,7 +3898,7 @@ public final class RemoteProtocol {
setIsOneWay(other.getIsOneWay());
}
if (other.hasSupervisorUuid()) {
setSupervisorUuid(other.getSupervisorUuid());
mergeSupervisorUuid(other.getSupervisorUuid());
}
if (other.hasSender()) {
mergeSender(other.getSender());
@ -3882,8 +3934,13 @@ public final class RemoteProtocol {
}
break;
}
case 8: {
setId(input.readUInt64());
case 10: {
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder();
if (hasUuid()) {
subBuilder.mergeFrom(getUuid());
}
input.readMessage(subBuilder, extensionRegistry);
setUuid(subBuilder.buildPartial());
break;
}
case 18: {
@ -3909,7 +3966,12 @@ public final class RemoteProtocol {
break;
}
case 42: {
setSupervisorUuid(input.readString());
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder();
if (hasSupervisorUuid()) {
subBuilder.mergeFrom(getSupervisorUuid());
}
input.readMessage(subBuilder, extensionRegistry);
setSupervisorUuid(subBuilder.buildPartial());
break;
}
case 50: {
@ -3932,21 +3994,40 @@ public final class RemoteProtocol {
}
// required uint64 id = 1;
public boolean hasId() {
return result.hasId();
// required .UuidProtocol uuid = 1;
public boolean hasUuid() {
return result.hasUuid();
}
public long getId() {
return result.getId();
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() {
return result.getUuid();
}
public Builder setId(long value) {
result.hasId = true;
result.id_ = value;
public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (value == null) {
throw new NullPointerException();
}
result.hasUuid = true;
result.uuid_ = value;
return this;
}
public Builder clearId() {
result.hasId = false;
result.id_ = 0L;
public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) {
result.hasUuid = true;
result.uuid_ = builderForValue.build();
return this;
}
public Builder mergeUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (result.hasUuid() &&
result.uuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) {
result.uuid_ =
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial();
} else {
result.uuid_ = value;
}
result.hasUuid = true;
return this;
}
public Builder clearUuid() {
result.hasUuid = false;
result.uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
return this;
}
@ -4042,24 +4123,40 @@ public final class RemoteProtocol {
return this;
}
// optional string supervisorUuid = 5;
// optional .UuidProtocol supervisorUuid = 5;
public boolean hasSupervisorUuid() {
return result.hasSupervisorUuid();
}
public java.lang.String getSupervisorUuid() {
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() {
return result.getSupervisorUuid();
}
public Builder setSupervisorUuid(java.lang.String value) {
public Builder setSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (value == null) {
throw new NullPointerException();
}
result.hasSupervisorUuid = true;
throw new NullPointerException();
}
result.hasSupervisorUuid = true;
result.supervisorUuid_ = value;
return this;
}
public Builder setSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) {
result.hasSupervisorUuid = true;
result.supervisorUuid_ = builderForValue.build();
return this;
}
public Builder mergeSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (result.hasSupervisorUuid() &&
result.supervisorUuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) {
result.supervisorUuid_ =
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.supervisorUuid_).mergeFrom(value).buildPartial();
} else {
result.supervisorUuid_ = value;
}
result.hasSupervisorUuid = true;
return this;
}
public Builder clearSupervisorUuid() {
result.hasSupervisorUuid = false;
result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid();
result.supervisorUuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
return this;
}
@ -4190,12 +4287,12 @@ public final class RemoteProtocol {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_fieldAccessorTable;
}
// required uint64 id = 1;
public static final int ID_FIELD_NUMBER = 1;
private boolean hasId;
private long id_ = 0L;
public boolean hasId() { return hasId; }
public long getId() { return id_; }
// required .UuidProtocol uuid = 1;
public static final int UUID_FIELD_NUMBER = 1;
private boolean hasUuid;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_;
public boolean hasUuid() { return hasUuid; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; }
// optional .MessageProtocol message = 2;
public static final int MESSAGE_FIELD_NUMBER = 2;
@ -4211,12 +4308,12 @@ public final class RemoteProtocol {
public boolean hasException() { return hasException; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { return exception_; }
// optional string supervisorUuid = 4;
// optional .UuidProtocol supervisorUuid = 4;
public static final int SUPERVISORUUID_FIELD_NUMBER = 4;
private boolean hasSupervisorUuid;
private java.lang.String supervisorUuid_ = "";
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol supervisorUuid_;
public boolean hasSupervisorUuid() { return hasSupervisorUuid; }
public java.lang.String getSupervisorUuid() { return supervisorUuid_; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { return supervisorUuid_; }
// required bool isActor = 5;
public static final int ISACTOR_FIELD_NUMBER = 5;
@ -4245,19 +4342,25 @@ public final class RemoteProtocol {
}
private void initFields() {
uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance();
supervisorUuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
}
public final boolean isInitialized() {
if (!hasId) return false;
if (!hasUuid) return false;
if (!hasIsActor) return false;
if (!hasIsSuccessful) return false;
if (!getUuid().isInitialized()) return false;
if (hasMessage()) {
if (!getMessage().isInitialized()) return false;
}
if (hasException()) {
if (!getException().isInitialized()) return false;
}
if (hasSupervisorUuid()) {
if (!getSupervisorUuid().isInitialized()) return false;
}
for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) {
if (!element.isInitialized()) return false;
}
@ -4267,8 +4370,8 @@ public final class RemoteProtocol {
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
if (hasId()) {
output.writeUInt64(1, getId());
if (hasUuid()) {
output.writeMessage(1, getUuid());
}
if (hasMessage()) {
output.writeMessage(2, getMessage());
@ -4277,7 +4380,7 @@ public final class RemoteProtocol {
output.writeMessage(3, getException());
}
if (hasSupervisorUuid()) {
output.writeString(4, getSupervisorUuid());
output.writeMessage(4, getSupervisorUuid());
}
if (hasIsActor()) {
output.writeBool(5, getIsActor());
@ -4297,9 +4400,9 @@ public final class RemoteProtocol {
if (size != -1) return size;
size = 0;
if (hasId()) {
if (hasUuid()) {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(1, getId());
.computeMessageSize(1, getUuid());
}
if (hasMessage()) {
size += com.google.protobuf.CodedOutputStream
@ -4311,7 +4414,7 @@ public final class RemoteProtocol {
}
if (hasSupervisorUuid()) {
size += com.google.protobuf.CodedOutputStream
.computeStringSize(4, getSupervisorUuid());
.computeMessageSize(4, getSupervisorUuid());
}
if (hasIsActor()) {
size += com.google.protobuf.CodedOutputStream
@ -4487,8 +4590,8 @@ public final class RemoteProtocol {
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance()) return this;
if (other.hasId()) {
setId(other.getId());
if (other.hasUuid()) {
mergeUuid(other.getUuid());
}
if (other.hasMessage()) {
mergeMessage(other.getMessage());
@ -4497,7 +4600,7 @@ public final class RemoteProtocol {
mergeException(other.getException());
}
if (other.hasSupervisorUuid()) {
setSupervisorUuid(other.getSupervisorUuid());
mergeSupervisorUuid(other.getSupervisorUuid());
}
if (other.hasIsActor()) {
setIsActor(other.getIsActor());
@ -4536,8 +4639,13 @@ public final class RemoteProtocol {
}
break;
}
case 8: {
setId(input.readUInt64());
case 10: {
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder();
if (hasUuid()) {
subBuilder.mergeFrom(getUuid());
}
input.readMessage(subBuilder, extensionRegistry);
setUuid(subBuilder.buildPartial());
break;
}
case 18: {
@ -4559,7 +4667,12 @@ public final class RemoteProtocol {
break;
}
case 34: {
setSupervisorUuid(input.readString());
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder();
if (hasSupervisorUuid()) {
subBuilder.mergeFrom(getSupervisorUuid());
}
input.readMessage(subBuilder, extensionRegistry);
setSupervisorUuid(subBuilder.buildPartial());
break;
}
case 40: {
@ -4581,21 +4694,40 @@ public final class RemoteProtocol {
}
// required uint64 id = 1;
public boolean hasId() {
return result.hasId();
// required .UuidProtocol uuid = 1;
public boolean hasUuid() {
return result.hasUuid();
}
public long getId() {
return result.getId();
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() {
return result.getUuid();
}
public Builder setId(long value) {
result.hasId = true;
result.id_ = value;
public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (value == null) {
throw new NullPointerException();
}
result.hasUuid = true;
result.uuid_ = value;
return this;
}
public Builder clearId() {
result.hasId = false;
result.id_ = 0L;
public Builder setUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) {
result.hasUuid = true;
result.uuid_ = builderForValue.build();
return this;
}
public Builder mergeUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (result.hasUuid() &&
result.uuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) {
result.uuid_ =
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial();
} else {
result.uuid_ = value;
}
result.hasUuid = true;
return this;
}
public Builder clearUuid() {
result.hasUuid = false;
result.uuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
return this;
}
@ -4673,24 +4805,40 @@ public final class RemoteProtocol {
return this;
}
// optional string supervisorUuid = 4;
// optional .UuidProtocol supervisorUuid = 4;
public boolean hasSupervisorUuid() {
return result.hasSupervisorUuid();
}
public java.lang.String getSupervisorUuid() {
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() {
return result.getSupervisorUuid();
}
public Builder setSupervisorUuid(java.lang.String value) {
public Builder setSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (value == null) {
throw new NullPointerException();
}
result.hasSupervisorUuid = true;
throw new NullPointerException();
}
result.hasSupervisorUuid = true;
result.supervisorUuid_ = value;
return this;
}
public Builder setSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) {
result.hasSupervisorUuid = true;
result.supervisorUuid_ = builderForValue.build();
return this;
}
public Builder mergeSupervisorUuid(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol value) {
if (result.hasSupervisorUuid() &&
result.supervisorUuid_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) {
result.supervisorUuid_ =
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.supervisorUuid_).mergeFrom(value).buildPartial();
} else {
result.supervisorUuid_ = value;
}
result.hasSupervisorUuid = true;
return this;
}
public Builder clearSupervisorUuid() {
result.hasSupervisorUuid = false;
result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid();
result.supervisorUuid_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
return this;
}
@ -6482,57 +6630,60 @@ public final class RemoteProtocol {
descriptor;
static {
java.lang.String[] descriptorData = {
"\n\024RemoteProtocol.proto\"v\n\026RemoteActorRef" +
"Protocol\022\014\n\004uuid\030\001 \002(\t\022\026\n\016actorClassname" +
"\030\002 \002(\t\022%\n\013homeAddress\030\003 \002(\0132\020.AddressPro" +
"tocol\022\017\n\007timeout\030\004 \001(\004\"_\n\033RemoteTypedAct" +
"orRefProtocol\022)\n\010actorRef\030\001 \002(\0132\027.Remote" +
"ActorRefProtocol\022\025\n\rinterfaceName\030\002 \002(\t\"" +
"\200\003\n\032SerializedActorRefProtocol\022\014\n\004uuid\030\001" +
" \002(\t\022\n\n\002id\030\002 \002(\t\022\026\n\016actorClassname\030\003 \002(\t" +
"\022)\n\017originalAddress\030\004 \002(\0132\020.AddressProto" +
"col\022\025\n\ractorInstance\030\005 \001(\014\022\033\n\023serializer",
"Classname\030\006 \001(\t\022\024\n\014isTransactor\030\007 \001(\010\022\017\n" +
"\007timeout\030\010 \001(\004\022\026\n\016receiveTimeout\030\t \001(\004\022%" +
"\n\tlifeCycle\030\n \001(\0132\022.LifeCycleProtocol\022+\n" +
"\nsupervisor\030\013 \001(\0132\027.RemoteActorRefProtoc" +
"ol\022\024\n\014hotswapStack\030\014 \001(\014\022(\n\010messages\030\r \003" +
"(\0132\026.RemoteRequestProtocol\"g\n\037Serialized" +
"TypedActorRefProtocol\022-\n\010actorRef\030\001 \002(\0132" +
"\033.SerializedActorRefProtocol\022\025\n\rinterfac" +
"eName\030\002 \002(\t\"r\n\017MessageProtocol\0225\n\023serial" +
"izationScheme\030\001 \002(\0162\030.SerializationSchem",
"eType\022\017\n\007message\030\002 \002(\014\022\027\n\017messageManifes" +
"t\030\003 \001(\014\"\236\001\n\021ActorInfoProtocol\022\014\n\004uuid\030\001 " +
"\002(\t\022\016\n\006target\030\002 \002(\t\022\017\n\007timeout\030\003 \002(\004\022\035\n\t" +
"actorType\030\004 \002(\0162\n.ActorType\022/\n\016typedActo" +
"rInfo\030\005 \001(\0132\027.TypedActorInfoProtocol\022\n\n\002" +
"id\030\006 \001(\t\";\n\026TypedActorInfoProtocol\022\021\n\tin" +
"terface\030\001 \002(\t\022\016\n\006method\030\002 \002(\t\"\352\001\n\025Remote" +
"RequestProtocol\022\n\n\002id\030\001 \002(\004\022!\n\007message\030\002" +
" \002(\0132\020.MessageProtocol\022%\n\tactorInfo\030\003 \002(" +
"\0132\022.ActorInfoProtocol\022\020\n\010isOneWay\030\004 \002(\010\022",
"\026\n\016supervisorUuid\030\005 \001(\t\022\'\n\006sender\030\006 \001(\0132" +
"\027.RemoteActorRefProtocol\022(\n\010metadata\030\007 \003" +
"(\0132\026.MetadataEntryProtocol\"\324\001\n\023RemoteRep" +
"lyProtocol\022\n\n\002id\030\001 \002(\004\022!\n\007message\030\002 \001(\0132" +
"\020.MessageProtocol\022%\n\texception\030\003 \001(\0132\022.E" +
"xceptionProtocol\022\026\n\016supervisorUuid\030\004 \001(\t" +
"\022\017\n\007isActor\030\005 \002(\010\022\024\n\014isSuccessful\030\006 \002(\010\022" +
"(\n\010metadata\030\007 \003(\0132\026.MetadataEntryProtoco" +
"l\")\n\014UuidProtocol\022\014\n\004high\030\001 \002(\004\022\013\n\003low\030\002" +
" \002(\004\"3\n\025MetadataEntryProtocol\022\013\n\003key\030\001 \002",
"(\t\022\r\n\005value\030\002 \002(\014\"6\n\021LifeCycleProtocol\022!" +
"\n\tlifeCycle\030\001 \002(\0162\016.LifeCycleType\"1\n\017Add" +
"ressProtocol\022\020\n\010hostname\030\001 \002(\t\022\014\n\004port\030\002" +
" \002(\r\"7\n\021ExceptionProtocol\022\021\n\tclassname\030\001" +
" \002(\t\022\017\n\007message\030\002 \002(\t*=\n\tActorType\022\017\n\013SC" +
"ALA_ACTOR\020\001\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACT" +
"OR\020\003*]\n\027SerializationSchemeType\022\010\n\004JAVA\020" +
"\001\022\013\n\007SBINARY\020\002\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_J" +
"SON\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\t" +
"PERMANENT\020\001\022\r\n\tTEMPORARY\020\002B-\n)se.scalabl",
"esolutions.akka.remote.protocolH\001"
"\n\024RemoteProtocol.proto\"\204\001\n\026RemoteActorRe" +
"fProtocol\022\032\n\022classOrServiceName\030\001 \002(\t\022\026\n" +
"\016actorClassname\030\002 \002(\t\022%\n\013homeAddress\030\003 \002" +
"(\0132\020.AddressProtocol\022\017\n\007timeout\030\004 \001(\004\"_\n" +
"\033RemoteTypedActorRefProtocol\022)\n\010actorRef" +
"\030\001 \002(\0132\027.RemoteActorRefProtocol\022\025\n\rinter" +
"faceName\030\002 \002(\t\"\217\003\n\032SerializedActorRefPro" +
"tocol\022\033\n\004uuid\030\001 \002(\0132\r.UuidProtocol\022\n\n\002id" +
"\030\002 \002(\t\022\026\n\016actorClassname\030\003 \002(\t\022)\n\017origin" +
"alAddress\030\004 \002(\0132\020.AddressProtocol\022\025\n\ract",
"orInstance\030\005 \001(\014\022\033\n\023serializerClassname\030" +
"\006 \001(\t\022\024\n\014isTransactor\030\007 \001(\010\022\017\n\007timeout\030\010" +
" \001(\004\022\026\n\016receiveTimeout\030\t \001(\004\022%\n\tlifeCycl" +
"e\030\n \001(\0132\022.LifeCycleProtocol\022+\n\nsuperviso" +
"r\030\013 \001(\0132\027.RemoteActorRefProtocol\022\024\n\014hots" +
"wapStack\030\014 \001(\014\022(\n\010messages\030\r \003(\0132\026.Remot" +
"eRequestProtocol\"g\n\037SerializedTypedActor" +
"RefProtocol\022-\n\010actorRef\030\001 \002(\0132\033.Serializ" +
"edActorRefProtocol\022\025\n\rinterfaceName\030\002 \002(" +
"\t\"r\n\017MessageProtocol\0225\n\023serializationSch",
"eme\030\001 \002(\0162\030.SerializationSchemeType\022\017\n\007m" +
"essage\030\002 \002(\014\022\027\n\017messageManifest\030\003 \001(\014\"\255\001" +
"\n\021ActorInfoProtocol\022\033\n\004uuid\030\001 \002(\0132\r.Uuid" +
"Protocol\022\016\n\006target\030\002 \002(\t\022\017\n\007timeout\030\003 \002(" +
"\004\022\035\n\tactorType\030\004 \002(\0162\n.ActorType\022/\n\016type" +
"dActorInfo\030\005 \001(\0132\027.TypedActorInfoProtoco" +
"l\022\n\n\002id\030\006 \001(\t\";\n\026TypedActorInfoProtocol\022" +
"\021\n\tinterface\030\001 \002(\t\022\016\n\006method\030\002 \002(\t\"\212\002\n\025R" +
"emoteRequestProtocol\022\033\n\004uuid\030\001 \002(\0132\r.Uui" +
"dProtocol\022!\n\007message\030\002 \002(\0132\020.MessageProt",
"ocol\022%\n\tactorInfo\030\003 \002(\0132\022.ActorInfoProto" +
"col\022\020\n\010isOneWay\030\004 \002(\010\022%\n\016supervisorUuid\030" +
"\005 \001(\0132\r.UuidProtocol\022\'\n\006sender\030\006 \001(\0132\027.R" +
"emoteActorRefProtocol\022(\n\010metadata\030\007 \003(\0132" +
"\026.MetadataEntryProtocol\"\364\001\n\023RemoteReplyP" +
"rotocol\022\033\n\004uuid\030\001 \002(\0132\r.UuidProtocol\022!\n\007" +
"message\030\002 \001(\0132\020.MessageProtocol\022%\n\texcep" +
"tion\030\003 \001(\0132\022.ExceptionProtocol\022%\n\016superv" +
"isorUuid\030\004 \001(\0132\r.UuidProtocol\022\017\n\007isActor" +
"\030\005 \002(\010\022\024\n\014isSuccessful\030\006 \002(\010\022(\n\010metadata",
"\030\007 \003(\0132\026.MetadataEntryProtocol\")\n\014UuidPr" +
"otocol\022\014\n\004high\030\001 \002(\004\022\013\n\003low\030\002 \002(\004\"3\n\025Met" +
"adataEntryProtocol\022\013\n\003key\030\001 \002(\t\022\r\n\005value" +
"\030\002 \002(\014\"6\n\021LifeCycleProtocol\022!\n\tlifeCycle" +
"\030\001 \002(\0162\016.LifeCycleType\"1\n\017AddressProtoco" +
"l\022\020\n\010hostname\030\001 \002(\t\022\014\n\004port\030\002 \002(\r\"7\n\021Exc" +
"eptionProtocol\022\021\n\tclassname\030\001 \002(\t\022\017\n\007mes" +
"sage\030\002 \002(\t*=\n\tActorType\022\017\n\013SCALA_ACTOR\020\001" +
"\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACTOR\020\003*]\n\027Ser" +
"ializationSchemeType\022\010\n\004JAVA\020\001\022\013\n\007SBINAR",
"Y\020\002\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_JSON\020\004\022\014\n\010PR" +
"OTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tPERMANENT\020\001" +
"\022\r\n\tTEMPORARY\020\002B-\n)se.scalablesolutions." +
"akka.remote.protocolH\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -6544,7 +6695,7 @@ public final class RemoteProtocol {
internal_static_RemoteActorRefProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_RemoteActorRefProtocol_descriptor,
new java.lang.String[] { "Uuid", "ActorClassname", "HomeAddress", "Timeout", },
new java.lang.String[] { "ClassOrServiceName", "ActorClassname", "HomeAddress", "Timeout", },
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.class,
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder.class);
internal_static_RemoteTypedActorRefProtocol_descriptor =
@ -6600,7 +6751,7 @@ public final class RemoteProtocol {
internal_static_RemoteRequestProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_RemoteRequestProtocol_descriptor,
new java.lang.String[] { "Id", "Message", "ActorInfo", "IsOneWay", "SupervisorUuid", "Sender", "Metadata", },
new java.lang.String[] { "Uuid", "Message", "ActorInfo", "IsOneWay", "SupervisorUuid", "Sender", "Metadata", },
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.class,
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder.class);
internal_static_RemoteReplyProtocol_descriptor =
@ -6608,7 +6759,7 @@ public final class RemoteProtocol {
internal_static_RemoteReplyProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_RemoteReplyProtocol_descriptor,
new java.lang.String[] { "Id", "Message", "Exception", "SupervisorUuid", "IsActor", "IsSuccessful", "Metadata", },
new java.lang.String[] { "Uuid", "Message", "Exception", "SupervisorUuid", "IsActor", "IsSuccessful", "Metadata", },
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.class,
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.Builder.class);
internal_static_UuidProtocol_descriptor =

View file

@ -16,7 +16,7 @@ option optimize_for = SPEED;
* on the original node.
*/
message RemoteActorRefProtocol {
required string uuid = 1;
required string classOrServiceName = 1;
required string actorClassname = 2;
required AddressProtocol homeAddress = 3;
optional uint64 timeout = 4;
@ -37,7 +37,7 @@ message RemoteTypedActorRefProtocol {
* from its original host.
*/
message SerializedActorRefProtocol {
required string uuid = 1;
required UuidProtocol uuid = 1;
required string id = 2;
required string actorClassname = 3;
required AddressProtocol originalAddress = 4;
@ -75,7 +75,7 @@ message MessageProtocol {
* Defines the actor info.
*/
message ActorInfoProtocol {
required string uuid = 1;
required UuidProtocol uuid = 1;
required string target = 2;
required uint64 timeout = 3;
required ActorType actorType = 4;
@ -95,11 +95,11 @@ message TypedActorInfoProtocol {
* Defines a remote message request.
*/
message RemoteRequestProtocol {
required uint64 id = 1;
required UuidProtocol uuid = 1;
required MessageProtocol message = 2;
required ActorInfoProtocol actorInfo = 3;
required bool isOneWay = 4;
optional string supervisorUuid = 5;
optional UuidProtocol supervisorUuid = 5;
optional RemoteActorRefProtocol sender = 6;
repeated MetadataEntryProtocol metadata = 7;
}
@ -108,10 +108,10 @@ message RemoteRequestProtocol {
* Defines a remote message reply.
*/
message RemoteReplyProtocol {
required uint64 id = 1;
required UuidProtocol uuid = 1;
optional MessageProtocol message = 2;
optional ExceptionProtocol exception = 3;
optional string supervisorUuid = 4;
optional UuidProtocol supervisorUuid = 4;
required bool isActor = 5;
required bool isSuccessful = 6;
repeated MetadataEntryProtocol metadata = 7;

View file

@ -28,17 +28,21 @@ object MessageSerializer extends Logging {
messageProtocol.getSerializationScheme match {
case SerializationSchemeType.JAVA =>
unbox(SERIALIZER_JAVA.fromBinary(messageProtocol.getMessage.toByteArray, None))
case SerializationSchemeType.PROTOBUF =>
val clazz = loadManifest(SERIALIZER_PROTOBUF.classLoader, messageProtocol)
SERIALIZER_PROTOBUF.fromBinary(messageProtocol.getMessage.toByteArray, Some(clazz))
case SerializationSchemeType.SBINARY =>
val clazz = loadManifest(SERIALIZER_SBINARY.classLoader, messageProtocol)
val renderer = clazz.newInstance.asInstanceOf[Serializable.SBinary[_ <: AnyRef]]
renderer.fromBytes(messageProtocol.getMessage.toByteArray)
case SerializationSchemeType.SCALA_JSON =>
val clazz = loadManifest(SERIALIZER_SCALA_JSON.classLoader, messageProtocol)
import scala.reflect._
SERIALIZER_SCALA_JSON.fromBinary(messageProtocol.getMessage.toByteArray)(Manifest.classType(clazz))
val renderer = clazz.newInstance.asInstanceOf[Serializable.ScalaJSON[_]]
renderer.fromBytes(messageProtocol.getMessage.toByteArray)
case SerializationSchemeType.JAVA_JSON =>
val clazz = loadManifest(SERIALIZER_JAVA_JSON.classLoader, messageProtocol)
SERIALIZER_JAVA_JSON.fromBinary(messageProtocol.getMessage.toByteArray, Some(clazz))
@ -52,9 +56,9 @@ object MessageSerializer extends Logging {
builder.setSerializationScheme(SerializationSchemeType.PROTOBUF)
builder.setMessage(ByteString.copyFrom(serializable.toByteArray))
builder.setMessageManifest(ByteString.copyFromUtf8(serializable.getClass.getName))
} else if (message.isInstanceOf[Serializable.ScalaJSON]) {
} else if (message.isInstanceOf[Serializable.ScalaJSON[_]]) {
builder.setSerializationScheme(SerializationSchemeType.SCALA_JSON)
setMessageAndManifest(builder, message.asInstanceOf[Serializable.ScalaJSON])
setMessageAndManifest(builder, message.asInstanceOf[Serializable.ScalaJSON[_ <: Any]])
} else if (message.isInstanceOf[Serializable.SBinary[_]]) {
builder.setSerializationScheme(SerializationSchemeType.SBINARY)
setMessageAndManifest(builder, message.asInstanceOf[Serializable.SBinary[_ <: Any]])

View file

@ -7,12 +7,12 @@ package se.scalablesolutions.akka.remote
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _}
import se.scalablesolutions.akka.actor.{Exit, Actor, ActorRef, ActorType, RemoteActorRef, IllegalActorStateException}
import se.scalablesolutions.akka.dispatch.{DefaultCompletableFuture, CompletableFuture}
import se.scalablesolutions.akka.util.{ListenerManagement, UUID, Logging, Duration}
import se.scalablesolutions.akka.util.{ListenerManagement, Logging, Duration}
import se.scalablesolutions.akka.actor.{Uuid,newUuid,uuidFrom}
import se.scalablesolutions.akka.config.Config._
import se.scalablesolutions.akka.serialization.RemoteActorSerialization._
import se.scalablesolutions.akka.AkkaException
import Actor._
import org.jboss.netty.channel._
import group.DefaultChannelGroup
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
@ -32,18 +32,6 @@ import scala.collection.mutable.{HashSet, HashMap}
import scala.reflect.BeanProperty
import se.scalablesolutions.akka.actor._
/**
* Atomic remote request/reply message id generator.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object RemoteRequestProtocolIdFactory {
private val nodeId = UUID.newUuid
private val id = new AtomicLong
def nextId: Long = id.getAndIncrement + nodeId
}
/**
* Life-cycle events for RemoteClient.
*/
@ -75,7 +63,7 @@ object RemoteClient extends Logging {
val RECONNECT_DELAY = Duration(config.getInt("akka.remote.client.reconnect-delay", 5), TIME_UNIT)
private val remoteClients = new HashMap[String, RemoteClient]
private val remoteActors = new HashMap[RemoteServer.Address, HashSet[String]]
private val remoteActors = new HashMap[RemoteServer.Address, HashSet[Uuid]]
def actorFor(classNameOrServiceId: String, hostname: String, port: Int): ActorRef =
actorFor(classNameOrServiceId, classNameOrServiceId, 5000L, hostname, port, None)
@ -174,21 +162,21 @@ object RemoteClient extends Logging {
remoteClients.clear
}
def register(hostname: String, port: Int, uuid: String) = synchronized {
def register(hostname: String, port: Int, uuid: Uuid) = synchronized {
actorsFor(RemoteServer.Address(hostname, port)) += uuid
}
private[akka] def unregister(hostname: String, port: Int, uuid: String) = synchronized {
private[akka] def unregister(hostname: String, port: Int, uuid: Uuid) = synchronized {
val set = actorsFor(RemoteServer.Address(hostname, port))
set -= uuid
if (set.isEmpty) shutdownClientFor(new InetSocketAddress(hostname, port))
}
private[akka] def actorsFor(remoteServerAddress: RemoteServer.Address): HashSet[String] = {
private[akka] def actorsFor(remoteServerAddress: RemoteServer.Address): HashSet[Uuid] = {
val set = remoteActors.get(remoteServerAddress)
if (set.isDefined && (set.get ne null)) set.get
else {
val remoteActorSet = new HashSet[String]
val remoteActorSet = new HashSet[Uuid]
remoteActors.put(remoteServerAddress, remoteActorSet)
remoteActorSet
}
@ -206,8 +194,8 @@ class RemoteClient private[akka] (
val name = "RemoteClient@" + hostname + "::" + port
//FIXME Should these be clear:ed on postStop?
private val futures = new ConcurrentHashMap[Long, CompletableFuture[_]]
private val supervisors = new ConcurrentHashMap[String, ActorRef]
private val futures = new ConcurrentHashMap[Uuid, CompletableFuture[_]]
private val supervisors = new ConcurrentHashMap[Uuid, ActorRef]
private val remoteAddress = new InetSocketAddress(hostname, port)
@ -299,7 +287,7 @@ class RemoteClient private[akka] (
futures.synchronized {
val futureResult = if (senderFuture.isDefined) senderFuture.get
else new DefaultCompletableFuture[T](request.getActorInfo.getTimeout)
futures.put(request.getId, futureResult)
futures.put(uuidFrom(request.getUuid.getHigh,request.getUuid.getLow), futureResult)
connection.getChannel.write(request)
Some(futureResult)
}
@ -342,8 +330,8 @@ class RemoteClient private[akka] (
*/
class RemoteClientPipelineFactory(
name: String,
futures: ConcurrentMap[Long, CompletableFuture[_]],
supervisors: ConcurrentMap[String, ActorRef],
futures: ConcurrentMap[Uuid, CompletableFuture[_]],
supervisors: ConcurrentMap[Uuid, ActorRef],
bootstrap: ClientBootstrap,
remoteAddress: SocketAddress,
timer: HashedWheelTimer,
@ -382,8 +370,8 @@ class RemoteClientPipelineFactory(
@ChannelHandler.Sharable
class RemoteClientHandler(
val name: String,
val futures: ConcurrentMap[Long, CompletableFuture[_]],
val supervisors: ConcurrentMap[String, ActorRef],
val futures: ConcurrentMap[Uuid, CompletableFuture[_]],
val supervisors: ConcurrentMap[Uuid, ActorRef],
val bootstrap: ClientBootstrap,
val remoteAddress: SocketAddress,
val timer: HashedWheelTimer,
@ -403,14 +391,15 @@ class RemoteClientHandler(
val result = event.getMessage
if (result.isInstanceOf[RemoteReplyProtocol]) {
val reply = result.asInstanceOf[RemoteReplyProtocol]
val replyUuid = uuidFrom(reply.getUuid.getHigh,reply.getUuid.getLow)
log.debug("Remote client received RemoteReplyProtocol[\n%s]", reply.toString)
val future = futures.get(reply.getId).asInstanceOf[CompletableFuture[Any]]
val future = futures.get(replyUuid).asInstanceOf[CompletableFuture[Any]]
if (reply.getIsSuccessful) {
val message = MessageSerializer.deserialize(reply.getMessage)
future.completeWithResult(message)
} else {
if (reply.hasSupervisorUuid()) {
val supervisorUuid = reply.getSupervisorUuid
val supervisorUuid = uuidFrom(reply.getSupervisorUuid.getHigh,reply.getSupervisorUuid.getLow)
if (!supervisors.containsKey(supervisorUuid)) throw new IllegalActorStateException(
"Expected a registered supervisor for UUID [" + supervisorUuid + "] but none was found")
val supervisedActor = supervisors.get(supervisorUuid)
@ -420,7 +409,7 @@ class RemoteClientHandler(
}
future.completeWithException(parseException(reply, client.loader))
}
futures.remove(reply.getId)
futures remove replyUuid
} else {
val exception = new RemoteClientException("Unknown message received in remote client handler: " + result, client)
client.notifyListeners(RemoteClientError(exception, client))

View file

@ -10,7 +10,7 @@ import java.util.concurrent.{ConcurrentHashMap, Executors}
import java.util.{Map => JMap}
import se.scalablesolutions.akka.actor.{
Actor, TypedActor, ActorRef, IllegalActorStateException, RemoteActorSystemMessage}
Actor, TypedActor, ActorRef, IllegalActorStateException, RemoteActorSystemMessage,uuidFrom,Uuid}
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.util._
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
@ -314,7 +314,7 @@ class RemoteServer extends Logging with ListenerManagement {
}
}
private def register(id: String, actorRef: ActorRef, registry: ConcurrentHashMap[String, ActorRef]) {
private def register[Key](id: Key, actorRef: ActorRef, registry: ConcurrentHashMap[Key, ActorRef]) {
if (_isRunning) {
if (!registry.contains(id)) {
if (!actorRef.isRunning) actorRef.start
@ -323,7 +323,7 @@ class RemoteServer extends Logging with ListenerManagement {
}
}
private def registerTypedActor(id: String, typedActor: AnyRef, registry: ConcurrentHashMap[String, AnyRef]) {
private def registerTypedActor[Key](id: Key, typedActor: AnyRef, registry: ConcurrentHashMap[Key, AnyRef]) {
if (_isRunning) {
if (!registry.contains(id)) {
registry.put(id, typedActor)
@ -337,9 +337,8 @@ class RemoteServer extends Logging with ListenerManagement {
def unregister(actorRef: ActorRef):Unit = synchronized {
if (_isRunning) {
log.debug("Unregistering server side remote actor [%s] with id [%s:%s]", actorRef.actorClass.getName, actorRef.id, actorRef.uuid)
val actorMap = actors()
actorMap remove actorRef.id
if (actorRef.registeredInRemoteNodeDuringSerialization) actorsByUuid() remove actorRef.uuid
actors().remove(actorRef.id,actorRef)
actorsByUuid().remove(actorRef.uuid,actorRef)
}
}
@ -354,11 +353,9 @@ class RemoteServer extends Logging with ListenerManagement {
if (id.startsWith(UUID_PREFIX)) {
actorsByUuid().remove(id.substring(UUID_PREFIX.length))
} else {
val actorRef = actors().get(id)
if (actorRef.registeredInRemoteNodeDuringSerialization) {
actorsByUuid() remove actorRef.uuid
}
actors() remove id
val actorRef = actors() get id
actorsByUuid().remove(actorRef.uuid,actorRef)
actors().remove(id,actorRef)
}
}
}
@ -536,10 +533,10 @@ class RemoteServerHandler(
override def onComplete(result: AnyRef) {
log.debug("Returning result from actor invocation [%s]", result)
val replyBuilder = RemoteReplyProtocol.newBuilder
.setId(request.getId)
.setMessage(MessageSerializer.serialize(result))
.setIsSuccessful(true)
.setIsActor(true)
.setUuid(request.getUuid)
.setMessage(MessageSerializer.serialize(result))
.setIsSuccessful(true)
.setIsActor(true)
if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
@ -580,7 +577,7 @@ class RemoteServerHandler(
val result = messageReceiver.invoke(typedActor, args: _*)
log.debug("Returning result from remote typed actor invocation [%s]", result)
val replyBuilder = RemoteReplyProtocol.newBuilder
.setId(request.getId)
.setUuid(request.getUuid)
.setMessage(MessageSerializer.serialize(result))
.setIsSuccessful(true)
.setIsActor(false)
@ -640,7 +637,7 @@ class RemoteServerHandler(
val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name)
else Class.forName(name)
val actorRef = Actor.actorOf(clazz.newInstance.asInstanceOf[Actor])
actorRef.uuid = uuid
actorRef.uuid = uuidFrom(uuid.getHigh,uuid.getLow)
actorRef.id = id
actorRef.timeout = timeout
actorRef.remoteAddress = None
@ -695,7 +692,7 @@ class RemoteServerHandler(
val actorInfo = request.getActorInfo
log.error(e, "Could not invoke remote typed actor [%s :: %s]", actorInfo.getTypedActorInfo.getMethod, actorInfo.getTarget)
val replyBuilder = RemoteReplyProtocol.newBuilder
.setId(request.getId)
.setUuid(request.getUuid)
.setException(ExceptionProtocol.newBuilder.setClassname(e.getClass.getName).setMessage(e.getMessage).build)
.setIsSuccessful(false)
.setIsActor(isActor)

View file

@ -91,10 +91,32 @@ object Serializable {
}
/**
* case class Address(street: String, city: String, zip: String)
* extends ScalaJSON[Address] {
*
* implicit val AddressFormat: Format[Address] =
* asProduct3("street", "city", "zip")(Address)(Address.unapply(_).get)
*
* import dispatch.json._
* import sjson.json._
* import sjson.json.JsonSerialization._
*
* def toJSON: String = JsValue.toJson(tojson(this))
* def toBytes: Array[Byte] = tobinary(this)
* def fromBytes(bytes: Array[Byte]): Address = frombinary[Address](bytes)
* def fromJSON(js: String): Address = fromjson[Address](Js(js))
* }
*
* val a = Address(...)
* val js = tojson(a)
* val add = fromjson[Address](js)
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait ScalaJSON extends JSON {
def toJSON: String = new String(toBytes, "UTF-8")
def toBytes: Array[Byte] = SJSONSerializer.SJSON.out(this)
trait ScalaJSON[T] extends JSON {
def toJSON: String
def fromJSON(js: String): T
def toBytes: Array[Byte]
def fromBytes(bytes: Array[Byte]): T
}
}

View file

@ -8,12 +8,12 @@ import se.scalablesolutions.akka.stm.global._
import se.scalablesolutions.akka.stm.TransactionManagement._
import se.scalablesolutions.akka.stm.TransactionManagement
import se.scalablesolutions.akka.dispatch.MessageInvocation
import se.scalablesolutions.akka.remote.{RemoteServer, RemoteRequestProtocolIdFactory, MessageSerializer}
import se.scalablesolutions.akka.remote.{RemoteServer, MessageSerializer}
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _}
import ActorTypeProtocol._
import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy}
import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.actor.{uuidFrom,newUuid}
import com.google.protobuf.ByteString
import se.scalablesolutions.akka.actor._
@ -110,12 +110,13 @@ object ActorSerialization {
.build
val builder = SerializedActorRefProtocol.newBuilder
.setUuid(actorRef.uuid)
.setId(actorRef.id)
.setActorClassname(actorRef.actorClass.getName)
.setOriginalAddress(originalAddress)
.setIsTransactor(actorRef.isTransactor)
.setTimeout(actorRef.timeout)
.setUuid(UuidProtocol.newBuilder.setHigh(actorRef.uuid.getTime).setLow(actorRef.uuid.getClockSeqAndNode).build)
.setId(actorRef.id)
.setActorClassname(actorRef.actorClass.getName)
.setOriginalAddress(originalAddress)
.setIsTransactor(actorRef.isTransactor)
.setTimeout(actorRef.timeout)
if (serializeMailBox == true) {
val messages =
@ -191,7 +192,7 @@ object ActorSerialization {
}
val ar = new LocalActorRef(
protocol.getUuid,
uuidFrom(protocol.getUuid.getHigh,protocol.getUuid.getLow),
protocol.getId,
protocol.getOriginalAddress.getHostname,
protocol.getOriginalAddress.getPort,
@ -232,7 +233,7 @@ object RemoteActorSerialization {
private[akka] def fromProtobufToRemoteActorRef(protocol: RemoteActorRefProtocol, loader: Option[ClassLoader]): ActorRef = {
Actor.log.debug("Deserializing RemoteActorRefProtocol to RemoteActorRef:\n" + protocol)
RemoteActorRef(
protocol.getUuid,
protocol.getClassOrServiceName,
protocol.getActorClassname,
protocol.getHomeAddress.getHostname,
protocol.getHomeAddress.getPort,
@ -248,19 +249,16 @@ object RemoteActorSerialization {
val host = homeAddress.getHostName
val port = homeAddress.getPort
if (!registeredInRemoteNodeDuringSerialization) {
Actor.log.debug("Register serialized Actor [%s] as remote @ [%s:%s]", actorClass.getName, host, port)
RemoteServer.getOrCreateServer(homeAddress)
RemoteServer.registerActorByUuid(homeAddress, uuid, ar)
registeredInRemoteNodeDuringSerialization = true
}
Actor.log.debug("Register serialized Actor [%s] as remote @ [%s:%s]", actorClass.getName, host, port)
RemoteServer.getOrCreateServer(homeAddress)
RemoteServer.registerActorByUuid(homeAddress, uuid.toString, ar)
RemoteActorRefProtocol.newBuilder
.setUuid(uuid)
.setActorClassname(actorClass.getName)
.setHomeAddress(AddressProtocol.newBuilder.setHostname(host).setPort(port).build)
.setTimeout(timeout)
.build
.setClassOrServiceName(uuid.toString)
.setActorClassname(actorClass.getName)
.setHomeAddress(AddressProtocol.newBuilder.setHostname(host).setPort(port).build)
.setTimeout(timeout)
.build
}
def createRemoteRequestProtocolBuilder(
@ -274,7 +272,7 @@ object RemoteActorSerialization {
import actorRef._
val actorInfoBuilder = ActorInfoProtocol.newBuilder
.setUuid(uuid)
.setUuid(UuidProtocol.newBuilder.setHigh(uuid.getTime).setLow(uuid.getClockSeqAndNode).build)
.setId(actorRef.id)
.setTarget(actorClassName)
.setTimeout(timeout)
@ -295,18 +293,18 @@ object RemoteActorSerialization {
val actorInfo = actorInfoBuilder.build
val requestBuilder = RemoteRequestProtocol.newBuilder
.setId(RemoteRequestProtocolIdFactory.nextId)
.setUuid(UuidProtocol.newBuilder.setHigh(uuid.getTime).setLow(uuid.getClockSeqAndNode).build)
.setMessage(MessageSerializer.serialize(message))
.setActorInfo(actorInfo)
.setIsOneWay(isOneWay)
val id = registerSupervisorAsRemoteActor
if (id.isDefined) requestBuilder.setSupervisorUuid(id.get)
if (id.isDefined) requestBuilder.setSupervisorUuid(UuidProtocol.newBuilder.setHigh(id.get.getTime).setLow(id.get.getClockSeqAndNode).build)
senderOption.foreach { sender =>
RemoteServer.getOrCreateServer(sender.homeAddress).register(sender.uuid.toString, sender)
requestBuilder.setSender(toRemoteActorRefProtocol(sender))
senderOption.foreach {
sender =>
RemoteServer.getOrCreateServer(sender.homeAddress).register(sender.uuid, sender)
requestBuilder.setSender(toRemoteActorRefProtocol(sender))
}
requestBuilder
}

View file

@ -128,11 +128,23 @@ object Serializer {
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object ScalaJSON extends ScalaJSON
trait ScalaJSON extends Serializer {
trait ScalaJSON {
import sjson.json._
var classLoader: Option[ClassLoader] = None
def tojson[T](o: T)(implicit tjs: Writes[T]): JsValue = JsonSerialization.tojson(o)(tjs)
def fromjson[T](json: JsValue)(implicit fjs: Reads[T]): T = JsonSerialization.fromjson(json)(fjs)
def tobinary[T](o: T)(implicit tjs: Writes[T]): Array[Byte] = JsonSerialization.tobinary(o)(tjs)
def frombinary[T](bytes: Array[Byte])(implicit fjs: Reads[T]): T = JsonSerialization.frombinary(bytes)(fjs)
// backward compatibility
// implemented using refelction based json serialization
def toBinary(obj: AnyRef): Array[Byte] = SJSONSerializer.SJSON.out(obj)
// FIXME set ClassLoader on SJSONSerializer.SJSON
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = SJSONSerializer.SJSON.in(bytes)
import scala.reflect.Manifest
@ -144,6 +156,7 @@ object Serializer {
SJSONSerializer.SJSON.in(bytes)(m)
}
}
object ScalaJSON extends ScalaJSON
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>

View file

@ -0,0 +1,9 @@
package se.scalablesolutions.akka
package object serialization {
type JsValue = _root_.dispatch.json.JsValue
val JsValue = _root_.dispatch.json.JsValue
val Js = _root_.dispatch.json.Js
val JsonSerialization = sjson.json.JsonSerialization
val DefaultProtocol = sjson.json.DefaultProtocol
}

View file

@ -79,6 +79,8 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
@After
def finished() {
s1.shutdown
val s2 = RemoteServer.serverFor(HOSTNAME, PORT2)
if (s2.isDefined) s2.get.shutdown
RemoteClient.shutdownAll
Thread.sleep(1000)
}

View file

@ -72,6 +72,8 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite {
def finished {
try {
server.shutdown
val s2 = RemoteServer.serverFor(HOSTNAME, PORT + 1)
if (s2.isDefined) s2.get.shutdown
RemoteClient.shutdownAll
Thread.sleep(1000)
} catch {
@ -208,7 +210,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite {
def shouldRegisterAndUnregisterByUuid {
val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
server.register("uuid:" + actor1.uuid, actor1)
assert(server.actorsByUuid().get(actor1.uuid) != null, "actor registered")
assert(server.actorsByUuid().get(actor1.uuid.toString) != null, "actor registered")
server.unregister("uuid:" + actor1.uuid)
assert(server.actorsByUuid().get(actor1.uuid) == null, "actor unregistered")
}

View file

@ -103,9 +103,9 @@ class ServerInitiatedRemoteTypedActorSpec extends
it("should register and unregister typed actors") {
val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000)
server.registerTypedActor("my-test-service", typedActor)
assert(server.typedActors().get("my-test-service") != null, "typed actor registered")
assert(server.typedActors().get("my-test-service") ne null, "typed actor registered")
server.unregisterTypedActor("my-test-service")
assert(server.typedActors().get("my-test-service") == null, "typed actor unregistered")
assert(server.typedActors().get("my-test-service") eq null, "typed actor unregistered")
}
it("should register and unregister typed actors by uuid") {
@ -113,9 +113,9 @@ class ServerInitiatedRemoteTypedActorSpec extends
val init = AspectInitRegistry.initFor(typedActor)
val uuid = "uuid:" + init.actorRef.uuid
server.registerTypedActor(uuid, typedActor)
assert(server.typedActorsByUuid().get(init.actorRef.uuid) != null, "typed actor registered")
assert(server.typedActorsByUuid().get(init.actorRef.uuid.toString) ne null, "typed actor registered")
server.unregisterTypedActor(uuid)
assert(server.typedActorsByUuid().get(init.actorRef.uuid) == null, "typed actor unregistered")
assert(server.typedActorsByUuid().get(init.actorRef.uuid.toString) eq null, "typed actor unregistered")
}
it("should find typed actors by uuid") {
@ -123,7 +123,7 @@ class ServerInitiatedRemoteTypedActorSpec extends
val init = AspectInitRegistry.initFor(typedActor)
val uuid = "uuid:" + init.actorRef.uuid
server.registerTypedActor(uuid, typedActor)
assert(server.typedActorsByUuid().get(init.actorRef.uuid) != null, "typed actor registered")
assert(server.typedActorsByUuid().get(init.actorRef.uuid.toString) ne null, "typed actor registered")
val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], uuid, HOSTNAME, PORT)
expect("oneway") {

View file

@ -0,0 +1,63 @@
package se.scalablesolutions.akka.serialization
import org.scalatest.Spec
import org.scalatest.matchers.ShouldMatchers
import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import se.scalablesolutions.akka.serialization.Serializable.ScalaJSON
object Serializables {
import DefaultProtocol._
import JsonSerialization._
case class Shop(store: String, item: String, price: Int) extends
ScalaJSON[Shop] {
implicit val ShopFormat: sjson.json.Format[Shop] =
asProduct3("store", "item", "price")(Shop)(Shop.unapply(_).get)
def toJSON: String = JsValue.toJson(tojson(this))
def toBytes: Array[Byte] = tobinary(this)
def fromBytes(bytes: Array[Byte]) = frombinary[Shop](bytes)
def fromJSON(js: String) = fromjson[Shop](Js(js))
}
case class MyMessage(val id: String, val value: Tuple2[String, Int])
implicit val MyMessageFormat: sjson.json.Format[MyMessage] =
asProduct2("id", "value")(MyMessage)(MyMessage.unapply(_).get)
case class MyJsonObject(val key: String, val map: Map[String, Int],
val standAloneInt: Int) extends ScalaJSON[MyJsonObject] {
implicit val MyJsonObjectFormat: sjson.json.Format[MyJsonObject] =
asProduct3("key", "map", "standAloneInt")(MyJsonObject)(MyJsonObject.unapply(_).get)
def toJSON: String = JsValue.toJson(tojson(this))
def toBytes: Array[Byte] = tobinary(this)
def fromBytes(bytes: Array[Byte]) = frombinary[MyJsonObject](bytes)
def fromJSON(js: String) = fromjson[MyJsonObject](Js(js))
}
}
@RunWith(classOf[JUnitRunner])
class ScalaJSONSerializableSpec extends
Spec with
ShouldMatchers with
BeforeAndAfterAll {
import Serializables._
describe("Serialization of case classes") {
it("should be able to serialize and de-serialize") {
val s = Shop("Target", "cooker", 120)
s.fromBytes(s.toBytes) should equal(s)
s.fromJSON(s.toJSON) should equal(s)
val key: String = "myKey"
val value: Int = 123
val standAloneInt: Int = 35
val message = MyJsonObject(key, Map(key -> value), standAloneInt)
message.fromBytes(message.toBytes) should equal(message)
message.fromJSON(message.toJSON) should equal(message)
}
}
}

View file

@ -0,0 +1,52 @@
package se.scalablesolutions.akka.serialization
import org.scalatest.Spec
import org.scalatest.matchers.ShouldMatchers
import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import se.scalablesolutions.akka.serialization.Serializer.ScalaJSON
object Protocols {
import sjson.json.DefaultProtocol._
case class Shop(store: String, item: String, price: Int)
implicit val ShopFormat: sjson.json.Format[Shop] =
asProduct3("store", "item", "price")(Shop)(Shop.unapply(_).get)
case class MyMessage(val id: String, val value: Tuple2[String, Int])
implicit val MyMessageFormat: sjson.json.Format[MyMessage] =
asProduct2("id", "value")(MyMessage)(MyMessage.unapply(_).get)
case class MyJsonObject(val key: String, val map: Map[String, Int],
val standAloneInt: Int)
implicit val MyJsonObjectFormat: sjson.json.Format[MyJsonObject] =
asProduct3("key", "map", "standAloneInt")(MyJsonObject)(MyJsonObject.unapply(_).get)
}
@RunWith(classOf[JUnitRunner])
class ScalaJSONSerializerSpec extends
Spec with
ShouldMatchers with
BeforeAndAfterAll {
import Protocols._
import ScalaJSON._
describe("Serialization of case classes") {
it("should be able to serialize and de-serialize") {
val s = Shop("Target", "cooker", 120)
fromjson[Shop](tojson(s)) should equal(s)
frombinary[Shop](tobinary(s)) should equal(s)
val o = MyMessage("dg", ("akka", 100))
fromjson[MyMessage](tojson(o)) should equal(o)
frombinary[MyMessage](tobinary(o)) should equal(o)
val key: String = "myKey"
val value: Int = 123
val standAloneInt: Int = 35
val message = MyJsonObject(key, Map(key -> value), standAloneInt)
fromjson[MyJsonObject](tojson(message)) should equal(message)
}
}
}

View file

@ -8,6 +8,7 @@ import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import se.scalablesolutions.akka.serialization._
// import dispatch.json._
import se.scalablesolutions.akka.actor._
import ActorSerialization._
import Actor._
@ -52,6 +53,10 @@ class SerializableTypeClassActorSpec extends
implicit object MyStatelessActorFormat extends StatelessActorFormat[MyStatelessActorWithMessagesInMailbox]
}
object BinaryFormatMyActorWithSerializableMessages {
implicit object MyActorWithSerializableMessagesFormat extends StatelessActorFormat[MyActorWithSerializableMessages]
}
object BinaryFormatMyJavaSerializableActor {
implicit object MyJavaSerializableActorFormat extends SerializerBasedActorFormat[MyJavaSerializableActor] {
val serializer = Serializer.Java
@ -139,6 +144,29 @@ class SerializableTypeClassActorSpec extends
(actor3 !! "hello-reply").getOrElse("_") should equal("world")
}
}
describe("Custom serializable actors") {
it("should serialize and de-serialize") {
import BinaryFormatMyActorWithSerializableMessages._
val actor1 = actorOf[MyActorWithSerializableMessages].start
(actor1 ! MyMessage("hello1", ("akka", 100)))
(actor1 ! MyMessage("hello2", ("akka", 200)))
(actor1 ! MyMessage("hello3", ("akka", 300)))
(actor1 ! MyMessage("hello4", ("akka", 400)))
(actor1 ! MyMessage("hello5", ("akka", 500)))
actor1.mailboxSize should be > (0)
val actor2 = fromBinary(toBinary(actor1))
Thread.sleep(1000)
actor2.mailboxSize should be > (0)
(actor2 !! "hello-reply").getOrElse("_") should equal("world")
val actor3 = fromBinary(toBinary(actor1, false))
Thread.sleep(1000)
actor3.mailboxSize should equal(0)
(actor3 !! "hello-reply").getOrElse("_") should equal("world")
}
}
}
class MyActorWithDualCounter extends Actor {
@ -188,3 +216,29 @@ class MyStatelessActorWithMessagesInMailbox extends Actor {
self.reply("world " + count)
}
}
class MyActorWithSerializableMessages extends Actor {
def receive = {
case MyMessage(s, t) =>
println("# messages in mailbox " + self.mailboxSize)
Thread.sleep(500)
case "hello-reply" => self.reply("world")
}
}
case class MyMessage(val id: String, val value: Tuple2[String, Int])
extends Serializable.ScalaJSON[MyMessage] {
def this() = this(null, null)
import DefaultProtocol._
import JsonSerialization._
implicit val MyMessageFormat: sjson.json.Format[MyMessage] =
asProduct2("id", "value")(MyMessage)(MyMessage.unapply(_).get)
def toJSON: String = JsValue.toJson(tojson(this))
def toBytes: Array[Byte] = tobinary(this)
def fromBytes(bytes: Array[Byte]) = frombinary[MyMessage](bytes)
def fromJSON(js: String) = fromjson[MyMessage](Js(js))
}

View file

@ -1,49 +0,0 @@
package se.scalablesolutions.akka.actor.serialization
import org.scalatest.Spec
import org.scalatest.matchers.ShouldMatchers
import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import se.scalablesolutions.akka.serialization.Serializer
import se.scalablesolutions.akka.serialization.Serializable.ScalaJSON
import scala.reflect._
import scala.annotation.target._
import sjson.json.JSONTypeHint
@BeanInfo class MyJsonObject(val key: String,
@(JSONTypeHint @field)(value = classOf[Int])
val map: Map[String, Int],
val standAloneInt: Int) extends ScalaJSON {
private def this() = this(null, null, -1)
override def toString(): String = try {
val mapValue: Int = map.getOrElse(key, -1)
println("Map value: %s".format(mapValue.asInstanceOf[AnyRef].getClass))
"Key: %s, Map value: %d, Stand Alone Int: %d".format(key, mapValue, standAloneInt)
} catch {
case e: ClassCastException => e.getMessage
case _ => "Unknown error"
}
}
@RunWith(classOf[JUnitRunner])
class Ticket436Spec extends
Spec with
ShouldMatchers with
BeforeAndAfterAll {
describe("Serialization of Maps containing Int") {
it("should be able to serialize and de-serialize preserving the data types of the Map") {
val key: String = "myKey"
val value: Int = 123
val standAloneInt: Int = 35
val message = new MyJsonObject(key, Map(key -> value), standAloneInt)
val json = message.toJSON
val copy = Serializer.ScalaJSON.fromJSON[MyJsonObject](json)
copy.asInstanceOf[MyJsonObject].map.get("myKey").get.isInstanceOf[Int] should equal(true)
}
}
}

View file

@ -6,6 +6,7 @@ package se.scalablesolutions.akka.actor.ticket
import org.scalatest.Spec
import org.scalatest.matchers.ShouldMatchers
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.actor.{Uuid,newUuid,uuidFrom}
import se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec.RemoteActorSpecActorUnidirectional
import java.util.concurrent.TimeUnit
import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer}
@ -14,12 +15,15 @@ import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
class Ticket434Spec extends Spec with ShouldMatchers {
val HOSTNAME = "localhost"
val PORT = 9991
describe("A server managed remote actor") {
it("should possible be use a custom service name containing ':'") {
val server = new RemoteServer().start("localhost", 9999)
it("can use a custom service name containing ':'") {
val server = new RemoteServer().start(HOSTNAME, PORT)
server.register("my:service", actorOf[RemoteActorSpecActorUnidirectional])
val actor = RemoteClient.actorFor("my:service", 5000L, "localhost", 9999)
val actor = RemoteClient.actorFor("my:service", 5000L, HOSTNAME, PORT)
actor ! "OneWay"
assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS))
@ -32,14 +36,15 @@ class Ticket434Spec extends Spec with ShouldMatchers {
describe("The ActorInfoProtocol") {
it("should be possible to set the acor id and uuuid") {
val uuid = newUuid
val actorInfoBuilder = ActorInfoProtocol.newBuilder
.setUuid("unique-id")
.setUuid(UuidProtocol.newBuilder.setHigh(uuid.getTime).setLow(uuid.getClockSeqAndNode).build)
.setId("some-id")
.setTarget("actorClassName")
.setTimeout(5000L)
.setActorType(ActorType.SCALA_ACTOR)
val actorInfo = actorInfoBuilder.build
assert(actorInfo.getUuid === "unique-id")
assert(uuidFrom(actorInfo.getUuid.getHigh,actorInfo.getUuid.getLow) === uuid)
assert(actorInfo.getId === "some-id")
}
}

View file

@ -17,9 +17,8 @@ import se.scalablesolutions.akka.routing.Routing._
/**
* @author Martin Krasser
*/
@Ignore // do not run concurrency test by default
class HttpConcurrencyTest extends JUnitSuite {
import HttpConcurrencyTest._
class HttpConcurrencyTestStress extends JUnitSuite {
import HttpConcurrencyTestStress._
@Test def shouldProcessMessagesConcurrently = {
val num = 50
@ -43,7 +42,7 @@ class HttpConcurrencyTest extends JUnitSuite {
}
}
object HttpConcurrencyTest {
object HttpConcurrencyTestStress {
@BeforeClass
def beforeClass = {
startCamelService

View file

@ -633,7 +633,7 @@ private[akka] abstract class ActorAspect {
protected var typedActor: TypedActor = _
protected var actorRef: ActorRef = _
protected var timeout: Long = _
protected var uuid: String = _
protected var uuid: Uuid = _
protected var remoteAddress: Option[InetSocketAddress] = _
protected def localDispatch(joinPoint: JoinPoint): AnyRef = {

View file

@ -167,7 +167,21 @@ akka {
}
hbase {
zookeeper-quorum = "localhost"
zookeeper-quorum = "localhost" # A comma separated list of the hostnames or IP belonging to the zookeeper quorum
}
voldemort {
store {
ref = "Refs" # Voldemort Store Used to Persist Refs. Use string serializer for keys, identity serializer for values
map-keys = "MapKeys" # Voldemort Store Used to Persist Map Keys. Use string serializer for keys, identity serializer for values
map-values = "MapValues" # Voldemort Store Used to Persist Map Values. Use identity serializer for keys, identity serializer for values
vector = "Vectors" # Voldemort Store Used to Persist Vector Sizes. Use string serializer for keys, identity serializer for values
queue = "Queues" # Voldemort Store Used to Persist Vector Values. Use identity serializer for keys, identity serializer for values
}
client { # The KeyValue pairs under client are converted to java Properties and used to construct the ClientConfig
bootstrap_urls = "tcp://localhost:6666" # All Valid Voldemort Client properties are valid here, in string form
}
}
}
}

View file

@ -16,6 +16,10 @@
</encoder>
</appender>
<logger name="se.scalablesolutions" level="DEBUG"/>
<logger name="org.mortbay.log" level="ERROR"/>
<logger name="org.apache.jasper" level="ERROR"/>
<root level="DEBUG">
<appender-ref ref="stdout"/>
</root>

Binary file not shown.

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.eaio</groupId>
<artifactId>uuid</artifactId>
<version>3.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>org.apache.hbase</groupId>
<artifactId>hbase-core</artifactId>
<version>0.20.6</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>org.apache.hbase</groupId>
<artifactId>hbase-test</artifactId>
<version>0.20.6</version>
<packaging>jar</packaging>
</project>

View file

@ -53,6 +53,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val SunJDMKRepo = MavenRepository("Sun JDMK Repo", "http://wp5.e-taxonomy.eu/cdmlib/mavenrepo")
lazy val CasbahRepoReleases = MavenRepository("Casbah Release Repo", "http://repo.bumnetworks.com/releases")
lazy val ZookeeperRepo = MavenRepository("Zookeeper Repo", "http://lilycms.org/maven/maven2/deploy/")
lazy val ClojarsRepo = MavenRepository("Clojars Repo", "http://clojars.org/repo")
}
// -------------------------------------------------------------------------------------------------------------------
@ -83,6 +84,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val zookeeperRelease = ModuleConfiguration("org.apache.hadoop.zookeeper",ZookeeperRepo)
lazy val casbahModuleConfig = ModuleConfiguration("com.novus", CasbahRepo)
lazy val timeModuleConfig = ModuleConfiguration("org.scala-tools", "time", CasbahSnapshotRepo)
lazy val voldemortModuleConfig = ModuleConfiguration("voldemort", ClojarsRepo)
lazy val embeddedRepo = EmbeddedRepo // This is the only exception, because the embedded repo is fast!
// -------------------------------------------------------------------------------------------------------------------
@ -95,7 +97,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val DISPATCH_VERSION = "0.7.4"
lazy val HAWT_DISPATCH_VERSION = "1.0"
lazy val JACKSON_VERSION = "1.2.1"
lazy val JERSEY_VERSION = "1.2"
lazy val JERSEY_VERSION = "1.3"
lazy val MULTIVERSE_VERSION = "0.6.1"
lazy val SCALATEST_VERSION = "1.2-for-scala-2.8.0.final-SNAPSHOT"
lazy val LOGBACK_VERSION = "0.9.24"
@ -147,6 +149,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val jetty_xml = "org.eclipse.jetty" % "jetty-xml" % JETTY_VERSION % "compile"
lazy val jetty_servlet = "org.eclipse.jetty" % "jetty-servlet" % JETTY_VERSION % "compile"
lazy val uuid = "com.eaio" % "uuid" % "3.2" % "compile"
lazy val guicey = "org.guiceyfruit" % "guice-all" % "2.0" % "compile"
lazy val h2_lzf = "voldemort.store.compress" % "h2-lzf" % "1.0" % "compile"
@ -191,6 +195,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val sbinary = "sbinary" % "sbinary" % "2.8.0-0.3.1" % "compile"
lazy val sjson = "sjson.json" % "sjson" % "0.8-2.8.0" % "compile"
lazy val sjson_test = "sjson.json" % "sjson" % "0.8-2.8.0" % "test"
lazy val slf4j = "org.slf4j" % "slf4j-api" % SLF4J_VERSION % "compile"
@ -204,6 +209,10 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val thrift = "com.facebook" % "thrift" % "r917130" % "compile"
lazy val voldemort = "voldemort" % "voldemort" % "0.81" % "compile"
lazy val voldemort_contrib = "voldemort" % "voldemort-contrib" % "0.81" % "compile"
lazy val voldemort_needs_log4j = "org.slf4j" % "log4j-over-slf4j" % SLF4J_VERSION % "compile"
lazy val werkz = "org.codehaus.aspectwerkz" % "aspectwerkz-nodeps-jdk5" % ASPECTWERKZ_VERSION % "compile"
lazy val werkz_core = "org.codehaus.aspectwerkz" % "aspectwerkz-jdk5" % ASPECTWERKZ_VERSION % "compile"
@ -226,10 +235,18 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val junit = "junit" % "junit" % "4.5" % "test"
lazy val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test"
lazy val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "test"
//HBase testing
lazy val hadoop_test = "org.apache.hadoop" % "hadoop-test" % "0.20.2" % "test"
lazy val hbase_test = "org.apache.hbase" % "hbase-test" % "0.20.6" % "test"
lazy val log4j = "log4j" % "log4j" % "1.2.15" % "test"
lazy val jett_mortbay = "org.mortbay.jetty" % "jetty" % "6.1.14" % "test"
lazy val jetty_mortbay = "org.mortbay.jetty" % "jetty" % "6.1.14" % "test"
//voldemort testing
lazy val jdom = "org.jdom" % "jdom" % "1.1" % "test"
lazy val vold_jetty = "org.mortbay.jetty" % "jetty" % "6.1.18" % "test"
lazy val velocity = "org.apache.velocity" % "velocity" % "1.6.2" % "test"
lazy val dbcp = "commons-dbcp" % "commons-dbcp" % "1.2.2" % "test"
}
// -------------------------------------------------------------------------------------------------------------------
@ -354,6 +371,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
// -------------------------------------------------------------------------------------------------------------------
class AkkaActorProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
val uuid = Dependencies.uuid
val configgy = Dependencies.configgy
val hawtdispatch = Dependencies.hawtdispatch
val multiverse = Dependencies.multiverse
@ -460,6 +478,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
class AkkaCamelProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
val camel_core = Dependencies.camel_core
override def testOptions = createTestFilter( _.endsWith("Test"))
}
// -------------------------------------------------------------------------------------------------------------------
@ -477,6 +497,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
new AkkaCassandraProject(_), akka_persistence_common)
lazy val akka_persistence_hbase = project("akka-persistence-hbase", "akka-persistence-hbase",
new AkkaHbaseProject(_), akka_persistence_common)
lazy val akka_persistence_voldemort = project("akka-persistence-voldemort", "akka-persistence-voldemort",
new AkkaVoldemortProject(_), akka_persistence_common)
}
// -------------------------------------------------------------------------------------------------------------------
@ -496,7 +518,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
val commons_codec = Dependencies.commons_codec
val redis = Dependencies.redis
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
override def testOptions = createTestFilter( _.endsWith("Test"))
}
// -------------------------------------------------------------------------------------------------------------------
@ -507,7 +529,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
val mongo = Dependencies.mongo
val casbah = Dependencies.casbah
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
override def testOptions = createTestFilter( _.endsWith("Test"))
}
// -------------------------------------------------------------------------------------------------------------------
@ -523,7 +545,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
val google_coll = Dependencies.google_coll
val high_scale = Dependencies.high_scale
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
override def testOptions = createTestFilter( _.endsWith("Test"))
}
// -------------------------------------------------------------------------------------------------------------------
@ -531,17 +553,52 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
// -------------------------------------------------------------------------------------------------------------------
class AkkaHbaseProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
val zookeeper = Dependencies.zookeeper
val hadoop_core = Dependencies.hadoop_core
val hbase_core = Dependencies.hbase_core
// testing
val hadoop_test = Dependencies.hadoop_test
val hbase_test = Dependencies.hbase_test
val jetty = Dependencies.jett_mortbay
val log4j = Dependencies.log4j
override def ivyXML =
<dependencies>
<dependency org="org.apache.hadoop.zookeeper" name="zookeeper" rev="3.2.2" conf="compile">
</dependency>
<dependency org="org.apache.hadoop" name="hadoop-core" rev="0.20.2" conf="compile">
</dependency>
<dependency org="org.apache.hbase" name="hbase-core" rev="0.20.6" conf="compile">
</dependency>
<dependency org="org.apache.hadoop" name="hadoop-test" rev="0.20.2" conf="test">
<exclude module="slf4j-api"/>
</dependency>
<dependency org="org.slf4j" name="slf4j-api" rev={SLF4J_VERSION} conf="test">
</dependency>
<dependency org="org.apache.hbase" name="hbase-test" rev="0.20.6" conf="test">
</dependency>
<dependency org="log4j" name="log4j" rev="1.2.15" conf="test">
</dependency>
<dependency org="org.mortbay.jetty" name="jetty" rev="6.1.14" conf="test">
</dependency>
</dependencies>
override def testOptions = createTestFilter( _.endsWith("Test") )
}
// akka-persistence-voldemort subproject
// -------------------------------------------------------------------------------------------------------------------
class AkkaVoldemortProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
val voldemort = Dependencies.voldemort
val voldemort_contrib = Dependencies.voldemort_contrib
val voldemort_needs_log4j = Dependencies.voldemort_needs_log4j
//testing
val scalatest = Dependencies.scalatest
val google_coll = Dependencies.google_coll
val jdom = Dependencies.jdom
val jetty = Dependencies.vold_jetty
val velocity = Dependencies.velocity
val dbcp = Dependencies.dbcp
val sjson = Dependencies.sjson_test
override def testOptions = createTestFilter( _.endsWith("Suite"))
}
// -------------------------------------------------------------------------------------------------------------------
// akka-kernel subproject
// -------------------------------------------------------------------------------------------------------------------
@ -585,7 +642,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
new AkkaOSGiAssemblyProject(_), akka_osgi_dependencies_bundle, akka_remote, akka_amqp, akka_http,
akka_camel, akka_spring, akka_jta, akka_persistence.akka_persistence_common,
akka_persistence.akka_persistence_redis, akka_persistence.akka_persistence_mongo,
akka_persistence.akka_persistence_cassandra)
akka_persistence.akka_persistence_cassandra,akka_persistence.akka_persistence_hbase,
akka_persistence.akka_persistence_voldemort)
}
class AkkaOSGiDependenciesBundleProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) with BNDPlugin {
@ -706,6 +764,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
<dependency org="org.apache.activemq" name="activemq-core" rev="5.3.2">
</dependency>
</dependencies>
override def testOptions = createTestFilter( _.endsWith("Test"))
}
class AkkaSampleSecurityProject(info: ProjectInfo) extends AkkaDefaultProject(info, deployPath) {
@ -772,6 +832,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
}
def akkaArtifacts = descendents(info.projectPath / "dist", "*" + buildScalaVersion + "-" + version + ".jar")
lazy val integrationTestsEnabled = systemOptional[Boolean]("integration.tests",false)
lazy val stressTestsEnabled = systemOptional[Boolean]("stress.tests",false)
// ------------------------------------------------------------
class AkkaDefaultProject(info: ProjectInfo, val deployPath: Path) extends DefaultProject(info) with DeployProject with OSGiProject {
@ -782,6 +844,21 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
override def packageDocsJar = this.defaultJarPath("-docs.jar")
override def packageSrcJar = this.defaultJarPath("-sources.jar")
override def packageToPublishActions = super.packageToPublishActions ++ Seq(this.packageDocs, this.packageSrc)
/**
* Used for testOptions, possibility to enable the running of integration and or stresstests
*
* To enable set true and disable set false
* set integration.tests true
* set stress.tests true
*/
def createTestFilter(defaultTests: (String) => Boolean) = { TestFilter({
case s: String if defaultTests(s) => true
case s: String if integrationTestsEnabled.value => s.endsWith("TestIntegration")
case s: String if stressTestsEnabled.value => s.endsWith("TestStress")
case _ => false
}) :: Nil
}
}
}