Merge branch 'master' of git@github.com:jboner/akka
Conflicts: akka-actor/src/main/scala/dispatch/Dispatchers.scala akka-actor/src/main/scala/dispatch/MessageHandling.scala akka-actor/src/main/scala/util/ReflectiveAccess.scala
This commit is contained in:
commit
fe42fdf782
69 changed files with 2026 additions and 633 deletions
|
|
@ -1,59 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
|
||||
*/
|
||||
package se.scalablesolutions.akka.util;
|
||||
|
||||
/**
|
||||
* NOTE:
|
||||
* <p/>
|
||||
* This code is based on code from the [Plasmid Replication Engine] project.
|
||||
* <br/>
|
||||
* Licensed under [Mozilla Public License 1.0 (MPL)].
|
||||
* <p/>
|
||||
* Original JavaDoc:
|
||||
* <p/>
|
||||
* Our distributed objects are generally named most efficiently (and cleanly)
|
||||
* by their UUID's. This class provides some static helpers for using UUID's.
|
||||
* If it was efficient to do in Java, I would make the uuid an normal class
|
||||
* and use instances of it. However, in current JVM's, we would end up using an
|
||||
* Object to represent a long, which is pretty expensive. Maybe someday. ###
|
||||
* <p/>
|
||||
* UUID format: currently using currentTimeMillis() for the low bits. This uses
|
||||
* about 40 bits for the next 1000 years, leaving 24 bits for debugging
|
||||
* and consistency data. I'm using 8 of those for a magic asci 'U' byte.
|
||||
* <p/>
|
||||
* Future: use one instance of Uuid per type of object for better performance
|
||||
* and more detailed info (instance could be matched to its uuid's via a map or
|
||||
* array). This all static version bites.###
|
||||
*/
|
||||
public final class UUID {
|
||||
|
||||
public static final long UUID_NONE = 0;
|
||||
public static final long UUID_WILD = -1;
|
||||
public static final long UUID_MAGICMASK = 0xff << 56;
|
||||
public static final long UUID_MAGIC = 'U' << 56;
|
||||
|
||||
protected static long lastTime;
|
||||
|
||||
/**
|
||||
* Generate and return a new Universally Unique ID.
|
||||
* Happens to be monotonically increasing.
|
||||
*/
|
||||
public synchronized static long newUuid() {
|
||||
long time = System.currentTimeMillis();
|
||||
|
||||
if (time <= lastTime) {
|
||||
time = lastTime + 1;
|
||||
}
|
||||
lastTime = time;
|
||||
return UUID_MAGIC | time;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if uuid could have been generated by Uuid.
|
||||
*/
|
||||
public static boolean isValid(final long uuid) {
|
||||
return (uuid & UUID_MAGICMASK) == UUID_MAGIC
|
||||
&& (uuid & ~UUID_MAGICMASK) != 0;
|
||||
}
|
||||
}
|
||||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -35,7 +35,7 @@ case class ActorUnregistered(actor: ActorRef) extends ActorRegistryEvent
|
|||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
object ActorRegistry extends ListenerManagement {
|
||||
private val actorsByUUID = new ConcurrentHashMap[String, ActorRef]
|
||||
private val actorsByUUID = new ConcurrentHashMap[Uuid, ActorRef]
|
||||
private val actorsById = new Index[String,ActorRef]
|
||||
|
||||
/**
|
||||
|
|
@ -112,7 +112,7 @@ object ActorRegistry extends ListenerManagement {
|
|||
/**
|
||||
* Finds the actor that has a specific UUID.
|
||||
*/
|
||||
def actorFor(uuid: String): Option[ActorRef] = Option(actorsByUUID get uuid)
|
||||
def actorFor(uuid: Uuid): Option[ActorRef] = Option(actorsByUUID get uuid)
|
||||
|
||||
/**
|
||||
* Registers an actor in the ActorRegistry.
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 => {
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
|
||||
package se.scalablesolutions.akka.dispatch
|
||||
|
||||
import se.scalablesolutions.akka.actor.{Actor, ActorRef, ActorInitializationException}
|
||||
import se.scalablesolutions.akka.actor.{Actor, ActorRef, Uuid, ActorInitializationException}
|
||||
import se.scalablesolutions.akka.util.{SimpleLock, Duration, HashCode, Logging}
|
||||
import se.scalablesolutions.akka.util.ReflectiveAccess.EnterpriseModule
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
|
@ -13,7 +13,6 @@ import org.multiverse.commitbarriers.CountDownCommitBarrier
|
|||
|
||||
import java.util.{Queue, List}
|
||||
import java.util.concurrent._
|
||||
import concurrent.forkjoin.LinkedTransferQueue
|
||||
|
||||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
|
|
@ -62,7 +61,7 @@ final class MessageInvocation(val receiver: ActorRef,
|
|||
*/
|
||||
trait MessageDispatcher extends MailboxFactory with Logging {
|
||||
|
||||
protected val uuids = new ConcurrentSkipListSet[String]
|
||||
protected val uuids = new ConcurrentSkipListSet[Uuid]
|
||||
|
||||
def dispatch(invocation: MessageInvocation): Unit
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -13,7 +13,7 @@ import se.scalablesolutions.akka.camel.component.TypedActorComponent
|
|||
import se.scalablesolutions.akka.util.Logging
|
||||
|
||||
/**
|
||||
* Defines the lifecycle of a CamelContext. Allowed state transitions are
|
||||
* Manages the lifecycle of a CamelContext. Allowed transitions are
|
||||
* init -> start -> stop -> init -> ... etc.
|
||||
*
|
||||
* @author Martin Krasser
|
||||
|
|
@ -35,7 +35,7 @@ trait CamelContextLifecycle extends Logging {
|
|||
|
||||
/**
|
||||
* Registry in which typed actors are TEMPORARILY registered during
|
||||
* creation of Camel routes to typed actors.
|
||||
* creation of Camel routes to these actors.
|
||||
*/
|
||||
private[camel] var typedActorRegistry: Map[String, AnyRef] = _
|
||||
|
||||
|
|
@ -63,7 +63,7 @@ trait CamelContextLifecycle extends Logging {
|
|||
def started = _started
|
||||
|
||||
/**
|
||||
* Starts the CamelContext and ProducerTemplate.
|
||||
* Starts the CamelContext and an associated ProducerTemplate.
|
||||
*/
|
||||
def start = {
|
||||
context.start
|
||||
|
|
@ -73,7 +73,7 @@ trait CamelContextLifecycle extends Logging {
|
|||
}
|
||||
|
||||
/**
|
||||
* Stops the CamelContext and ProducerTemplate.
|
||||
* Stops the CamelContext and the associated ProducerTemplate.
|
||||
*/
|
||||
def stop = {
|
||||
template.stop
|
||||
|
|
@ -90,11 +90,10 @@ trait CamelContextLifecycle extends Logging {
|
|||
|
||||
/**
|
||||
* Initializes this lifecycle object with the given CamelContext. For the passed
|
||||
* CamelContext stream-caching is enabled. If applications want to disable stream-
|
||||
* CamelContext, stream-caching is enabled. If applications want to disable stream-
|
||||
* caching they can do so after this method returned and prior to calling start.
|
||||
* This method also registers a new
|
||||
* {@link se.scalablesolutions.akka.camel.component.TypedActorComponent} at
|
||||
* <code>context</code> under a name defined by TypedActorComponent.InternalSchema.
|
||||
* This method also registers a new TypedActorComponent at the passes CamelContext
|
||||
* under a name defined by TypedActorComponent.InternalSchema.
|
||||
*/
|
||||
def init(context: CamelContext) {
|
||||
this.typedActorComponent = new TypedActorComponent
|
||||
|
|
@ -109,8 +108,7 @@ trait CamelContextLifecycle extends Logging {
|
|||
}
|
||||
|
||||
/**
|
||||
* Makes a global CamelContext and ProducerTemplate accessible to applications. The lifecycle
|
||||
* of these objects is managed by se.scalablesolutions.akka.camel.CamelService.
|
||||
* Manages a global CamelContext and an associated ProducerTemplate.
|
||||
*/
|
||||
object CamelContextManager extends CamelContextLifecycle {
|
||||
override def context: CamelContext = super.context
|
||||
|
|
|
|||
|
|
@ -12,9 +12,10 @@ import se.scalablesolutions.akka.actor.{AspectInitRegistry, ActorRegistry}
|
|||
import se.scalablesolutions.akka.util.{Bootable, Logging}
|
||||
|
||||
/**
|
||||
* Used by applications (and the Kernel) to publish consumer actors and typed actors via
|
||||
* Camel endpoints and to manage the life cycle of a a global CamelContext which can be
|
||||
* accessed via <code>se.scalablesolutions.akka.camel.CamelContextManager.context</code>.
|
||||
* Publishes (untyped) consumer actors and typed consumer actors via Camel endpoints. Actors
|
||||
* are published (asynchronously) when they are started and unpublished (asynchronously) when
|
||||
* they are stopped. The CamelService is notified about actor start- and stop-events by
|
||||
* registering listeners at ActorRegistry and AspectInitRegistry.
|
||||
*
|
||||
* @author Martin Krasser
|
||||
*/
|
||||
|
|
@ -29,11 +30,11 @@ trait CamelService extends Bootable with Logging {
|
|||
AspectInitRegistry.addListener(publishRequestor)
|
||||
|
||||
/**
|
||||
* Starts the CamelService. Any started actor that is a consumer actor will be (asynchronously)
|
||||
* Starts this CamelService. Any started actor that is a consumer actor will be (asynchronously)
|
||||
* published as Camel endpoint. Consumer actors that are started after this method returned will
|
||||
* be published as well. Actor publishing is done asynchronously. A started (loaded) CamelService
|
||||
* also publishes <code>@consume</code> annotated methods of typed actors that have been created
|
||||
* with <code>TypedActor.newInstance(..)</code> (and <code>TypedActor.newInstance(..)</code>
|
||||
* with <code>TypedActor.newInstance(..)</code> (and <code>TypedActor.newRemoteInstance(..)</code>
|
||||
* on a remote node).
|
||||
*/
|
||||
abstract override def onLoad = {
|
||||
|
|
@ -54,7 +55,8 @@ trait CamelService extends Bootable with Logging {
|
|||
}
|
||||
|
||||
/**
|
||||
* Stops the CamelService.
|
||||
* Stops this CamelService. All published consumer actors and typed consumer actor methods will be
|
||||
* unpublished asynchronously.
|
||||
*/
|
||||
abstract override def onUnload = {
|
||||
// Unregister this instance as current CamelService
|
||||
|
|
@ -98,24 +100,24 @@ trait CamelService extends Bootable with Logging {
|
|||
def stop = onUnload
|
||||
|
||||
/**
|
||||
* Sets an expectation of the number of upcoming endpoint activations and returns
|
||||
* a {@link CountDownLatch} that can be used to wait for the activations to occur.
|
||||
* Endpoint activations that occurred in the past are not considered.
|
||||
* Sets an expectation on the number of upcoming endpoint activations and returns
|
||||
* a CountDownLatch that can be used to wait for the activations to occur. Endpoint
|
||||
* activations that occurred in the past are not considered.
|
||||
*/
|
||||
def expectEndpointActivationCount(count: Int): CountDownLatch =
|
||||
(consumerPublisher !! SetExpectedRegistrationCount(count)).as[CountDownLatch].get
|
||||
|
||||
/**
|
||||
* Sets an expectation of the number of upcoming endpoint de-activations and returns
|
||||
* a {@link CountDownLatch} that can be used to wait for the de-activations to occur.
|
||||
* Endpoint de-activations that occurred in the past are not considered.
|
||||
* Sets an expectation on the number of upcoming endpoint de-activations and returns
|
||||
* a CountDownLatch that can be used to wait for the de-activations to occur. Endpoint
|
||||
* de-activations that occurred in the past are not considered.
|
||||
*/
|
||||
def expectEndpointDeactivationCount(count: Int): CountDownLatch =
|
||||
(consumerPublisher !! SetExpectedUnregistrationCount(count)).as[CountDownLatch].get
|
||||
}
|
||||
|
||||
/**
|
||||
* ...
|
||||
* Manages a global CamelService (the 'current' CamelService).
|
||||
*
|
||||
* @author Martin Krasser
|
||||
*/
|
||||
|
|
@ -128,11 +130,17 @@ object CamelServiceManager {
|
|||
|
||||
/**
|
||||
* Starts a new CamelService and makes it the current CamelService.
|
||||
*
|
||||
* @see CamelService#start
|
||||
* @see CamelService#onLoad
|
||||
*/
|
||||
def startCamelService = CamelServiceFactory.createCamelService.start
|
||||
|
||||
/**
|
||||
* Stops the current CamelService.
|
||||
*
|
||||
* @see CamelService#stop
|
||||
* @see CamelService#onUnload
|
||||
*/
|
||||
def stopCamelService = service.stop
|
||||
|
||||
|
|
@ -159,12 +167,12 @@ object CamelServiceManager {
|
|||
*/
|
||||
object CamelServiceFactory {
|
||||
/**
|
||||
* Creates a new CamelService instance
|
||||
* Creates a new CamelService instance.
|
||||
*/
|
||||
def createCamelService: CamelService = new CamelService { }
|
||||
|
||||
/**
|
||||
* Creates a new CamelService instance
|
||||
* Creates a new CamelService instance and initializes it with the given CamelContext.
|
||||
*/
|
||||
def createCamelService(camelContext: CamelContext): CamelService = {
|
||||
CamelContextManager.init(camelContext)
|
||||
|
|
|
|||
|
|
@ -20,30 +20,24 @@ trait Consumer { self: Actor =>
|
|||
def endpointUri: String
|
||||
|
||||
/**
|
||||
* Determines whether two-way communications with this consumer actor should
|
||||
* be done in blocking or non-blocking mode (default is non-blocking). One-way
|
||||
* communications never block.
|
||||
* Determines whether two-way communications between an endpoint and this consumer actor
|
||||
* should be done in blocking or non-blocking mode (default is non-blocking). This method
|
||||
* doesn't have any effect on one-way communications (they'll never block).
|
||||
*/
|
||||
def blocking = false
|
||||
}
|
||||
|
||||
/**
|
||||
* Java-friendly {@link Consumer} inherited by
|
||||
* Java-friendly Consumer.
|
||||
*
|
||||
* <ul>
|
||||
* <li>{@link UntypedConsumerActor}</li>
|
||||
* <li>{@link RemoteUntypedConsumerActor}</li>
|
||||
* <li>{@link UntypedConsumerTransactor}</li>
|
||||
* </ul>
|
||||
*
|
||||
* implementations.
|
||||
* @see UntypedConsumerActor
|
||||
* @see RemoteUntypedConsumerActor
|
||||
* @see UntypedConsumerTransactor
|
||||
*
|
||||
* @author Martin Krasser
|
||||
*/
|
||||
trait UntypedConsumer extends Consumer { self: UntypedActor =>
|
||||
|
||||
final override def endpointUri = getEndpointUri
|
||||
|
||||
final override def blocking = isBlocking
|
||||
|
||||
/**
|
||||
|
|
@ -52,9 +46,9 @@ trait UntypedConsumer extends Consumer { self: UntypedActor =>
|
|||
def getEndpointUri(): String
|
||||
|
||||
/**
|
||||
* Determines whether two-way communications with this consumer actor should
|
||||
* be done in blocking or non-blocking mode (default is non-blocking). One-way
|
||||
* communications never block.
|
||||
* Determines whether two-way communications between an endpoint and this consumer actor
|
||||
* should be done in blocking or non-blocking mode (default is non-blocking). This method
|
||||
* doesn't have any effect on one-way communications (they'll never block).
|
||||
*/
|
||||
def isBlocking() = super.blocking
|
||||
}
|
||||
|
|
@ -89,7 +83,7 @@ private[camel] object Consumer {
|
|||
* reference with a target actor that implements the <code>Consumer</code> trait. The
|
||||
* target <code>Consumer</code> object is passed as argument to <code>f</code>. This
|
||||
* method returns <code>None</code> if <code>actorRef</code> is not a valid reference
|
||||
* to a consumer actor, <code>Some</code> result otherwise.
|
||||
* to a consumer actor, <code>Some</code> consumer actor otherwise.
|
||||
*/
|
||||
def forConsumer[T](actorRef: ActorRef)(f: Consumer => T): Option[T] = {
|
||||
if (!actorRef.actor.isInstanceOf[Consumer]) None
|
||||
|
|
|
|||
|
|
@ -28,10 +28,10 @@ private[camel] object ConsumerPublisher extends Logging {
|
|||
}
|
||||
|
||||
/**
|
||||
* Stops route to the already un-registered consumer actor.
|
||||
* Stops the route to the already un-registered consumer actor.
|
||||
*/
|
||||
def handleConsumerUnregistered(event: ConsumerUnregistered) {
|
||||
CamelContextManager.context.stopRoute(event.uuid)
|
||||
CamelContextManager.context.stopRoute(event.uuid.toString)
|
||||
log.info("unpublished actor %s from endpoint %s" format (event.actorRef, event.uri))
|
||||
}
|
||||
|
||||
|
|
@ -48,7 +48,7 @@ private[camel] object ConsumerPublisher extends Logging {
|
|||
}
|
||||
|
||||
/**
|
||||
* Stops route to the already un-registered consumer actor method.
|
||||
* Stops the route to the already un-registered consumer actor method.
|
||||
*/
|
||||
def handleConsumerMethodUnregistered(event: ConsumerMethodUnregistered) {
|
||||
val targetMethod = event.method.getName
|
||||
|
|
@ -62,10 +62,10 @@ private[camel] object ConsumerPublisher extends Logging {
|
|||
|
||||
/**
|
||||
* Actor that publishes consumer actors and typed actor methods at Camel endpoints.
|
||||
* The Camel context used for publishing is CamelContextManager.context. This actor
|
||||
* accepts messages of type
|
||||
* The Camel context used for publishing is obtained via CamelContextManager.context.
|
||||
* This actor accepts messages of type
|
||||
* se.scalablesolutions.akka.camel.ConsumerRegistered,
|
||||
* se.scalablesolutions.akka.camel.ConsumerUnregistered.
|
||||
* se.scalablesolutions.akka.camel.ConsumerUnregistered,
|
||||
* se.scalablesolutions.akka.camel.ConsumerMethodRegistered and
|
||||
* se.scalablesolutions.akka.camel.ConsumerMethodUnregistered.
|
||||
*
|
||||
|
|
@ -110,7 +110,7 @@ private[camel] case class SetExpectedRegistrationCount(num: Int)
|
|||
private[camel] case class SetExpectedUnregistrationCount(num: Int)
|
||||
|
||||
/**
|
||||
* Defines an abstract route to a target which is either an actor or an typed actor method..
|
||||
* Abstract route to a target which is either an actor or an typed actor method.
|
||||
*
|
||||
* @param endpointUri endpoint URI of the consumer actor or typed actor method.
|
||||
* @param id actor identifier or typed actor identifier (registry key).
|
||||
|
|
@ -135,20 +135,20 @@ private[camel] abstract class ConsumerRoute(endpointUri: String, id: String) ext
|
|||
}
|
||||
|
||||
/**
|
||||
* Defines the route to a consumer actor.
|
||||
* Defines the route to a (untyped) consumer actor.
|
||||
*
|
||||
* @param endpointUri endpoint URI of the consumer actor
|
||||
* @param endpointUri endpoint URI of the (untyped) consumer actor
|
||||
* @param uuid actor uuid
|
||||
* @param blocking true for blocking in-out exchanges, false otherwise
|
||||
*
|
||||
* @author Martin Krasser
|
||||
*/
|
||||
private[camel] class ConsumerActorRoute(endpointUri: String, uuid: String, blocking: Boolean) extends ConsumerRoute(endpointUri, uuid) {
|
||||
private[camel] class ConsumerActorRoute(endpointUri: String, uuid: Uuid, blocking: Boolean) extends ConsumerRoute(endpointUri, uuid.toString) {
|
||||
protected override def targetUri = "actor:uuid:%s?blocking=%s" format (uuid, blocking)
|
||||
}
|
||||
|
||||
/**
|
||||
* Defines the route to an typed actor method..
|
||||
* Defines the route to a typed actor method.
|
||||
*
|
||||
* @param endpointUri endpoint URI of the consumer actor method
|
||||
* @param id typed actor identifier
|
||||
|
|
@ -162,10 +162,10 @@ private[camel] class ConsumerMethodRoute(val endpointUri: String, id: String, me
|
|||
|
||||
/**
|
||||
* A registration listener that triggers publication of consumer actors and typed actor
|
||||
* methods as well as un-publication of consumer actors. This actor needs to be initialized
|
||||
* with a <code>PublishRequestorInit</code> command message for obtaining a reference to
|
||||
* a <code>publisher</code> actor. Before initialization it buffers all outbound messages
|
||||
* and delivers them to the <code>publisher</code> when receiving a
|
||||
* methods as well as un-publication of consumer actors and typed actor methods. This actor
|
||||
* needs to be initialized with a <code>PublishRequestorInit</code> command message for
|
||||
* obtaining a reference to a <code>publisher</code> actor. Before initialization it buffers
|
||||
* all outbound messages and delivers them to the <code>publisher</code> when receiving a
|
||||
* <code>PublishRequestorInit</code> message. After initialization, outbound messages are
|
||||
* delivered directly without buffering.
|
||||
*
|
||||
|
|
@ -229,7 +229,7 @@ private[camel] sealed trait ConsumerEvent
|
|||
*
|
||||
* @author Martin Krasser
|
||||
*/
|
||||
private[camel] case class ConsumerRegistered(actorRef: ActorRef, uri: String, uuid: String, blocking: Boolean) extends ConsumerEvent
|
||||
private[camel] case class ConsumerRegistered(actorRef: ActorRef, uri: String, uuid: Uuid, blocking: Boolean) extends ConsumerEvent
|
||||
|
||||
/**
|
||||
* Event indicating that a consumer actor has been unregistered from the actor registry.
|
||||
|
|
@ -240,7 +240,7 @@ private[camel] case class ConsumerRegistered(actorRef: ActorRef, uri: String, uu
|
|||
*
|
||||
* @author Martin Krasser
|
||||
*/
|
||||
private[camel] case class ConsumerUnregistered(actorRef: ActorRef, uri: String, uuid: String) extends ConsumerEvent
|
||||
private[camel] case class ConsumerUnregistered(actorRef: ActorRef, uri: String, uuid: Uuid) extends ConsumerEvent
|
||||
|
||||
/**
|
||||
* Event indicating that an typed actor proxy has been created for a typed actor. For each <code>@consume</code>
|
||||
|
|
@ -273,7 +273,7 @@ private[camel] case class ConsumerMethodUnregistered(typedActor: AnyRef, init: A
|
|||
*/
|
||||
private[camel] object ConsumerRegistered {
|
||||
/**
|
||||
* Optionally creates an ConsumerRegistered event message for a consumer actor or None if
|
||||
* Creates an ConsumerRegistered event message for a consumer actor or None if
|
||||
* <code>actorRef</code> is not a consumer actor.
|
||||
*/
|
||||
def forConsumer(actorRef: ActorRef): Option[ConsumerRegistered] = {
|
||||
|
|
@ -288,7 +288,7 @@ private[camel] object ConsumerRegistered {
|
|||
*/
|
||||
private[camel] object ConsumerUnregistered {
|
||||
/**
|
||||
* Optionally creates an ConsumerUnregistered event message for a consumer actor or None if
|
||||
* Creates an ConsumerUnregistered event message for a consumer actor or None if
|
||||
* <code>actorRef</code> is not a consumer actor.
|
||||
*/
|
||||
def forConsumer(actorRef: ActorRef): Option[ConsumerUnregistered] = {
|
||||
|
|
@ -327,8 +327,8 @@ private[camel] object ConsumerMethod {
|
|||
*/
|
||||
private[camel] object ConsumerMethodRegistered {
|
||||
/**
|
||||
* Creates a list of ConsumerMethodRegistered event messages for an typed actor or an empty
|
||||
* list if the typed actor is a proxy for an remote typed actor or the typed actor doesn't
|
||||
* Creates a list of ConsumerMethodRegistered event messages for a typed actor or an empty
|
||||
* list if the typed actor is a proxy for a remote typed actor or the typed actor doesn't
|
||||
* have any <code>@consume</code> annotated methods.
|
||||
*/
|
||||
def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodRegistered] = {
|
||||
|
|
@ -343,8 +343,8 @@ private[camel] object ConsumerMethodRegistered {
|
|||
*/
|
||||
private[camel] object ConsumerMethodUnregistered {
|
||||
/**
|
||||
* Creates a list of ConsumerMethodUnregistered event messages for an typed actor or an empty
|
||||
* list if the typed actor is a proxy for an remote typed actor or the typed actor doesn't
|
||||
* Creates a list of ConsumerMethodUnregistered event messages for a typed actor or an empty
|
||||
* list if the typed actor is a proxy for a remote typed actor or the typed actor doesn't
|
||||
* have any <code>@consume</code> annotated methods.
|
||||
*/
|
||||
def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodUnregistered] = {
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ trait ProducerSupport { this: Actor =>
|
|||
private val headersToCopyDefault = Set(Message.MessageExchangeId)
|
||||
|
||||
/**
|
||||
* <code>Endpoint</code> object resolved from current CamelContext with
|
||||
* <code>Endpoint</code> object resolved from the current CamelContext with
|
||||
* <code>endpointUri</code>.
|
||||
*/
|
||||
private lazy val endpoint = CamelContextManager.context.getEndpoint(endpointUri)
|
||||
|
|
@ -36,8 +36,8 @@ trait ProducerSupport { this: Actor =>
|
|||
|
||||
/**
|
||||
* If set to false (default), this producer expects a response message from the Camel endpoint.
|
||||
* If set to true, this producer communicates with the Camel endpoint with an in-only message
|
||||
* exchange pattern (fire and forget).
|
||||
* If set to true, this producer initiates an in-only message exchange with the Camel endpoint
|
||||
* (fire and forget).
|
||||
*/
|
||||
def oneway: Boolean = false
|
||||
|
||||
|
|
@ -62,13 +62,17 @@ trait ProducerSupport { this: Actor =>
|
|||
}
|
||||
|
||||
/**
|
||||
* Produces <code>msg</code> as exchange of given <code>pattern</code> to the endpoint specified by
|
||||
* <code>endpointUri</code>. After producing to the endpoint the processing result is passed as argument
|
||||
* to <code>receiveAfterProduce</code>. If the result was returned synchronously by the endpoint then
|
||||
* <code>receiveAfterProduce</code> is called synchronously as well. If the result was returned asynchronously,
|
||||
* the <code>receiveAfterProduce</code> is called asynchronously as well. This is done by wrapping the result,
|
||||
* adding it to this producers mailbox, unwrapping it once it is received and calling
|
||||
* <code>receiveAfterProduce</code>. The original sender and senderFuture are thereby preserved.
|
||||
* Initiates a message exchange of given <code>pattern</code> with the endpoint specified by
|
||||
* <code>endpointUri</code>. The in-message of the initiated exchange is the canonical form
|
||||
* of <code>msg</code>. After sending the in-message, the processing result (response) is passed
|
||||
* as argument to <code>receiveAfterProduce</code>. If the response is received synchronously from
|
||||
* the endpoint then <code>receiveAfterProduce</code> is called synchronously as well. If the
|
||||
* response is received asynchronously, the <code>receiveAfterProduce</code> is called
|
||||
* asynchronously. This is done by wrapping the response, adding it to this producers
|
||||
* mailbox, unwrapping it and calling <code>receiveAfterProduce</code>. The original
|
||||
* sender and senderFuture are thereby preserved.
|
||||
*
|
||||
* @see Message#canonicalize(Any)
|
||||
*
|
||||
* @param msg message to produce
|
||||
* @param pattern exchange pattern
|
||||
|
|
@ -106,8 +110,8 @@ trait ProducerSupport { this: Actor =>
|
|||
|
||||
/**
|
||||
* Produces <code>msg</code> to the endpoint specified by <code>endpointUri</code>. Before the message is
|
||||
* actually produced it is pre-processed by calling <code>receiveBeforeProduce</code>. If <code>oneway</code>
|
||||
* is true an in-only message exchange is initiated, otherwise an in-out message exchange.
|
||||
* actually sent it is pre-processed by calling <code>receiveBeforeProduce</code>. If <code>oneway</code>
|
||||
* is <code>true</code>, an in-only message exchange is initiated, otherwise an in-out message exchange.
|
||||
*
|
||||
* @see Producer#produce(Any, ExchangePattern)
|
||||
*/
|
||||
|
|
@ -132,17 +136,18 @@ trait ProducerSupport { this: Actor =>
|
|||
}
|
||||
|
||||
/**
|
||||
* Called after the a result was received from the endpoint specified by <code>endpointUri</code>. The
|
||||
* result is passed as argument. By default, this method replies the result back to the original sender
|
||||
* if <code>oneway</code> is false. If <code>oneway</code> is true then nothing is done. This method may
|
||||
* be overridden by subtraits or subclasses.
|
||||
* Called after a response was received from the endpoint specified by <code>endpointUri</code>. The
|
||||
* response is passed as argument. By default, this method sends the response back to the original sender
|
||||
* if <code>oneway</code> is <code>false</code>. If <code>oneway</code> is <code>true</code>, nothing is
|
||||
* done. This method may be overridden by subtraits or subclasses (e.g. to forward responses to another
|
||||
* actor).
|
||||
*/
|
||||
protected def receiveAfterProduce: Receive = {
|
||||
case msg => if (!oneway) self.reply(msg)
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new Exchange with given <code>pattern</code> from the endpoint specified by
|
||||
* Creates a new Exchange of given <code>pattern</code> from the endpoint specified by
|
||||
* <code>endpointUri</code>.
|
||||
*/
|
||||
private def createExchange(pattern: ExchangePattern): Exchange = endpoint.createExchange(pattern)
|
||||
|
|
@ -158,25 +163,26 @@ trait ProducerSupport { this: Actor =>
|
|||
}
|
||||
|
||||
/**
|
||||
* Mixed in by Actor implementations that produce messages to Camel endpoints.
|
||||
* Mixed in by Actor implementations to produce messages to Camel endpoints.
|
||||
*/
|
||||
trait Producer extends ProducerSupport { this: Actor =>
|
||||
|
||||
/**
|
||||
* Default implementation of Actor.receive
|
||||
* Default implementation of Actor.receive. Any messages received by this actors
|
||||
* will be produced to the endpoint specified by <code>endpointUri</code>.
|
||||
*/
|
||||
protected def receive = produce
|
||||
}
|
||||
|
||||
/**
|
||||
* Java-friendly {@link ProducerSupport} inherited by {@link UntypedProducerActor} implementations.
|
||||
* Java-friendly ProducerSupport.
|
||||
*
|
||||
* @see UntypedProducerActor
|
||||
*
|
||||
* @author Martin Krasser
|
||||
*/
|
||||
trait UntypedProducer extends ProducerSupport { this: UntypedActor =>
|
||||
|
||||
final override def endpointUri = getEndpointUri
|
||||
|
||||
final override def oneway = isOneway
|
||||
|
||||
final override def receiveBeforeProduce = {
|
||||
|
|
@ -213,10 +219,10 @@ trait UntypedProducer extends ProducerSupport { this: UntypedActor =>
|
|||
def onReceiveBeforeProduce(message: Any): Any = super.receiveBeforeProduce(message)
|
||||
|
||||
/**
|
||||
* Called after the a result was received from the endpoint specified by <code>getEndpointUri</code>. The
|
||||
* result is passed as argument. By default, this method replies the result back to the original sender
|
||||
* if <code>isOneway</code> returns false. If <code>isOneway</code> returns true then nothing is done. This
|
||||
* method may be overridden by subclasses.
|
||||
* Called after a response was received from the endpoint specified by <code>endpointUri</code>. The
|
||||
* response is passed as argument. By default, this method sends the response back to the original sender
|
||||
* if <code>oneway</code> is <code>false</code>. If <code>oneway</code> is <code>true</code>, nothing is
|
||||
* done. This method may be overridden by subclasses (e.g. to forward responses to another actor).
|
||||
*/
|
||||
@throws(classOf[Exception])
|
||||
def onReceiveAfterProduce(message: Any): Unit = super.receiveAfterProduce(message)
|
||||
|
|
|
|||
|
|
@ -14,14 +14,13 @@ import jsr166x.Deque
|
|||
import org.apache.camel._
|
||||
import org.apache.camel.impl.{DefaultProducer, DefaultEndpoint, DefaultComponent}
|
||||
|
||||
import se.scalablesolutions.akka.camel.{Failure, CamelMessageConversion, Message}
|
||||
import CamelMessageConversion.toExchangeAdapter
|
||||
import se.scalablesolutions.akka.actor._
|
||||
import se.scalablesolutions.akka.camel.{Failure, Message}
|
||||
import se.scalablesolutions.akka.camel.CamelMessageConversion.toExchangeAdapter
|
||||
import se.scalablesolutions.akka.dispatch.{CompletableFuture, MessageInvocation, MessageDispatcher}
|
||||
import se.scalablesolutions.akka.stm.TransactionConfig
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
|
||||
import scala.reflect.BeanProperty
|
||||
import se.scalablesolutions.akka.actor._
|
||||
|
||||
/**
|
||||
* Camel component for sending messages to and receiving replies from (untyped) actors.
|
||||
|
|
@ -33,28 +32,26 @@ import se.scalablesolutions.akka.actor._
|
|||
*/
|
||||
class ActorComponent extends DefaultComponent {
|
||||
def createEndpoint(uri: String, remaining: String, parameters: JavaMap[String, Object]): ActorEndpoint = {
|
||||
val idAndUuid = idAndUuidPair(remaining)
|
||||
new ActorEndpoint(uri, this, idAndUuid._1, idAndUuid._2)
|
||||
val (id,uuid) = idAndUuidPair(remaining)
|
||||
new ActorEndpoint(uri, this, id, uuid)
|
||||
}
|
||||
|
||||
private def idAndUuidPair(remaining: String): Tuple2[Option[String], Option[String]] = {
|
||||
remaining split ":" toList match {
|
||||
case id :: Nil => (Some(id), None)
|
||||
case "id" :: id :: Nil => (Some(id), None)
|
||||
case "uuid" :: uuid :: Nil => (None, Some(uuid))
|
||||
case _ => throw new IllegalArgumentException(
|
||||
"invalid path format: %s - should be <actorid> or id:<actorid> or uuid:<actoruuid>" format remaining)
|
||||
}
|
||||
private def idAndUuidPair(remaining: String): Tuple2[Option[String],Option[Uuid]] = remaining match {
|
||||
case null | "" => throw new IllegalArgumentException("invalid path format: [%s] - should be <actorid> or id:<actorid> or uuid:<actoruuid>" format remaining)
|
||||
case id if id startsWith "id:" => (Some(id substring 3),None)
|
||||
case uuid if uuid startsWith "uuid:" => (None,Some(uuidFrom(uuid substring 5)))
|
||||
case id => (Some(id),None)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Camel endpoint for referencing an (untyped) actor. The actor reference is given by the endpoint URI.
|
||||
* An actor can be referenced by its <code>ActorRef.id</code> or its <code>ActorRef.uuid</code>.
|
||||
* Supported endpoint URI formats are
|
||||
* <code>actor:<actorid></code>,
|
||||
* <code>actor:id:<actorid></code> and
|
||||
* <code>actor:uuid:<actoruuid></code>.
|
||||
* Camel endpoint for sending messages to and receiving replies from (untyped) actors. Actors
|
||||
* are referenced using <code>actor</code> endpoint URIs of the following format:
|
||||
* <code>actor:<actor-id></code>,
|
||||
* <code>actor:id:<actor-id></code> and
|
||||
* <code>actor:uuid:<actor-uuid></code>,
|
||||
* where <code>actor-id</code> refers to <code>ActorRef.id</code> and <code>actor-uuid</code>
|
||||
* refers to the String-representation od <code>ActorRef.uuid</code>.
|
||||
*
|
||||
* @see se.scalablesolutions.akka.camel.component.ActorComponent
|
||||
* @see se.scalablesolutions.akka.camel.component.ActorProducer
|
||||
|
|
@ -64,11 +61,12 @@ class ActorComponent extends DefaultComponent {
|
|||
class ActorEndpoint(uri: String,
|
||||
comp: ActorComponent,
|
||||
val id: Option[String],
|
||||
val uuid: Option[String]) extends DefaultEndpoint(uri, comp) {
|
||||
val uuid: Option[Uuid]) extends DefaultEndpoint(uri, comp) {
|
||||
|
||||
/**
|
||||
* Blocking of caller thread during two-way message exchanges with consumer actors. This is set
|
||||
* via the <code>blocking=true|false</code> endpoint URI parameter. If omitted blocking is false.
|
||||
* Whether to block caller thread during two-way message exchanges with (untyped) actors. This is
|
||||
* set via the <code>blocking=true|false</code> endpoint URI parameter. Default value is
|
||||
* <code>false</code>.
|
||||
*/
|
||||
@BeanProperty var blocking: Boolean = false
|
||||
|
||||
|
|
@ -90,9 +88,18 @@ class ActorEndpoint(uri: String,
|
|||
}
|
||||
|
||||
/**
|
||||
* Sends the in-message of an exchange to an (untyped) actor. If the exchange pattern is out-capable and
|
||||
* <code>blocking</code> is enabled then the producer waits for a reply (using the !! operator),
|
||||
* otherwise the ! operator is used for sending the message.
|
||||
* Sends the in-message of an exchange to an (untyped) actor.
|
||||
* <ul>
|
||||
* <li>If the exchange pattern is out-capable and <code>blocking</code> is set to
|
||||
* <code>true</code> then the producer waits for a reply, using the !! operator.</li>
|
||||
* <li>If the exchange pattern is out-capable and <code>blocking</code> is set to
|
||||
* <code>false</code> then the producer sends the message using the ! operator, together
|
||||
* with a callback handler. The callback handler is an <code>ActorRef</code> that can be
|
||||
* used by the receiving actor to asynchronously reply to the route that is sending the
|
||||
* message.</li>
|
||||
* <li>If the exchange pattern is in-only then the producer sends the message using the
|
||||
* ! operator.</li>
|
||||
* </ul>
|
||||
*
|
||||
* @see se.scalablesolutions.akka.camel.component.ActorComponent
|
||||
* @see se.scalablesolutions.akka.camel.component.ActorEndpoint
|
||||
|
|
@ -151,7 +158,7 @@ class ActorProducer(val ep: ActorEndpoint) extends DefaultProducer(ep) with Asyn
|
|||
case actors => Some(actors(0))
|
||||
}
|
||||
|
||||
private def targetByUuid(uuid: String) = ActorRegistry.actorFor(uuid)
|
||||
private def targetByUuid(uuid: Uuid) = ActorRegistry.actorFor(uuid)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -187,11 +194,11 @@ private[akka] object AsyncCallbackAdapter {
|
|||
}
|
||||
|
||||
/**
|
||||
* Adapts an <code>AsyncCallback</code> to <code>ActorRef.!</code>. Used by other actors to reply
|
||||
* asynchronously to Camel with <code>ActorRef.reply</code>.
|
||||
* Adapts an <code>ActorRef</code> to a Camel <code>AsyncCallback</code>. Used by receiving actors to reply
|
||||
* asynchronously to Camel routes with <code>ActorRef.reply</code>.
|
||||
* <p>
|
||||
* <em>Please note</em> that this adapter can only be used locally at the moment which should not
|
||||
* be a problem is most situations as Camel endpoints are only activated for local actor references,
|
||||
* be a problem is most situations since Camel endpoints are only activated for local actor references,
|
||||
* never for remote references.
|
||||
*
|
||||
* @author Martin Krasser
|
||||
|
|
@ -208,8 +215,9 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
|
|||
}
|
||||
|
||||
/**
|
||||
* Writes the reply <code>message</code> to <code>exchange</code> and uses <code>callback</code> to
|
||||
* generate completion notifications.
|
||||
* Populates the initial <code>exchange</code> with the reply <code>message</code> and uses the
|
||||
* <code>callback</code> handler to notify Camel about the asynchronous completion of the message
|
||||
* exchange.
|
||||
*
|
||||
* @param message reply message
|
||||
* @param sender ignored
|
||||
|
|
@ -249,7 +257,7 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
|
|||
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
|
||||
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
|
||||
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
|
||||
protected[akka] def linkedActors: JavaMap[String, ActorRef] = unsupported
|
||||
protected[akka] def linkedActors: JavaMap[Uuid, ActorRef] = unsupported
|
||||
protected[akka] def linkedActorsAsList: List[ActorRef] = unsupported
|
||||
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported
|
||||
protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@ object TypedActorComponent {
|
|||
|
||||
/**
|
||||
* Camel component for exchanging messages with typed actors. This component
|
||||
* tries to obtain the typed actor from the <code>typedActorRegistry</code>
|
||||
* tries to obtain the typed actor from its <code>typedActorRegistry</code>
|
||||
* first. If it's not there it tries to obtain it from the CamelContext's registry.
|
||||
*
|
||||
* @see org.apache.camel.component.bean.BeanComponent
|
||||
|
|
@ -32,9 +32,9 @@ class TypedActorComponent extends BeanComponent {
|
|||
val typedActorRegistry = new ConcurrentHashMap[String, AnyRef]
|
||||
|
||||
/**
|
||||
* Creates a {@link org.apache.camel.component.bean.BeanEndpoint} with a custom
|
||||
* bean holder that uses <code>typedActorRegistry</code> for getting access to
|
||||
* typed actors (beans).
|
||||
* Creates an <code>org.apache.camel.component.bean.BeanEndpoint</code> with a custom
|
||||
* bean holder that uses <code>typedActorRegistry</code> for getting access to typed
|
||||
* actors (beans).
|
||||
*
|
||||
* @see se.scalablesolutions.akka.camel.component.TypedActorHolder
|
||||
*/
|
||||
|
|
@ -51,7 +51,7 @@ class TypedActorComponent extends BeanComponent {
|
|||
}
|
||||
|
||||
/**
|
||||
* {@link org.apache.camel.component.bean.BeanHolder} implementation that uses a custom
|
||||
* <code>org.apache.camel.component.bean.BeanHolder</code> implementation that uses a custom
|
||||
* registry for getting access to typed actors.
|
||||
*
|
||||
* @author Martin Krasser
|
||||
|
|
@ -60,13 +60,16 @@ class TypedActorHolder(typedActorRegistry: Map[String, AnyRef], context: CamelCo
|
|||
extends RegistryBean(context, name) {
|
||||
|
||||
/**
|
||||
* Returns an {@link se.scalablesolutions.akka.camel.component.TypedActorInfo} instance.
|
||||
* Returns an <code>se.scalablesolutions.akka.camel.component.TypedActorInfo</code> instance.
|
||||
*/
|
||||
override def getBeanInfo: BeanInfo =
|
||||
new TypedActorInfo(getContext, getBean.getClass, getParameterMappingStrategy)
|
||||
|
||||
/**
|
||||
* Obtains an typed actor from <code>typedActorRegistry</code>.
|
||||
* Obtains a typed actor from <code>typedActorRegistry</code>. If the typed actor cannot
|
||||
* be found then this method tries to obtain the actor from the CamelContext's registry.
|
||||
*
|
||||
* @return a typed actor or <code>null</code>.
|
||||
*/
|
||||
override def getBean: AnyRef = {
|
||||
val bean = typedActorRegistry.get(getName)
|
||||
|
|
@ -75,7 +78,7 @@ class TypedActorHolder(typedActorRegistry: Map[String, AnyRef], context: CamelCo
|
|||
}
|
||||
|
||||
/**
|
||||
* Provides typed actor meta information.
|
||||
* Typed actor meta information.
|
||||
*
|
||||
* @author Martin Krasser
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
@ -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 = {
|
||||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -76,7 +76,7 @@ class PersistentFailerActor extends Transactor {
|
|||
}
|
||||
}
|
||||
|
||||
class HbasePersistentActorSpec extends JUnitSuite with BeforeAndAfterAll {
|
||||
class HbasePersistentActorSpecTestIntegration extends JUnitSuite with BeforeAndAfterAll {
|
||||
|
||||
val testUtil = new HBaseTestingUtility
|
||||
|
||||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
||||
|
|
@ -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
|
||||
|
||||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -0,0 +1,43 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
|
||||
*/
|
||||
|
||||
package se.scalablesolutions.akka.persistence.voldemort
|
||||
|
||||
import se.scalablesolutions.akka.actor.{newUuid}
|
||||
import se.scalablesolutions.akka.stm._
|
||||
import se.scalablesolutions.akka.persistence.common._
|
||||
|
||||
|
||||
object VoldemortStorage extends Storage {
|
||||
|
||||
type ElementType = Array[Byte]
|
||||
def newMap: PersistentMap[ElementType, ElementType] = newMap(newUuid.toString)
|
||||
def newVector: PersistentVector[ElementType] = newVector(newUuid.toString)
|
||||
def newRef: PersistentRef[ElementType] = newRef(newUuid.toString)
|
||||
|
||||
def getMap(id: String): PersistentMap[ElementType, ElementType] = newMap(id)
|
||||
def getVector(id: String): PersistentVector[ElementType] = newVector(id)
|
||||
def getRef(id: String): PersistentRef[ElementType] = newRef(id)
|
||||
|
||||
def newMap(id: String): PersistentMap[ElementType, ElementType] = new VoldemortPersistentMap(id)
|
||||
def newVector(id: String): PersistentVector[ElementType] = new VoldemortPersistentVector(id)
|
||||
def newRef(id: String): PersistentRef[ElementType] = new VoldemortPersistentRef(id)
|
||||
}
|
||||
|
||||
|
||||
class VoldemortPersistentMap(id: String) extends PersistentMapBinary {
|
||||
val uuid = id
|
||||
val storage = VoldemortStorageBackend
|
||||
}
|
||||
|
||||
|
||||
class VoldemortPersistentVector(id: String) extends PersistentVector[Array[Byte]] {
|
||||
val uuid = id
|
||||
val storage = VoldemortStorageBackend
|
||||
}
|
||||
|
||||
class VoldemortPersistentRef(id: String) extends PersistentRef[Array[Byte]] {
|
||||
val uuid = id
|
||||
val storage = VoldemortStorageBackend
|
||||
}
|
||||
|
|
@ -0,0 +1,379 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
|
||||
*/
|
||||
|
||||
package se.scalablesolutions.akka.persistence.voldemort
|
||||
|
||||
import se.scalablesolutions.akka.stm._
|
||||
import se.scalablesolutions.akka.persistence.common._
|
||||
import se.scalablesolutions.akka.util.Logging
|
||||
import se.scalablesolutions.akka.util.Helpers._
|
||||
import se.scalablesolutions.akka.config.Config.config
|
||||
|
||||
import voldemort.client._
|
||||
import java.lang.String
|
||||
import voldemort.utils.ByteUtils
|
||||
import voldemort.versioning.Versioned
|
||||
import collection.JavaConversions
|
||||
import java.nio.ByteBuffer
|
||||
import collection.Map
|
||||
import collection.immutable.{IndexedSeq, SortedSet, TreeSet, HashMap}
|
||||
import collection.mutable.{Set, HashSet, ArrayBuffer}
|
||||
import java.util.{Properties, Map => JMap}
|
||||
|
||||
private[akka] object VoldemortStorageBackend extends
|
||||
MapStorageBackend[Array[Byte], Array[Byte]] with
|
||||
VectorStorageBackend[Array[Byte]] with
|
||||
RefStorageBackend[Array[Byte]] with
|
||||
QueueStorageBackend[Array[Byte]] with
|
||||
Logging {
|
||||
val bootstrapUrlsProp = "bootstrap_urls"
|
||||
val clientConfig = config.getConfigMap("akka.storage.voldemort.client") match {
|
||||
case Some(configMap) => getClientConfig(configMap.asMap)
|
||||
case None => getClientConfig(new HashMap[String, String] + (bootstrapUrlsProp -> "tcp://localhost:6666"))
|
||||
}
|
||||
val refStore = config.getString("akka.storage.voldemort.store.ref", "Refs")
|
||||
val mapKeyStore = config.getString("akka.storage.voldemort.store.map-key", "MapKeys")
|
||||
val mapValueStore = config.getString("akka.storage.voldemort.store.map-value", "MapValues")
|
||||
val vectorStore = config.getString("akka.storage.voldemort.store.vector", "Vectors")
|
||||
val queueStore = config.getString("akka.storage.voldemort.store.queue", "Queues")
|
||||
|
||||
var storeClientFactory: StoreClientFactory = null
|
||||
var refClient: StoreClient[String, Array[Byte]] = null
|
||||
var mapKeyClient: StoreClient[String, Array[Byte]] = null
|
||||
var mapValueClient: StoreClient[Array[Byte], Array[Byte]] = null
|
||||
var vectorClient: StoreClient[Array[Byte], Array[Byte]] = null
|
||||
var queueClient: StoreClient[Array[Byte], Array[Byte]] = null
|
||||
initStoreClients
|
||||
|
||||
val underscoreBytesUTF8 = "_".getBytes("UTF-8")
|
||||
val vectorSizeIndex = -1
|
||||
val queueHeadIndex = -1
|
||||
val queueTailIndex = -2
|
||||
case class QueueMetadata(head: Int, tail: Int) {
|
||||
def size = tail - head
|
||||
//worry about wrapping etc
|
||||
}
|
||||
|
||||
implicit val byteOrder = new Ordering[Array[Byte]] {
|
||||
override def compare(x: Array[Byte], y: Array[Byte]) = ByteUtils.compare(x, y)
|
||||
}
|
||||
|
||||
|
||||
def getRefStorageFor(name: String): Option[Array[Byte]] = {
|
||||
val result: Array[Byte] = refClient.getValue(name)
|
||||
result match {
|
||||
case null => None
|
||||
case _ => Some(result)
|
||||
}
|
||||
}
|
||||
|
||||
def insertRefStorageFor(name: String, element: Array[Byte]) = {
|
||||
refClient.put(name, element)
|
||||
}
|
||||
|
||||
def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = {
|
||||
val allkeys: SortedSet[Array[Byte]] = getMapKeys(name)
|
||||
val range = allkeys.rangeImpl(start, finish).take(count)
|
||||
getKeyValues(name, range)
|
||||
}
|
||||
|
||||
def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = {
|
||||
val keys = getMapKeys(name)
|
||||
getKeyValues(name, keys)
|
||||
}
|
||||
|
||||
private def getKeyValues(name: String, keys: SortedSet[Array[Byte]]): List[(Array[Byte], Array[Byte])] = {
|
||||
val all: JMap[Array[Byte], Versioned[Array[Byte]]] =
|
||||
mapValueClient.getAll(JavaConversions.asIterable(keys.map {
|
||||
mapKey => getKey(name, mapKey)
|
||||
}))
|
||||
|
||||
val buf = new ArrayBuffer[(Array[Byte], Array[Byte])](all.size)
|
||||
JavaConversions.asMap(all).foreach {
|
||||
(entry) => {
|
||||
entry match {
|
||||
case (key: Array[Byte], versioned: Versioned[Array[Byte]]) => {
|
||||
buf += key -> versioned.getValue
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
buf.toList
|
||||
}
|
||||
|
||||
def getMapStorageSizeFor(name: String): Int = {
|
||||
val keys = getMapKeys(name)
|
||||
keys.size
|
||||
}
|
||||
|
||||
def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = {
|
||||
val result: Array[Byte] = mapValueClient.getValue(getKey(name, key))
|
||||
result match {
|
||||
case null => None
|
||||
case _ => Some(result)
|
||||
}
|
||||
}
|
||||
|
||||
def removeMapStorageFor(name: String, key: Array[Byte]) = {
|
||||
var keys = getMapKeys(name)
|
||||
keys -= key
|
||||
putMapKeys(name, keys)
|
||||
mapValueClient.delete(getKey(name, key))
|
||||
}
|
||||
|
||||
|
||||
def removeMapStorageFor(name: String) = {
|
||||
val keys = getMapKeys(name)
|
||||
keys.foreach {
|
||||
key =>
|
||||
mapValueClient.delete(getKey(name, key))
|
||||
}
|
||||
mapKeyClient.delete(name)
|
||||
}
|
||||
|
||||
def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]) = {
|
||||
mapValueClient.put(getKey(name, key), value)
|
||||
var keys = getMapKeys(name)
|
||||
keys += key
|
||||
putMapKeys(name, keys)
|
||||
}
|
||||
|
||||
def insertMapStorageEntriesFor(name: String, entries: List[(Array[Byte], Array[Byte])]) = {
|
||||
val newKeys = entries.map {
|
||||
case (key, value) => {
|
||||
mapValueClient.put(getKey(name, key), value)
|
||||
key
|
||||
}
|
||||
}
|
||||
var keys = getMapKeys(name)
|
||||
keys ++= newKeys
|
||||
putMapKeys(name, keys)
|
||||
}
|
||||
|
||||
def putMapKeys(name: String, keys: SortedSet[Array[Byte]]) = {
|
||||
mapKeyClient.put(name, SortedSetSerializer.toBytes(keys))
|
||||
}
|
||||
|
||||
def getMapKeys(name: String): SortedSet[Array[Byte]] = {
|
||||
SortedSetSerializer.fromBytes(mapKeyClient.getValue(name, Array.empty[Byte]))
|
||||
}
|
||||
|
||||
|
||||
def getVectorStorageSizeFor(name: String): Int = {
|
||||
IntSerializer.fromBytes(vectorClient.getValue(getIndexedKey(name, vectorSizeIndex), IntSerializer.toBytes(0)))
|
||||
}
|
||||
|
||||
|
||||
def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = {
|
||||
val size = getVectorStorageSizeFor(name)
|
||||
val st = start.getOrElse(0)
|
||||
val cnt =
|
||||
if (finish.isDefined) {
|
||||
val f = finish.get
|
||||
if (f >= st) (f - st) else count
|
||||
} else {
|
||||
count
|
||||
}
|
||||
val seq: IndexedSeq[Array[Byte]] = (st until st + cnt).map {
|
||||
index => getIndexedKey(name, index)
|
||||
}
|
||||
|
||||
val all: JMap[Array[Byte], Versioned[Array[Byte]]] = vectorClient.getAll(JavaConversions.asIterable(seq))
|
||||
|
||||
var storage = new ArrayBuffer[Array[Byte]](seq.size)
|
||||
storage = storage.padTo(seq.size, Array.empty[Byte])
|
||||
var idx = 0;
|
||||
seq.foreach {
|
||||
key => {
|
||||
if (all.containsKey(key)) {
|
||||
storage.update(idx, all.get(key).getValue)
|
||||
}
|
||||
idx += 1
|
||||
}
|
||||
}
|
||||
|
||||
storage.toList
|
||||
}
|
||||
|
||||
|
||||
def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = {
|
||||
vectorClient.getValue(getIndexedKey(name, index), Array.empty[Byte])
|
||||
}
|
||||
|
||||
def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) = {
|
||||
val size = getVectorStorageSizeFor(name)
|
||||
vectorClient.put(getIndexedKey(name, index), elem)
|
||||
if (size < index + 1) {
|
||||
vectorClient.put(getIndexedKey(name, vectorSizeIndex), IntSerializer.toBytes(index + 1))
|
||||
}
|
||||
}
|
||||
|
||||
def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) = {
|
||||
var size = getVectorStorageSizeFor(name)
|
||||
elements.foreach {
|
||||
element =>
|
||||
vectorClient.put(getIndexedKey(name, size), element)
|
||||
size += 1
|
||||
}
|
||||
vectorClient.put(getIndexedKey(name, vectorSizeIndex), IntSerializer.toBytes(size))
|
||||
}
|
||||
|
||||
def insertVectorStorageEntryFor(name: String, element: Array[Byte]) = {
|
||||
insertVectorStorageEntriesFor(name, List(element))
|
||||
}
|
||||
|
||||
|
||||
def remove(name: String): Boolean = {
|
||||
false
|
||||
}
|
||||
|
||||
def peek(name: String, start: Int, count: Int): List[Array[Byte]] = {
|
||||
List(Array.empty[Byte])
|
||||
}
|
||||
|
||||
def size(name: String): Int = {
|
||||
getQueueMetadata(name).size
|
||||
}
|
||||
|
||||
def dequeue(name: String): Option[Array[Byte]] = {
|
||||
None
|
||||
}
|
||||
|
||||
def enqueue(name: String, item: Array[Byte]): Option[Int] = {
|
||||
val mdata = getQueueMetadata(name)
|
||||
val key = getIndexedKey(name, mdata.tail)
|
||||
queueClient.put(key, item)
|
||||
queueClient.put(getIndexedKey(name, queueTailIndex), IntSerializer.toBytes(mdata.tail + 1))
|
||||
Some (mdata.size + 1)
|
||||
}
|
||||
|
||||
|
||||
def getQueueMetadata(name: String): QueueMetadata = {
|
||||
val keys = List(getIndexedKey(name, queueHeadIndex), getIndexedKey(name, queueTailIndex))
|
||||
val qdata = JavaConversions.asMap(queueClient.getAll(JavaConversions.asIterable(keys)))
|
||||
val values = keys.map {
|
||||
qdata.get(_) match {
|
||||
case Some(versioned) => IntSerializer.fromBytes(versioned.getValue)
|
||||
case None => 0
|
||||
}
|
||||
}
|
||||
QueueMetadata(values.head, values.tail.head)
|
||||
}
|
||||
|
||||
/**
|
||||
* Concat the ownerlenght+owner+key+ of owner so owned data will be colocated
|
||||
* Store the length of owner as first byte to work around the rare case
|
||||
* where ownerbytes1 + keybytes1 == ownerbytes2 + keybytes2 but ownerbytes1 != ownerbytes2
|
||||
*/
|
||||
|
||||
|
||||
def getKey(owner: String, key: Array[Byte]): Array[Byte] = {
|
||||
val ownerBytes: Array[Byte] = owner.getBytes("UTF-8")
|
||||
val ownerLenghtBytes: Array[Byte] = IntSerializer.toBytes(owner.length)
|
||||
val theKey = new Array[Byte](ownerLenghtBytes.length + ownerBytes.length + key.length)
|
||||
System.arraycopy(ownerLenghtBytes, 0, theKey, 0, ownerLenghtBytes.length)
|
||||
System.arraycopy(ownerBytes, 0, theKey, ownerLenghtBytes.length, ownerBytes.length)
|
||||
System.arraycopy(key, 0, theKey, ownerLenghtBytes.length + ownerBytes.length, key.length)
|
||||
theKey
|
||||
}
|
||||
|
||||
def getIndexedKey(owner: String, index: Int): Array[Byte] = {
|
||||
val indexbytes = IntSerializer.toBytes(index)
|
||||
val theIndexKey = new Array[Byte](underscoreBytesUTF8.length + indexbytes.length)
|
||||
System.arraycopy(underscoreBytesUTF8, 0, theIndexKey, 0, underscoreBytesUTF8.length)
|
||||
System.arraycopy(indexbytes, 0, theIndexKey, underscoreBytesUTF8.length, indexbytes.length)
|
||||
getKey(owner, theIndexKey)
|
||||
}
|
||||
|
||||
def getIndexFromVectorValueKey(owner: String, key: Array[Byte]): Int = {
|
||||
val indexBytes = new Array[Byte](IntSerializer.bytesPerInt)
|
||||
System.arraycopy(key, key.length - IntSerializer.bytesPerInt, indexBytes, 0, IntSerializer.bytesPerInt)
|
||||
IntSerializer.fromBytes(indexBytes)
|
||||
}
|
||||
|
||||
|
||||
def getClientConfig(configMap: Map[String, String]): Properties = {
|
||||
val properites = new Properties
|
||||
configMap.foreach {
|
||||
keyval => keyval match {
|
||||
case (key, value) => properites.setProperty(key.asInstanceOf[java.lang.String], value.asInstanceOf[java.lang.String])
|
||||
}
|
||||
}
|
||||
properites
|
||||
}
|
||||
|
||||
def initStoreClients() = {
|
||||
if (storeClientFactory != null) {
|
||||
storeClientFactory.close
|
||||
}
|
||||
|
||||
storeClientFactory = {
|
||||
if (clientConfig.getProperty(bootstrapUrlsProp, "none").startsWith("tcp")) {
|
||||
new SocketStoreClientFactory(new ClientConfig(clientConfig))
|
||||
} else if (clientConfig.getProperty(bootstrapUrlsProp, "none").startsWith("http")) {
|
||||
new HttpStoreClientFactory(new ClientConfig(clientConfig))
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown boostrapUrl syntax" + clientConfig.getProperty(bootstrapUrlsProp, "No Bootstrap URLs defined"))
|
||||
}
|
||||
}
|
||||
refClient = storeClientFactory.getStoreClient(refStore)
|
||||
mapKeyClient = storeClientFactory.getStoreClient(mapKeyStore)
|
||||
mapValueClient = storeClientFactory.getStoreClient(mapValueStore)
|
||||
vectorClient = storeClientFactory.getStoreClient(vectorStore)
|
||||
queueClient = storeClientFactory.getStoreClient(queueStore)
|
||||
}
|
||||
|
||||
object IntSerializer {
|
||||
val bytesPerInt = java.lang.Integer.SIZE / java.lang.Byte.SIZE
|
||||
|
||||
def toBytes(i: Int) = ByteBuffer.wrap(new Array[Byte](bytesPerInt)).putInt(i).array()
|
||||
|
||||
def fromBytes(bytes: Array[Byte]) = ByteBuffer.wrap(bytes).getInt()
|
||||
|
||||
def toString(obj: Int) = obj.toString
|
||||
|
||||
def fromString(str: String) = str.toInt
|
||||
}
|
||||
|
||||
object SortedSetSerializer {
|
||||
def toBytes(set: SortedSet[Array[Byte]]): Array[Byte] = {
|
||||
val length = set.foldLeft(0) {
|
||||
(total, bytes) => {
|
||||
total + bytes.length + IntSerializer.bytesPerInt
|
||||
}
|
||||
}
|
||||
val allBytes = new Array[Byte](length)
|
||||
val written = set.foldLeft(0) {
|
||||
(total, bytes) => {
|
||||
val sizeBytes = IntSerializer.toBytes(bytes.length)
|
||||
System.arraycopy(sizeBytes, 0, allBytes, total, sizeBytes.length)
|
||||
System.arraycopy(bytes, 0, allBytes, total + sizeBytes.length, bytes.length)
|
||||
total + sizeBytes.length + bytes.length
|
||||
}
|
||||
}
|
||||
require(length == written, "Bytes Written Did not equal Calculated Length, written %d, length %d".format(written, length))
|
||||
allBytes
|
||||
}
|
||||
|
||||
def fromBytes(bytes: Array[Byte]): SortedSet[Array[Byte]] = {
|
||||
var set = new TreeSet[Array[Byte]]
|
||||
if (bytes.length > IntSerializer.bytesPerInt) {
|
||||
var pos = 0
|
||||
while (pos < bytes.length) {
|
||||
val lengthBytes = new Array[Byte](IntSerializer.bytesPerInt)
|
||||
System.arraycopy(bytes, pos, lengthBytes, 0, IntSerializer.bytesPerInt)
|
||||
pos += IntSerializer.bytesPerInt
|
||||
val length = IntSerializer.fromBytes(lengthBytes)
|
||||
val item = new Array[Byte](length)
|
||||
System.arraycopy(bytes, pos, item, 0, length)
|
||||
set = set + item
|
||||
pos += length
|
||||
}
|
||||
}
|
||||
set
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,14 @@
|
|||
<cluster>
|
||||
<!-- The name is just to help users identify this cluster from the gui -->
|
||||
<name>akka-test</name>
|
||||
<server>
|
||||
<!-- The node id is a unique, sequential id beginning with 0 that identifies each server in the cluster-->
|
||||
<id>0</id>
|
||||
<host>localhost</host>
|
||||
<http-port>8081</http-port>
|
||||
<socket-port>6666</socket-port>
|
||||
<admin-port>6667</admin-port>
|
||||
<!-- A list of data partitions assigned to this server -->
|
||||
<partitions>0,1,2,3</partitions>
|
||||
</server>
|
||||
</cluster>
|
||||
|
|
@ -0,0 +1,4 @@
|
|||
node.id=0
|
||||
enable.rebalancing=false
|
||||
enable.bdb.engine=false
|
||||
slop.enable=false
|
||||
|
|
@ -0,0 +1,85 @@
|
|||
<stores>
|
||||
<store>
|
||||
<name>Refs</name>
|
||||
<replication-factor>1</replication-factor>
|
||||
<preferred-reads>1</preferred-reads>
|
||||
<required-reads>1</required-reads>
|
||||
<preferred-writes>1</preferred-writes>
|
||||
<required-writes>1</required-writes>
|
||||
<persistence>memory</persistence>
|
||||
<routing>client</routing>
|
||||
<key-serializer>
|
||||
<type>string</type>
|
||||
<schema-info>utf8</schema-info>
|
||||
</key-serializer>
|
||||
<value-serializer>
|
||||
<type>identity</type>
|
||||
</value-serializer>
|
||||
</store>
|
||||
<store>
|
||||
<name>MapValues</name>
|
||||
<replication-factor>1</replication-factor>
|
||||
<preferred-reads>1</preferred-reads>
|
||||
<required-reads>1</required-reads>
|
||||
<preferred-writes>1</preferred-writes>
|
||||
<required-writes>1</required-writes>
|
||||
<persistence>memory</persistence>
|
||||
<routing>client</routing>
|
||||
<key-serializer>
|
||||
<type>identity</type>
|
||||
</key-serializer>
|
||||
<value-serializer>
|
||||
<type>identity</type>
|
||||
</value-serializer>
|
||||
</store>
|
||||
<store>
|
||||
<name>MapKeys</name>
|
||||
<replication-factor>1</replication-factor>
|
||||
<preferred-reads>1</preferred-reads>
|
||||
<required-reads>1</required-reads>
|
||||
<preferred-writes>1</preferred-writes>
|
||||
<required-writes>1</required-writes>
|
||||
<persistence>memory</persistence>
|
||||
<routing>client</routing>
|
||||
<key-serializer>
|
||||
<type>string</type>
|
||||
<schema-info>utf8</schema-info>
|
||||
</key-serializer>
|
||||
<value-serializer>
|
||||
<type>identity</type>
|
||||
</value-serializer>
|
||||
</store>
|
||||
<store>
|
||||
<name>Vectors</name>
|
||||
<replication-factor>1</replication-factor>
|
||||
<preferred-reads>1</preferred-reads>
|
||||
<required-reads>1</required-reads>
|
||||
<preferred-writes>1</preferred-writes>
|
||||
<required-writes>1</required-writes>
|
||||
<persistence>memory</persistence>
|
||||
<routing>client</routing>
|
||||
<key-serializer>
|
||||
<type>identity</type>
|
||||
</key-serializer>
|
||||
<value-serializer>
|
||||
<type>identity</type>
|
||||
</value-serializer>
|
||||
</store>
|
||||
<store>
|
||||
<name>Queues</name>
|
||||
<replication-factor>1</replication-factor>
|
||||
<preferred-reads>1</preferred-reads>
|
||||
<required-reads>1</required-reads>
|
||||
<preferred-writes>1</preferred-writes>
|
||||
<required-writes>1</required-writes>
|
||||
<persistence>memory</persistence>
|
||||
<routing>client</routing>
|
||||
<key-serializer>
|
||||
<type>identity</type>
|
||||
</key-serializer>
|
||||
<value-serializer>
|
||||
<type>identity</type>
|
||||
</value-serializer>
|
||||
</store>
|
||||
|
||||
</stores>
|
||||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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")
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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 =
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
|
|
|||
|
|
@ -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]])
|
||||
|
|
|
|||
|
|
@ -7,12 +7,12 @@ package se.scalablesolutions.akka.remote
|
|||
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _}
|
||||
import se.scalablesolutions.akka.actor.{Exit, Actor, ActorRef, ActorType, RemoteActorRef, IllegalActorStateException}
|
||||
import se.scalablesolutions.akka.dispatch.{DefaultCompletableFuture, CompletableFuture}
|
||||
import se.scalablesolutions.akka.util.{ListenerManagement, UUID, Logging, Duration}
|
||||
import se.scalablesolutions.akka.util.{ListenerManagement, Logging, Duration}
|
||||
import se.scalablesolutions.akka.actor.{Uuid,newUuid,uuidFrom}
|
||||
import se.scalablesolutions.akka.config.Config._
|
||||
import se.scalablesolutions.akka.serialization.RemoteActorSerialization._
|
||||
import se.scalablesolutions.akka.AkkaException
|
||||
import Actor._
|
||||
|
||||
import org.jboss.netty.channel._
|
||||
import group.DefaultChannelGroup
|
||||
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
|
||||
|
|
@ -32,18 +32,6 @@ import scala.collection.mutable.{HashSet, HashMap}
|
|||
import scala.reflect.BeanProperty
|
||||
import se.scalablesolutions.akka.actor._
|
||||
|
||||
/**
|
||||
* Atomic remote request/reply message id generator.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
object RemoteRequestProtocolIdFactory {
|
||||
private val nodeId = UUID.newUuid
|
||||
private val id = new AtomicLong
|
||||
|
||||
def nextId: Long = id.getAndIncrement + nodeId
|
||||
}
|
||||
|
||||
/**
|
||||
* Life-cycle events for RemoteClient.
|
||||
*/
|
||||
|
|
@ -75,7 +63,7 @@ object RemoteClient extends Logging {
|
|||
val RECONNECT_DELAY = Duration(config.getInt("akka.remote.client.reconnect-delay", 5), TIME_UNIT)
|
||||
|
||||
private val remoteClients = new HashMap[String, RemoteClient]
|
||||
private val remoteActors = new HashMap[RemoteServer.Address, HashSet[String]]
|
||||
private val remoteActors = new HashMap[RemoteServer.Address, HashSet[Uuid]]
|
||||
|
||||
def actorFor(classNameOrServiceId: String, hostname: String, port: Int): ActorRef =
|
||||
actorFor(classNameOrServiceId, classNameOrServiceId, 5000L, hostname, port, None)
|
||||
|
|
@ -174,21 +162,21 @@ object RemoteClient extends Logging {
|
|||
remoteClients.clear
|
||||
}
|
||||
|
||||
def register(hostname: String, port: Int, uuid: String) = synchronized {
|
||||
def register(hostname: String, port: Int, uuid: Uuid) = synchronized {
|
||||
actorsFor(RemoteServer.Address(hostname, port)) += uuid
|
||||
}
|
||||
|
||||
private[akka] def unregister(hostname: String, port: Int, uuid: String) = synchronized {
|
||||
private[akka] def unregister(hostname: String, port: Int, uuid: Uuid) = synchronized {
|
||||
val set = actorsFor(RemoteServer.Address(hostname, port))
|
||||
set -= uuid
|
||||
if (set.isEmpty) shutdownClientFor(new InetSocketAddress(hostname, port))
|
||||
}
|
||||
|
||||
private[akka] def actorsFor(remoteServerAddress: RemoteServer.Address): HashSet[String] = {
|
||||
private[akka] def actorsFor(remoteServerAddress: RemoteServer.Address): HashSet[Uuid] = {
|
||||
val set = remoteActors.get(remoteServerAddress)
|
||||
if (set.isDefined && (set.get ne null)) set.get
|
||||
else {
|
||||
val remoteActorSet = new HashSet[String]
|
||||
val remoteActorSet = new HashSet[Uuid]
|
||||
remoteActors.put(remoteServerAddress, remoteActorSet)
|
||||
remoteActorSet
|
||||
}
|
||||
|
|
@ -206,8 +194,8 @@ class RemoteClient private[akka] (
|
|||
val name = "RemoteClient@" + hostname + "::" + port
|
||||
|
||||
//FIXME Should these be clear:ed on postStop?
|
||||
private val futures = new ConcurrentHashMap[Long, CompletableFuture[_]]
|
||||
private val supervisors = new ConcurrentHashMap[String, ActorRef]
|
||||
private val futures = new ConcurrentHashMap[Uuid, CompletableFuture[_]]
|
||||
private val supervisors = new ConcurrentHashMap[Uuid, ActorRef]
|
||||
|
||||
private val remoteAddress = new InetSocketAddress(hostname, port)
|
||||
|
||||
|
|
@ -299,7 +287,7 @@ class RemoteClient private[akka] (
|
|||
futures.synchronized {
|
||||
val futureResult = if (senderFuture.isDefined) senderFuture.get
|
||||
else new DefaultCompletableFuture[T](request.getActorInfo.getTimeout)
|
||||
futures.put(request.getId, futureResult)
|
||||
futures.put(uuidFrom(request.getUuid.getHigh,request.getUuid.getLow), futureResult)
|
||||
connection.getChannel.write(request)
|
||||
Some(futureResult)
|
||||
}
|
||||
|
|
@ -342,8 +330,8 @@ class RemoteClient private[akka] (
|
|||
*/
|
||||
class RemoteClientPipelineFactory(
|
||||
name: String,
|
||||
futures: ConcurrentMap[Long, CompletableFuture[_]],
|
||||
supervisors: ConcurrentMap[String, ActorRef],
|
||||
futures: ConcurrentMap[Uuid, CompletableFuture[_]],
|
||||
supervisors: ConcurrentMap[Uuid, ActorRef],
|
||||
bootstrap: ClientBootstrap,
|
||||
remoteAddress: SocketAddress,
|
||||
timer: HashedWheelTimer,
|
||||
|
|
@ -382,8 +370,8 @@ class RemoteClientPipelineFactory(
|
|||
@ChannelHandler.Sharable
|
||||
class RemoteClientHandler(
|
||||
val name: String,
|
||||
val futures: ConcurrentMap[Long, CompletableFuture[_]],
|
||||
val supervisors: ConcurrentMap[String, ActorRef],
|
||||
val futures: ConcurrentMap[Uuid, CompletableFuture[_]],
|
||||
val supervisors: ConcurrentMap[Uuid, ActorRef],
|
||||
val bootstrap: ClientBootstrap,
|
||||
val remoteAddress: SocketAddress,
|
||||
val timer: HashedWheelTimer,
|
||||
|
|
@ -403,14 +391,15 @@ class RemoteClientHandler(
|
|||
val result = event.getMessage
|
||||
if (result.isInstanceOf[RemoteReplyProtocol]) {
|
||||
val reply = result.asInstanceOf[RemoteReplyProtocol]
|
||||
val replyUuid = uuidFrom(reply.getUuid.getHigh,reply.getUuid.getLow)
|
||||
log.debug("Remote client received RemoteReplyProtocol[\n%s]", reply.toString)
|
||||
val future = futures.get(reply.getId).asInstanceOf[CompletableFuture[Any]]
|
||||
val future = futures.get(replyUuid).asInstanceOf[CompletableFuture[Any]]
|
||||
if (reply.getIsSuccessful) {
|
||||
val message = MessageSerializer.deserialize(reply.getMessage)
|
||||
future.completeWithResult(message)
|
||||
} else {
|
||||
if (reply.hasSupervisorUuid()) {
|
||||
val supervisorUuid = reply.getSupervisorUuid
|
||||
val supervisorUuid = uuidFrom(reply.getSupervisorUuid.getHigh,reply.getSupervisorUuid.getLow)
|
||||
if (!supervisors.containsKey(supervisorUuid)) throw new IllegalActorStateException(
|
||||
"Expected a registered supervisor for UUID [" + supervisorUuid + "] but none was found")
|
||||
val supervisedActor = supervisors.get(supervisorUuid)
|
||||
|
|
@ -420,7 +409,7 @@ class RemoteClientHandler(
|
|||
}
|
||||
future.completeWithException(parseException(reply, client.loader))
|
||||
}
|
||||
futures.remove(reply.getId)
|
||||
futures remove replyUuid
|
||||
} else {
|
||||
val exception = new RemoteClientException("Unknown message received in remote client handler: " + result, client)
|
||||
client.notifyListeners(RemoteClientError(exception, client))
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -91,10 +91,32 @@ object Serializable {
|
|||
}
|
||||
|
||||
/**
|
||||
* case class Address(street: String, city: String, zip: String)
|
||||
* extends ScalaJSON[Address] {
|
||||
*
|
||||
* implicit val AddressFormat: Format[Address] =
|
||||
* asProduct3("street", "city", "zip")(Address)(Address.unapply(_).get)
|
||||
*
|
||||
* import dispatch.json._
|
||||
* import sjson.json._
|
||||
* import sjson.json.JsonSerialization._
|
||||
*
|
||||
* def toJSON: String = JsValue.toJson(tojson(this))
|
||||
* def toBytes: Array[Byte] = tobinary(this)
|
||||
* def fromBytes(bytes: Array[Byte]): Address = frombinary[Address](bytes)
|
||||
* def fromJSON(js: String): Address = fromjson[Address](Js(js))
|
||||
* }
|
||||
*
|
||||
* val a = Address(...)
|
||||
* val js = tojson(a)
|
||||
* val add = fromjson[Address](js)
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
trait ScalaJSON extends JSON {
|
||||
def toJSON: String = new String(toBytes, "UTF-8")
|
||||
def toBytes: Array[Byte] = SJSONSerializer.SJSON.out(this)
|
||||
trait ScalaJSON[T] extends JSON {
|
||||
def toJSON: String
|
||||
def fromJSON(js: String): T
|
||||
def toBytes: Array[Byte]
|
||||
def fromBytes(bytes: Array[Byte]): T
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -128,11 +128,23 @@ object Serializer {
|
|||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
object ScalaJSON extends ScalaJSON
|
||||
trait ScalaJSON extends Serializer {
|
||||
trait ScalaJSON {
|
||||
import sjson.json._
|
||||
|
||||
var classLoader: Option[ClassLoader] = None
|
||||
|
||||
def tojson[T](o: T)(implicit tjs: Writes[T]): JsValue = JsonSerialization.tojson(o)(tjs)
|
||||
|
||||
def fromjson[T](json: JsValue)(implicit fjs: Reads[T]): T = JsonSerialization.fromjson(json)(fjs)
|
||||
|
||||
def tobinary[T](o: T)(implicit tjs: Writes[T]): Array[Byte] = JsonSerialization.tobinary(o)(tjs)
|
||||
|
||||
def frombinary[T](bytes: Array[Byte])(implicit fjs: Reads[T]): T = JsonSerialization.frombinary(bytes)(fjs)
|
||||
|
||||
// backward compatibility
|
||||
// implemented using refelction based json serialization
|
||||
def toBinary(obj: AnyRef): Array[Byte] = SJSONSerializer.SJSON.out(obj)
|
||||
|
||||
// FIXME set ClassLoader on SJSONSerializer.SJSON
|
||||
def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = SJSONSerializer.SJSON.in(bytes)
|
||||
|
||||
import scala.reflect.Manifest
|
||||
|
|
@ -144,6 +156,7 @@ object Serializer {
|
|||
SJSONSerializer.SJSON.in(bytes)(m)
|
||||
}
|
||||
}
|
||||
object ScalaJSON extends ScalaJSON
|
||||
|
||||
/**
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
|
|
|
|||
9
akka-remote/src/main/scala/serialization/package.scala
Normal file
9
akka-remote/src/main/scala/serialization/package.scala
Normal file
|
|
@ -0,0 +1,9 @@
|
|||
package se.scalablesolutions.akka
|
||||
|
||||
package object serialization {
|
||||
type JsValue = _root_.dispatch.json.JsValue
|
||||
val JsValue = _root_.dispatch.json.JsValue
|
||||
val Js = _root_.dispatch.json.Js
|
||||
val JsonSerialization = sjson.json.JsonSerialization
|
||||
val DefaultProtocol = sjson.json.DefaultProtocol
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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") {
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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")
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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 = {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -16,6 +16,10 @@
|
|||
</encoder>
|
||||
</appender>
|
||||
<logger name="se.scalablesolutions" level="DEBUG"/>
|
||||
|
||||
<logger name="org.mortbay.log" level="ERROR"/>
|
||||
<logger name="org.apache.jasper" level="ERROR"/>
|
||||
|
||||
<root level="DEBUG">
|
||||
<appender-ref ref="stdout"/>
|
||||
</root>
|
||||
|
|
|
|||
BIN
embedded-repo/com/eaio/uuid/3.2/uuid-3.2.jar
Normal file
BIN
embedded-repo/com/eaio/uuid/3.2/uuid-3.2.jar
Normal file
Binary file not shown.
8
embedded-repo/com/eaio/uuid/3.2/uuid-3.2.pom
Normal file
8
embedded-repo/com/eaio/uuid/3.2/uuid-3.2.pom
Normal file
|
|
@ -0,0 +1,8 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>com.eaio</groupId>
|
||||
<artifactId>uuid</artifactId>
|
||||
<version>3.2</version>
|
||||
<packaging>jar</packaging>
|
||||
</project>
|
||||
|
|
@ -0,0 +1,8 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-core</artifactId>
|
||||
<version>0.20.6</version>
|
||||
<packaging>jar</packaging>
|
||||
</project>
|
||||
|
|
@ -0,0 +1,8 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
<artifactId>hbase-test</artifactId>
|
||||
<version>0.20.6</version>
|
||||
<packaging>jar</packaging>
|
||||
</project>
|
||||
|
|
@ -53,6 +53,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
lazy val SunJDMKRepo = MavenRepository("Sun JDMK Repo", "http://wp5.e-taxonomy.eu/cdmlib/mavenrepo")
|
||||
lazy val CasbahRepoReleases = MavenRepository("Casbah Release Repo", "http://repo.bumnetworks.com/releases")
|
||||
lazy val ZookeeperRepo = MavenRepository("Zookeeper Repo", "http://lilycms.org/maven/maven2/deploy/")
|
||||
lazy val ClojarsRepo = MavenRepository("Clojars Repo", "http://clojars.org/repo")
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
|
@ -83,6 +84,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
lazy val zookeeperRelease = ModuleConfiguration("org.apache.hadoop.zookeeper",ZookeeperRepo)
|
||||
lazy val casbahModuleConfig = ModuleConfiguration("com.novus", CasbahRepo)
|
||||
lazy val timeModuleConfig = ModuleConfiguration("org.scala-tools", "time", CasbahSnapshotRepo)
|
||||
lazy val voldemortModuleConfig = ModuleConfiguration("voldemort", ClojarsRepo)
|
||||
lazy val embeddedRepo = EmbeddedRepo // This is the only exception, because the embedded repo is fast!
|
||||
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
|
@ -95,7 +97,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
lazy val DISPATCH_VERSION = "0.7.4"
|
||||
lazy val HAWT_DISPATCH_VERSION = "1.0"
|
||||
lazy val JACKSON_VERSION = "1.2.1"
|
||||
lazy val JERSEY_VERSION = "1.2"
|
||||
lazy val JERSEY_VERSION = "1.3"
|
||||
lazy val MULTIVERSE_VERSION = "0.6.1"
|
||||
lazy val SCALATEST_VERSION = "1.2-for-scala-2.8.0.final-SNAPSHOT"
|
||||
lazy val LOGBACK_VERSION = "0.9.24"
|
||||
|
|
@ -147,6 +149,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
lazy val jetty_xml = "org.eclipse.jetty" % "jetty-xml" % JETTY_VERSION % "compile"
|
||||
lazy val jetty_servlet = "org.eclipse.jetty" % "jetty-servlet" % JETTY_VERSION % "compile"
|
||||
|
||||
lazy val uuid = "com.eaio" % "uuid" % "3.2" % "compile"
|
||||
|
||||
lazy val guicey = "org.guiceyfruit" % "guice-all" % "2.0" % "compile"
|
||||
|
||||
lazy val h2_lzf = "voldemort.store.compress" % "h2-lzf" % "1.0" % "compile"
|
||||
|
|
@ -191,6 +195,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
lazy val sbinary = "sbinary" % "sbinary" % "2.8.0-0.3.1" % "compile"
|
||||
|
||||
lazy val sjson = "sjson.json" % "sjson" % "0.8-2.8.0" % "compile"
|
||||
lazy val sjson_test = "sjson.json" % "sjson" % "0.8-2.8.0" % "test"
|
||||
|
||||
lazy val slf4j = "org.slf4j" % "slf4j-api" % SLF4J_VERSION % "compile"
|
||||
|
||||
|
|
@ -204,6 +209,10 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
|
||||
lazy val thrift = "com.facebook" % "thrift" % "r917130" % "compile"
|
||||
|
||||
lazy val voldemort = "voldemort" % "voldemort" % "0.81" % "compile"
|
||||
lazy val voldemort_contrib = "voldemort" % "voldemort-contrib" % "0.81" % "compile"
|
||||
lazy val voldemort_needs_log4j = "org.slf4j" % "log4j-over-slf4j" % SLF4J_VERSION % "compile"
|
||||
|
||||
lazy val werkz = "org.codehaus.aspectwerkz" % "aspectwerkz-nodeps-jdk5" % ASPECTWERKZ_VERSION % "compile"
|
||||
lazy val werkz_core = "org.codehaus.aspectwerkz" % "aspectwerkz-jdk5" % ASPECTWERKZ_VERSION % "compile"
|
||||
|
||||
|
|
@ -226,10 +235,18 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
lazy val junit = "junit" % "junit" % "4.5" % "test"
|
||||
lazy val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test"
|
||||
lazy val scalatest = "org.scalatest" % "scalatest" % SCALATEST_VERSION % "test"
|
||||
|
||||
//HBase testing
|
||||
lazy val hadoop_test = "org.apache.hadoop" % "hadoop-test" % "0.20.2" % "test"
|
||||
lazy val hbase_test = "org.apache.hbase" % "hbase-test" % "0.20.6" % "test"
|
||||
lazy val log4j = "log4j" % "log4j" % "1.2.15" % "test"
|
||||
lazy val jett_mortbay = "org.mortbay.jetty" % "jetty" % "6.1.14" % "test"
|
||||
lazy val jetty_mortbay = "org.mortbay.jetty" % "jetty" % "6.1.14" % "test"
|
||||
|
||||
//voldemort testing
|
||||
lazy val jdom = "org.jdom" % "jdom" % "1.1" % "test"
|
||||
lazy val vold_jetty = "org.mortbay.jetty" % "jetty" % "6.1.18" % "test"
|
||||
lazy val velocity = "org.apache.velocity" % "velocity" % "1.6.2" % "test"
|
||||
lazy val dbcp = "commons-dbcp" % "commons-dbcp" % "1.2.2" % "test"
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
|
@ -354,6 +371,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
||||
class AkkaActorProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
|
||||
val uuid = Dependencies.uuid
|
||||
val configgy = Dependencies.configgy
|
||||
val hawtdispatch = Dependencies.hawtdispatch
|
||||
val multiverse = Dependencies.multiverse
|
||||
|
|
@ -460,6 +478,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
|
||||
class AkkaCamelProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
|
||||
val camel_core = Dependencies.camel_core
|
||||
|
||||
override def testOptions = createTestFilter( _.endsWith("Test"))
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
|
@ -477,6 +497,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
new AkkaCassandraProject(_), akka_persistence_common)
|
||||
lazy val akka_persistence_hbase = project("akka-persistence-hbase", "akka-persistence-hbase",
|
||||
new AkkaHbaseProject(_), akka_persistence_common)
|
||||
lazy val akka_persistence_voldemort = project("akka-persistence-voldemort", "akka-persistence-voldemort",
|
||||
new AkkaVoldemortProject(_), akka_persistence_common)
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
|
@ -496,7 +518,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
val commons_codec = Dependencies.commons_codec
|
||||
val redis = Dependencies.redis
|
||||
|
||||
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
|
||||
override def testOptions = createTestFilter( _.endsWith("Test"))
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
|
@ -507,7 +529,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
val mongo = Dependencies.mongo
|
||||
val casbah = Dependencies.casbah
|
||||
|
||||
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
|
||||
override def testOptions = createTestFilter( _.endsWith("Test"))
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
|
@ -523,7 +545,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
val google_coll = Dependencies.google_coll
|
||||
val high_scale = Dependencies.high_scale
|
||||
|
||||
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
|
||||
override def testOptions = createTestFilter( _.endsWith("Test"))
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
|
@ -531,17 +553,52 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
||||
class AkkaHbaseProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
|
||||
val zookeeper = Dependencies.zookeeper
|
||||
val hadoop_core = Dependencies.hadoop_core
|
||||
val hbase_core = Dependencies.hbase_core
|
||||
|
||||
// testing
|
||||
val hadoop_test = Dependencies.hadoop_test
|
||||
val hbase_test = Dependencies.hbase_test
|
||||
val jetty = Dependencies.jett_mortbay
|
||||
val log4j = Dependencies.log4j
|
||||
override def ivyXML =
|
||||
<dependencies>
|
||||
<dependency org="org.apache.hadoop.zookeeper" name="zookeeper" rev="3.2.2" conf="compile">
|
||||
</dependency>
|
||||
<dependency org="org.apache.hadoop" name="hadoop-core" rev="0.20.2" conf="compile">
|
||||
</dependency>
|
||||
<dependency org="org.apache.hbase" name="hbase-core" rev="0.20.6" conf="compile">
|
||||
</dependency>
|
||||
|
||||
<dependency org="org.apache.hadoop" name="hadoop-test" rev="0.20.2" conf="test">
|
||||
<exclude module="slf4j-api"/>
|
||||
</dependency>
|
||||
<dependency org="org.slf4j" name="slf4j-api" rev={SLF4J_VERSION} conf="test">
|
||||
</dependency>
|
||||
<dependency org="org.apache.hbase" name="hbase-test" rev="0.20.6" conf="test">
|
||||
</dependency>
|
||||
<dependency org="log4j" name="log4j" rev="1.2.15" conf="test">
|
||||
</dependency>
|
||||
<dependency org="org.mortbay.jetty" name="jetty" rev="6.1.14" conf="test">
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
override def testOptions = createTestFilter( _.endsWith("Test") )
|
||||
}
|
||||
|
||||
// akka-persistence-voldemort subproject
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
||||
class AkkaVoldemortProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
|
||||
val voldemort = Dependencies.voldemort
|
||||
val voldemort_contrib = Dependencies.voldemort_contrib
|
||||
val voldemort_needs_log4j = Dependencies.voldemort_needs_log4j
|
||||
|
||||
//testing
|
||||
val scalatest = Dependencies.scalatest
|
||||
val google_coll = Dependencies.google_coll
|
||||
val jdom = Dependencies.jdom
|
||||
val jetty = Dependencies.vold_jetty
|
||||
val velocity = Dependencies.velocity
|
||||
val dbcp = Dependencies.dbcp
|
||||
val sjson = Dependencies.sjson_test
|
||||
|
||||
override def testOptions = createTestFilter( _.endsWith("Suite"))
|
||||
}
|
||||
|
||||
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
// akka-kernel subproject
|
||||
// -------------------------------------------------------------------------------------------------------------------
|
||||
|
|
@ -585,7 +642,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
new AkkaOSGiAssemblyProject(_), akka_osgi_dependencies_bundle, akka_remote, akka_amqp, akka_http,
|
||||
akka_camel, akka_spring, akka_jta, akka_persistence.akka_persistence_common,
|
||||
akka_persistence.akka_persistence_redis, akka_persistence.akka_persistence_mongo,
|
||||
akka_persistence.akka_persistence_cassandra)
|
||||
akka_persistence.akka_persistence_cassandra,akka_persistence.akka_persistence_hbase,
|
||||
akka_persistence.akka_persistence_voldemort)
|
||||
}
|
||||
|
||||
class AkkaOSGiDependenciesBundleProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) with BNDPlugin {
|
||||
|
|
@ -706,6 +764,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
<dependency org="org.apache.activemq" name="activemq-core" rev="5.3.2">
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
override def testOptions = createTestFilter( _.endsWith("Test"))
|
||||
}
|
||||
|
||||
class AkkaSampleSecurityProject(info: ProjectInfo) extends AkkaDefaultProject(info, deployPath) {
|
||||
|
|
@ -772,6 +832,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
}
|
||||
|
||||
def akkaArtifacts = descendents(info.projectPath / "dist", "*" + buildScalaVersion + "-" + version + ".jar")
|
||||
lazy val integrationTestsEnabled = systemOptional[Boolean]("integration.tests",false)
|
||||
lazy val stressTestsEnabled = systemOptional[Boolean]("stress.tests",false)
|
||||
|
||||
// ------------------------------------------------------------
|
||||
class AkkaDefaultProject(info: ProjectInfo, val deployPath: Path) extends DefaultProject(info) with DeployProject with OSGiProject {
|
||||
|
|
@ -782,6 +844,21 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
|
|||
override def packageDocsJar = this.defaultJarPath("-docs.jar")
|
||||
override def packageSrcJar = this.defaultJarPath("-sources.jar")
|
||||
override def packageToPublishActions = super.packageToPublishActions ++ Seq(this.packageDocs, this.packageSrc)
|
||||
|
||||
/**
|
||||
* Used for testOptions, possibility to enable the running of integration and or stresstests
|
||||
*
|
||||
* To enable set true and disable set false
|
||||
* set integration.tests true
|
||||
* set stress.tests true
|
||||
*/
|
||||
def createTestFilter(defaultTests: (String) => Boolean) = { TestFilter({
|
||||
case s: String if defaultTests(s) => true
|
||||
case s: String if integrationTestsEnabled.value => s.endsWith("TestIntegration")
|
||||
case s: String if stressTestsEnabled.value => s.endsWith("TestStress")
|
||||
case _ => false
|
||||
}) :: Nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue