diff --git a/akka-actor/src/main/java/se/scalablesolutions/akka/util/UUID.java b/akka-actor/src/main/java/se/scalablesolutions/akka/util/UUID.java deleted file mode 100644 index f06aab34df..0000000000 --- a/akka-actor/src/main/java/se/scalablesolutions/akka/util/UUID.java +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Copyright (C) 2009-2010 Scalable Solutions AB - */ -package se.scalablesolutions.akka.util; - -/** - * NOTE: - *

- * This code is based on code from the [Plasmid Replication Engine] project. - *
- * Licensed under [Mozilla Public License 1.0 (MPL)]. - *

- * Original JavaDoc: - *

- * 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. ### - *

- * 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. - *

- * 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; - } -} \ No newline at end of file diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala index 158cc2cc25..05e8b18eae 100644 --- a/akka-actor/src/main/scala/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/actor/ActorRef.scala @@ -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. diff --git a/akka-actor/src/main/scala/actor/ActorRegistry.scala b/akka-actor/src/main/scala/actor/ActorRegistry.scala index 51bbfd3477..e425451470 100644 --- a/akka-actor/src/main/scala/actor/ActorRegistry.scala +++ b/akka-actor/src/main/scala/actor/ActorRegistry.scala @@ -35,7 +35,7 @@ case class ActorUnregistered(actor: ActorRef) extends ActorRegistryEvent * @author Jonas Bonér */ 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. diff --git a/akka-actor/src/main/scala/actor/Implicits.scala b/akka-actor/src/main/scala/actor/Implicits.scala index 604e5189b9..9992cd36a1 100644 --- a/akka-actor/src/main/scala/actor/Implicits.scala +++ b/akka-actor/src/main/scala/actor/Implicits.scala @@ -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) + } } diff --git a/akka-actor/src/main/scala/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/dispatch/Dispatchers.scala index 0cca2ebe0b..55e819a2c8 100644 --- a/akka-actor/src/main/scala/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/dispatch/Dispatchers.scala @@ -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 => { diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala index 4843c70fbb..06516e52d4 100644 --- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala @@ -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 Jonas Bonér @@ -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 diff --git a/akka-actor/src/main/scala/stm/Ref.scala b/akka-actor/src/main/scala/stm/Ref.scala index b0ae15c957..fc81dbafa9 100644 --- a/akka-actor/src/main/scala/stm/Ref.scala +++ b/akka-actor/src/main/scala/stm/Ref.scala @@ -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) diff --git a/akka-actor/src/main/scala/stm/TransactionalMap.scala b/akka-actor/src/main/scala/stm/TransactionalMap.scala index d45396ad25..457774f755 100644 --- a/akka-actor/src/main/scala/stm/TransactionalMap.scala +++ b/akka-actor/src/main/scala/stm/TransactionalMap.scala @@ -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) diff --git a/akka-actor/src/main/scala/stm/TransactionalVector.scala b/akka-actor/src/main/scala/stm/TransactionalVector.scala index 2beeeecef0..3004e97db8 100644 --- a/akka-actor/src/main/scala/stm/TransactionalVector.scala +++ b/akka-actor/src/main/scala/stm/TransactionalVector.scala @@ -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) diff --git a/akka-actor/src/main/scala/util/AkkaException.scala b/akka-actor/src/main/scala/util/AkkaException.scala index 3e28c17390..0eb06549b4 100644 --- a/akka-actor/src/main/scala/util/AkkaException.scala +++ b/akka-actor/src/main/scala/util/AkkaException.scala @@ -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) diff --git a/akka-actor/src/main/scala/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/util/ReflectiveAccess.scala index 974250bc84..6a719d3834 100644 --- a/akka-actor/src/main/scala/util/ReflectiveAccess.scala +++ b/akka-actor/src/main/scala/util/ReflectiveAccess.scala @@ -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 } diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala index e2b2e0bdaa..9942c5812e 100644 --- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala +++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala @@ -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 } diff --git a/akka-camel/src/main/scala/CamelContextLifecycle.scala b/akka-camel/src/main/scala/CamelContextLifecycle.scala index 05c18396b8..32bb386dee 100644 --- a/akka-camel/src/main/scala/CamelContextLifecycle.scala +++ b/akka-camel/src/main/scala/CamelContextLifecycle.scala @@ -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 - * context 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 diff --git a/akka-camel/src/main/scala/CamelService.scala b/akka-camel/src/main/scala/CamelService.scala index 5fd8c9a66c..033fc2d01b 100644 --- a/akka-camel/src/main/scala/CamelService.scala +++ b/akka-camel/src/main/scala/CamelService.scala @@ -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 se.scalablesolutions.akka.camel.CamelContextManager.context. + * 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 @consume annotated methods of typed actors that have been created - * with TypedActor.newInstance(..) (and TypedActor.newInstance(..) + * with TypedActor.newInstance(..) (and TypedActor.newRemoteInstance(..) * 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) diff --git a/akka-camel/src/main/scala/Consumer.scala b/akka-camel/src/main/scala/Consumer.scala index ea07757a9c..db04c46abf 100644 --- a/akka-camel/src/main/scala/Consumer.scala +++ b/akka-camel/src/main/scala/Consumer.scala @@ -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. * - *

- * - * 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 Consumer trait. The * target Consumer object is passed as argument to f. This * method returns None if actorRef is not a valid reference - * to a consumer actor, Some result otherwise. + * to a consumer actor, Some consumer actor otherwise. */ def forConsumer[T](actorRef: ActorRef)(f: Consumer => T): Option[T] = { if (!actorRef.actor.isInstanceOf[Consumer]) None diff --git a/akka-camel/src/main/scala/ConsumerPublisher.scala b/akka-camel/src/main/scala/ConsumerPublisher.scala index c0b64021af..dfc1510ea6 100644 --- a/akka-camel/src/main/scala/ConsumerPublisher.scala +++ b/akka-camel/src/main/scala/ConsumerPublisher.scala @@ -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 PublishRequestorInit command message for obtaining a reference to - * a publisher actor. Before initialization it buffers all outbound messages - * and delivers them to the publisher when receiving a + * methods as well as un-publication of consumer actors and typed actor methods. This actor + * needs to be initialized with a PublishRequestorInit command message for + * obtaining a reference to a publisher actor. Before initialization it buffers + * all outbound messages and delivers them to the publisher when receiving a * PublishRequestorInit 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 @consume @@ -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 * actorRef 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 * actorRef 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 @consume 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 @consume annotated methods. */ def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodUnregistered] = { diff --git a/akka-camel/src/main/scala/Message.scala b/akka-camel/src/main/scala/Message.scala index a834568a22..d5cfd23faf 100644 --- a/akka-camel/src/main/scala/Message.scala +++ b/akka-camel/src/main/scala/Message.scala @@ -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 { diff --git a/akka-camel/src/main/scala/Producer.scala b/akka-camel/src/main/scala/Producer.scala index 0be07e9737..3313c1d0be 100644 --- a/akka-camel/src/main/scala/Producer.scala +++ b/akka-camel/src/main/scala/Producer.scala @@ -24,7 +24,7 @@ trait ProducerSupport { this: Actor => private val headersToCopyDefault = Set(Message.MessageExchangeId) /** - * Endpoint object resolved from current CamelContext with + * Endpoint object resolved from the current CamelContext with * endpointUri. */ 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 msg as exchange of given pattern to the endpoint specified by - * endpointUri. After producing to the endpoint the processing result is passed as argument - * to receiveAfterProduce. If the result was returned synchronously by the endpoint then - * receiveAfterProduce is called synchronously as well. If the result was returned asynchronously, - * the receiveAfterProduce 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 - * receiveAfterProduce. The original sender and senderFuture are thereby preserved. + * Initiates a message exchange of given pattern with the endpoint specified by + * endpointUri. The in-message of the initiated exchange is the canonical form + * of msg. After sending the in-message, the processing result (response) is passed + * as argument to receiveAfterProduce. If the response is received synchronously from + * the endpoint then receiveAfterProduce is called synchronously as well. If the + * response is received asynchronously, the receiveAfterProduce is called + * asynchronously. This is done by wrapping the response, adding it to this producers + * mailbox, unwrapping it and calling receiveAfterProduce. 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 msg to the endpoint specified by endpointUri. Before the message is - * actually produced it is pre-processed by calling receiveBeforeProduce. If oneway - * is true an in-only message exchange is initiated, otherwise an in-out message exchange. + * actually sent it is pre-processed by calling receiveBeforeProduce. If oneway + * is true, 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 endpointUri. The - * result is passed as argument. By default, this method replies the result back to the original sender - * if oneway is false. If oneway 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 endpointUri. The + * response is passed as argument. By default, this method sends the response back to the original sender + * if oneway is false. If oneway is true, 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 pattern from the endpoint specified by + * Creates a new Exchange of given pattern from the endpoint specified by * endpointUri. */ 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 endpointUri. */ 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 getEndpointUri. The - * result is passed as argument. By default, this method replies the result back to the original sender - * if isOneway returns false. If isOneway returns true then nothing is done. This - * method may be overridden by subclasses. + * Called after a response was received from the endpoint specified by endpointUri. The + * response is passed as argument. By default, this method sends the response back to the original sender + * if oneway is false. If oneway is true, 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) diff --git a/akka-camel/src/main/scala/component/ActorComponent.scala b/akka-camel/src/main/scala/component/ActorComponent.scala index 89cc0d4d3e..297a4c3a84 100644 --- a/akka-camel/src/main/scala/component/ActorComponent.scala +++ b/akka-camel/src/main/scala/component/ActorComponent.scala @@ -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 or id: or uuid:" 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 or id: or uuid:" 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 ActorRef.id or its ActorRef.uuid. - * Supported endpoint URI formats are - * actor:<actorid>, - * actor:id:<actorid> and - * actor:uuid:<actoruuid>. + * Camel endpoint for sending messages to and receiving replies from (untyped) actors. Actors + * are referenced using actor endpoint URIs of the following format: + * actor:, + * actor:id: and + * actor:uuid:, + * where actor-id refers to ActorRef.id and actor-uuid + * refers to the String-representation od ActorRef.uuid. * * @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 blocking=true|false 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 blocking=true|false endpoint URI parameter. Default value is + * false. */ @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 - * blocking 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. + *
    + *
  • If the exchange pattern is out-capable and blocking is set to + * true then the producer waits for a reply, using the !! operator.
  • + *
  • If the exchange pattern is out-capable and blocking is set to + * false then the producer sends the message using the ! operator, together + * with a callback handler. The callback handler is an ActorRef that can be + * used by the receiving actor to asynchronously reply to the route that is sending the + * message.
  • + *
  • If the exchange pattern is in-only then the producer sends the message using the + * ! operator.
  • + *
* * @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 AsyncCallback to ActorRef.!. Used by other actors to reply - * asynchronously to Camel with ActorRef.reply. + * Adapts an ActorRef to a Camel AsyncCallback. Used by receiving actors to reply + * asynchronously to Camel routes with ActorRef.reply. *

* Please note 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 message to exchange and uses callback to - * generate completion notifications. + * Populates the initial exchange with the reply message and uses the + * callback 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 diff --git a/akka-camel/src/main/scala/component/TypedActorComponent.scala b/akka-camel/src/main/scala/component/TypedActorComponent.scala index 2a48cf9fc4..542705d0c6 100644 --- a/akka-camel/src/main/scala/component/TypedActorComponent.scala +++ b/akka-camel/src/main/scala/component/TypedActorComponent.scala @@ -21,7 +21,7 @@ object TypedActorComponent { /** * Camel component for exchanging messages with typed actors. This component - * tries to obtain the typed actor from the typedActorRegistry + * tries to obtain the typed actor from its typedActorRegistry * 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 typedActorRegistry for getting access to - * typed actors (beans). + * Creates an org.apache.camel.component.bean.BeanEndpoint with a custom + * bean holder that uses typedActorRegistry 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 + * org.apache.camel.component.bean.BeanHolder 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 se.scalablesolutions.akka.camel.component.TypedActorInfo instance. */ override def getBeanInfo: BeanInfo = new TypedActorInfo(getContext, getBean.getClass, getParameterMappingStrategy) /** - * Obtains an typed actor from typedActorRegistry. + * Obtains a typed actor from typedActorRegistry. 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 null. */ 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 */ diff --git a/akka-camel/src/test/scala/CamelServiceManagerSpec.scala b/akka-camel/src/test/scala/CamelServiceManagerTest.scala similarity index 97% rename from akka-camel/src/test/scala/CamelServiceManagerSpec.scala rename to akka-camel/src/test/scala/CamelServiceManagerTest.scala index 222c1a17c6..fd15ce7154 100644 --- a/akka-camel/src/test/scala/CamelServiceManagerSpec.scala +++ b/akka-camel/src/test/scala/CamelServiceManagerTest.scala @@ -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 diff --git a/akka-camel/src/test/scala/ConsumerSpec.scala b/akka-camel/src/test/scala/ConsumerTest.scala similarity index 98% rename from akka-camel/src/test/scala/ConsumerSpec.scala rename to akka-camel/src/test/scala/ConsumerTest.scala index 678ed70057..2a2cc0b11f 100644 --- a/akka-camel/src/test/scala/ConsumerSpec.scala +++ b/akka-camel/src/test/scala/ConsumerTest.scala @@ -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 = { diff --git a/akka-camel/src/test/scala/component/ActorComponentTest.scala b/akka-camel/src/test/scala/component/ActorComponentTest.scala index e27e8c5875..50c6e664e7 100644 --- a/akka-camel/src/test/scala/component/ActorComponentTest.scala +++ b/akka-camel/src/test/scala/component/ActorComponentTest.scala @@ -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) } diff --git a/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala b/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala index 06f7e29173..37352eb154 100644 --- a/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala +++ b/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala @@ -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 diff --git a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala index 0c6f239ef7..b3867b9492 100644 --- a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala +++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala @@ -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) diff --git a/akka-persistence/akka-persistence-hbase/src/main/resources/log4j.properties b/akka-persistence/akka-persistence-hbase/src/main/resources/log4j.properties new file mode 100644 index 0000000000..5763ff8232 --- /dev/null +++ b/akka-persistence/akka-persistence-hbase/src/main/resources/log4j.properties @@ -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 diff --git a/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorage.scala b/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorage.scala index 1c3abdff4e..fc8114b7dd 100644 --- a/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorage.scala +++ b/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorage.scala @@ -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) diff --git a/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorageBackend.scala b/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorageBackend.scala index 69c393f455..30873bf036 100644 --- a/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorageBackend.scala +++ b/akka-persistence/akka-persistence-hbase/src/main/scala/HbaseStorageBackend.scala @@ -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" diff --git a/akka-persistence/akka-persistence-hbase/src/test/scala/HbasePersistentActorSpec.scala b/akka-persistence/akka-persistence-hbase/src/test/scala/HbasePersistentActorSpecTestIntegration.scala similarity index 98% rename from akka-persistence/akka-persistence-hbase/src/test/scala/HbasePersistentActorSpec.scala rename to akka-persistence/akka-persistence-hbase/src/test/scala/HbasePersistentActorSpecTestIntegration.scala index 468cd800ce..fc496ed480 100644 --- a/akka-persistence/akka-persistence-hbase/src/test/scala/HbasePersistentActorSpec.scala +++ b/akka-persistence/akka-persistence-hbase/src/test/scala/HbasePersistentActorSpecTestIntegration.scala @@ -76,7 +76,7 @@ class PersistentFailerActor extends Transactor { } } -class HbasePersistentActorSpec extends JUnitSuite with BeforeAndAfterAll { +class HbasePersistentActorSpecTestIntegration extends JUnitSuite with BeforeAndAfterAll { val testUtil = new HBaseTestingUtility diff --git a/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpec.scala b/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpecTestIntegration.scala similarity index 99% rename from akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpec.scala rename to akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpecTestIntegration.scala index 1bad777675..4d118850f0 100644 --- a/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpec.scala +++ b/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseStorageSpecTestIntegration.scala @@ -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 diff --git a/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343Spec.scala b/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343SpecTestIntegration.scala similarity index 99% rename from akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343Spec.scala rename to akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343SpecTestIntegration.scala index d61b82fa87..26210ba52f 100644 --- a/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343Spec.scala +++ b/akka-persistence/akka-persistence-hbase/src/test/scala/HbaseTicket343SpecTestIntegration.scala @@ -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 diff --git a/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseTest.scala b/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala similarity index 95% rename from akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseTest.scala rename to akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala index f59e3ae55e..8df7bbc7c9 100644 --- a/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseTest.scala +++ b/akka-persistence/akka-persistence-hbase/src/test/scala/SimpleHbaseSpecTestIntegration.scala @@ -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 diff --git a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala index 83e47e3ba5..6bd99ee3e1 100644 --- a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala +++ b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala @@ -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) diff --git a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala index 1eca775567..9055b6717c 100644 --- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala +++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala @@ -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) diff --git a/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorage.scala b/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorage.scala new file mode 100644 index 0000000000..4e237267a5 --- /dev/null +++ b/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorage.scala @@ -0,0 +1,43 @@ +/** + * Copyright (C) 2009-2010 Scalable Solutions AB + */ + +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 +} diff --git a/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorageBackend.scala b/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorageBackend.scala new file mode 100644 index 0000000000..b5c1023970 --- /dev/null +++ b/akka-persistence/akka-persistence-voldemort/src/main/scala/VoldemortStorageBackend.scala @@ -0,0 +1,379 @@ +/** + * Copyright (C) 2009-2010 Scalable Solutions AB + */ + +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 + } + + } + +} \ No newline at end of file diff --git a/akka-persistence/akka-persistence-voldemort/src/test/resources/config/cluster.xml b/akka-persistence/akka-persistence-voldemort/src/test/resources/config/cluster.xml new file mode 100644 index 0000000000..dcf806b0ca --- /dev/null +++ b/akka-persistence/akka-persistence-voldemort/src/test/resources/config/cluster.xml @@ -0,0 +1,14 @@ + + + akka-test + + + 0 + localhost + 8081 + 6666 + 6667 + + 0,1,2,3 + + diff --git a/akka-persistence/akka-persistence-voldemort/src/test/resources/config/server.properties b/akka-persistence/akka-persistence-voldemort/src/test/resources/config/server.properties new file mode 100644 index 0000000000..6dcd5bb340 --- /dev/null +++ b/akka-persistence/akka-persistence-voldemort/src/test/resources/config/server.properties @@ -0,0 +1,4 @@ +node.id=0 +enable.rebalancing=false +enable.bdb.engine=false +slop.enable=false diff --git a/akka-persistence/akka-persistence-voldemort/src/test/resources/config/stores.xml b/akka-persistence/akka-persistence-voldemort/src/test/resources/config/stores.xml new file mode 100644 index 0000000000..de666a219f --- /dev/null +++ b/akka-persistence/akka-persistence-voldemort/src/test/resources/config/stores.xml @@ -0,0 +1,85 @@ + + + Refs + 1 + 1 + 1 + 1 + 1 + memory + client + + string + utf8 + + + identity + + + + MapValues + 1 + 1 + 1 + 1 + 1 + memory + client + + identity + + + identity + + + + MapKeys + 1 + 1 + 1 + 1 + 1 + memory + client + + string + utf8 + + + identity + + + + Vectors + 1 + 1 + 1 + 1 + 1 + memory + client + + identity + + + identity + + + + Queues + 1 + 1 + 1 + 1 + 1 + memory + client + + identity + + + identity + + + + \ No newline at end of file diff --git a/akka-persistence/akka-persistence-voldemort/src/test/scala/EmbeddedVoldemort.scala b/akka-persistence/akka-persistence-voldemort/src/test/scala/EmbeddedVoldemort.scala new file mode 100644 index 0000000000..ce87309fb9 --- /dev/null +++ b/akka-persistence/akka-persistence-voldemort/src/test/scala/EmbeddedVoldemort.scala @@ -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 + } +} \ No newline at end of file diff --git a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortPersistentActorSuite.scala b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortPersistentActorSuite.scala new file mode 100644 index 0000000000..e39732dabf --- /dev/null +++ b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortPersistentActorSuite.scala @@ -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) + } + } +} diff --git a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortPersistentDatastructureSuite.scala b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortPersistentDatastructureSuite.scala new file mode 100644 index 0000000000..76bb989ac9 --- /dev/null +++ b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortPersistentDatastructureSuite.scala @@ -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) + } + + + } + +} \ No newline at end of file diff --git a/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendSuite.scala b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendSuite.scala new file mode 100644 index 0000000000..5f27771bae --- /dev/null +++ b/akka-persistence/akka-persistence-voldemort/src/test/scala/VoldemortStorageBackendSuite.scala @@ -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") + } + +} \ No newline at end of file diff --git a/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java b/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java index 31ae9650d4..e5265ea396 100644 --- a/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java +++ b/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java @@ -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 = diff --git a/akka-remote/src/main/protocol/RemoteProtocol.proto b/akka-remote/src/main/protocol/RemoteProtocol.proto index e84ae9f18e..1ea9d8f986 100644 --- a/akka-remote/src/main/protocol/RemoteProtocol.proto +++ b/akka-remote/src/main/protocol/RemoteProtocol.proto @@ -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; diff --git a/akka-remote/src/main/scala/remote/MessageSerializer.scala b/akka-remote/src/main/scala/remote/MessageSerializer.scala index 49f38524f9..7cda9e4b4c 100644 --- a/akka-remote/src/main/scala/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/remote/MessageSerializer.scala @@ -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]]) diff --git a/akka-remote/src/main/scala/remote/RemoteClient.scala b/akka-remote/src/main/scala/remote/RemoteClient.scala index 97e2f3070f..bb9714bc61 100644 --- a/akka-remote/src/main/scala/remote/RemoteClient.scala +++ b/akka-remote/src/main/scala/remote/RemoteClient.scala @@ -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 Jonas Bonér - */ -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)) diff --git a/akka-remote/src/main/scala/remote/RemoteServer.scala b/akka-remote/src/main/scala/remote/RemoteServer.scala index b258c4867d..bed9e9f933 100644 --- a/akka-remote/src/main/scala/remote/RemoteServer.scala +++ b/akka-remote/src/main/scala/remote/RemoteServer.scala @@ -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) diff --git a/akka-remote/src/main/scala/serialization/Serializable.scala b/akka-remote/src/main/scala/serialization/Serializable.scala index 216869d7a6..b15784f9ce 100644 --- a/akka-remote/src/main/scala/serialization/Serializable.scala +++ b/akka-remote/src/main/scala/serialization/Serializable.scala @@ -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 Jonas Bonér */ - 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 } } diff --git a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala index c9b443f4ec..c07417c0e2 100644 --- a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala +++ b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala @@ -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 } diff --git a/akka-remote/src/main/scala/serialization/Serializer.scala b/akka-remote/src/main/scala/serialization/Serializer.scala index 1365a7d4c1..871ce1f681 100644 --- a/akka-remote/src/main/scala/serialization/Serializer.scala +++ b/akka-remote/src/main/scala/serialization/Serializer.scala @@ -128,11 +128,23 @@ object Serializer { /** * @author Jonas Bonér */ - 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 Jonas Bonér diff --git a/akka-remote/src/main/scala/serialization/package.scala b/akka-remote/src/main/scala/serialization/package.scala new file mode 100644 index 0000000000..1a3c83341f --- /dev/null +++ b/akka-remote/src/main/scala/serialization/package.scala @@ -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 +} diff --git a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala index 6670722b02..d39b58d41d 100644 --- a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala +++ b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala @@ -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) } diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala index fbf723ece5..8b61b30600 100644 --- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala +++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala @@ -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") } diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala index f50c3e6652..71ece9792e 100644 --- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala +++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala @@ -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") { diff --git a/akka-remote/src/test/scala/serialization/ScalaJSONSerializableSpec.scala b/akka-remote/src/test/scala/serialization/ScalaJSONSerializableSpec.scala new file mode 100644 index 0000000000..68b2f171e4 --- /dev/null +++ b/akka-remote/src/test/scala/serialization/ScalaJSONSerializableSpec.scala @@ -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) + } + } +} diff --git a/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala b/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala new file mode 100644 index 0000000000..7d1ef4c7a0 --- /dev/null +++ b/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala @@ -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) + } + } +} diff --git a/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala index 832a655c22..90c445dd3c 100644 --- a/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala +++ b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala @@ -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)) +} diff --git a/akka-remote/src/test/scala/serialization/Ticket436Spec.scala b/akka-remote/src/test/scala/serialization/Ticket436Spec.scala deleted file mode 100644 index 042f3f07be..0000000000 --- a/akka-remote/src/test/scala/serialization/Ticket436Spec.scala +++ /dev/null @@ -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) - } - } -} diff --git a/akka-remote/src/test/scala/ticket/Ticket434Spec.scala b/akka-remote/src/test/scala/ticket/Ticket434Spec.scala index b27c17cfe5..29944d8e0b 100644 --- a/akka-remote/src/test/scala/ticket/Ticket434Spec.scala +++ b/akka-remote/src/test/scala/ticket/Ticket434Spec.scala @@ -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") } } diff --git a/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTest.scala b/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala similarity index 94% rename from akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTest.scala rename to akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala index 1a115c6f76..3813463601 100644 --- a/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTest.scala +++ b/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala @@ -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 diff --git a/akka-typed-actor/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala index f6cf2e2337..a4c7ddada1 100644 --- a/akka-typed-actor/src/main/scala/actor/TypedActor.scala +++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala @@ -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 = { diff --git a/config/akka-reference.conf b/config/akka-reference.conf index 8d3b361fe3..a8bfcb332d 100644 --- a/config/akka-reference.conf +++ b/config/akka-reference.conf @@ -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 + } } } } diff --git a/config/logback-test.xml b/config/logback-test.xml index fdb912d463..0671694e3f 100644 --- a/config/logback-test.xml +++ b/config/logback-test.xml @@ -16,6 +16,10 @@ + + + + diff --git a/embedded-repo/com/eaio/uuid/3.2/uuid-3.2.jar b/embedded-repo/com/eaio/uuid/3.2/uuid-3.2.jar new file mode 100644 index 0000000000..b205a2488e Binary files /dev/null and b/embedded-repo/com/eaio/uuid/3.2/uuid-3.2.jar differ diff --git a/embedded-repo/com/eaio/uuid/3.2/uuid-3.2.pom b/embedded-repo/com/eaio/uuid/3.2/uuid-3.2.pom new file mode 100644 index 0000000000..bb20c22466 --- /dev/null +++ b/embedded-repo/com/eaio/uuid/3.2/uuid-3.2.pom @@ -0,0 +1,8 @@ + + + 4.0.0 + com.eaio + uuid + 3.2 + jar + \ No newline at end of file diff --git a/embedded-repo/org/apache/hbase/hbase-core/0.20.6/hbase-core-0.20.6.pom b/embedded-repo/org/apache/hbase/hbase-core/0.20.6/hbase-core-0.20.6.pom new file mode 100644 index 0000000000..19a8b54700 --- /dev/null +++ b/embedded-repo/org/apache/hbase/hbase-core/0.20.6/hbase-core-0.20.6.pom @@ -0,0 +1,8 @@ + + + 4.0.0 + org.apache.hbase + hbase-core + 0.20.6 + jar + \ No newline at end of file diff --git a/embedded-repo/org/apache/hbase/hbase-test/0.20.6/hbase-test-0.20.6.pom b/embedded-repo/org/apache/hbase/hbase-test/0.20.6/hbase-test-0.20.6.pom new file mode 100644 index 0000000000..bdc80cc8b7 --- /dev/null +++ b/embedded-repo/org/apache/hbase/hbase-test/0.20.6/hbase-test-0.20.6.pom @@ -0,0 +1,8 @@ + + + 4.0.0 + org.apache.hbase + hbase-test + 0.20.6 + jar + \ No newline at end of file diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala index bb468c7b9f..a1c2a9fdf6 100644 --- a/project/build/AkkaProject.scala +++ b/project/build/AkkaProject.scala @@ -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 = + + + + + + + + + + + + + + + + + + + + + + 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) { + + 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 + } } }